From cba1600d0f6a82f1ea194f3214a80f283fe8dc27 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sat, 23 Mar 2024 14:52:56 +0800 Subject: [PATCH 001/580] [fix] [broker] Close dispatchers stuck due to mismatch between dispatcher.consumerList and dispatcher.consumerSet (#22270) --- ...PersistentDispatcherMultipleConsumers.java | 36 +++++-- ...istentDispatcherMultipleConsumersTest.java | 101 ++++++++++++++++++ 2 files changed, 127 insertions(+), 10 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index be82b190ffb32..35204e7af72bf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -217,15 +217,7 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE consumerList.remove(consumer); log.info("Removed consumer {} with pending {} acks", consumer, consumer.getPendingAcks().size()); if (consumerList.isEmpty()) { - cancelPendingRead(); - - redeliveryMessages.clear(); - redeliveryTracker.clear(); - if (closeFuture != null) { - log.info("[{}] All consumers removed. Subscription is disconnected", name); - closeFuture.complete(null); - } - totalAvailablePermits = 0; + clearComponentsAfterRemovedAllConsumers(); } else { if (log.isDebugEnabled()) { log.debug("[{}] Consumer are left, reading more entries", name); @@ -242,8 +234,29 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE readMoreEntries(); } } else { - log.info("[{}] Trying to remove a non-connected consumer: {}", name, consumer); + /** + * This is not an expected scenario, it will never happen in expected. + * Just add a defensive code to avoid the topic can not be unloaded anymore: remove the consumers which + * are not mismatch with {@link #consumerSet}. See more detail: https://github.com/apache/pulsar/pull/22270. + */ + log.error("[{}] Trying to remove a non-connected consumer: {}", name, consumer); + consumerList.removeIf(c -> consumer.equals(c)); + if (consumerList.isEmpty()) { + clearComponentsAfterRemovedAllConsumers(); + } + } + } + + private synchronized void clearComponentsAfterRemovedAllConsumers() { + cancelPendingRead(); + + redeliveryMessages.clear(); + redeliveryTracker.clear(); + if (closeFuture != null) { + log.info("[{}] All consumers removed. Subscription is disconnected", name); + closeFuture.complete(null); } + totalAvailablePermits = 0; } @Override @@ -554,6 +567,9 @@ public synchronized CompletableFuture disconnectAllConsumers( if (consumerList.isEmpty()) { closeFuture.complete(null); } else { + // Iterator of CopyOnWriteArrayList uses the internal array to do the for-each, and CopyOnWriteArrayList + // will create a new internal array when adding/removing a new item. So remove items in the for-each + // block is safety when the for-each and add/remove are using a same lock. consumerList.forEach(consumer -> consumer.disconnect(isResetCursor, assignedBrokerLookupData)); cancelPendingRead(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java new file mode 100644 index 0000000000000..f24c5c5933e5b --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java @@ -0,0 +1,101 @@ +/* + * 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.service.persistent; + +import com.carrotsearch.hppc.ObjectSet; +import java.util.List; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class PersistentDispatcherMultipleConsumersTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test(timeOut = 30 * 1000) + public void testTopicDeleteIfConsumerSetMismatchConsumerList() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subscription, MessageId.earliest); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).subscribe(); + // Make an error that "consumerSet" is mismatch with "consumerList". + Dispatcher dispatcher = pulsar.getBrokerService() + .getTopic(topicName, false).join().get() + .getSubscription(subscription).getDispatcher(); + ObjectSet consumerSet = + WhiteboxImpl.getInternalState(dispatcher, "consumerSet"); + List consumerList = + WhiteboxImpl.getInternalState(dispatcher, "consumerList"); + + org.apache.pulsar.broker.service.Consumer serviceConsumer = consumerList.get(0); + consumerSet.add(serviceConsumer); + consumerList.add(serviceConsumer); + + // Verify: the topic can be deleted successfully. + consumer.close(); + admin.topics().delete(topicName, false); + } + + @Test(timeOut = 30 * 1000) + public void testTopicDeleteIfConsumerSetMismatchConsumerList2() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subscription, MessageId.earliest); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).subscribe(); + // Make an error that "consumerSet" is mismatch with "consumerList". + Dispatcher dispatcher = pulsar.getBrokerService() + .getTopic(topicName, false).join().get() + .getSubscription(subscription).getDispatcher(); + ObjectSet consumerSet = + WhiteboxImpl.getInternalState(dispatcher, "consumerSet"); + consumerSet.clear(); + + // Verify: the topic can be deleted successfully. + consumer.close(); + admin.topics().delete(topicName, false); + } +} From afe4261e2b4c07df5498649d617f76e263ab1119 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 25 Mar 2024 15:20:48 +0800 Subject: [PATCH 002/580] [improve] [pip] PIP-344 Correct the behavior of the public API pulsarClient.getPartitionsForTopic(topicName) (#22182) --- pip/pip-344.md | 127 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 127 insertions(+) create mode 100644 pip/pip-344.md diff --git a/pip/pip-344.md b/pip/pip-344.md new file mode 100644 index 0000000000000..5eafc6fd5c279 --- /dev/null +++ b/pip/pip-344.md @@ -0,0 +1,127 @@ +# PIP-344: Correct the behavior of the public API pulsarClient.getPartitionsForTopic(topicName) + +# Background knowledge + +**Topic auto-creation** +- The partitioned topic auto-creation is dependent on `pulsarClient.getPartitionsForTopic` + - It triggers partitioned metadata creation by `pulsarClient.getPartitionsForTopic` + - And triggers the topic partition creation by producers' registration and consumers' registration. +- When calling `pulsarClient.getPartitionsForTopic(topicName)`, Pulsar will automatically create the partitioned topic metadata if it does not exist, either using `HttpLookupService` or `BinaryProtoLookupService`. + +**Now `pulsarClient.getPartitionsForTopic`'s behavior** +| case | broker allow `auto-create` | param allow
`create if not exists` | non-partitioned topic | partitioned topic | current behavior | +| --- | --- | --- | --- | --- | --- | +| 1 | `true/false` | `true/false` | `exists: true` | | REST API: `partitions: 0`
Binary API: `partitions: 0` | +| 2 | `true/false` | `true/false` | | `exists: true`
`partitions: 3` | REST API: `partitions: 3`
Binary API: `partitions: 3` | +| 3 | `true` | `true` | | | REST API:
  - `create new: true`
  - `partitions: 3`
Binary API:
  - `create new: true`
  - `partitions: 3`
| +| 4 | `true` | `false` | | | REST API:
  - `create new: false`
  - `partitions: 0`
Binary API:
  not support
| +| 5 | `false` | `true` | | | REST API:
  - `create new: false`
  - `partitions: 0`
Binary API:
  - `create new: false`
  - `partitions: 0`
| + +- Broker allows `auto-create`: see also the config `allowAutoTopicCreation` in `broker.conf`. +- Param allow
`create if not exists` + - Regarding the HTTP API `PersistentTopics.getPartitionedMetadata`, it is an optional param which named `checkAllowAutoCreation,` and the default value is `false`. + - Regarding the `pulsar-admin` API, it depends on the HTTP API `PersistentTopics.getPartitionedMetadata`, and it always sets the param `checkAllowAutoCreation` to `false` and can not be set manually. + - Regarding the client API `HttpLookupService.getPartitionedTopicMetadata`, it depends on the HTTP API `PersistentTopics.getPartitionedMetadata`, and it always sets the param `checkAllowAutoCreation` to `true` and can not be set manually. + - Regarding the client API `BinaryProtoLookupService.getPartitionedTopicMetadata`, it always tries to create partitioned metadata. +- `REST API & HTTP API`: Since there are only two implementations of the 4 ways to get partitioned metadata, we call HTTP API `PersistentTopics.getPartitionedMetadata`, `pulsar-admin`, and `HttpLookupService.getPartitionedTopicMetadata` HTTP API, and call `BinaryProtoLookupService.getPartitionedTopicMetadata` Binary API. + +# Motivation + +The param `create if not exists` of the Binary API is always `true.` + +- For case 4 of `pulsarClient.getPartitionsForTopic`'s behavior, it always tries to create the partitioned metadata, but the API name is `getxxx`. +- For case 5 of `pulsarClient.getPartitionsForTopic`'s behavior, it returns a `0` partitioned metadata, but the topic does not exist. For the correct behavior of this case, we had discussed [here](https://github.com/apache/pulsar/issues/8813) before. +- BTW, [flink-connector-pulsar](https://github.com/apache/flink-connector-pulsar/blob/main/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegister.java#L221-L227) is using this API to create partitioned topic metadata. + +# Goals + +- Regarding the case 4: Add a new API `PulsarClient.getPartitionsForTopic(String, boolean)` to support the feature that just get partitioned topic metadata and do not try to create one. See detail below. +- Regarding the case 5: Instead of returning a `0` partitioned metadata, respond to a not found error when calling `pulsarClient.getPartitionsForTopic(String)` if the topic does not exist. + +# Detailed Design + +## Public-facing Changes + +When you call the public API `pulsarClient.getPartitionsForTopic`, pulsar will not create the partitioned metadata anymore. + +### Public API +**LookupService.java** +```java + +- CompletableFuture getPartitionedTopicMetadata(TopicName topicName); + +/** + * 1. Get the partitions if the topic exists. Return "{partition: n}" if a partitioned topic exists; return "{partition: 0}" if a non-partitioned topic exists. + * 2. When {@param createIfAutoCreationEnabled} is "false," neither partitioned topic nor non-partitioned topic does not exist. You will get an {@link PulsarClientException.NotFoundException}. + * 2-1. You will get a {@link PulsarClientException.NotSupportedException} if the broker's version is an older one that does not support this feature and the Pulsar client is using a binary protocol "serviceUrl". + * 3. When {@param createIfAutoCreationEnabled} is "true," it will trigger an auto-creation for this topic(using the default topic auto-creation strategy you set for the broker), and the corresponding result is returned. For the result, see case 1. + * @version 3.3.0 + */ ++ CompletableFuture getPartitionedTopicMetadata(TopicName topicName, boolean createIfAutoCreationEnabled); +``` + +The behavior of the new API `LookupService.getPartitionedTopicMetadata(TopicName, boolean)`. + +| case | client-side param: `createIfAutoCreationEnabled` | non-partitioned topic | partitioned topic | broker-side: topic auto-creation strategy | current behavior | +|------|--------------------------------------------------|-----------------------|-------------------------------------|--------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------| +| 1 | `true/false` | `exists: true` | | | REST/Binary API: `{partitions: 0}` | +| 2 | `true/false` | | `exists: true`
`partitions: 2` | | REST/Binary API: `{partitions: 2}` | +| 3 | `true` | | | `allowAutoTopicCreation`: `true` `allowAutoTopicCreationType`: `non-partitioned` | REST/Binary API:
  - `create new: true`
  - `{partitions: 0}` | +| 4 | `true` | | | `allowAutoTopicCreation`: `true` `allowAutoTopicCreationType`: `partitioned`
`defaultNumPartitions`: `2` | REST/Binary API:
  - `create new: true`
  - `{partitions: 2}` | +| 5 | `false` | | | `allowAutoTopicCreation`: `true` | REST/Binary API:
  - Not found error | +| 6 | `true` | | | `allowAutoTopicCreation`: `false` | REST/Binary API:
  - Not found error | + + +**PulsarClient.java** +```java +// This API existed before. Not change it, thus ensuring compatibility. ++ @Deprecated it is not suggested to use now; please use {@link #getPartitionsForTopic(TopicName, boolean)}. +- CompletableFuture> getPartitionsForTopic(String topic); ++ default CompletableFuture> getPartitionsForTopic(String topic) { ++ getPartitionsForTopic(topic, true); ++ } + +/** + * 1. Get the partitions if the topic exists. Return "[{partition-0}, {partition-1}....{partition-n}}]" if a partitioned topic exists; return "[{topic}]" if a non-partitioned topic exists. + * 2. When {@param createIfAutoCreationEnabled} is "false", neither the partitioned topic nor non-partitioned topic does not exist. You will get an {@link PulsarClientException.NotFoundException}. + * 2-1. You will get a {@link PulsarClientException.NotSupportedException} if the broker's version is an older one that does not support this feature and the Pulsar client is using a binary protocol "serviceUrl". + * 3. When {@param createIfAutoCreationEnabled} is "true," it will trigger an auto-creation for this topic(using the default topic auto-creation strategy you set for the broker), and the corresponding result is returned. For the result, see case 1. + * @version 3.3.0 + */ +CompletableFuture> getPartitionsForTopic(String topic, boolean createIfAutoCreationEnabled); +``` + +The behavior of the new API `PulsarClient.getPartitionsForTopic(String, boolean)`. + +| case | client-side param: `createIfAutoCreationEnabled` | non-partitioned topic | partitioned topic | broker-side: topic autp-creation strategy | current behavior | +|------|--------------------------------------------------|----------------------|-------------------------------------|--------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------| +| 1 | `true/false` | `exists: true` | | | REST/Binary API: `["{tenat}/{ns}/topic"]` | +| 2 | `true/false` | | `exists: true`
`partitions: 2` | | REST/Binary `API`: `["{tenat}/{ns}/topic-partition-0", "{tenat}/{ns}/topic-partition-1"]` | +| 3 | `true` | | | `allowAutoTopicCreation`: `true` `allowAutoTopicCreationType`: `non-partitioned` | REST/Binary API:
  - `create new: true`
  - `["{tenat}/{ns}/topic"]` | +| 4 | `true` | | | `allowAutoTopicCreation`: `true` `allowAutoTopicCreationType`: `partitioned`
`defaultNumPartitions`: `2` | REST/Binary API:
  - `create new: true`
  - `["{tenat}/{ns}/topic-partition-0", "{tenat}/{ns}/topic-partition-1"]` | +| 5 | `false` | | | `allowAutoTopicCreation`: `true` | REST/Binary API:
  - Not found error | +| 5 | `true` | | | `allowAutoTopicCreation`: `false` | REST/Binary API:
  - Not found error | + + + +### Binary protocol + +**CommandPartitionedTopicMetadata** +``` +message CommandPartitionedTopicMetadata { + + optional bool metadata_auto_creation_enabled = 6 [default = true]; +} +``` + +**FeatureFlags** +``` +message FeatureFlags { + + optional bool supports_binary_api_get_partitioned_meta_with_param_created_false = 5 [default = false]; +} +``` + +# Backward & Forward Compatibility + +- Old version client and New version Broker: The client will call the old API. + +- New version client and Old version Broker: The feature flag `supports_binary_api_get_partitioned_meta_with_param_created_false` will be `false`. The client will get a not-support error if the param `createIfAutoCreationEnabled` is false. From 567174f43528c0f7ae917bfb5166213973c62c29 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 25 Mar 2024 15:52:19 +0800 Subject: [PATCH 003/580] [improve][cli] PIP-343: Use picocli instead of jcommander in pulsar-function (#22331) Signed-off-by: Zixuan Liu --- pulsar-functions/instance/pom.xml | 4 +- pulsar-functions/localrun-shaded/pom.xml | 6 +- .../apache/pulsar/functions/LocalRunner.java | 70 +++++++++--------- pulsar-functions/runtime/pom.xml | 4 +- .../runtime/JavaInstanceStarter.java | 72 +++++++++---------- .../pulsar/io/docs/ConnectorDocGenerator.java | 57 ++++++--------- 6 files changed, 98 insertions(+), 115 deletions(-) diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index 0929d5ff2101b..b8d197c0683d3 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -153,8 +153,8 @@ - com.beust - jcommander + info.picocli + picocli diff --git a/pulsar-functions/localrun-shaded/pom.xml b/pulsar-functions/localrun-shaded/pom.xml index e8ac0f2faf814..ac075f7ee26fb 100644 --- a/pulsar-functions/localrun-shaded/pom.xml +++ b/pulsar-functions/localrun-shaded/pom.xml @@ -133,7 +133,7 @@ org.rocksdb:* org.eclipse.jetty*:* org.apache.avro:avro - com.beust:* + info.picocli:* net.jodah:* io.airlift:* com.yahoo.datasketches:* @@ -385,8 +385,8 @@ org.apache.pulsar.shaded.com.yahoo.sketches - com.beust - org.apache.pulsar.functions.runtime.shaded.com.beust + info.picocli + org.apache.pulsar.functions.runtime.shaded.info.picocli diff --git a/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java b/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java index 711fa33edb2a2..3b1c86a68c285 100644 --- a/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java +++ b/pulsar-functions/localrun/src/main/java/org/apache/pulsar/functions/LocalRunner.java @@ -20,9 +20,6 @@ import static org.apache.commons.lang3.StringUtils.isNotEmpty; import static org.apache.pulsar.common.functions.Utils.inferMissingArguments; -import com.beust.jcommander.IStringConverter; -import com.beust.jcommander.JCommander; -import com.beust.jcommander.Parameter; import com.google.gson.Gson; import com.google.gson.GsonBuilder; import com.google.gson.JsonParser; @@ -87,6 +84,10 @@ import org.apache.pulsar.functions.utils.functions.FunctionUtils; import org.apache.pulsar.functions.utils.io.Connector; import org.apache.pulsar.functions.utils.io.ConnectorUtils; +import picocli.CommandLine; +import picocli.CommandLine.ITypeConverter; +import picocli.CommandLine.Option; +import picocli.CommandLine.TypeConversionException; @Slf4j public class LocalRunner implements AutoCloseable { @@ -115,95 +116,95 @@ private static class UserCodeClassLoader { boolean classLoaderCreated; } - public static class FunctionConfigConverter implements IStringConverter { + public static class FunctionConfigConverter implements ITypeConverter { @Override public FunctionConfig convert(String value) { try { return ObjectMapperFactory.getMapper().reader().readValue(value, FunctionConfig.class); } catch (IOException e) { - throw new RuntimeException("Failed to parse function config:", e); + throw new TypeConversionException(e.getMessage()); } } } - public static class SourceConfigConverter implements IStringConverter { + public static class SourceConfigConverter implements ITypeConverter { @Override public SourceConfig convert(String value) { try { return ObjectMapperFactory.getMapper().reader().readValue(value, SourceConfig.class); } catch (IOException e) { - throw new RuntimeException("Failed to parse source config:", e); + throw new TypeConversionException(e.getMessage()); } } } - public static class SinkConfigConverter implements IStringConverter { + public static class SinkConfigConverter implements ITypeConverter { @Override public SinkConfig convert(String value) { try { return ObjectMapperFactory.getMapper().reader().readValue(value, SinkConfig.class); } catch (IOException e) { - throw new RuntimeException("Failed to parse sink config:", e); + throw new TypeConversionException(e.getMessage()); } } } - public static class RuntimeConverter implements IStringConverter { + public static class RuntimeConverter implements ITypeConverter { @Override public RuntimeEnv convert(String value) { return RuntimeEnv.valueOf(value); } } - @Parameter(names = "--functionConfig", description = "The json representation of FunctionConfig", + @Option(names = "--functionConfig", description = "The json representation of FunctionConfig", hidden = true, converter = FunctionConfigConverter.class) protected FunctionConfig functionConfig; - @Parameter(names = "--sourceConfig", description = "The json representation of SourceConfig", + @Option(names = "--sourceConfig", description = "The json representation of SourceConfig", hidden = true, converter = SourceConfigConverter.class) protected SourceConfig sourceConfig; - @Parameter(names = "--sinkConfig", description = "The json representation of SinkConfig", + @Option(names = "--sinkConfig", description = "The json representation of SinkConfig", hidden = true, converter = SinkConfigConverter.class) protected SinkConfig sinkConfig; - @Parameter(names = "--stateStorageImplClass", description = "The implemenatation class " + @Option(names = "--stateStorageImplClass", description = "The implemenatation class " + "state storage service (by default Apache BookKeeper)", hidden = true, required = false) protected String stateStorageImplClass; - @Parameter(names = "--stateStorageServiceUrl", description = "The URL for the state storage service " + @Option(names = "--stateStorageServiceUrl", description = "The URL for the state storage service " + "(by default Apache BookKeeper)", hidden = true) protected String stateStorageServiceUrl; - @Parameter(names = "--brokerServiceUrl", description = "The URL for the Pulsar broker", hidden = true) + @Option(names = "--brokerServiceUrl", description = "The URL for the Pulsar broker", hidden = true) protected String brokerServiceUrl; - @Parameter(names = "--webServiceUrl", description = "The URL for the Pulsar web service", hidden = true) + @Option(names = "--webServiceUrl", description = "The URL for the Pulsar web service", hidden = true) protected String webServiceUrl = null; - @Parameter(names = "--clientAuthPlugin", description = "Client authentication plugin using which " + @Option(names = "--clientAuthPlugin", description = "Client authentication plugin using which " + "function-process can connect to broker", hidden = true) protected String clientAuthPlugin; - @Parameter(names = "--clientAuthParams", description = "Client authentication param", hidden = true) + @Option(names = "--clientAuthParams", description = "Client authentication param", hidden = true) protected String clientAuthParams; - @Parameter(names = "--useTls", description = "Use tls connection\n", hidden = true, arity = 1) + @Option(names = "--useTls", description = "Use tls connection\n", hidden = true, arity = "1") protected boolean useTls; - @Parameter(names = "--tlsAllowInsecureConnection", description = "Allow insecure tls connection\n", - hidden = true, arity = 1) + @Option(names = "--tlsAllowInsecureConnection", description = "Allow insecure tls connection\n", + hidden = true, arity = "1") protected boolean tlsAllowInsecureConnection; - @Parameter(names = "--tlsHostNameVerificationEnabled", description = "Enable hostname verification", hidden = true - , arity = 1) + @Option(names = "--tlsHostNameVerificationEnabled", description = "Enable hostname verification", hidden = true + , arity = "1") protected boolean tlsHostNameVerificationEnabled; - @Parameter(names = "--tlsTrustCertFilePath", description = "tls trust cert file path", hidden = true) + @Option(names = "--tlsTrustCertFilePath", description = "tls trust cert file path", hidden = true) protected String tlsTrustCertFilePath; - @Parameter(names = "--instanceIdOffset", description = "Start the instanceIds from this offset", hidden = true) + @Option(names = "--instanceIdOffset", description = "Start the instanceIds from this offset", hidden = true) protected int instanceIdOffset = 0; - @Parameter(names = "--runtime", description = "Function runtime to use (Thread/Process)", hidden = true, + @Option(names = "--runtime", description = "Function runtime to use (Thread/Process)", hidden = true, converter = RuntimeConverter.class) protected RuntimeEnv runtimeEnv; - @Parameter(names = "--secretsProviderClassName", + @Option(names = "--secretsProviderClassName", description = "Whats the classname of secrets provider", hidden = true) protected String secretsProviderClassName; - @Parameter(names = "--secretsProviderConfig", + @Option(names = "--secretsProviderConfig", description = "Whats the config for the secrets provider", hidden = true) protected String secretsProviderConfig; - @Parameter(names = "--metricsPortStart", description = "The starting port range for metrics server. When running " + @Option(names = "--metricsPortStart", description = "The starting port range for metrics server. When running " + "instances as threads, one metrics server is used to host the stats for all instances.", hidden = true) protected Integer metricsPortStart; - @Parameter(names = "--exitOnError", description = "The starting port range for metrics server. When running " + @Option(names = "--exitOnError", description = "The starting port range for metrics server. When running " + "instances as threads, one metrics server is used to host the stats for all instances.", hidden = true) protected boolean exitOnError; @@ -212,11 +213,10 @@ public RuntimeEnv convert(String value) { public static void main(String[] args) throws Exception { LocalRunner localRunner = LocalRunner.builder().build(); - JCommander jcommander = new JCommander(localRunner); - jcommander.setProgramName("LocalRunner"); + CommandLine jcommander = new CommandLine(localRunner); + jcommander.setCommandName("LocalRunner"); - // parse args by JCommander - jcommander.parse(args); + jcommander.parseArgs(args); try { localRunner.start(true); } catch (Exception e) { diff --git a/pulsar-functions/runtime/pom.xml b/pulsar-functions/runtime/pom.xml index 4c14a4302f188..ec35c3169e814 100644 --- a/pulsar-functions/runtime/pom.xml +++ b/pulsar-functions/runtime/pom.xml @@ -46,8 +46,8 @@ - com.beust - jcommander + info.picocli + picocli diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java index e23838cb34396..06cfca6c41a2a 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java @@ -20,9 +20,6 @@ import static org.apache.pulsar.functions.utils.FunctionCommon.getSinkType; import static org.apache.pulsar.functions.utils.FunctionCommon.getSourceType; -import com.beust.jcommander.JCommander; -import com.beust.jcommander.Parameter; -import com.beust.jcommander.converters.StringConverter; import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; import com.google.protobuf.Empty; @@ -59,104 +56,104 @@ import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.functioncache.FunctionCacheManager; import org.apache.pulsar.functions.utils.functioncache.FunctionCacheManagerImpl; +import picocli.CommandLine; +import picocli.CommandLine.Option; @Slf4j public class JavaInstanceStarter implements AutoCloseable { - @Parameter(names = "--function_details", description = "Function details json\n", required = true) + @Option(names = "--function_details", description = "Function details json\n", required = true) public String functionDetailsJsonString; - @Parameter( + @Option( names = "--jar", - description = "Path to Jar\n", - listConverter = StringConverter.class) + description = "Path to Jar\n") public String jarFile; - @Parameter( + @Option( names = "--transform_function_jar", - description = "Path to Transform Function Jar\n", - listConverter = StringConverter.class) + description = "Path to Transform Function Jar\n") public String transformFunctionJarFile; - @Parameter(names = "--instance_id", description = "Instance Id\n", required = true) + @Option(names = "--instance_id", description = "Instance Id\n", required = true) public int instanceId; - @Parameter(names = "--function_id", description = "Function Id\n", required = true) + @Option(names = "--function_id", description = "Function Id\n", required = true) public String functionId; - @Parameter(names = "--function_version", description = "Function Version\n", required = true) + @Option(names = "--function_version", description = "Function Version\n", required = true) public String functionVersion; - @Parameter(names = "--pulsar_serviceurl", description = "Pulsar Service Url\n", required = true) + @Option(names = "--pulsar_serviceurl", description = "Pulsar Service Url\n", required = true) public String pulsarServiceUrl; - @Parameter(names = "--transform_function_id", description = "Transform Function Id\n") + @Option(names = "--transform_function_id", description = "Transform Function Id\n") public String transformFunctionId; - @Parameter(names = "--client_auth_plugin", description = "Client auth plugin name\n") + @Option(names = "--client_auth_plugin", description = "Client auth plugin name\n") public String clientAuthenticationPlugin; - @Parameter(names = "--client_auth_params", description = "Client auth param\n") + @Option(names = "--client_auth_params", description = "Client auth param\n") public String clientAuthenticationParameters; - @Parameter(names = "--use_tls", description = "Use tls connection\n") + @Option(names = "--use_tls", description = "Use tls connection\n") public String useTls = Boolean.FALSE.toString(); - @Parameter(names = "--tls_allow_insecure", description = "Allow insecure tls connection\n") + @Option(names = "--tls_allow_insecure", description = "Allow insecure tls connection\n") public String tlsAllowInsecureConnection = Boolean.FALSE.toString(); - @Parameter(names = "--hostname_verification_enabled", description = "Enable hostname verification") + @Option(names = "--hostname_verification_enabled", description = "Enable hostname verification") public String tlsHostNameVerificationEnabled = Boolean.FALSE.toString(); - @Parameter(names = "--tls_trust_cert_path", description = "tls trust cert file path") + @Option(names = "--tls_trust_cert_path", description = "tls trust cert file path") public String tlsTrustCertFilePath; - @Parameter(names = "--state_storage_impl_class", description = "State Storage Service " + @Option(names = "--state_storage_impl_class", description = "State Storage Service " + "Implementation class\n", required = false) public String stateStorageImplClass; - @Parameter(names = "--state_storage_serviceurl", description = "State Storage Service Url\n", required = false) + @Option(names = "--state_storage_serviceurl", description = "State Storage Service Url\n", required = false) public String stateStorageServiceUrl; - @Parameter(names = "--port", description = "Port to listen on\n", required = true) + @Option(names = "--port", description = "Port to listen on\n", required = true) public int port; - @Parameter(names = "--metrics_port", description = "Port metrics will be exposed on\n", required = true) + @Option(names = "--metrics_port", description = "Port metrics will be exposed on\n", required = true) public int metricsPort; - @Parameter(names = "--max_buffered_tuples", description = "Maximum number of tuples to buffer\n", required = true) + @Option(names = "--max_buffered_tuples", description = "Maximum number of tuples to buffer\n", required = true) public int maxBufferedTuples; - @Parameter(names = "--expected_healthcheck_interval", description = "Expected interval in " + @Option(names = "--expected_healthcheck_interval", description = "Expected interval in " + "seconds between healtchecks", required = true) public int expectedHealthCheckInterval; - @Parameter(names = "--secrets_provider", description = "The classname of the secrets provider", required = false) + @Option(names = "--secrets_provider", description = "The classname of the secrets provider", required = false) public String secretsProviderClassName; - @Parameter(names = "--secrets_provider_config", description = "The config that needs to be " + @Option(names = "--secrets_provider_config", description = "The config that needs to be " + "passed to secrets provider", required = false) public String secretsProviderConfig; - @Parameter(names = "--cluster_name", description = "The name of the cluster this " + @Option(names = "--cluster_name", description = "The name of the cluster this " + "instance is running on", required = true) public String clusterName; - @Parameter(names = "--nar_extraction_directory", description = "The directory where " + @Option(names = "--nar_extraction_directory", description = "The directory where " + "extraction of nar packages happen", required = false) public String narExtractionDirectory = NarClassLoader.DEFAULT_NAR_EXTRACTION_DIR; - @Parameter(names = "--pending_async_requests", description = "Max pending async requests per instance", + @Option(names = "--pending_async_requests", description = "Max pending async requests per instance", required = false) public int maxPendingAsyncRequests = 1000; - @Parameter(names = "--web_serviceurl", description = "Pulsar Web Service Url", required = false) + @Option(names = "--web_serviceurl", description = "Pulsar Web Service Url", required = false) public String webServiceUrl = null; - @Parameter(names = "--expose_pulsaradmin", description = "Whether the pulsar admin client " + @Option(names = "--expose_pulsaradmin", description = "Whether the pulsar admin client " + "exposed to function context, default is disabled.", required = false) public Boolean exposePulsarAdminClientEnabled = false; - @Parameter(names = "--ignore_unknown_config_fields", + @Option(names = "--ignore_unknown_config_fields", description = "Whether to ignore unknown properties when deserializing the connector configuration.", required = false) public Boolean ignoreUnknownConfigFields = false; @@ -176,9 +173,8 @@ public void start(String[] args, ClassLoader functionInstanceClassLoader, ClassL throws Exception { Thread.currentThread().setContextClassLoader(functionInstanceClassLoader); - JCommander jcommander = new JCommander(this); - // parse args by JCommander - jcommander.parse(args); + CommandLine jcommander = new CommandLine(this); + jcommander.parseArgs(args); InstanceConfig instanceConfig = new InstanceConfig(); instanceConfig.setFunctionId(functionId); diff --git a/pulsar-io/docs/src/main/java/org/apache/pulsar/io/docs/ConnectorDocGenerator.java b/pulsar-io/docs/src/main/java/org/apache/pulsar/io/docs/ConnectorDocGenerator.java index fec7b12087977..2e9d6a9f27acc 100644 --- a/pulsar-io/docs/src/main/java/org/apache/pulsar/io/docs/ConnectorDocGenerator.java +++ b/pulsar-io/docs/src/main/java/org/apache/pulsar/io/docs/ConnectorDocGenerator.java @@ -18,8 +18,6 @@ */ package org.apache.pulsar.io.docs; -import com.beust.jcommander.JCommander; -import com.beust.jcommander.Parameter; import com.google.common.base.Strings; import java.io.File; import java.io.FileOutputStream; @@ -34,14 +32,19 @@ import java.util.ArrayList; import java.util.List; import java.util.Set; +import java.util.concurrent.Callable; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.io.core.annotations.Connector; import org.apache.pulsar.io.core.annotations.FieldDoc; import org.reflections.Reflections; import org.reflections.util.ConfigurationBuilder; +import picocli.CommandLine; +import picocli.CommandLine.Command; +import picocli.CommandLine.Option; @Slf4j -public class ConnectorDocGenerator { +@Command(name = "connector-doc-gen") +public class ConnectorDocGenerator implements Callable { private static final String INDENT = " "; @@ -118,41 +121,25 @@ private void generatorConnectorYamlFiles(String outputDir) throws IOException { } } - /** - * Args for stats generator. - */ - private static class MainArgs { - @Parameter( - names = {"-o", "--output-dir"}, - description = "The output dir to dump connector docs", - required = true) - String outputDir = null; - - @Parameter(names = {"-h", "--help"}, description = "Show this help message") - boolean help = false; - } + @Option( + names = {"-o", "--output-dir"}, + description = "The output dir to dump connector docs", + required = true) + String outputDir = null; - public static void main(String[] args) throws Exception { - MainArgs mainArgs = new MainArgs(); - - JCommander commander = new JCommander(); - try { - commander.setProgramName("connector-doc-gen"); - commander.addObject(mainArgs); - commander.parse(args); - if (mainArgs.help) { - commander.usage(); - Runtime.getRuntime().exit(0); - return; - } - } catch (Exception e) { - commander.usage(); - Runtime.getRuntime().exit(1); - return; - } + @Option(names = {"-h", "--help"}, usageHelp = true, description = "Show this help message") + boolean help = false; + @Override + public Integer call() throws Exception { ConnectorDocGenerator docGen = new ConnectorDocGenerator(); - docGen.generatorConnectorYamlFiles(mainArgs.outputDir); + docGen.generatorConnectorYamlFiles(outputDir); + return 0; + } + + public static void main(String[] args) throws Exception { + CommandLine commander = new CommandLine(new ConnectorDocGenerator()); + Runtime.getRuntime().exit(commander.execute(args)); } } From a52945b1c51fa874667eecb9fea9bf03e5d6153b Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 26 Mar 2024 07:41:07 +0800 Subject: [PATCH 004/580] [fix] [broker] fix mismatch between dispatcher.consumerList and dispatcher.consumerSet (#22283) --- .../pulsar/broker/service/ServerCnx.java | 16 ++- ...PersistentDispatcherMultipleConsumers.java | 8 +- .../pulsar/broker/service/ServerCnxTest.java | 5 +- ...ProducerConsumerMLInitializeDelayTest.java | 108 ++++++++++++++++++ 4 files changed, 131 insertions(+), 6 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java 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 3ab25eb098cdf..4f82f416ed2a5 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 @@ -1224,10 +1224,20 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { commandSender.sendErrorResponse(requestId, ServerError.ServiceNotReady, "Consumer is already present on the connection"); } else if (existingConsumerFuture.isCompletedExceptionally()){ + log.warn("[{}][{}][{}] A failed consumer with id is already present on the connection," + + " consumerId={}", remoteAddress, topicName, subscriptionName, consumerId); ServerError error = getErrorCodeWithErrorLog(existingConsumerFuture, true, - String.format("Consumer subscribe failure. remoteAddress: %s, subscription: %s", - remoteAddress, subscriptionName)); - consumers.remove(consumerId, existingConsumerFuture); + String.format("A failed consumer with id is already present on the connection." + + " consumerId: %s, remoteAddress: %s, subscription: %s", + consumerId, remoteAddress, subscriptionName)); + /** + * This future may was failed due to the client closed a in-progress subscribing. + * See {@link #handleCloseConsumer(CommandCloseConsumer)} + * Do not remove the failed future at current line, it will be removed after the progress of + * the previous subscribing is done. + * Before the previous subscribing is done, the new subscribe request will always fail. + * This mechanism is in order to prevent more complex logic to handle the race conditions. + */ commandSender.sendErrorResponse(requestId, error, "Consumer that failed is already present on the connection"); } else { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 35204e7af72bf..039104fe0221a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -190,9 +190,15 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { } if (isConsumersExceededOnSubscription()) { - log.warn("[{}] Attempting to add consumer to subscription which reached max consumers limit", name); + log.warn("[{}] Attempting to add consumer to subscription which reached max consumers limit {}", + name, consumer); return FutureUtil.failedFuture(new ConsumerBusyException("Subscription reached max consumers limit")); } + // This is not an expected scenario, it will never happen in expected. Just print a warn log if the unexpected + // scenario happens. See more detail: https://github.com/apache/pulsar/pull/22283. + if (consumerSet.contains(consumer)) { + log.warn("[{}] Attempting to add a consumer that already registered {}", name, consumer); + } consumerList.add(consumer); if (consumerList.size() > 1 diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index e195f220f87dd..1cb2f76c5e2b2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -3382,8 +3382,9 @@ public boolean isCompletedExceptionally() { }; // assert error response assertTrue(responseAssert.test(responseAssert)); - // assert consumer-delete event occur - assertEquals(1L, + // The delete event will only occur after the future is completed. + // assert consumer-delete event will not occur. + assertEquals(0L, deleteTimesMark.getAllValues().stream().filter(f -> f == existingConsumerFuture).count()); // Server will not close the connection assertTrue(channel.isOpen()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java new file mode 100644 index 0000000000000..ab4e063ae3d83 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java @@ -0,0 +1,108 @@ +/* + * 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 com.carrotsearch.hppc.ObjectSet; +import java.time.Duration; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.common.naming.TopicName; +import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class SimpleProducerConsumerMLInitializeDelayTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setTopicLoadTimeoutSeconds(60 * 5); + } + + @Test(timeOut = 30 * 1000) + public void testConsumerListMatchesConsumerSet() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subName = "sub"; + final int clientOperationTimeout = 3; + final int loadMLDelayMillis = clientOperationTimeout * 3 * 1000; + final int clientMaxBackoffSeconds = clientOperationTimeout * 2; + admin.topics().createNonPartitionedTopic(topicName); + // Create a client with a low operation timeout. + PulsarClient client = PulsarClient.builder() + .serviceUrl(lookupUrl.toString()) + .operationTimeout(clientOperationTimeout, TimeUnit.SECONDS) + .maxBackoffInterval(clientMaxBackoffSeconds, TimeUnit.SECONDS) + .build(); + Consumer consumer = client.newConsumer() + .topic(topicName) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + // Inject a delay for the initialization of ML, to make the consumer to register twice. + // Consumer register twice: the first will be timeout, and try again. + AtomicInteger delayTimes = new AtomicInteger(); + mockZooKeeper.delay(loadMLDelayMillis, (op, s) -> { + if (op.toString().equals("GET") && s.contains(TopicName.get(topicName).getPersistenceNamingEncoding())) { + return delayTimes.incrementAndGet() == 1; + } + return false; + }); + admin.topics().unload(topicName); + // Verify: at last, "dispatcher.consumers.size" equals "dispatcher.consumerList.size". + Awaitility.await().atMost(Duration.ofSeconds(loadMLDelayMillis * 3)) + .ignoreExceptions().untilAsserted(() -> { + Dispatcher dispatcher = pulsar.getBrokerService() + .getTopic(topicName, false).join().get() + .getSubscription(subName).getDispatcher(); + ObjectSet consumerSet = WhiteboxImpl.getInternalState(dispatcher, "consumerSet"); + List consumerList = WhiteboxImpl.getInternalState(dispatcher, "consumerList"); + log.info("consumerSet_size: {}, consumerList_size: {}", consumerSet.size(), consumerList.size()); + Assert.assertEquals(consumerList.size(), 1); + Assert.assertEquals(consumerSet.size(), 1); + }); + + // Verify: the topic can be deleted. + consumer.close(); + admin.topics().delete(topicName); + // cleanup. + client.close(); + } +} From 0b2b6d593bb6ee9094d4c0a7a311490e7362f68f Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 26 Mar 2024 14:37:10 +0800 Subject: [PATCH 005/580] [fix][broker] Fix ResourceGroup report local usage (#22340) Signed-off-by: Zixuan Liu --- .../broker/resourcegroup/ResourceGroup.java | 38 ++++--- .../resourcegroup/ResourceGroupService.java | 4 +- .../ResourceGroupReportLocalUsageTest.java | 105 ++++++++++++++++++ 3 files changed, 130 insertions(+), 17 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupReportLocalUsageTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java index effb6568a5378..f8ec52bfe3c5a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.resourcegroup; +import com.google.common.annotations.VisibleForTesting; import io.prometheus.client.Counter; import java.util.HashMap; import java.util.Set; @@ -218,24 +219,28 @@ public void rgFillResourceUsage(ResourceUsage resourceUsage) { resourceUsage.setOwner(this.getID()); p = resourceUsage.setPublish(); - this.setUsageInMonitoredEntity(ResourceGroupMonitoringClass.Publish, p); + if (!this.setUsageInMonitoredEntity(ResourceGroupMonitoringClass.Publish, p)) { + resourceUsage.clearPublish(); + } p = resourceUsage.setDispatch(); - this.setUsageInMonitoredEntity(ResourceGroupMonitoringClass.Dispatch, p); + if (!this.setUsageInMonitoredEntity(ResourceGroupMonitoringClass.Dispatch, p)) { + resourceUsage.clearDispatch(); + } // Punt storage for now. } // Transport manager mandated op. public void rgResourceUsageListener(String broker, ResourceUsage resourceUsage) { - NetworkUsage p; - - p = resourceUsage.getPublish(); - this.getUsageFromMonitoredEntity(ResourceGroupMonitoringClass.Publish, p, broker); - - p = resourceUsage.getDispatch(); - this.getUsageFromMonitoredEntity(ResourceGroupMonitoringClass.Dispatch, p, broker); + if (resourceUsage.hasPublish()) { + this.getUsageFromMonitoredEntity(ResourceGroupMonitoringClass.Publish, resourceUsage.getPublish(), broker); + } + if (resourceUsage.hasDispatch()) { + this.getUsageFromMonitoredEntity(ResourceGroupMonitoringClass.Dispatch, resourceUsage.getDispatch(), + broker); + } // Punt storage for now. } @@ -453,12 +458,6 @@ protected boolean setUsageInMonitoredEntity(ResourceGroupMonitoringClass monClas bytesUsed = monEntity.usedLocallySinceLastReport.bytes; messagesUsed = monEntity.usedLocallySinceLastReport.messages; - monEntity.usedLocallySinceLastReport.bytes = monEntity.usedLocallySinceLastReport.messages = 0; - - monEntity.totalUsedLocally.bytes += bytesUsed; - monEntity.totalUsedLocally.messages += messagesUsed; - - monEntity.lastResourceUsageFillTimeMSecsSinceEpoch = System.currentTimeMillis(); if (sendReport) { p.setBytesPerPeriod(bytesUsed); @@ -466,6 +465,10 @@ protected boolean setUsageInMonitoredEntity(ResourceGroupMonitoringClass monClas monEntity.lastReportedValues.bytes = bytesUsed; monEntity.lastReportedValues.messages = messagesUsed; monEntity.numSuppressedUsageReports = 0; + monEntity.usedLocallySinceLastReport.bytes = monEntity.usedLocallySinceLastReport.messages = 0; + monEntity.totalUsedLocally.bytes += bytesUsed; + monEntity.totalUsedLocally.messages += messagesUsed; + monEntity.lastResourceUsageFillTimeMSecsSinceEpoch = System.currentTimeMillis(); } else { numSuppressions = monEntity.numSuppressedUsageReports++; } @@ -598,6 +601,11 @@ public void acceptResourceUsage(String broker, ResourceUsage resourceUsage) { }; } + @VisibleForTesting + PerMonitoringClassFields getMonitoredEntity(ResourceGroupMonitoringClass monClass) { + return this.monitoringClassFields[monClass.ordinal()]; + } + public final String resourceGroupName; public PerMonitoringClassFields[] monitoringClassFields = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupService.java index e228c35cc11a4..29633ab19feff 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupService.java @@ -686,7 +686,7 @@ protected void calculateQuotaForAllResourceGroups() { timeUnitScale); this.resourceUsagePublishPeriodInSeconds = newPeriodInSeconds; maxIntervalForSuppressingReportsMSecs = - this.resourceUsagePublishPeriodInSeconds * MaxUsageReportSuppressRounds; + TimeUnit.SECONDS.toMillis(this.resourceUsagePublishPeriodInSeconds) * MaxUsageReportSuppressRounds; } } @@ -705,7 +705,7 @@ private void initialize() { periodInSecs, this.timeUnitScale); maxIntervalForSuppressingReportsMSecs = - this.resourceUsagePublishPeriodInSeconds * MaxUsageReportSuppressRounds; + TimeUnit.SECONDS.toMillis(this.resourceUsagePublishPeriodInSeconds) * MaxUsageReportSuppressRounds; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupReportLocalUsageTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupReportLocalUsageTest.java new file mode 100644 index 0000000000000..658b7c94165d9 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupReportLocalUsageTest.java @@ -0,0 +1,105 @@ +/* + * 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.resourcegroup; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.resourcegroup.ResourceGroup.BytesAndMessagesCount; +import org.apache.pulsar.broker.resourcegroup.ResourceGroup.PerMonitoringClassFields; +import org.apache.pulsar.broker.resourcegroup.ResourceGroup.ResourceGroupMonitoringClass; +import org.apache.pulsar.broker.service.resource.usage.ResourceUsage; +import org.apache.pulsar.common.policies.data.ResourceGroup; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class ResourceGroupReportLocalUsageTest extends MockedPulsarServiceBaseTest { + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + + @Test + public void testRgFillResourceUsage() throws Exception { + pulsar.getResourceGroupServiceManager().close(); + AtomicBoolean needReport = new AtomicBoolean(false); + ResourceGroupService service = new ResourceGroupService(pulsar, TimeUnit.HOURS, null, + new ResourceQuotaCalculator() { + @Override + public boolean needToReportLocalUsage(long currentBytesUsed, long lastReportedBytes, + long currentMessagesUsed, long lastReportedMessages, + long lastReportTimeMSecsSinceEpoch) { + return needReport.get(); + } + + @Override + public long computeLocalQuota(long confUsage, long myUsage, long[] allUsages) { + return 0; + } + }); + String rgName = "rg-1"; + ResourceGroup rgConfig = new ResourceGroup(); + rgConfig.setPublishRateInBytes(1000L); + rgConfig.setPublishRateInMsgs(2000); + service.resourceGroupCreate(rgName, rgConfig); + + org.apache.pulsar.broker.resourcegroup.ResourceGroup resourceGroup = service.resourceGroupGet(rgName); + BytesAndMessagesCount bytesAndMessagesCount = new BytesAndMessagesCount(); + bytesAndMessagesCount.bytes = 20; + bytesAndMessagesCount.messages = 10; + resourceGroup.incrementLocalUsageStats(ResourceGroupMonitoringClass.Publish, bytesAndMessagesCount); + ResourceUsage resourceUsage = new ResourceUsage(); + resourceGroup.rgFillResourceUsage(resourceUsage); + assertFalse(resourceUsage.hasDispatch()); + assertFalse(resourceUsage.hasPublish()); + + PerMonitoringClassFields publishMonitoredEntity = + resourceGroup.getMonitoredEntity(ResourceGroupMonitoringClass.Publish); + assertEquals(publishMonitoredEntity.usedLocallySinceLastReport.messages, bytesAndMessagesCount.messages); + assertEquals(publishMonitoredEntity.usedLocallySinceLastReport.bytes, bytesAndMessagesCount.bytes); + assertEquals(publishMonitoredEntity.totalUsedLocally.messages, 0); + assertEquals(publishMonitoredEntity.totalUsedLocally.bytes, 0); + assertEquals(publishMonitoredEntity.lastReportedValues.messages, 0); + assertEquals(publishMonitoredEntity.lastReportedValues.bytes, 0); + + needReport.set(true); + resourceGroup.rgFillResourceUsage(resourceUsage); + assertTrue(resourceUsage.hasDispatch()); + assertTrue(resourceUsage.hasPublish()); + assertEquals(publishMonitoredEntity.usedLocallySinceLastReport.messages, 0); + assertEquals(publishMonitoredEntity.usedLocallySinceLastReport.bytes, 0); + assertEquals(publishMonitoredEntity.totalUsedLocally.messages, bytesAndMessagesCount.messages); + assertEquals(publishMonitoredEntity.totalUsedLocally.bytes, bytesAndMessagesCount.bytes); + assertEquals(publishMonitoredEntity.lastReportedValues.messages, bytesAndMessagesCount.messages); + assertEquals(publishMonitoredEntity.lastReportedValues.bytes, bytesAndMessagesCount.bytes); + } +} \ No newline at end of file From 80b491dab0fd8a948db0a3d85a3ccb8490ecf266 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 26 Mar 2024 16:49:58 +0800 Subject: [PATCH 006/580] [fix][broker] Fix ResourceGroups loading (#21781) Signed-off-by: Zixuan Liu --- .../ResourceGroupConfigListener.java | 76 ++++++++++++++----- .../ResourceGroupConfigListenerTest.java | 48 ++++++++++++ 2 files changed, 104 insertions(+), 20 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListener.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListener.java index c15edd2be4e43..4a5b8a8bcc244 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListener.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListener.java @@ -18,15 +18,22 @@ */ package org.apache.pulsar.broker.resourcegroup; +import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; +import java.util.ArrayList; import java.util.HashSet; +import java.util.List; import java.util.Optional; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.resources.ResourceGroupResources; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.policies.data.ResourceGroup; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; import org.slf4j.Logger; @@ -47,24 +54,32 @@ public class ResourceGroupConfigListener implements Consumer { private final ResourceGroupService rgService; private final PulsarService pulsarService; private final ResourceGroupResources rgResources; - private final ResourceGroupNamespaceConfigListener rgNamespaceConfigListener; + private volatile ResourceGroupNamespaceConfigListener rgNamespaceConfigListener; public ResourceGroupConfigListener(ResourceGroupService rgService, PulsarService pulsarService) { this.rgService = rgService; this.pulsarService = pulsarService; this.rgResources = pulsarService.getPulsarResources().getResourcegroupResources(); - loadAllResourceGroups(); this.rgResources.getStore().registerListener(this); - rgNamespaceConfigListener = new ResourceGroupNamespaceConfigListener( - rgService, pulsarService, this); + execute(() -> loadAllResourceGroupsWithRetryAsync(0)); } - private void loadAllResourceGroups() { - rgResources.listResourceGroupsAsync().whenCompleteAsync((rgList, ex) -> { - if (ex != null) { - LOG.error("Exception when fetching resource groups", ex); - return; + private void loadAllResourceGroupsWithRetryAsync(long retry) { + loadAllResourceGroupsAsync().thenAccept(__ -> { + if (rgNamespaceConfigListener == null) { + rgNamespaceConfigListener = new ResourceGroupNamespaceConfigListener(rgService, pulsarService, this); } + }).exceptionally(e -> { + long nextRetry = retry + 1; + long delay = 500 * nextRetry; + LOG.error("Failed to load all resource groups during initialization, retrying after {}ms: ", delay, e); + schedule(() -> loadAllResourceGroupsWithRetryAsync(nextRetry), delay); + return null; + }); + } + + private CompletableFuture loadAllResourceGroupsAsync() { + return rgResources.listResourceGroupsAsync().thenCompose(rgList -> { final Set existingSet = rgService.resourceGroupGetAll(); HashSet newSet = new HashSet<>(); @@ -72,21 +87,26 @@ private void loadAllResourceGroups() { final Sets.SetView deleteList = Sets.difference(existingSet, newSet); - for (String rgName: deleteList) { + for (String rgName : deleteList) { deleteResourceGroup(rgName); } final Sets.SetView addList = Sets.difference(newSet, existingSet); - for (String rgName: addList) { - pulsarService.getPulsarResources().getResourcegroupResources() - .getResourceGroupAsync(rgName).thenAcceptAsync(optionalRg -> { - ResourceGroup rg = optionalRg.get(); - createResourceGroup(rgName, rg); - }).exceptionally((ex1) -> { - LOG.error("Failed to fetch resourceGroup", ex1); - return null; - }); + List> futures = new ArrayList<>(); + for (String rgName : addList) { + futures.add(pulsarService.getPulsarResources() + .getResourcegroupResources() + .getResourceGroupAsync(rgName) + .thenAccept(optionalRg -> { + if (optionalRg.isPresent()) { + ResourceGroup rg = optionalRg.get(); + createResourceGroup(rgName, rg); + } + }) + ); } + + return FutureUtil.waitForAll(futures); }); } @@ -140,7 +160,10 @@ public void accept(Notification notification) { Optional rgName = ResourceGroupResources.resourceGroupNameFromPath(notifyPath); if ((notification.getType() == NotificationType.ChildrenChanged) || (notification.getType() == NotificationType.Created)) { - loadAllResourceGroups(); + loadAllResourceGroupsAsync().exceptionally((ex) -> { + LOG.error("Exception when fetching resource groups", ex); + return null; + }); } else if (rgName.isPresent()) { switch (notification.getType()) { case Modified: @@ -151,4 +174,17 @@ public void accept(Notification notification) { } } } + + protected void execute(Runnable runnable) { + pulsarService.getExecutor().execute(catchingAndLoggingThrowables(runnable)); + } + + protected void schedule(Runnable runnable, long delayMs) { + pulsarService.getExecutor().schedule(catchingAndLoggingThrowables(runnable), delayMs, TimeUnit.MILLISECONDS); + } + + @VisibleForTesting + ResourceGroupNamespaceConfigListener getRgNamespaceConfigListener() { + return rgNamespaceConfigListener; + } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListenerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListenerTest.java index 90c26530850a3..4010635ed9952 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListenerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListenerTest.java @@ -18,20 +18,31 @@ */ package org.apache.pulsar.broker.resourcegroup; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertThrows; import com.google.common.collect.Sets; +import java.util.ArrayList; import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.resources.ResourceGroupResources; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ResourceGroup; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.metadata.api.MetadataStore; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -288,4 +299,41 @@ private void prepareData() throws PulsarAdminException { testAddRg.setDispatchRateInBytes(200L); } + + @Test + public void testNewResourceGroupNamespaceConfigListener() { + PulsarService pulsarService = mock(PulsarService.class); + PulsarResources pulsarResources = mock(PulsarResources.class); + doReturn(pulsarResources).when(pulsarService).getPulsarResources(); + ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); + doReturn(scheduledExecutorService).when(pulsarService).getExecutor(); + + ResourceGroupService resourceGroupService = mock(ResourceGroupService.class); + ResourceGroupResources resourceGroupResources = mock(ResourceGroupResources.class); + RuntimeException exception = new RuntimeException("listResourceGroupsAsync error"); + doReturn(CompletableFuture.failedFuture(exception)) + .when(resourceGroupResources).listResourceGroupsAsync(); + doReturn(mock(MetadataStore.class)) + .when(resourceGroupResources).getStore(); + doReturn(resourceGroupResources).when(pulsarResources).getResourcegroupResources(); + + ServiceConfiguration ServiceConfiguration = new ServiceConfiguration(); + doReturn(ServiceConfiguration).when(pulsarService).getConfiguration(); + + ResourceGroupConfigListener resourceGroupConfigListener = + new ResourceGroupConfigListener(resourceGroupService, pulsarService); + + // getResourcegroupResources() returns an error, ResourceGroupNamespaceConfigListener doesn't be created. + Awaitility.await().pollDelay(3, TimeUnit.SECONDS).untilAsserted(() -> { + assertNull(resourceGroupConfigListener.getRgNamespaceConfigListener()); + }); + + // ResourceGroupNamespaceConfigListener will be created, and uses real pulsar resource. + doReturn(CompletableFuture.completedFuture(new ArrayList())) + .when(resourceGroupResources).listResourceGroupsAsync(); + doReturn(pulsar.getPulsarResources()).when(pulsarService).getPulsarResources(); + Awaitility.await().untilAsserted(() -> { + assertNotNull(resourceGroupConfigListener.getRgNamespaceConfigListener()); + }); + } } From 023446b73287dea25c22c6db307e5d723306e765 Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 27 Mar 2024 11:40:28 +0800 Subject: [PATCH 007/580] [fix][cli] Fix typos in CmdSinks class (#22358) --- .../src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java | 2 +- .../test/java/org/apache/pulsar/admin/cli/TestCmdSinks.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java index 47af7e6794ca2..f3172a49b0154 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java @@ -617,7 +617,7 @@ protected Map parseConfigs(String str) throws JsonProcessingExce protected void validateSinkConfigs(SinkConfig sinkConfig) { if (isBlank(sinkConfig.getArchive())) { - throw new ParameterException("Sink archive not specfied"); + throw new ParameterException("Sink archive not specified"); } org.apache.pulsar.common.functions.Utils.inferMissingArguments(sinkConfig); diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSinks.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSinks.java index c68bbd20ab8b0..6fbe3bc5da26d 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSinks.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSinks.java @@ -283,7 +283,7 @@ public void testMissingProcessingGuarantees() throws Exception { } @Test(expectedExceptions = CliCommand.ParameterException.class, - expectedExceptionsMessageRegExp = "Sink archive not specfied") + expectedExceptionsMessageRegExp = "Sink archive not specified") public void testMissingArchive() throws Exception { SinkConfig sinkConfig = getSinkConfig(); sinkConfig.setArchive(null); @@ -503,7 +503,7 @@ public void testCmdSinkConfigFileMissingResources() throws Exception { testCmdSinkConfigFile(testSinkConfig, expectedSinkConfig); } - @Test(expectedExceptions = CliCommand.ParameterException.class, expectedExceptionsMessageRegExp = "Sink archive not specfied") + @Test(expectedExceptions = CliCommand.ParameterException.class, expectedExceptionsMessageRegExp = "Sink archive not specified") public void testCmdSinkConfigFileMissingJar() throws Exception { SinkConfig testSinkConfig = getSinkConfig(); testSinkConfig.setArchive(null); From cd49defc1383175ef32e18c7f0905567f734318c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 27 Mar 2024 14:08:39 +0800 Subject: [PATCH 008/580] [fix][ml]Expose ledger timestamp (#22338) --- .../bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java | 1 + .../bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java | 5 +++++ 2 files changed, 6 insertions(+) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 51c5c91234f21..5ce84b3ed850a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -694,6 +694,7 @@ public void operationComplete(MLDataFormats.ManagedLedgerInfo pbInfo, Stat stat) ledgerInfo.ledgerId = pbLedgerInfo.getLedgerId(); ledgerInfo.entries = pbLedgerInfo.hasEntries() ? pbLedgerInfo.getEntries() : null; ledgerInfo.size = pbLedgerInfo.hasSize() ? pbLedgerInfo.getSize() : null; + ledgerInfo.timestamp = pbLedgerInfo.hasTimestamp() ? pbLedgerInfo.getTimestamp() : null; ledgerInfo.isOffloaded = pbLedgerInfo.hasOffloadContext(); if (pbLedgerInfo.hasOffloadContext()) { MLDataFormats.OffloadContext offloadContext = pbLedgerInfo.getOffloadContext(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java index 708fda308b8c5..a953b140aba63 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java @@ -19,6 +19,7 @@ package org.apache.bookkeeper.mledger.impl; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -60,6 +61,10 @@ public void testGetManagedLedgerInfoWithClose() throws Exception { assertEquals(info.ledgers.get(2).ledgerId, 5); assertEquals(info.ledgers.get(3).ledgerId, 6); + for (ManagedLedgerInfo.LedgerInfo linfo : info.ledgers) { + assertNotNull(linfo.timestamp); + } + assertEquals(info.cursors.size(), 1); CursorInfo cursorInfo = info.cursors.get("c1"); From d23a8f64acbfb4179b9f2f64e1e9dd0756742a5b Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 27 Mar 2024 16:36:36 +0800 Subject: [PATCH 009/580] [cleanup][cli] Cleanup jcommander (#22337) Signed-off-by: Zixuan Liu --- .../shell/src/assemble/LICENSE.bin.txt | 1 - pom.xml | 7 -- pulsar-cli-utils/pom.xml | 5 -- .../TimeUnitToSecondsConverter.java | 42 ------------ .../IntegerMaxValueLongValidator.java | 30 --------- .../validators/MinNegativeOneValidator.java | 30 --------- .../validators/NonNegativeValueValidator.java | 30 --------- .../PositiveIntegerValueValidator.java | 31 --------- .../PositiveLongValueValidator.java | 31 --------- .../pulsar/cli/validators/package-info.java | 19 ------ .../cli/converters/TimeConversionTest.java | 5 +- .../cli/validators/CliUtilValidatorsTest.java | 64 ------------------- .../utils/NameValueParameterSplitterTest.java | 52 --------------- .../pulsar/admin/cli/utils/CmdUtils.java | 11 ++-- .../cli/utils/NameValueParameterSplitter.java | 61 ------------------ .../apache/pulsar/client/cli/CmdProduce.java | 9 ++- .../org/apache/pulsar/client/cli/CmdRead.java | 3 +- .../pulsar/admin/cli/TestCmdSources.java | 3 +- 18 files changed, 14 insertions(+), 420 deletions(-) delete mode 100644 pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/converters/TimeUnitToSecondsConverter.java delete mode 100644 pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/IntegerMaxValueLongValidator.java delete mode 100644 pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/MinNegativeOneValidator.java delete mode 100644 pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/NonNegativeValueValidator.java delete mode 100644 pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/PositiveIntegerValueValidator.java delete mode 100644 pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/PositiveLongValueValidator.java delete mode 100644 pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/package-info.java delete mode 100644 pulsar-cli-utils/src/test/java/org/apache/pulsar/cli/validators/CliUtilValidatorsTest.java delete mode 100644 pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/utils/NameValueParameterSplitterTest.java delete mode 100644 pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/NameValueParameterSplitter.java diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 2b2f1c26be112..e735bd454eed2 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -309,7 +309,6 @@ pulsar-client-cpp/lib/checksum/crc32c_sw.cc This projects includes binary packages with the following licenses: The Apache Software License, Version 2.0 - * JCommander -- jcommander-1.82.jar * Picocli - picocli-4.7.5.jar - picocli-shell-jline3-4.7.5.jar diff --git a/pom.xml b/pom.xml index caa2fc49b2781..da7f2c7642102 100644 --- a/pom.xml +++ b/pom.xml @@ -213,7 +213,6 @@ flexible messaging model and an intuitive client API. 6.2.8 0.20 2.12.1 - 1.82 3.11 1.10 2.8.0 @@ -693,12 +692,6 @@ flexible messaging model and an intuitive client API. linux-aarch_64 - - com.beust - jcommander - ${jcommander.version} - - info.picocli picocli diff --git a/pulsar-cli-utils/pom.xml b/pulsar-cli-utils/pom.xml index ac442b4004e8b..1638029f4c8ba 100644 --- a/pulsar-cli-utils/pom.xml +++ b/pulsar-cli-utils/pom.xml @@ -35,11 +35,6 @@ Isolated CLI utility module - - com.beust - jcommander - compile - info.picocli picocli diff --git a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/converters/TimeUnitToSecondsConverter.java b/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/converters/TimeUnitToSecondsConverter.java deleted file mode 100644 index 3aca2e95d2526..0000000000000 --- a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/converters/TimeUnitToSecondsConverter.java +++ /dev/null @@ -1,42 +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.cli.converters; - -import static org.apache.pulsar.cli.ValueValidationUtil.emptyCheck; -import com.beust.jcommander.ParameterException; -import com.beust.jcommander.converters.BaseConverter; -import java.util.concurrent.TimeUnit; - -public class TimeUnitToSecondsConverter extends BaseConverter { - - public TimeUnitToSecondsConverter(String optionName) { - super(optionName); - } - - @Override - public Long convert(String str) { - emptyCheck(getOptionName(), str); - try { - return TimeUnit.SECONDS.toSeconds( - RelativeTimeUtil.parseRelativeTimeInSeconds(str.trim())); - } catch (IllegalArgumentException exception) { - throw new ParameterException("For input " + getOptionName() + ": " + exception.getMessage()); - } - } -} diff --git a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/IntegerMaxValueLongValidator.java b/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/IntegerMaxValueLongValidator.java deleted file mode 100644 index 63115b1418793..0000000000000 --- a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/IntegerMaxValueLongValidator.java +++ /dev/null @@ -1,30 +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.cli.validators; - -import com.beust.jcommander.IValueValidator; -import com.beust.jcommander.ParameterException; -import org.apache.pulsar.cli.ValueValidationUtil; - -public class IntegerMaxValueLongValidator implements IValueValidator { - @Override - public void validate(String name, Long value) throws ParameterException { - ValueValidationUtil.maxValueCheck(name, value, Integer.MAX_VALUE); - } -} diff --git a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/MinNegativeOneValidator.java b/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/MinNegativeOneValidator.java deleted file mode 100644 index 320e36812bfc2..0000000000000 --- a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/MinNegativeOneValidator.java +++ /dev/null @@ -1,30 +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.cli.validators; - -import com.beust.jcommander.IValueValidator; -import com.beust.jcommander.ParameterException; -import org.apache.pulsar.cli.ValueValidationUtil; - -public class MinNegativeOneValidator implements IValueValidator { - @Override - public void validate(String name, Long value) throws ParameterException { - ValueValidationUtil.minValueCheck(name, value, -1L); - } -} diff --git a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/NonNegativeValueValidator.java b/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/NonNegativeValueValidator.java deleted file mode 100644 index 473961be06d83..0000000000000 --- a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/NonNegativeValueValidator.java +++ /dev/null @@ -1,30 +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.cli.validators; - -import com.beust.jcommander.IValueValidator; -import com.beust.jcommander.ParameterException; -import org.apache.pulsar.cli.ValueValidationUtil; - -public class NonNegativeValueValidator implements IValueValidator { - @Override - public void validate(String name, Long value) throws ParameterException { - ValueValidationUtil.minValueCheck(name, value, 0L); - } -} diff --git a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/PositiveIntegerValueValidator.java b/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/PositiveIntegerValueValidator.java deleted file mode 100644 index c6b4cc43d6825..0000000000000 --- a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/PositiveIntegerValueValidator.java +++ /dev/null @@ -1,31 +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.cli.validators; - -import com.beust.jcommander.IValueValidator; -import com.beust.jcommander.ParameterException; -import org.apache.pulsar.cli.ValueValidationUtil; - -public class PositiveIntegerValueValidator implements IValueValidator { - - @Override - public void validate(String name, Integer value) throws ParameterException { - ValueValidationUtil.positiveCheck(name, value); - } -} diff --git a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/PositiveLongValueValidator.java b/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/PositiveLongValueValidator.java deleted file mode 100644 index 849a55241c665..0000000000000 --- a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/PositiveLongValueValidator.java +++ /dev/null @@ -1,31 +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.cli.validators; - -import com.beust.jcommander.IValueValidator; -import com.beust.jcommander.ParameterException; -import org.apache.pulsar.cli.ValueValidationUtil; - -public class PositiveLongValueValidator implements IValueValidator { - - @Override - public void validate(String name, Long value) throws ParameterException { - ValueValidationUtil.positiveCheck(name, value); - } -} diff --git a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/package-info.java b/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/package-info.java deleted file mode 100644 index 4d132b984c244..0000000000000 --- a/pulsar-cli-utils/src/main/java/org/apache/pulsar/cli/validators/package-info.java +++ /dev/null @@ -1,19 +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.cli.validators; diff --git a/pulsar-cli-utils/src/test/java/org/apache/pulsar/cli/converters/TimeConversionTest.java b/pulsar-cli-utils/src/test/java/org/apache/pulsar/cli/converters/TimeConversionTest.java index cc50eed4d03e4..451a215bce313 100644 --- a/pulsar-cli-utils/src/test/java/org/apache/pulsar/cli/converters/TimeConversionTest.java +++ b/pulsar-cli-utils/src/test/java/org/apache/pulsar/cli/converters/TimeConversionTest.java @@ -22,6 +22,7 @@ import static org.testng.Assert.assertThrows; import java.util.concurrent.TimeUnit; import org.apache.pulsar.cli.converters.picocli.TimeUnitToMillisConverter; +import org.apache.pulsar.cli.converters.picocli.TimeUnitToSecondsConverter; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -57,8 +58,8 @@ public void testSuccessfulRelativeTimeUtilParsing(String input, long expected) { } @Test(dataProvider = "successfulRelativeTimeUtilTestCases") - public void testSuccessfulTimeUnitToSecondsConverter(String input, long expected) { - TimeUnitToSecondsConverter secondsConverter = new TimeUnitToSecondsConverter("optionName"); + public void testSuccessfulTimeUnitToSecondsConverter(String input, long expected) throws Exception { + TimeUnitToSecondsConverter secondsConverter = new TimeUnitToSecondsConverter(); assertEquals(secondsConverter.convert(input), Long.valueOf(expected)); } diff --git a/pulsar-cli-utils/src/test/java/org/apache/pulsar/cli/validators/CliUtilValidatorsTest.java b/pulsar-cli-utils/src/test/java/org/apache/pulsar/cli/validators/CliUtilValidatorsTest.java deleted file mode 100644 index ba7de23373892..0000000000000 --- a/pulsar-cli-utils/src/test/java/org/apache/pulsar/cli/validators/CliUtilValidatorsTest.java +++ /dev/null @@ -1,64 +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.cli.validators; - -import static org.testng.Assert.assertThrows; -import org.testng.annotations.Test; - -public class CliUtilValidatorsTest { - - @Test - public void testPositiveLongValueValidator() { - PositiveLongValueValidator validator = new PositiveLongValueValidator(); - assertThrows(IllegalArgumentException.class, () -> validator.validate("param", -1L)); - assertThrows(IllegalArgumentException.class, () -> validator.validate("param", 0L)); - validator.validate("param", 1L); - } - - @Test - public void testPositiveIntegerValueValidator() { - PositiveIntegerValueValidator validator = new PositiveIntegerValueValidator(); - assertThrows(IllegalArgumentException.class, () -> validator.validate("param", -1)); - assertThrows(IllegalArgumentException.class, () -> validator.validate("param", 0)); - validator.validate("param", 1); - } - - @Test - public void testNonNegativeValueValidator() { - NonNegativeValueValidator validator = new NonNegativeValueValidator(); - assertThrows(IllegalArgumentException.class, () -> validator.validate("param", -1L)); - validator.validate("param", 0L); - validator.validate("param", 1L); - } - - @Test - public void testMinNegativeOneValidator() { - MinNegativeOneValidator validator = new MinNegativeOneValidator(); - assertThrows(IllegalArgumentException.class, () -> validator.validate("param", -2L)); - validator.validate("param", -1L); - validator.validate("param", 0L); - } - - @Test - public void testIntegerMaxValueLongValidator() { - IntegerMaxValueLongValidator validator = new IntegerMaxValueLongValidator(); - assertThrows(IllegalArgumentException.class, () -> validator.validate("param", Integer.MAX_VALUE + 1L)); - validator.validate("param", (long) Integer.MAX_VALUE); - } -} diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/utils/NameValueParameterSplitterTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/utils/NameValueParameterSplitterTest.java deleted file mode 100644 index 1bf4f3fedeca4..0000000000000 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/utils/NameValueParameterSplitterTest.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.admin.cli.utils; - -import java.util.Map; - -import org.testng.Assert; -import org.testng.annotations.Test; - -public class NameValueParameterSplitterTest { - @Test(description = "Basic Test") - public void test1() { - NameValueParameterSplitter splitter = new NameValueParameterSplitter(); - Map result = splitter.convert("Name=Sunnyvale"); - Assert.assertEquals(result.get("Name"), "Sunnyvale"); - } - - @Test(description = "Check trimming of values") - public void test2() { - NameValueParameterSplitter splitter = new NameValueParameterSplitter(); - Map result = splitter.convert(" Name = Sunnyvale CA"); - Assert.assertEquals(result.get("Name"), "Sunnyvale CA"); - } - - @Test(description = "Check error on invalid input") - public void test3() { - try { - NameValueParameterSplitter splitter = new NameValueParameterSplitter(); - splitter.convert(" Name Sunnyvale CA"); - // Expecting exception - Assert.fail("' Name Sunnyvale CA' is not a valid name value pair"); - } catch (Exception e) { - // TODO: handle exception - } - } -} diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/CmdUtils.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/CmdUtils.java index a4db39f9cc92b..bfbd78601c4c1 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/CmdUtils.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/CmdUtils.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.admin.cli.utils; -import com.beust.jcommander.ParameterException; import com.fasterxml.jackson.databind.exc.InvalidFormatException; import com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException; import java.io.File; @@ -40,7 +39,7 @@ public static T loadConfig(String file, Class clazz) throws IOException { unrecognizedPropertyException.getLocation().getLineNr(), unrecognizedPropertyException.getLocation().getColumnNr(), unrecognizedPropertyException.getKnownPropertyIds()); - throw new ParameterException(exceptionMessage); + throw new IllegalArgumentException(exceptionMessage); } else if (ex instanceof InvalidFormatException) { InvalidFormatException invalidFormatException = (InvalidFormatException) ex; @@ -50,23 +49,23 @@ public static T loadConfig(String file, Class clazz) throws IOException { invalidFormatException.getLocation().getLineNr(), invalidFormatException.getLocation().getColumnNr()); - throw new ParameterException(exceptionMessage); + throw new IllegalArgumentException(exceptionMessage); } else { - throw new ParameterException(ex.getMessage()); + throw new IllegalArgumentException(ex.getMessage()); } } } public static boolean positiveCheck(String paramName, long value) { if (value <= 0) { - throw new ParameterException(paramName + " cannot be less than or equal to 0!"); + throw new IllegalArgumentException(paramName + " cannot be less than or equal to 0!"); } return true; } public static boolean maxValueCheck(String paramName, long value, long maxValue) { if (value > maxValue) { - throw new ParameterException(paramName + " cannot be greater than " + maxValue + "!"); + throw new IllegalArgumentException(paramName + " cannot be greater than " + maxValue + "!"); } return true; } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/NameValueParameterSplitter.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/NameValueParameterSplitter.java deleted file mode 100644 index 011f93e18f1f0..0000000000000 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/NameValueParameterSplitter.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.admin.cli.utils; - -import com.beust.jcommander.IStringConverter; -import com.beust.jcommander.ParameterException; -import java.util.HashMap; -import java.util.Map; - -public class NameValueParameterSplitter implements IStringConverter> { - - @Override - public Map convert(String value) { - boolean error = false; - Map map = new HashMap(); - - String[] nvpairs = value.split(","); - - for (String nvpair : nvpairs) { - error = true; - if (nvpair != null) { - String[] nv = nvpair.split("="); - if (nv != null && nv.length == 2) { - nv[0] = nv[0].trim(); - nv[1] = nv[1].trim(); - if (!nv[0].isEmpty() && !nv[1].isEmpty() && nv[0].charAt(0) != '\'') { - map.put(nv[0], nv[1]); - error = false; - } - } - } - - if (error) { - break; - } - } - - if (error) { - throw new ParameterException("unable to parse bad name=value parameter list: " + value); - } - - return map; - } - -} diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdProduce.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdProduce.java index b41aea4538c02..e5a8836602151 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdProduce.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdProduce.java @@ -19,7 +19,6 @@ package org.apache.pulsar.client.cli; import static org.apache.commons.lang3.StringUtils.isNotBlank; -import com.beust.jcommander.ParameterException; import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.RateLimiter; @@ -270,7 +269,7 @@ public int run() throws PulsarClientException { case KEY_VALUE_ENCODING_TYPE_INLINE: break; default: - throw (new ParameterException("--key-value-encoding-type " + throw (new IllegalArgumentException("--key-value-encoding-type " + keyValueEncodingType + " is not valid, only 'separated' or 'inline'")); } } @@ -279,7 +278,7 @@ public int run() throws PulsarClientException { if (totalMessages > MAX_MESSAGES) { String msg = "Attempting to send " + totalMessages + " messages. Please do not send more than " + MAX_MESSAGES + " messages"; - throw new ParameterException(msg); + throw new IllegalArgumentException(msg); } if (this.serviceURL.startsWith("ws")) { @@ -322,13 +321,13 @@ private int publish(String topic) { final byte[] keyValueKeyBytes; if (this.keyValueKey != null) { if (keyValueEncodingType == KEY_VALUE_ENCODING_TYPE_NOT_SET) { - throw new ParameterException( + throw new IllegalArgumentException( "Key value encoding type must be set when using --key-value-key"); } keyValueKeyBytes = this.keyValueKey.getBytes(StandardCharsets.UTF_8); } else if (this.keyValueKeyFile != null) { if (keyValueEncodingType == KEY_VALUE_ENCODING_TYPE_NOT_SET) { - throw new ParameterException( + throw new IllegalArgumentException( "Key value encoding type must be set when using --key-value-key-file"); } keyValueKeyBytes = Files.readAllBytes(Paths.get(this.keyValueKeyFile)); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java index 2e0a3e826aa61..daab436499219 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java @@ -19,7 +19,6 @@ package org.apache.pulsar.client.cli; import static org.apache.commons.lang3.StringUtils.isNotBlank; -import com.beust.jcommander.ParameterException; import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.RateLimiter; import java.io.IOException; @@ -114,7 +113,7 @@ public CmdRead() { */ public int run() throws PulsarClientException, IOException { if (this.numMessagesToRead < 0) { - throw (new ParameterException("Number of messages should be zero or positive.")); + throw (new IllegalArgumentException("Number of messages should be zero or positive.")); } diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSources.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSources.java index 13a632121e03e..d96b0933d3f84 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSources.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSources.java @@ -27,7 +27,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.testng.Assert.assertTrue; -import com.beust.jcommander.ParameterException; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.dataformat.yaml.YAMLMapper; import java.io.Closeable; @@ -460,7 +459,7 @@ public void testCmdSourcesThrowingExceptionOnFailure() throws Exception { private void verifyNoSuchFileParameterException(org.apache.pulsar.admin.cli.CmdSources.SourceDetailsCommand command) { command.sourceConfigFile = UUID.randomUUID().toString(); - ParameterException e = Assert.expectThrows(ParameterException.class, command::processArguments); + IllegalArgumentException e = Assert.expectThrows(IllegalArgumentException.class, command::processArguments); assertTrue(e.getMessage().endsWith("(No such file or directory)")); } From fc066d727b52f7e412476297995c2eb2f5ab61bf Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 27 Mar 2024 16:45:02 +0800 Subject: [PATCH 010/580] [fix] [test] Fix flaky test ManagedLedgerTest.testGetNumberOfEntriesInStorage (#22344) --- .../apache/bookkeeper/mledger/impl/ManagedLedgerTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 3b5fd0bcbdd66..0baafa7e1b01c 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -2642,10 +2642,10 @@ public void testGetNumberOfEntriesInStorage() throws Exception { managedLedger.addEntry(("entry-" + i).getBytes(Encoding)); } - //trigger ledger rollover and wait for the new ledger created - Field stateUpdater = ManagedLedgerImpl.class.getDeclaredField("state"); - stateUpdater.setAccessible(true); - stateUpdater.set(managedLedger, ManagedLedgerImpl.State.LedgerOpened); + // trigger ledger rollover and wait for the new ledger created + Awaitility.await().untilAsserted(() -> { + assertEquals("LedgerOpened", WhiteboxImpl.getInternalState(managedLedger, "state").toString()); + }); managedLedger.rollCurrentLedgerIfFull(); Awaitility.await().untilAsserted(() -> { assertEquals(managedLedger.getLedgersInfo().size(), 3); From c184209bfc5a61f143abfa701e5f1b0be2109d77 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 27 Mar 2024 04:10:47 -0700 Subject: [PATCH 011/580] [fix][test] Fix flaky ManagedLedgerErrorsTest.recoverAfterZnodeVersionError (#22368) --- .../apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java index 512e90d17f5e8..7b2f8228ad722 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java @@ -381,7 +381,6 @@ public void recoverAfterZnodeVersionError() throws Exception { ledger.addEntry("entry".getBytes()); fail("should fail"); } catch (ManagedLedgerFencedException e) { - assertEquals(e.getCause().getClass(), ManagedLedgerException.BadVersionException.class); // ok } From 149deaa5a79ed8570489bead4215ae213a4e9206 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 27 Mar 2024 19:49:27 +0800 Subject: [PATCH 012/580] [fix][client] Fix wrong results of hasMessageAvailable and readNext after seeking by timestamp (#22363) Co-authored-by: Lari Hotari --- .../apache/pulsar/client/impl/ReaderTest.java | 84 ++++++++++++++++--- .../pulsar/client/impl/ConsumerImpl.java | 25 ++++-- 2 files changed, 89 insertions(+), 20 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java index cee3ea09968dc..2d3e8d4c6e978 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java @@ -66,8 +66,8 @@ import org.apache.pulsar.schema.Schemas; import org.awaitility.Awaitility; import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -77,7 +77,7 @@ public class ReaderTest extends MockedPulsarServiceBaseTest { private static final String subscription = "reader-sub"; - @BeforeMethod + @BeforeClass(alwaysRun = true) @Override protected void setup() throws Exception { super.internalSetup(); @@ -89,7 +89,7 @@ protected void setup() throws Exception { admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); } - @AfterMethod(alwaysRun = true) + @AfterClass(alwaysRun = true) @Override protected void cleanup() throws Exception { super.internalCleanup(); @@ -198,21 +198,41 @@ public void testReadMessageWithBatching() throws Exception { testReadMessages(topic, true); } - @Test - public void testReadMessageWithBatchingWithMessageInclusive() throws Exception { + @DataProvider + public static Object[][] seekBeforeHasMessageAvailable() { + return new Object[][] { { true }, { false } }; + } + + @Test(timeOut = 20000, dataProvider = "seekBeforeHasMessageAvailable") + public void testReadMessageWithBatchingWithMessageInclusive(boolean seekBeforeHasMessageAvailable) + throws Exception { String topic = "persistent://my-property/my-ns/my-reader-topic-with-batching-inclusive"; Set keys = publishMessages(topic, 10, true); Reader reader = pulsarClient.newReader().topic(topic).startMessageId(MessageId.latest) .startMessageIdInclusive().readerName(subscription).create(); - while (reader.hasMessageAvailable()) { - Assert.assertTrue(keys.remove(reader.readNext().getKey())); + if (seekBeforeHasMessageAvailable) { + reader.seek(0L); // it should seek to the earliest } + + assertTrue(reader.hasMessageAvailable()); + final Message msg = reader.readNext(); + assertTrue(keys.remove(msg.getKey())); // start from latest with start message inclusive should only read the last message in batch assertEquals(keys.size(), 9); - Assert.assertFalse(keys.contains("key9")); - Assert.assertFalse(reader.hasMessageAvailable()); + + final MessageIdAdv msgId = (MessageIdAdv) msg.getMessageId(); + if (seekBeforeHasMessageAvailable) { + assertEquals(msgId.getBatchIndex(), 0); + assertFalse(keys.contains("key0")); + assertTrue(reader.hasMessageAvailable()); + } else { + assertEquals(msgId.getBatchIndex(), 9); + assertFalse(reader.hasMessageAvailable()); + assertFalse(keys.contains("key9")); + assertFalse(reader.hasMessageAvailable()); + } } private void testReadMessages(String topic, boolean enableBatch) throws Exception { @@ -310,7 +330,7 @@ public void testReadFromPartition() throws Exception { @Test public void testReaderWithTimeLong() throws Exception { String ns = "my-property/my-ns"; - String topic = "persistent://" + ns + "/testReadFromPartition"; + String topic = "persistent://" + ns + "/testReaderWithTimeLong"; RetentionPolicies retention = new RetentionPolicies(-1, -1); admin.namespaces().setRetention(ns, retention); @@ -840,4 +860,46 @@ public void testHasMessageAvailableAfterSeek(boolean initializeLastMessageIdInBr producer.send("msg"); assertTrue(reader.hasMessageAvailable()); } + + @Test(dataProvider = "initializeLastMessageIdInBroker") + public void testHasMessageAvailableAfterSeekTimestamp(boolean initializeLastMessageIdInBroker) throws Exception { + final String topic = "persistent://my-property/my-ns/test-has-message-available-after-seek-timestamp"; + + @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + final long timestampBeforeSend = System.currentTimeMillis(); + final MessageId sentMsgId = producer.send("msg"); + + final List messageIds = new ArrayList<>(); + messageIds.add(MessageId.earliest); + messageIds.add(sentMsgId); + messageIds.add(MessageId.latest); + + for (MessageId messageId : messageIds) { + @Cleanup Reader reader = pulsarClient.newReader(Schema.STRING).topic(topic).receiverQueueSize(1) + .startMessageId(messageId).create(); + if (initializeLastMessageIdInBroker) { + if (messageId == MessageId.earliest) { + assertTrue(reader.hasMessageAvailable()); + } else { + assertFalse(reader.hasMessageAvailable()); + } + } // else: lastMessageIdInBroker is earliest + reader.seek(System.currentTimeMillis()); + assertFalse(reader.hasMessageAvailable()); + } + + for (MessageId messageId : messageIds) { + @Cleanup Reader reader = pulsarClient.newReader(Schema.STRING).topic(topic).receiverQueueSize(1) + .startMessageId(messageId).create(); + if (initializeLastMessageIdInBroker) { + if (messageId == MessageId.earliest) { + assertTrue(reader.hasMessageAvailable()); + } else { + assertFalse(reader.hasMessageAvailable()); + } + } // else: lastMessageIdInBroker is earliest + reader.seek(timestampBeforeSend); + assertTrue(reader.hasMessageAvailable()); + } + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 6c2ded819a56f..5a0e5de330d31 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -218,6 +218,7 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private final AtomicReference clientCnxUsedForConsumerRegistration = new AtomicReference<>(); private final List previousExceptions = new CopyOnWriteArrayList(); + private volatile boolean hasSoughtByTimestamp = false; static ConsumerImpl newConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurationData conf, @@ -2252,7 +2253,8 @@ public CompletableFuture seekAsync(Function function) { new PulsarClientException("Only support seek by messageId or timestamp")); } - private CompletableFuture seekAsyncInternal(long requestId, ByteBuf seek, MessageId seekId, String seekBy) { + private CompletableFuture seekAsyncInternal(long requestId, ByteBuf seek, MessageId seekId, + Long seekTimestamp, String seekBy) { AtomicLong opTimeoutMs = new AtomicLong(client.getConfiguration().getOperationTimeoutMs()); Backoff backoff = new BackoffBuilder() .setInitialTime(100, TimeUnit.MILLISECONDS) @@ -2269,11 +2271,11 @@ private CompletableFuture seekAsyncInternal(long requestId, ByteBuf seek, return FutureUtil.failedFuture(new IllegalStateException(message)); } seekFuture = new CompletableFuture<>(); - seekAsyncInternal(requestId, seek, seekId, seekBy, backoff, opTimeoutMs); + seekAsyncInternal(requestId, seek, seekId, seekTimestamp, seekBy, backoff, opTimeoutMs); return seekFuture; } - private void seekAsyncInternal(long requestId, ByteBuf seek, MessageId seekId, String seekBy, + private void seekAsyncInternal(long requestId, ByteBuf seek, MessageId seekId, Long seekTimestamp, String seekBy, final Backoff backoff, final AtomicLong remainingTime) { ClientCnx cnx = cnx(); if (isConnected() && cnx != null) { @@ -2281,6 +2283,8 @@ private void seekAsyncInternal(long requestId, ByteBuf seek, MessageId seekId, S seekMessageId = (MessageIdAdv) seekId; log.info("[{}][{}] Seeking subscription to {}", topic, subscription, seekBy); + final boolean originalHasSoughtByTimestamp = hasSoughtByTimestamp; + hasSoughtByTimestamp = (seekTimestamp != null); cnx.sendRequestWithId(seek, requestId).thenRun(() -> { log.info("[{}][{}] Successfully reset subscription to {}", topic, subscription, seekBy); acknowledgmentsGroupingTracker.flushAndClean(); @@ -2304,6 +2308,7 @@ private void seekAsyncInternal(long requestId, ByteBuf seek, MessageId seekId, S } }).exceptionally(e -> { seekMessageId = originSeekMessageId; + hasSoughtByTimestamp = originalHasSoughtByTimestamp; log.error("[{}][{}] Failed to reset subscription: {}", topic, subscription, e.getCause().getMessage()); failSeek( @@ -2326,7 +2331,7 @@ private void seekAsyncInternal(long requestId, ByteBuf seek, MessageId seekId, S log.warn("[{}] [{}] Could not get connection while seek -- Will try again in {} ms", topic, getHandlerName(), nextDelay); remainingTime.addAndGet(-nextDelay); - seekAsyncInternal(requestId, seek, seekId, seekBy, backoff, remainingTime); + seekAsyncInternal(requestId, seek, seekId, seekTimestamp, seekBy, backoff, remainingTime); }, nextDelay, TimeUnit.MILLISECONDS); } } @@ -2343,7 +2348,7 @@ public CompletableFuture seekAsync(long timestamp) { String seekBy = String.format("the timestamp %d", timestamp); long requestId = client.newRequestId(); return seekAsyncInternal(requestId, Commands.newSeek(consumerId, requestId, timestamp), - MessageId.earliest, seekBy); + MessageId.earliest, timestamp, seekBy); } @Override @@ -2369,7 +2374,7 @@ public CompletableFuture seekAsync(MessageId messageId) { } seek = Commands.newSeek(consumerId, requestId, msgId.getLedgerId(), msgId.getEntryId(), ackSetArr); } - return seekAsyncInternal(requestId, seek, messageId, seekBy); + return seekAsyncInternal(requestId, seek, messageId, null, seekBy); } public boolean hasMessageAvailable() throws PulsarClientException { @@ -2389,13 +2394,15 @@ public CompletableFuture hasMessageAvailableAsync() { // we haven't read yet. use startMessageId for comparison if (lastDequeuedMessageId == MessageId.earliest) { + // If the last seek is called with timestamp, startMessageId cannot represent the position to start, so we + // have to get the mark-delete position from the GetLastMessageId response to compare as well. // if we are starting from latest, we should seek to the actual last message first. // allow the last one to be read when read head inclusively. - if (MessageId.latest.equals(startMessageId)) { - + final boolean hasSoughtByTimestamp = this.hasSoughtByTimestamp; + if (MessageId.latest.equals(startMessageId) || hasSoughtByTimestamp) { CompletableFuture future = internalGetLastMessageIdAsync(); // if the consumer is configured to read inclusive then we need to seek to the last message - if (resetIncludeHead) { + if (resetIncludeHead && !hasSoughtByTimestamp) { future = future.thenCompose((lastMessageIdResponse) -> seekAsync(lastMessageIdResponse.lastMessageId) .thenApply((ignore) -> lastMessageIdResponse)); From 404c0572a461908ffe09c483092d8df78356eae9 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Wed, 27 Mar 2024 20:12:08 +0800 Subject: [PATCH 013/580] [fix][broker] Fix OpReadEntry.skipCondition NPE issue (#22367) --- .../java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index 7b59c3903d5bc..a79ba3fb5e23b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -209,8 +209,8 @@ public void recycle() { entries = null; nextReadPosition = null; maxPosition = null; - recyclerHandle.recycle(this); skipCondition = null; + recyclerHandle.recycle(this); } private static final Logger log = LoggerFactory.getLogger(OpReadEntry.class); From 3fa2ae83312ead38a81fe82bc06c1784e6061d6f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=90=A7=E6=98=93=E5=AE=A2?= Date: Wed, 27 Mar 2024 20:12:39 +0800 Subject: [PATCH 014/580] [fix][client] Consumer lost message ack due to race condition in acknowledge with batch message (#22353) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Yunze Xu Co-authored-by: 汪苏诚 --- .../pulsar/client/impl/MessageIdAdvUtils.java | 19 +++-- ...sistentAcknowledgmentsGroupingTracker.java | 18 ++++- .../client/impl/MessageIdAdvUtilsTest.java | 76 +++++++++++++++++++ .../pulsar/client/api/MessageIdAdv.java | 2 + 4 files changed, 106 insertions(+), 9 deletions(-) create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageIdAdvUtilsTest.java diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdAdvUtils.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdAdvUtils.java index a0d1446ba3d55..f66bb64202115 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdAdvUtils.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdAdvUtils.java @@ -40,6 +40,13 @@ static boolean equals(MessageIdAdv lhs, Object o) { && lhs.getBatchIndex() == rhs.getBatchIndex(); } + /** + * Acknowledge batch message. + * + * @param msgId the message id + * @param individual whether to acknowledge the batch message individually + * @return true if the batch message is fully acknowledged + */ static boolean acknowledge(MessageIdAdv msgId, boolean individual) { if (!isBatch(msgId)) { return true; @@ -51,12 +58,14 @@ static boolean acknowledge(MessageIdAdv msgId, boolean individual) { return false; } int batchIndex = msgId.getBatchIndex(); - if (individual) { - ackSet.clear(batchIndex); - } else { - ackSet.clear(0, batchIndex + 1); + synchronized (ackSet) { + if (individual) { + ackSet.clear(batchIndex); + } else { + ackSet.clear(0, batchIndex + 1); + } + return ackSet.isEmpty(); } - return ackSet.isEmpty(); } static boolean isBatch(MessageIdAdv msgId) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index 0cf776aea5942..c0ee13b346a0b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -324,8 +324,15 @@ private CompletableFuture doIndividualBatchAckAsync(MessageIdAdv msgId) { MessageIdAdvUtils.discardBatch(msgId), __ -> { final BitSet ackSet = msgId.getAckSet(); final ConcurrentBitSetRecyclable value; - if (ackSet != null && !ackSet.isEmpty()) { - value = ConcurrentBitSetRecyclable.create(ackSet); + if (ackSet != null) { + synchronized (ackSet) { + if (!ackSet.isEmpty()) { + value = ConcurrentBitSetRecyclable.create(ackSet); + } else { + value = ConcurrentBitSetRecyclable.create(); + value.set(0, msgId.getBatchSize()); + } + } } else { value = ConcurrentBitSetRecyclable.create(); value.set(0, msgId.getBatchSize()); @@ -374,8 +381,11 @@ private CompletableFuture doImmediateBatchIndexAck(MessageIdAdv msgId, int .ConnectException("Consumer connect fail! consumer state:" + consumer.getState())); } BitSetRecyclable bitSet; - if (msgId.getAckSet() != null) { - bitSet = BitSetRecyclable.valueOf(msgId.getAckSet().toLongArray()); + BitSet ackSetFromMsgId = msgId.getAckSet(); + if (ackSetFromMsgId != null) { + synchronized (ackSetFromMsgId) { + bitSet = BitSetRecyclable.valueOf(ackSetFromMsgId.toLongArray()); + } } else { bitSet = BitSetRecyclable.create(); bitSet.set(0, batchSize); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageIdAdvUtilsTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageIdAdvUtilsTest.java new file mode 100644 index 0000000000000..704dfc9cbd77b --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageIdAdvUtilsTest.java @@ -0,0 +1,76 @@ +/* + * 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.impl; + +import static org.testng.Assert.assertEquals; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.util.BitSet; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Phaser; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; +import org.apache.pulsar.client.api.MessageIdAdv; +import org.testng.annotations.Test; + +/** + * Unit test for {@link MessageIdAdvUtils}. + */ +public class MessageIdAdvUtilsTest { + + /** + * Call acknowledge concurrently with batch message, and verify that only return true once + * + * @see MessageIdAdvUtils#acknowledge(MessageIdAdv, boolean) + * @see MessageIdAdv#getAckSet() + */ + @Test + public void testAcknowledgeIndividualConcurrently() throws InterruptedException { + ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat("pulsar-consumer-%d").build(); + @Cleanup("shutdown") + ExecutorService executorService = Executors.newCachedThreadPool(threadFactory); + for (int i = 0; i < 100; i++) { + int batchSize = 32; + BitSet bitSet = new BitSet(batchSize); + bitSet.set(0, batchSize); + AtomicInteger individualAcked = new AtomicInteger(); + Phaser phaser = new Phaser(1); + CountDownLatch finishLatch = new CountDownLatch(batchSize); + for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) { + phaser.register(); + BatchMessageIdImpl messageId = new BatchMessageIdImpl(1, 0, 0, batchIndex, batchSize, bitSet); + executorService.execute(() -> { + try { + phaser.arriveAndAwaitAdvance(); + if (MessageIdAdvUtils.acknowledge(messageId, true)) { + individualAcked.incrementAndGet(); + } + } finally { + finishLatch.countDown(); + } + }); + } + phaser.arriveAndDeregister(); + finishLatch.await(); + assertEquals(individualAcked.get(), 1); + } + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/client/api/MessageIdAdv.java b/pulsar-common/src/main/java/org/apache/pulsar/client/api/MessageIdAdv.java index 73ecfed0ad059..76d41a7d3d4fc 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/client/api/MessageIdAdv.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/client/api/MessageIdAdv.java @@ -75,6 +75,8 @@ default int getBatchSize() { * @implNote The message IDs of a batch should share a BitSet. For example, given 3 messages in the same batch whose * size is 3, all message IDs of them should return "111" (i.e. a BitSet whose size is 3 and all bits are 1). If the * 1st message has been acknowledged, the returned BitSet should become "011" (i.e. the 1st bit become 0). + * If the caller performs any read or write operations on the return value of this method, they should do so with + * lock protection. * * @return null if the message is a non-batched message */ From e4553391f96af3bda3d8252b97cac3de1f39a1b5 Mon Sep 17 00:00:00 2001 From: hanmz Date: Thu, 28 Mar 2024 00:07:54 +0800 Subject: [PATCH 015/580] [improve][broker] Optimize web interface deleteDynamicConfiguration return error message (#22356) --- .../java/org/apache/pulsar/broker/admin/impl/BrokersBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index 61b354610ac20..83067e9f296ef 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -526,7 +526,7 @@ private CompletableFuture healthCheckRecursiveReadNext(Reader read private CompletableFuture internalDeleteDynamicConfigurationOnMetadataAsync(String configName) { if (!pulsar().getBrokerService().isDynamicConfiguration(configName)) { - throw new RestException(Status.PRECONDITION_FAILED, " Can't update non-dynamic configuration"); + throw new RestException(Status.PRECONDITION_FAILED, "Can't delete non-dynamic configuration"); } else { return dynamicConfigurationResources().setDynamicConfigurationAsync(old -> { if (old != null) { From edd0076bd83f01a5fcbe81c8396667014f0fc36e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 27 Mar 2024 09:16:22 -0700 Subject: [PATCH 016/580] [fix][misc] Make ConcurrentBitSet thread safe (#22361) --- .../util/collections/ConcurrentBitSet.java | 363 ++++++++++++++++-- 1 file changed, 331 insertions(+), 32 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSet.java index 23842fe5b556c..a37628cb300b8 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSet.java @@ -20,12 +20,13 @@ import java.util.BitSet; import java.util.concurrent.locks.StampedLock; -import lombok.EqualsAndHashCode; +import java.util.stream.IntStream; /** - * Safe multithreaded version of {@code BitSet}. + * A {@code BitSet} that is protected by a {@code StampedLock} to provide thread-safe access. + * The {@link #length()} method is not thread safe and is not overridden because StampedLock is not reentrant. + * Use the {@link #safeLength()} method to get the length of the bit set in a thread-safe manner. */ -@EqualsAndHashCode(callSuper = true) public class ConcurrentBitSet extends BitSet { private static final long serialVersionUID = 1L; @@ -39,10 +40,8 @@ public ConcurrentBitSet() { * Creates a bit set whose initial size is large enough to explicitly represent bits with indices in the range * {@code 0} through {@code nbits-1}. All bits are initially {@code false}. * - * @param nbits - * the initial size of the bit set - * @throws NegativeArraySizeException - * if the specified initial size is negative + * @param nbits the initial size of the bit set + * @throws NegativeArraySizeException if the specified initial size is negative */ public ConcurrentBitSet(int nbits) { super(nbits); @@ -65,105 +64,405 @@ public boolean get(int bitIndex) { @Override public void set(int bitIndex) { + long stamp = rwLock.writeLock(); + try { + super.set(bitIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void clear(int bitIndex) { + long stamp = rwLock.writeLock(); + try { + super.clear(bitIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void set(int fromIndex, int toIndex) { + long stamp = rwLock.writeLock(); + try { + super.set(fromIndex, toIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void clear(int fromIndex, int toIndex) { + long stamp = rwLock.writeLock(); + try { + super.clear(fromIndex, toIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void clear() { + long stamp = rwLock.writeLock(); + try { + super.clear(); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public int nextSetBit(int fromIndex) { long stamp = rwLock.tryOptimisticRead(); - super.set(bitIndex); + int nextSetBit = super.nextSetBit(fromIndex); if (!rwLock.validate(stamp)) { + // Fallback to read lock stamp = rwLock.readLock(); try { - super.set(bitIndex); + nextSetBit = super.nextSetBit(fromIndex); } finally { rwLock.unlockRead(stamp); } } + return nextSetBit; } @Override - public void set(int fromIndex, int toIndex) { + public int nextClearBit(int fromIndex) { long stamp = rwLock.tryOptimisticRead(); - super.set(fromIndex, toIndex); + int nextClearBit = super.nextClearBit(fromIndex); if (!rwLock.validate(stamp)) { + // Fallback to read lock stamp = rwLock.readLock(); try { - super.set(fromIndex, toIndex); + nextClearBit = super.nextClearBit(fromIndex); } finally { rwLock.unlockRead(stamp); } } + return nextClearBit; } @Override - public int nextSetBit(int fromIndex) { + public int previousSetBit(int fromIndex) { long stamp = rwLock.tryOptimisticRead(); - int bit = super.nextSetBit(fromIndex); + int previousSetBit = super.previousSetBit(fromIndex); if (!rwLock.validate(stamp)) { + // Fallback to read lock stamp = rwLock.readLock(); try { - bit = super.nextSetBit(fromIndex); + previousSetBit = super.previousSetBit(fromIndex); } finally { rwLock.unlockRead(stamp); } } - return bit; + return previousSetBit; } @Override - public int nextClearBit(int fromIndex) { + public int previousClearBit(int fromIndex) { long stamp = rwLock.tryOptimisticRead(); - int bit = super.nextClearBit(fromIndex); + int previousClearBit = super.previousClearBit(fromIndex); if (!rwLock.validate(stamp)) { + // Fallback to read lock stamp = rwLock.readLock(); try { - bit = super.nextClearBit(fromIndex); + previousClearBit = super.previousClearBit(fromIndex); } finally { rwLock.unlockRead(stamp); } } - return bit; + return previousClearBit; } @Override - public int previousSetBit(int fromIndex) { + public boolean isEmpty() { long stamp = rwLock.tryOptimisticRead(); - int bit = super.previousSetBit(fromIndex); + boolean isEmpty = super.isEmpty(); if (!rwLock.validate(stamp)) { + // Fallback to read lock stamp = rwLock.readLock(); try { - bit = super.previousSetBit(fromIndex); + isEmpty = super.isEmpty(); } finally { rwLock.unlockRead(stamp); } } - return bit; + return isEmpty; } @Override - public int previousClearBit(int fromIndex) { + public int cardinality() { long stamp = rwLock.tryOptimisticRead(); - int bit = super.previousClearBit(fromIndex); + int cardinality = super.cardinality(); if (!rwLock.validate(stamp)) { + // Fallback to read lock stamp = rwLock.readLock(); try { - bit = super.previousClearBit(fromIndex); + cardinality = super.cardinality(); } finally { rwLock.unlockRead(stamp); } } - return bit; + return cardinality; } @Override - public boolean isEmpty() { + public int size() { long stamp = rwLock.tryOptimisticRead(); - boolean isEmpty = super.isEmpty(); + int size = super.size(); if (!rwLock.validate(stamp)) { // Fallback to read lock stamp = rwLock.readLock(); try { - isEmpty = super.isEmpty(); + size = super.size(); } finally { rwLock.unlockRead(stamp); } } - return isEmpty; + return size; + } + + @Override + public byte[] toByteArray() { + long stamp = rwLock.tryOptimisticRead(); + byte[] byteArray = super.toByteArray(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + byteArray = super.toByteArray(); + } finally { + rwLock.unlockRead(stamp); + } + } + return byteArray; + } + + @Override + public long[] toLongArray() { + long stamp = rwLock.tryOptimisticRead(); + long[] longArray = super.toLongArray(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + longArray = super.toLongArray(); + } finally { + rwLock.unlockRead(stamp); + } + } + return longArray; + } + + @Override + public void flip(int bitIndex) { + long stamp = rwLock.writeLock(); + try { + super.flip(bitIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void flip(int fromIndex, int toIndex) { + long stamp = rwLock.writeLock(); + try { + super.flip(fromIndex, toIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void set(int bitIndex, boolean value) { + long stamp = rwLock.writeLock(); + try { + super.set(bitIndex, value); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void set(int fromIndex, int toIndex, boolean value) { + long stamp = rwLock.writeLock(); + try { + super.set(fromIndex, toIndex, value); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public BitSet get(int fromIndex, int toIndex) { + long stamp = rwLock.tryOptimisticRead(); + BitSet bitSet = super.get(fromIndex, toIndex); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + bitSet = super.get(fromIndex, toIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return bitSet; + } + + /** + * Thread-safe version of {@code length()}. + * StampedLock is not reentrant and that's why the length() method is not overridden. Overriding length() method + * would require to use a reentrant lock which would be less performant. + * + * @return length of the bit set + */ + public int safeLength() { + long stamp = rwLock.tryOptimisticRead(); + int length = super.length(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + length = super.length(); + } finally { + rwLock.unlockRead(stamp); + } + } + return length; + } + + @Override + public boolean intersects(BitSet set) { + long stamp = rwLock.writeLock(); + try { + return super.intersects(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void and(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.and(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void or(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.or(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void xor(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.xor(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void andNot(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.andNot(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + /** + * Returns the clone of the internal wrapped {@code BitSet}. + * This won't be a clone of the {@code ConcurrentBitSet} object. + * + * @return a clone of the internal wrapped {@code BitSet} + */ + @Override + public Object clone() { + long stamp = rwLock.tryOptimisticRead(); + BitSet clonedBitSet = (BitSet) super.clone(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + clonedBitSet = (BitSet) super.clone(); + } finally { + rwLock.unlockRead(stamp); + } + } + return clonedBitSet; + } + + @Override + public String toString() { + long stamp = rwLock.tryOptimisticRead(); + String str = super.toString(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + str = super.toString(); + } finally { + rwLock.unlockRead(stamp); + } + } + return str; + } + + /** + * This operation is not supported on {@code ConcurrentBitSet}. + */ + @Override + public IntStream stream() { + throw new UnsupportedOperationException("stream is not supported"); + } + + public boolean equals(final Object o) { + if (o == this) { + return true; + } + if (!(o instanceof ConcurrentBitSet)) { + return false; + } + long stamp = rwLock.tryOptimisticRead(); + boolean isEqual = super.equals(o); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + isEqual = super.equals(o); + } finally { + rwLock.unlockRead(stamp); + } + } + return isEqual; + } + + public int hashCode() { + long stamp = rwLock.tryOptimisticRead(); + int hashCode = super.hashCode(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + hashCode = super.hashCode(); + } finally { + rwLock.unlockRead(stamp); + } + } + return hashCode; } } From be0a9d9d9bb23dabc065f091b853f27c0ebcaa16 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 27 Mar 2024 12:34:14 -0700 Subject: [PATCH 017/580] [improve][misc] Upgrade to Netty 4.1.108 and tcnative 2.0.65 (#22369) --- .../server/src/assemble/LICENSE.bin.txt | 56 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 54 +++++++++--------- pom.xml | 2 +- 3 files changed, 56 insertions(+), 56 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index cb99d62edfeb7..cab23db279aca 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -292,34 +292,34 @@ The Apache Software License, Version 2.0 - org.apache.commons-commons-lang3-3.11.jar - org.apache.commons-commons-text-1.10.0.jar * Netty - - io.netty-netty-buffer-4.1.105.Final.jar - - io.netty-netty-codec-4.1.105.Final.jar - - io.netty-netty-codec-dns-4.1.105.Final.jar - - io.netty-netty-codec-http-4.1.105.Final.jar - - io.netty-netty-codec-http2-4.1.105.Final.jar - - io.netty-netty-codec-socks-4.1.105.Final.jar - - io.netty-netty-codec-haproxy-4.1.105.Final.jar - - io.netty-netty-common-4.1.105.Final.jar - - io.netty-netty-handler-4.1.105.Final.jar - - io.netty-netty-handler-proxy-4.1.105.Final.jar - - io.netty-netty-resolver-4.1.105.Final.jar - - io.netty-netty-resolver-dns-4.1.105.Final.jar - - io.netty-netty-resolver-dns-classes-macos-4.1.105.Final.jar - - io.netty-netty-resolver-dns-native-macos-4.1.105.Final-osx-aarch_64.jar - - io.netty-netty-resolver-dns-native-macos-4.1.105.Final-osx-x86_64.jar - - io.netty-netty-transport-4.1.105.Final.jar - - io.netty-netty-transport-classes-epoll-4.1.105.Final.jar - - io.netty-netty-transport-native-epoll-4.1.105.Final-linux-aarch_64.jar - - io.netty-netty-transport-native-epoll-4.1.105.Final-linux-x86_64.jar - - io.netty-netty-transport-native-unix-common-4.1.105.Final.jar - - io.netty-netty-transport-native-unix-common-4.1.105.Final-linux-x86_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.61.Final.jar - - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-osx-aarch_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-osx-x86_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-windows-x86_64.jar - - io.netty-netty-tcnative-classes-2.0.61.Final.jar + - io.netty-netty-buffer-4.1.108.Final.jar + - io.netty-netty-codec-4.1.108.Final.jar + - io.netty-netty-codec-dns-4.1.108.Final.jar + - io.netty-netty-codec-http-4.1.108.Final.jar + - io.netty-netty-codec-http2-4.1.108.Final.jar + - io.netty-netty-codec-socks-4.1.108.Final.jar + - io.netty-netty-codec-haproxy-4.1.108.Final.jar + - io.netty-netty-common-4.1.108.Final.jar + - io.netty-netty-handler-4.1.108.Final.jar + - io.netty-netty-handler-proxy-4.1.108.Final.jar + - io.netty-netty-resolver-4.1.108.Final.jar + - io.netty-netty-resolver-dns-4.1.108.Final.jar + - io.netty-netty-resolver-dns-classes-macos-4.1.108.Final.jar + - io.netty-netty-resolver-dns-native-macos-4.1.108.Final-osx-aarch_64.jar + - io.netty-netty-resolver-dns-native-macos-4.1.108.Final-osx-x86_64.jar + - io.netty-netty-transport-4.1.108.Final.jar + - io.netty-netty-transport-classes-epoll-4.1.108.Final.jar + - io.netty-netty-transport-native-epoll-4.1.108.Final-linux-aarch_64.jar + - io.netty-netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar + - io.netty-netty-transport-native-unix-common-4.1.108.Final.jar + - io.netty-netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.65.Final.jar + - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-osx-aarch_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-osx-x86_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-windows-x86_64.jar + - io.netty-netty-tcnative-classes-2.0.65.Final.jar - io.netty.incubator-netty-incubator-transport-classes-io_uring-0.0.24.Final.jar - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.24.Final-linux-x86_64.jar - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.24.Final-linux-aarch_64.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index e735bd454eed2..7e3ebbe06358d 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -346,36 +346,36 @@ The Apache Software License, Version 2.0 - commons-text-1.10.0.jar - commons-compress-1.26.0.jar * Netty - - netty-buffer-4.1.105.Final.jar - - netty-codec-4.1.105.Final.jar - - netty-codec-dns-4.1.105.Final.jar - - netty-codec-http-4.1.105.Final.jar - - netty-codec-socks-4.1.105.Final.jar - - netty-codec-haproxy-4.1.105.Final.jar - - netty-common-4.1.105.Final.jar - - netty-handler-4.1.105.Final.jar - - netty-handler-proxy-4.1.105.Final.jar - - netty-resolver-4.1.105.Final.jar - - netty-resolver-dns-4.1.105.Final.jar - - netty-transport-4.1.105.Final.jar - - netty-transport-classes-epoll-4.1.105.Final.jar - - netty-transport-native-epoll-4.1.105.Final-linux-aarch_64.jar - - netty-transport-native-epoll-4.1.105.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.105.Final.jar - - netty-transport-native-unix-common-4.1.105.Final-linux-x86_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final.jar - - netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final-osx-aarch_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final-osx-x86_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final-windows-x86_64.jar - - netty-tcnative-classes-2.0.61.Final.jar + - netty-buffer-4.1.108.Final.jar + - netty-codec-4.1.108.Final.jar + - netty-codec-dns-4.1.108.Final.jar + - netty-codec-http-4.1.108.Final.jar + - netty-codec-socks-4.1.108.Final.jar + - netty-codec-haproxy-4.1.108.Final.jar + - netty-common-4.1.108.Final.jar + - netty-handler-4.1.108.Final.jar + - netty-handler-proxy-4.1.108.Final.jar + - netty-resolver-4.1.108.Final.jar + - netty-resolver-dns-4.1.108.Final.jar + - netty-transport-4.1.108.Final.jar + - netty-transport-classes-epoll-4.1.108.Final.jar + - netty-transport-native-epoll-4.1.108.Final-linux-aarch_64.jar + - netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.108.Final.jar + - netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final.jar + - netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final-osx-aarch_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final-osx-x86_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final-windows-x86_64.jar + - netty-tcnative-classes-2.0.65.Final.jar - netty-incubator-transport-classes-io_uring-0.0.24.Final.jar - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-aarch_64.jar - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-x86_64.jar - - netty-resolver-dns-classes-macos-4.1.105.Final.jar - - netty-resolver-dns-native-macos-4.1.105.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.105.Final-osx-x86_64.jar + - netty-resolver-dns-classes-macos-4.1.108.Final.jar + - netty-resolver-dns-native-macos-4.1.108.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.108.Final-osx-x86_64.jar * Prometheus client - simpleclient-0.16.0.jar - simpleclient_log4j2-0.16.0.jar diff --git a/pom.xml b/pom.xml index da7f2c7642102..86a5be07c2a8f 100644 --- a/pom.xml +++ b/pom.xml @@ -144,7 +144,7 @@ flexible messaging model and an intuitive client API. 1.1.10.5 4.1.12.1 5.1.0 - 4.1.105.Final + 4.1.108.Final 0.0.24.Final 9.4.54.v20240208 2.5.2 From f77fe5f099f7ecc334509db07bba477c4226cf19 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Thu, 28 Mar 2024 03:42:15 +0800 Subject: [PATCH 018/580] [fix][broker] Avoid expired unclosed ledgers when checking expired messages by ledger closure time (#22335) --- .../PersistentMessageExpiryMonitor.java | 4 +- .../service/PersistentMessageFinderTest.java | 51 ++++++++++++++++--- 2 files changed, 47 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java index ac391c1050340..2478a7a2538d0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java @@ -121,8 +121,8 @@ private void checkExpiryByLedgerClosureTime(ManagedCursor cursor, int messageTTL managedLedger.getLedgersInfo().lastKey(), true); MLDataFormats.ManagedLedgerInfo.LedgerInfo info = null; for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo : ledgerInfoSortedMap.values()) { - if (!ledgerInfo.hasTimestamp() || !MessageImpl.isEntryExpired(messageTTLInSeconds, - ledgerInfo.getTimestamp())) { + if (!ledgerInfo.hasTimestamp() || ledgerInfo.getTimestamp() == 0L + || !MessageImpl.isEntryExpired(messageTTLInSeconds, ledgerInfo.getTimestamp())) { break; } info = ledgerInfo; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java index ace552a55a72a..6883c0467e481 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java @@ -33,10 +33,8 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; - import io.netty.buffer.ByteBuf; import io.netty.buffer.UnpooledByteBufAllocator; - import java.lang.reflect.Field; import java.util.ArrayList; import java.util.HashSet; @@ -46,7 +44,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; - +import java.util.concurrent.atomic.AtomicReference; +import javax.ws.rs.client.Entity; +import javax.ws.rs.core.MediaType; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -59,6 +59,7 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.service.persistent.PersistentMessageExpiryMonitor; import org.apache.pulsar.broker.service.persistent.PersistentMessageFinder; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -72,11 +73,10 @@ import org.apache.pulsar.common.protocol.ByteBufPair; import org.apache.pulsar.common.protocol.Commands; import org.awaitility.Awaitility; +import org.mockito.Mockito; +import org.testng.Assert; import org.testng.annotations.Test; -import javax.ws.rs.client.Entity; -import javax.ws.rs.core.MediaType; - @Test(groups = "broker") public class PersistentMessageFinderTest extends MockedBookKeeperTestCase { @@ -463,6 +463,45 @@ public void testIncorrectClientClock() throws Exception { assertEquals(c1.getNumberOfEntriesInBacklog(true), 0); } + @Test + public void testCheckExpiryByLedgerClosureTimeWithAckUnclosedLedger() throws Throwable { + final String ledgerAndCursorName = "testCheckExpiryByLedgerClosureTimeWithAckUnclosedLedger"; + int maxTTLSeconds = 1; + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setMaxEntriesPerLedger(5); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(ledgerAndCursorName, config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(ledgerAndCursorName); + // set client clock to 10 days later + long incorrectPublishTimestamp = System.currentTimeMillis() + TimeUnit.DAYS.toMillis(10); + for (int i = 0; i < 7; i++) { + ledger.addEntry(createMessageWrittenToLedger("msg" + i, incorrectPublishTimestamp)); + } + assertEquals(ledger.getLedgersInfoAsList().size(), 2); + PersistentTopic mock = mock(PersistentTopic.class); + when(mock.getName()).thenReturn("topicname"); + when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); + PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null); + AsyncCallbacks.MarkDeleteCallback markDeleteCallback = + (AsyncCallbacks.MarkDeleteCallback) spy( + FieldUtils.readDeclaredField(monitor, "markDeleteCallback", true)); + FieldUtils.writeField(monitor, "markDeleteCallback", markDeleteCallback, true); + + AtomicReference throwableAtomicReference = new AtomicReference<>(); + Mockito.doAnswer(invocation -> { + ManagedLedgerException argument = invocation.getArgument(0, ManagedLedgerException.class); + throwableAtomicReference.set(argument); + return invocation.callRealMethod(); + }).when(markDeleteCallback).markDeleteFailed(any(), any()); + + PositionImpl position = (PositionImpl) ledger.getLastConfirmedEntry(); + c1.markDelete(position); + Thread.sleep(TimeUnit.SECONDS.toMillis(maxTTLSeconds)); + monitor.expireMessages(maxTTLSeconds); + assertEquals(c1.getNumberOfEntriesInBacklog(true), 0); + + Assert.assertNull(throwableAtomicReference.get()); + } + @Test void testMessageExpiryWithPosition() throws Exception { final String ledgerAndCursorName = "testPersistentMessageExpiryWithPositionNonRecoverableLedgers"; From b702d440dc5e5a4cfd845bf60d5e310efe665ff5 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Thu, 28 Mar 2024 06:53:21 +0800 Subject: [PATCH 019/580] [fix][broker] Check cursor state before adding it to the `waitingCursors` (#22191) --- .../mledger/impl/ManagedCursorImpl.java | 2 +- .../mledger/impl/ManagedLedgerImpl.java | 10 ++++ .../persistent/PersistentTopicTest.java | 46 +++++++++++++++++++ 3 files changed, 57 insertions(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 8b13fc0f3424e..b253da72fa92b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -992,7 +992,7 @@ private void checkForNewEntries(OpReadEntry op, ReadEntriesCallback callback, Ob name); } // Let the managed ledger know we want to be notified whenever a new entry is published - ledger.waitingCursors.add(this); + ledger.addWaitingCursor(this); } else { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Skip notification registering since we do have entries available", diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 1c0a0465507a1..0f089ef4a8573 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -3813,6 +3813,16 @@ public void removeWaitingCursor(ManagedCursor cursor) { this.waitingCursors.remove(cursor); } + public void addWaitingCursor(ManagedCursorImpl cursor) { + if (cursor instanceof NonDurableCursorImpl) { + if (cursor.isActive()) { + this.waitingCursors.add(cursor); + } + } else { + this.waitingCursors.add(cursor); + } + } + public boolean isCursorActive(ManagedCursor cursor) { return activeCursors.get(cursor.getName()) != null; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index ea1a68bb0c280..d42b1d92007aa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -59,10 +59,15 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -75,6 +80,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.TopicName; @@ -662,4 +668,44 @@ public void testDynamicConfigurationAutoSkipNonRecoverableData() throws Exceptio subscribe.close(); admin.topics().delete(topicName); } + + @Test + public void testAddWaitingCursorsForNonDurable() throws Exception { + final String ns = "prop/ns-test"; + admin.namespaces().createNamespace(ns, 2); + final String topicName = "persistent://prop/ns-test/testAddWaitingCursors"; + admin.topics().createNonPartitionedTopic(topicName); + final Optional topic = pulsar.getBrokerService().getTopic(topicName, false).join(); + assertNotNull(topic.get()); + PersistentTopic persistentTopic = (PersistentTopic) topic.get(); + ManagedLedgerImpl ledger = (ManagedLedgerImpl)persistentTopic.getManagedLedger(); + final ManagedCursor spyCursor= spy(ledger.newNonDurableCursor(PositionImpl.LATEST, "sub-2")); + doAnswer((invocation) -> { + Thread.sleep(10_000); + invocation.callRealMethod(); + return null; + }).when(spyCursor).asyncReadEntriesOrWait(any(int.class), any(long.class), + any(AsyncCallbacks.ReadEntriesCallback.class), any(Object.class), any(PositionImpl.class)); + Field cursorField = ManagedLedgerImpl.class.getDeclaredField("cursors"); + cursorField.setAccessible(true); + ManagedCursorContainer container = (ManagedCursorContainer) cursorField.get(ledger); + container.removeCursor("sub-2"); + container.add(spyCursor, null); + final Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionMode(SubscriptionMode.NonDurable) + .subscriptionType(SubscriptionType.Exclusive) + .subscriptionName("sub-2").subscribe(); + final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + producer.send("test"); + producer.close(); + final Message receive = consumer.receive(); + assertEquals("test", receive.getValue()); + consumer.close(); + Awaitility.await() + .pollDelay(5, TimeUnit.SECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .untilAsserted(() -> { + assertEquals(ledger.getWaitingCursorsCount(), 0); + }); + } } From cce0b058efd55e2d5ac42c4ecaceddacee648a7c Mon Sep 17 00:00:00 2001 From: Jian Yun Date: Thu, 28 Mar 2024 06:59:28 +0800 Subject: [PATCH 020/580] [improve][misc] Remove the call to sun InetAddressCachePolicy (#22329) Co-authored-by: Lari Hotari --- .../common/util/netty/DnsResolverUtil.java | 52 ++++++++++++++----- .../common/util/netty/DnsResolverTest.java | 44 ++++++++++++++++ 2 files changed, 82 insertions(+), 14 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/DnsResolverUtil.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/DnsResolverUtil.java index f49a6453c72b3..bcff83acd949f 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/DnsResolverUtil.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/netty/DnsResolverUtil.java @@ -19,12 +19,20 @@ package org.apache.pulsar.common.util.netty; import io.netty.resolver.dns.DnsNameResolverBuilder; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; +import java.security.Security; +import java.util.Optional; import lombok.extern.slf4j.Slf4j; @Slf4j public class DnsResolverUtil { + + private static final String CACHE_POLICY_PROP = "networkaddress.cache.ttl"; + private static final String CACHE_POLICY_PROP_FALLBACK = "sun.net.inetaddr.ttl"; + private static final String NEGATIVE_CACHE_POLICY_PROP = "networkaddress.cache.negative.ttl"; + private static final String NEGATIVE_CACHE_POLICY_PROP_FALLBACK = "sun.net.inetaddr.negative.ttl"; + /* default ttl value from sun.net.InetAddressCachePolicy.DEFAULT_POSITIVE, which is used when no security manager + is used */ + private static final int JDK_DEFAULT_TTL = 30; private static final int MIN_TTL = 0; private static final int TTL; private static final int NEGATIVE_TTL; @@ -39,19 +47,35 @@ public class DnsResolverUtil { int ttl = DEFAULT_TTL; int negativeTtl = DEFAULT_NEGATIVE_TTL; try { - // use reflection to call sun.net.InetAddressCachePolicy's get and getNegative methods for getting - // effective JDK settings for DNS caching - Class inetAddressCachePolicyClass = Class.forName("sun.net.InetAddressCachePolicy"); - Method getTTLMethod = inetAddressCachePolicyClass.getMethod("get"); - ttl = (Integer) getTTLMethod.invoke(null); - Method getNegativeTTLMethod = inetAddressCachePolicyClass.getMethod("getNegative"); - negativeTtl = (Integer) getNegativeTTLMethod.invoke(null); - } catch (NoSuchMethodException | ClassNotFoundException | InvocationTargetException - | IllegalAccessException e) { - log.warn("Cannot get DNS TTL settings from sun.net.InetAddressCachePolicy class", e); + String ttlStr = Security.getProperty(CACHE_POLICY_PROP); + if (ttlStr == null) { + // Compatible with sun.net.inetaddr.ttl settings + ttlStr = System.getProperty(CACHE_POLICY_PROP_FALLBACK); + } + String negativeTtlStr = Security.getProperty(NEGATIVE_CACHE_POLICY_PROP); + if (negativeTtlStr == null) { + // Compatible with sun.net.inetaddr.negative.ttl settings + negativeTtlStr = System.getProperty(NEGATIVE_CACHE_POLICY_PROP_FALLBACK); + } + ttl = Optional.ofNullable(ttlStr) + .map(Integer::decode) + .filter(i -> i > 0) + .orElseGet(() -> { + if (System.getSecurityManager() == null) { + return JDK_DEFAULT_TTL; + } + return DEFAULT_TTL; + }); + + negativeTtl = Optional.ofNullable(negativeTtlStr) + .map(Integer::decode) + .filter(i -> i >= 0) + .orElse(DEFAULT_NEGATIVE_TTL); + } catch (NumberFormatException e) { + log.warn("Cannot get DNS TTL settings", e); } - TTL = ttl <= 0 ? DEFAULT_TTL : ttl; - NEGATIVE_TTL = negativeTtl < 0 ? DEFAULT_NEGATIVE_TTL : negativeTtl; + TTL = ttl; + NEGATIVE_TTL = negativeTtl; } private DnsResolverUtil() { diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/netty/DnsResolverTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/netty/DnsResolverTest.java index 0ccb960e79887..46599cc45a090 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/netty/DnsResolverTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/netty/DnsResolverTest.java @@ -18,13 +18,57 @@ */ package org.apache.pulsar.common.util.netty; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.testng.Assert.assertEquals; import io.netty.channel.EventLoop; import io.netty.resolver.dns.DnsNameResolverBuilder; +import java.security.Security; +import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; public class DnsResolverTest { + private static final int MIN_TTL = 0; + private static final int TTL = 101; + private static final int NEGATIVE_TTL = 121; + private static final String CACHE_POLICY_PROP = "networkaddress.cache.ttl"; + private static final String NEGATIVE_CACHE_POLICY_PROP = "networkaddress.cache.negative.ttl"; + + private String originalCachePolicy; + private String originalNegativeCachePolicy; + + @BeforeClass(alwaysRun = true) + public void beforeClass() { + originalCachePolicy = Security.getProperty(CACHE_POLICY_PROP); + originalNegativeCachePolicy = Security.getProperty(NEGATIVE_CACHE_POLICY_PROP); + Security.setProperty(CACHE_POLICY_PROP, Integer.toString(TTL)); + Security.setProperty(NEGATIVE_CACHE_POLICY_PROP, Integer.toString(NEGATIVE_TTL)); + } + + @AfterClass(alwaysRun = true) + public void afterClass() { + Security.setProperty(CACHE_POLICY_PROP, originalCachePolicy != null ? originalCachePolicy : "-1"); + Security.setProperty(NEGATIVE_CACHE_POLICY_PROP, + originalNegativeCachePolicy != null ? originalNegativeCachePolicy : "0"); + } + + @Test + public void testTTl() { + final DnsNameResolverBuilder builder = mock(DnsNameResolverBuilder.class); + ArgumentCaptor minTtlCaptor = ArgumentCaptor.forClass(Integer.class); + ArgumentCaptor maxTtlCaptor = ArgumentCaptor.forClass(Integer.class); + ArgumentCaptor negativeTtlCaptor = ArgumentCaptor.forClass(Integer.class); + DnsResolverUtil.applyJdkDnsCacheSettings(builder); + verify(builder).ttl(minTtlCaptor.capture(), maxTtlCaptor.capture()); + verify(builder).negativeTtl(negativeTtlCaptor.capture()); + assertEquals(minTtlCaptor.getValue(), MIN_TTL); + assertEquals(maxTtlCaptor.getValue(), TTL); + assertEquals(negativeTtlCaptor.getValue(), NEGATIVE_TTL); + } @Test public void testMaxTtl() { From 32037c3b0982aa00a7cb5ee7e17a6b235a8c2d7f Mon Sep 17 00:00:00 2001 From: hanmz Date: Thu, 28 Mar 2024 10:31:09 +0800 Subject: [PATCH 021/580] [fix][broker] Fix typos in PersistentTopic class (#22364) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 6179e73169fde..1650e449a3fd6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -2753,7 +2753,7 @@ public CompletableFuture checkClusterMigration() { ledger.asyncMigrate(); } if (log.isDebugEnabled()) { - log.debug("{} has replication backlog and applied migraiton", topic); + log.debug("{} has replication backlog and applied migration", topic); } return CompletableFuture.completedFuture(null); } From 6f9c8e7f70ec201d65c7fc270480bed3aa3b5aba Mon Sep 17 00:00:00 2001 From: sherlock-lin <1193179897@qq.com> Date: Thu, 28 Mar 2024 11:58:47 +0800 Subject: [PATCH 022/580] [fix][broker] Fix PersistentSubscription duplicate implementation interface Subscription (#22359) --- .../service/nonpersistent/NonPersistentSubscription.java | 3 +-- .../broker/service/persistent/PersistentSubscription.java | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java index 92aba6221da73..cfe05cc32b77d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java @@ -40,7 +40,6 @@ import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.GetStatsOptions; -import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; @@ -53,7 +52,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class NonPersistentSubscription extends AbstractSubscription implements Subscription { +public class NonPersistentSubscription extends AbstractSubscription { private final NonPersistentTopic topic; private volatile NonPersistentDispatcher dispatcher; private final String topicName; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 50e84310ac183..6e8e94baeae23 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -97,7 +97,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class PersistentSubscription extends AbstractSubscription implements Subscription { +public class PersistentSubscription extends AbstractSubscription { protected final PersistentTopic topic; protected final ManagedCursor cursor; protected volatile Dispatcher dispatcher; From d8903da3d5ea5bab207d119186f2be6fa1147f60 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Thu, 28 Mar 2024 23:14:19 +0800 Subject: [PATCH 023/580] [fix][broker] Fix issue of field 'topic' is not set when handle GetSchema request (#22377) --- .../pulsar/broker/service/ServerCnx.java | 5 +-- .../org/apache/pulsar/schema/SchemaTest.java | 33 +++++++++++++++++++ 2 files changed, 36 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 4f82f416ed2a5..716f3a1a04c25 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 @@ -2504,9 +2504,10 @@ remoteAddress, new String(commandGetSchema.getSchemaVersion()), schemaVersion = schemaService.versionFromBytes(commandGetSchema.getSchemaVersion()); } + final String topic = commandGetSchema.getTopic(); String schemaName; try { - schemaName = TopicName.get(commandGetSchema.getTopic()).getSchemaName(); + schemaName = TopicName.get(topic).getSchemaName(); } catch (Throwable t) { commandSender.sendGetSchemaErrorResponse(requestId, ServerError.InvalidTopicName, t.getMessage()); return; @@ -2515,7 +2516,7 @@ remoteAddress, new String(commandGetSchema.getSchemaVersion()), schemaService.getSchema(schemaName, schemaVersion).thenAccept(schemaAndMetadata -> { if (schemaAndMetadata == null) { commandSender.sendGetSchemaErrorResponse(requestId, ServerError.TopicNotFound, - String.format("Topic not found or no-schema %s", commandGetSchema.getTopic())); + String.format("Topic not found or no-schema %s", topic)); } else { commandSender.sendGetSchemaResponse(requestId, SchemaInfoUtil.newSchemaInfo(schemaName, schemaAndMetadata.schema), schemaAndMetadata.version); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java index d4ef041f6dea6..aa47c378fc38c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java @@ -46,6 +46,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; import java.util.stream.Collectors; import lombok.Cleanup; import lombok.EqualsAndHashCode; @@ -69,6 +70,8 @@ import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.client.api.schema.SchemaDefinition; +import org.apache.pulsar.client.impl.ConsumerImpl; +import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; import org.apache.pulsar.client.impl.schema.KeyValueSchemaImpl; import org.apache.pulsar.client.impl.schema.ProtobufSchema; import org.apache.pulsar.client.impl.schema.SchemaInfoImpl; @@ -98,6 +101,7 @@ public class SchemaTest extends MockedPulsarServiceBaseTest { @BeforeMethod @Override public void setup() throws Exception { + isTcpLookup = true; super.internalSetup(); // Setup namespaces @@ -106,6 +110,7 @@ public void setup() throws Exception { .allowedClusters(Collections.singleton(CLUSTER_NAME)) .build(); admin.tenants().createTenant(PUBLIC_TENANT, tenantInfo); + admin.namespaces().createNamespace(PUBLIC_TENANT + "/my-ns"); } @AfterMethod(alwaysRun = true) @@ -130,6 +135,34 @@ public void testGetSchemaWhenCreateAutoProduceBytesProducer() throws Exception{ pulsarClient.newProducer(org.apache.pulsar.client.api.Schema.AUTO_PRODUCE_BYTES()).topic(topic).create(); } + @Test + public void testGetSchemaWithPatternTopic() throws Exception { + final String topicPrefix = "persistent://public/my-ns/test-getSchema"; + + int topicNums = 10; + for (int i = 0; i < topicNums; i++) { + String topic = topicPrefix + "-" + i; + admin.topics().createNonPartitionedTopic(topic); + } + + Pattern pattern = Pattern.compile(topicPrefix + "-.*"); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.AUTO_CONSUME()) + .topicsPattern(pattern) + .subscriptionName("sub") + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + List> consumers = + ((MultiTopicsConsumerImpl) consumer).getConsumers(); + Assert.assertEquals(topicNums, consumers.size()); + + for (int i = 0; i < topicNums; i++) { + String topic = topicPrefix + "-" + i; + admin.topics().delete(topic, true); + } + } + @Test public void testMultiTopicSetSchemaProvider() throws Exception { final String tenant = PUBLIC_TENANT; From 6b2938223cf45a9298f9d40ab6ae108bea9a5a6d Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 28 Mar 2024 09:58:53 -0700 Subject: [PATCH 024/580] [improve] PIP-342: OTel client metrics support (#22179) --- .../shell/src/assemble/LICENSE.bin.txt | 4 + pulsar-broker/pom.xml | 6 + .../broker/service/BrokerServiceTest.java | 4 +- .../service/BrokerServiceThrottlingTest.java | 3 +- .../service/EnableProxyProtocolTest.java | 5 +- .../api/InjectedClientCnxClientBuilder.java | 3 +- ...ListenersWithInternalListenerNameTest.java | 5 +- .../client/impl/ConnectionPoolTest.java | 18 +- .../client/impl/MessageChecksumTest.java | 3 +- .../client/impl/MessageChunkingTest.java | 3 +- .../impl/PatternTopicsConsumerImplTest.java | 3 +- .../pulsar/client/impl/PulsarTestClient.java | 5 +- .../client/metrics/ClientMetricsTest.java | 336 +++++++++++++ pulsar-client-api/pom.xml | 6 + .../pulsar/client/api/ClientBuilder.java | 22 + .../pulsar/client/api/ConsumerStats.java | 6 +- .../pulsar/client/api/ProducerStats.java | 6 +- pulsar-client/pom.xml | 10 + .../impl/BatchMessageContainerImpl.java | 6 +- .../client/impl/BinaryProtoLookupService.java | 34 ++ .../pulsar/client/impl/ClientBuilderImpl.java | 7 + .../apache/pulsar/client/impl/ClientCnx.java | 25 +- .../pulsar/client/impl/ConnectionPool.java | 29 +- .../pulsar/client/impl/ConsumerImpl.java | 58 +++ .../pulsar/client/impl/HttpLookupService.java | 66 ++- .../pulsar/client/impl/ProducerImpl.java | 475 ++++++++++-------- .../pulsar/client/impl/PulsarClientImpl.java | 17 +- .../client/impl/UnAckedMessageTracker.java | 18 +- .../impl/conf/ClientConfigurationData.java | 3 + .../pulsar/client/impl/metrics/Counter.java | 60 +++ .../impl/metrics/InstrumentProvider.java | 58 +++ .../client/impl/metrics/LatencyHistogram.java | 110 ++++ .../client/impl/metrics/MetricsUtil.java | 59 +++ .../pulsar/client/impl/metrics/Unit.java | 59 +++ .../client/impl/metrics/UpDownCounter.java | 68 +++ .../client/impl/metrics/package-info.java | 23 + .../AcknowledgementsGroupingTrackerTest.java | 5 +- .../impl/BinaryProtoLookupServiceTest.java | 2 + .../ClientCnxRequestTimeoutQueueTest.java | 3 +- .../pulsar/client/impl/ClientCnxTest.java | 19 +- .../client/impl/OpSendMsgQueueTest.java | 3 +- .../impl/PartitionedProducerImplTest.java | 2 + .../pulsar/client/impl/ProducerImplTest.java | 2 + .../client/impl/PulsarClientImplTest.java | 5 +- .../impl/UnAckedMessageTrackerTest.java | 3 + .../conf/ClientConfigurationDataTest.java | 32 ++ .../pulsar/proxy/server/ProxyClientCnx.java | 3 +- .../pulsar/proxy/server/ProxyConnection.java | 6 +- .../pulsar/proxy/server/ProxyParserTest.java | 5 +- .../apache/pulsar/proxy/server/ProxyTest.java | 5 +- pulsar-testclient/pom.xml | 17 + .../pulsar/testclient/PerfClientUtils.java | 5 +- 52 files changed, 1476 insertions(+), 264 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/MetricsUtil.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Unit.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/package-info.java diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 7e3ebbe06358d..9042257f34c67 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -387,6 +387,10 @@ The Apache Software License, Version 2.0 - log4j-core-2.18.0.jar - log4j-slf4j-impl-2.18.0.jar - log4j-web-2.18.0.jar + * OpenTelemetry + - opentelemetry-api-1.34.1.jar + - opentelemetry-context-1.34.1.jar + - opentelemetry-extension-incubator-1.34.1-alpha.jar * BookKeeper - bookkeeper-common-allocator-4.16.4.jar diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 8264459c6d9ab..e15e024ea8158 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -149,6 +149,12 @@ ${project.version} + + io.opentelemetry + opentelemetry-sdk-testing + test + + ${project.groupId} pulsar-io-batch-discovery-triggerers diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index b6a73274f440b..8ebba5c9aeabd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -99,6 +99,7 @@ import org.apache.pulsar.client.impl.PulsarServiceNameResolver; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse; import org.apache.pulsar.common.api.proto.CommandPartitionedTopicMetadataResponse; import org.apache.pulsar.common.naming.NamespaceBundle; @@ -997,7 +998,8 @@ public void testLookupThrottlingForClientByClient() throws Exception { // Using an AtomicReference in order to reset a new CountDownLatch AtomicReference latchRef = new AtomicReference<>(); latchRef.set(new CountDownLatch(1)); - try (ConnectionPool pool = new ConnectionPool(conf, eventLoop, () -> new ClientCnx(conf, eventLoop) { + try (ConnectionPool pool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoop, + () -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop) { @Override protected void handleLookupResponse(CommandLookupTopicResponse lookupResult) { try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java index 707c350feb59c..0d517c014b315 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java @@ -47,6 +47,7 @@ import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarServiceNameResolver; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.testng.annotations.AfterMethod; @@ -197,7 +198,7 @@ public void testLookupThrottlingForClientByBroker() throws Exception { EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(20, false, new DefaultThreadFactory("test-pool", Thread.currentThread().isDaemon())); ExecutorService executor = Executors.newFixedThreadPool(10); - try (ConnectionPool pool = new ConnectionPool(conf, eventLoop)) { + try (ConnectionPool pool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoop)) { final int totalConsumers = 20; List> futures = new ArrayList<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java index a596e1ed32d6b..725b895fe6e14 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.impl.ClientBuilderImpl; import org.apache.pulsar.client.impl.ClientCnx; import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TopicStats; import org.awaitility.Awaitility; @@ -99,7 +100,7 @@ public void testProxyProtocol() throws Exception { ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); @Cleanup PulsarClientImpl protocolClient = InjectedClientCnxClientBuilder.create(clientBuilder, - (conf, eventLoopGroup) -> new ClientCnx(conf, eventLoopGroup) { + (conf, eventLoopGroup) -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup) { public void channelActive(ChannelHandlerContext ctx) throws Exception { byte[] bs = "PROXY TCP4 198.51.100.22 203.0.113.7 35646 80\r\n".getBytes(); ctx.writeAndFlush(Unpooled.copiedBuffer(bs)); @@ -124,7 +125,7 @@ public void testPubSubWhenSlowNetwork() throws Exception { ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); @Cleanup PulsarClientImpl protocolClient = InjectedClientCnxClientBuilder.create(clientBuilder, - (conf, eventLoopGroup) -> new ClientCnx(conf, eventLoopGroup) { + (conf, eventLoopGroup) -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup) { public void channelActive(ChannelHandlerContext ctx) throws Exception { Thread task = new Thread(() -> { try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java index 2a7908242707b..13447e089eab8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java @@ -28,6 +28,7 @@ import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.util.ExecutorProvider; import org.apache.pulsar.common.util.netty.EventLoopUtil; @@ -42,7 +43,7 @@ public static PulsarClientImpl create(final ClientBuilderImpl clientBuilder, EventLoopUtil.newEventLoopGroup(conf.getNumIoThreads(), conf.isEnableBusyWait(), threadFactory); // Inject into ClientCnx. - ConnectionPool pool = new ConnectionPool(conf, eventLoopGroup, + ConnectionPool pool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoopGroup, () -> clientCnxFactory.generate(conf, eventLoopGroup)); return new InjectedClientCnxPulsarClientImpl(conf, eventLoopGroup, pool); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java index 956b834e33435..a076e20b33218 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PulsarMultiListenersWithInternalListenerNameTest.java @@ -44,6 +44,7 @@ import org.apache.pulsar.client.impl.LookupService; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfo; @@ -137,7 +138,7 @@ private void doFindBrokerWithListenerName(boolean useHttp) throws Exception { conf.setMaxLookupRedirects(10); @Cleanup - LookupService lookupService = useHttp ? new HttpLookupService(conf, eventExecutors) : + LookupService lookupService = useHttp ? new HttpLookupService(InstrumentProvider.NOOP, conf, eventExecutors) : new BinaryProtoLookupService((PulsarClientImpl) this.pulsarClient, lookupUrl.toString(), "internal", false, this.executorService); TopicName topicName = TopicName.get("persistent://public/default/test"); @@ -172,7 +173,7 @@ public void testHttpLookupRedirect() throws Exception { conf.setMaxLookupRedirects(10); @Cleanup - HttpLookupService lookupService = new HttpLookupService(conf, eventExecutors); + HttpLookupService lookupService = new HttpLookupService(InstrumentProvider.NOOP, conf, eventExecutors); NamespaceService namespaceService = pulsar.getNamespaceService(); LookupResult lookupResult = new LookupResult(pulsar.getWebServiceAddress(), null, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java index 79ffada4a90c8..1037019d608ab 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java @@ -35,6 +35,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.CommandCloseProducer; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.awaitility.Awaitility; @@ -68,7 +69,8 @@ protected void cleanup() throws Exception { public void testSingleIpAddress() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, new DefaultThreadFactory("test")); - ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, conf, eventLoop); + ConnectionPool pool = + spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop); conf.setServiceUrl(serviceUrl); PulsarClientImpl client = new PulsarClientImpl(conf, eventLoop, pool); @@ -118,7 +120,7 @@ public void testSelectConnectionForSameProducer() throws Exception { public void testDoubleIpAddress() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, new DefaultThreadFactory("test")); - ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, conf, eventLoop); + ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop); conf.setServiceUrl(serviceUrl); PulsarClientImpl client = new PulsarClientImpl(conf, eventLoop, pool); @@ -143,7 +145,8 @@ public void testNoConnectionPool() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setConnectionsPerBroker(0); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(8, false, new DefaultThreadFactory("test")); - ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, conf, eventLoop); + ConnectionPool pool = + spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop); InetSocketAddress brokerAddress = InetSocketAddress.createUnresolved("127.0.0.1", brokerPort); @@ -166,7 +169,8 @@ public void testEnableConnectionPool() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setConnectionsPerBroker(5); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(8, false, new DefaultThreadFactory("test")); - ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, conf, eventLoop); + ConnectionPool pool = + spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop); InetSocketAddress brokerAddress = InetSocketAddress.createUnresolved("127.0.0.1", brokerPort); @@ -233,8 +237,10 @@ protected void doResolveAll(SocketAddress socketAddress, Promise promise) throws } }; - ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, conf, eventLoop, - (Supplier) () -> new ClientCnx(conf, eventLoop), Optional.of(resolver)); + ConnectionPool pool = + spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop, + (Supplier) () -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop), + Optional.of(resolver)); ClientCnx cnx = pool.getConnection( diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java index 515b34db8509d..0b25e3409563a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java @@ -37,6 +37,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.api.proto.ProtocolVersion; import org.apache.pulsar.common.protocol.ByteBufPair; @@ -233,7 +234,7 @@ public void testTamperingMessageIsDetected() throws Exception { // WHEN // protocol message is created with checksum ByteBufPair cmd = Commands.newSend(1, 1, 1, ChecksumType.Crc32c, msgMetadata, payload); - OpSendMsg op = OpSendMsg.create((MessageImpl) msgBuilder.getMessage(), cmd, 1, null); + OpSendMsg op = OpSendMsg.create(LatencyHistogram.NOOP, (MessageImpl) msgBuilder.getMessage(), cmd, 1, null); // THEN // the checksum validation passes diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java index 6686edd2b67d2..da359a6aeb9c5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java @@ -56,6 +56,7 @@ import org.apache.pulsar.client.api.SizeUnit; import org.apache.pulsar.client.impl.MessageImpl.SchemaState; import org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.policies.data.PublisherStats; import org.apache.pulsar.common.protocol.ByteBufPair; @@ -499,7 +500,7 @@ public void testExpireIncompleteChunkMessage() throws Exception{ ByteBufPair cmd = Commands.newSend(producerId, 1, 1, ChecksumType.Crc32c, msgMetadata, payload); MessageImpl msgImpl = ((MessageImpl) msg.getMessage()); msgImpl.setSchemaState(SchemaState.Ready); - OpSendMsg op = OpSendMsg.create(msgImpl, cmd, 1, null); + OpSendMsg op = OpSendMsg.create(LatencyHistogram.NOOP, msgImpl, cmd, 1, null); producer.processOpSendMsg(op); retryStrategically((test) -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java index 7707abafde8de..94d78e418ab87 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java @@ -49,6 +49,7 @@ import org.apache.pulsar.client.api.RegexSubscriptionMode; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.BaseCommand; import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; import org.apache.pulsar.common.naming.NamespaceName; @@ -811,7 +812,7 @@ public void testPreciseRegexpSubscribeDisabledTopicWatcher(boolean partitioned) private PulsarClient createDelayWatchTopicsClient() throws Exception { ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); return InjectedClientCnxClientBuilder.create(clientBuilder, - (conf, eventLoopGroup) -> new ClientCnx(conf, eventLoopGroup) { + (conf, eventLoopGroup) -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup) { public CompletableFuture newWatchTopicList( BaseCommand command, long requestId) { // Inject 2 seconds delay when sending command New Watch Topics. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java index ab273913fde29..8a79eb502439f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java @@ -35,6 +35,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.awaitility.Awaitility; @@ -79,7 +80,7 @@ public static PulsarTestClient create(ClientBuilder clientBuilder) throws Pulsar new DefaultThreadFactory("pulsar-test-client-io", Thread.currentThread().isDaemon())); AtomicReference> clientCnxSupplierReference = new AtomicReference<>(); - ConnectionPool connectionPool = new ConnectionPool(clientConfigurationData, eventLoopGroup, + ConnectionPool connectionPool = new ConnectionPool(InstrumentProvider.NOOP, clientConfigurationData, eventLoopGroup, () -> clientCnxSupplierReference.get().get()); return new PulsarTestClient(clientConfigurationData, eventLoopGroup, connectionPool, @@ -101,7 +102,7 @@ private PulsarTestClient(ClientConfigurationData conf, EventLoopGroup eventLoopG * @return new ClientCnx instance */ protected ClientCnx createClientCnx() { - return new ClientCnx(conf, eventLoopGroup) { + return new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup) { @Override public int getRemoteEndpointProtocolVersion() { return overrideRemoteEndpointProtocolVersion != 0 diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java new file mode 100644 index 0000000000000..31305123c4148 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/metrics/ClientMetricsTest.java @@ -0,0 +1,336 @@ +/* + * 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.metrics; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.fail; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.sdk.OpenTelemetrySdk; +import io.opentelemetry.sdk.metrics.SdkMeterProvider; +import io.opentelemetry.sdk.metrics.data.MetricData; +import io.opentelemetry.sdk.metrics.data.MetricDataType; +import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.assertj.core.api.Assertions; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker-api") +public class ClientMetricsTest extends ProducerConsumerBase { + + InMemoryMetricReader reader; + OpenTelemetry otel; + + @BeforeMethod + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + + this.reader = InMemoryMetricReader.create(); + SdkMeterProvider sdkMeterProvider = SdkMeterProvider.builder() + .registerMetricReader(reader) + .build(); + this.otel = OpenTelemetrySdk.builder().setMeterProvider(sdkMeterProvider).build(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + private Map collectMetrics() { + Map metrics = new TreeMap<>(); + for (MetricData md : reader.collectAllMetrics()) { + metrics.put(md.getName(), md); + } + return metrics; + } + + private void assertCounterValue(Map metrics, String name, long expectedValue, + Attributes expectedAttributes) { + assertEquals(getCounterValue(metrics, name, expectedAttributes), expectedValue); + } + + private long getCounterValue(Map metrics, String name, + Attributes expectedAttributes) { + MetricData md = metrics.get(name); + assertNotNull(md, "metric not found: " + name); + assertEquals(md.getType(), MetricDataType.LONG_SUM); + + for (var ex : md.getLongSumData().getPoints()) { + if (ex.getAttributes().equals(expectedAttributes)) { + return ex.getValue(); + } + } + + fail("metric attributes not found: " + expectedAttributes); + return -1; + } + + private void assertHistoCountValue(Map metrics, String name, long expectedCount, + Attributes expectedAttributes) { + assertEquals(getHistoCountValue(metrics, name, expectedAttributes), expectedCount); + } + + private long getHistoCountValue(Map metrics, String name, + Attributes expectedAttributes) { + MetricData md = metrics.get(name); + assertNotNull(md, "metric not found: " + name); + assertEquals(md.getType(), MetricDataType.HISTOGRAM); + + for (var ex : md.getHistogramData().getPoints()) { + if (ex.getAttributes().equals(expectedAttributes)) { + return ex.getCount(); + } + } + + fail("metric attributes not found: " + expectedAttributes); + return -1; + } + + @Test + public void testProducerMetrics() throws Exception { + String topic = newTopicName(); + + PulsarClient client = PulsarClient.builder() + .serviceUrl(pulsar.getBrokerServiceUrl()) + .openTelemetry(otel) + .build(); + + Producer producer = client.newProducer(Schema.STRING) + .topic(topic) + .create(); + + for (int i = 0; i < 5; i++) { + producer.send("Hello"); + } + + Attributes nsAttrs = Attributes.builder() + .put("pulsar.tenant", "my-property") + .put("pulsar.namespace", "my-property/my-ns") + .build(); + Attributes nsAttrsSuccess = nsAttrs.toBuilder() + .put("pulsar.response.status", "success") + .build(); + + var metrics = collectMetrics(); + + assertCounterValue(metrics, "pulsar.client.connection.opened", 1, Attributes.empty()); + assertCounterValue(metrics, "pulsar.client.producer.message.pending.count", 0, nsAttrs); + assertCounterValue(metrics, "pulsar.client.producer.message.pending.size", 0, nsAttrs); + + assertHistoCountValue(metrics, "pulsar.client.lookup.duration", 1, + Attributes.builder() + .put("pulsar.lookup.transport-type", "binary") + .put("pulsar.lookup.type", "topic") + .put("pulsar.response.status", "success") + .build()); + assertHistoCountValue(metrics, "pulsar.client.lookup.duration", 1, + Attributes.builder() + .put("pulsar.lookup.transport-type", "binary") + .put("pulsar.lookup.type", "metadata") + .put("pulsar.response.status", "success") + .build()); + + assertHistoCountValue(metrics, "pulsar.client.producer.message.send.duration", 5, nsAttrsSuccess); + assertHistoCountValue(metrics, "pulsar.client.producer.rpc.send.duration", 5, nsAttrsSuccess); + assertCounterValue(metrics, "pulsar.client.producer.message.send.size", "hello".length() * 5, nsAttrs); + + + assertCounterValue(metrics, "pulsar.client.producer.opened", 1, nsAttrs); + + producer.close(); + client.close(); + + metrics = collectMetrics(); + assertCounterValue(metrics, "pulsar.client.producer.closed", 1, nsAttrs); + assertCounterValue(metrics, "pulsar.client.connection.closed", 1, Attributes.empty()); + } + + @Test + public void testConnectionsFailedMetrics() throws Exception { + String topic = newTopicName(); + + @Cleanup + PulsarClient client = PulsarClient.builder() + .serviceUrl("pulsar://invalid-pulsar-address:1234") + .operationTimeout(3, TimeUnit.SECONDS) + .openTelemetry(otel) + .build(); + + Assertions.assertThatThrownBy(() -> { + client.newProducer(Schema.STRING) + .topic(topic) + .create(); + }).isInstanceOf(Exception.class); + + + var metrics = collectMetrics(); + + Assertions.assertThat( + getCounterValue(metrics, "pulsar.client.connection.failed", + Attributes.builder().put("pulsar.failure.type", "tcp-failed").build())) + .isGreaterThanOrEqualTo(1L); + } + + @Test + public void testPublishFailedMetrics() throws Exception { + String topic = newTopicName(); + + @Cleanup + PulsarClient client = PulsarClient.builder() + .serviceUrl(admin.getServiceUrl()) + .operationTimeout(3, TimeUnit.SECONDS) + .openTelemetry(otel) + .build(); + + @Cleanup + Producer producer = client.newProducer(Schema.STRING) + .topic(topic) + .sendTimeout(3, TimeUnit.SECONDS) + .create(); + + // Make the client switch to non-existing broker to make publish fail + client.updateServiceUrl("pulsar://invalid-address:6650"); + + + try { + producer.send("Hello"); + fail("Should have failed to publish"); + } catch (Exception e) { + // expected + } + + var metrics = collectMetrics(); + + Attributes nsAttrs = Attributes.builder() + .put("pulsar.tenant", "my-property") + .put("pulsar.namespace", "my-property/my-ns") + .build(); + Attributes nsAttrsFailure = nsAttrs.toBuilder() + .put("pulsar.response.status", "failed") + .build(); + + assertCounterValue(metrics, "pulsar.client.producer.message.pending.count", 0, nsAttrs); + assertCounterValue(metrics, "pulsar.client.producer.message.pending.size", 0, nsAttrs); + assertHistoCountValue(metrics, "pulsar.client.producer.message.send.duration", 1, nsAttrsFailure); + assertHistoCountValue(metrics, "pulsar.client.producer.rpc.send.duration", 1, nsAttrsFailure); + } + + @Test + public void testConsumerMetrics() throws Exception { + String topic = newTopicName(); + + PulsarClient client = PulsarClient.builder() + .serviceUrl(pulsar.getBrokerServiceUrl()) + .openTelemetry(otel) + .build(); + + @Cleanup + Producer producer = client.newProducer(Schema.STRING) + .topic(topic) + .create(); + + Consumer consumer = client.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("my-sub") + .ackTimeout(1, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + for (int i = 0; i < 10; i++) { + producer.send("Hello"); + } + + Thread.sleep(1000); + + Attributes nsAttrs = Attributes.builder() + .put("pulsar.tenant", "my-property") + .put("pulsar.namespace", "my-property/my-ns") + .put("pulsar.subscription", "my-sub") + .build(); + var metrics = collectMetrics(); + + assertCounterValue(metrics, "pulsar.client.connection.opened", 1, Attributes.empty()); + + assertHistoCountValue(metrics, "pulsar.client.lookup.duration", 2, + Attributes.builder() + .put("pulsar.lookup.transport-type", "binary") + .put("pulsar.lookup.type", "topic") + .put("pulsar.response.status", "success") + .build()); + assertHistoCountValue(metrics, "pulsar.client.lookup.duration", 2, + Attributes.builder() + .put("pulsar.lookup.transport-type", "binary") + .put("pulsar.lookup.type", "metadata") + .put("pulsar.response.status", "success") + .build()); + + assertCounterValue(metrics, "pulsar.client.consumer.receive_queue.count", 10, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.receive_queue.size", "hello".length() * 10, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.opened", 1, nsAttrs); + + Message msg1 = consumer.receive(); + consumer.acknowledge(msg1); + + Message msg2 = consumer.receive(); + consumer.negativeAcknowledge(msg2); + + /* Message msg3 = */ consumer.receive(); + + metrics = collectMetrics(); + assertCounterValue(metrics, "pulsar.client.consumer.receive_queue.count", 7, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.receive_queue.size", "hello".length() * 7, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.message.received.count", 3, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.message.received.size", "hello".length() * 3, nsAttrs); + + + // Let msg3 to reach ack-timeout + Thread.sleep(3000); + + metrics = collectMetrics(); + assertCounterValue(metrics, "pulsar.client.consumer.receive_queue.count", 8, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.receive_queue.size", "hello".length() * 8, nsAttrs); + + assertCounterValue(metrics, "pulsar.client.consumer.message.ack", 1, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.message.nack", 1, nsAttrs); + assertCounterValue(metrics, "pulsar.client.consumer.message.ack.timeout", 1, nsAttrs); + + client.close(); + + metrics = collectMetrics(); + assertCounterValue(metrics, "pulsar.client.consumer.closed", 1, nsAttrs); + assertCounterValue(metrics, "pulsar.client.connection.closed", 1, Attributes.empty()); + } +} diff --git a/pulsar-client-api/pom.xml b/pulsar-client-api/pom.xml index d8b51713da832..35bdf73374b3e 100644 --- a/pulsar-client-api/pom.xml +++ b/pulsar-client-api/pom.xml @@ -46,6 +46,12 @@ protobuf-java provided + + + io.opentelemetry + opentelemetry-api + provided + diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java index b180f6ba7f906..735aeeed55916 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import io.opentelemetry.api.OpenTelemetry; import java.io.Serializable; import java.net.InetSocketAddress; import java.time.Clock; @@ -459,7 +460,10 @@ ClientBuilder authentication(String authPluginClassName, Map aut * @param unit * time unit for {@code statsInterval} * @return the client builder instance + * + * @deprecated @see {@link #openTelemetry(OpenTelemetry)} */ + @Deprecated ClientBuilder statsInterval(long statsInterval, TimeUnit unit); /** @@ -554,6 +558,24 @@ ClientBuilder authentication(String authPluginClassName, Map aut */ ClientBuilder enableBusyWait(boolean enableBusyWait); + /** + * Configure OpenTelemetry for Pulsar Client + *

+ * When you pass an OpenTelemetry instance, Pulsar client will emit metrics that can be exported in a variety + * of different methods. + *

+ * Refer to OpenTelemetry Java SDK documentation for + * how to configure OpenTelemetry and the metrics exporter. + *

+ * By default, Pulsar client will use the {@link io.opentelemetry.api.GlobalOpenTelemetry} instance. If an + * OpenTelemetry JVM agent is configured, the metrics will be reported, otherwise the metrics will be + * completely disabled. + * + * @param openTelemetry the OpenTelemetry instance + * @return the client builder instance + */ + ClientBuilder openTelemetry(io.opentelemetry.api.OpenTelemetry openTelemetry); + /** * The clock used by the pulsar client. * diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerStats.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerStats.java index 7935e05d55b66..e488aa81151ce 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerStats.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerStats.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import io.opentelemetry.api.OpenTelemetry; import java.io.Serializable; import java.util.Collections; import java.util.Map; @@ -29,9 +30,12 @@ * *

All the stats are relative to the last recording period. The interval of the stats refreshes is configured with * {@link ClientBuilder#statsInterval(long, java.util.concurrent.TimeUnit)} with a default of 1 minute. + * + * @deprecated use {@link ClientBuilder#openTelemetry(OpenTelemetry)} to enable stats */ @InterfaceAudience.Public -@InterfaceStability.Stable +@InterfaceStability.Evolving +@Deprecated public interface ConsumerStats extends Serializable { /** diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ProducerStats.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ProducerStats.java index a26c20e740d37..9a9ade73669dd 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ProducerStats.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ProducerStats.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import io.opentelemetry.api.OpenTelemetry; import java.io.Serializable; import java.util.Collections; import java.util.Map; @@ -29,9 +30,12 @@ * *

All the stats are relative to the last recording period. The interval of the stats refreshes is configured with * {@link ClientBuilder#statsInterval(long, java.util.concurrent.TimeUnit)} with a default of 1 minute. + * + * @deprecated use {@link ClientBuilder#openTelemetry(OpenTelemetry)} to enable stats */ @InterfaceAudience.Public -@InterfaceStability.Stable +@InterfaceStability.Evolving +@Deprecated public interface ProducerStats extends Serializable { /** * @return the number of messages published in the last interval diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index 7424b12db5aa2..3917e2996e180 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -52,6 +52,16 @@ pkg + + io.opentelemetry + opentelemetry-api + + + + io.opentelemetry + opentelemetry-extension-incubator + + ${project.groupId} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java index fc5c3a3c6798b..a3c9d1bc9ab48 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java @@ -263,8 +263,8 @@ public OpSendMsg createOpSendMsg() throws IOException { // Because when invoke `ProducerImpl.processOpSendMsg` on flush, // if `op.msg != null && isBatchMessagingEnabled()` checks true, it will call `batchMessageAndSend` to flush // messageContainers before publishing this one-batch message. - op = OpSendMsg.create(messages, cmd, messageMetadata.getSequenceId(), firstCallback, - batchAllocatedSizeBytes); + op = OpSendMsg.create(producer.rpcLatencyHistogram, messages, cmd, messageMetadata.getSequenceId(), + firstCallback, batchAllocatedSizeBytes); // NumMessagesInBatch and BatchSizeByte will not be serialized to the binary cmd. It's just useful for the // ProducerStats @@ -314,7 +314,7 @@ public OpSendMsg createOpSendMsg() throws IOException { messageMetadata.getUncompressedSize(), encryptedPayload.readableBytes()); } - OpSendMsg op = OpSendMsg.create(messages, cmd, messageMetadata.getSequenceId(), + OpSendMsg op = OpSendMsg.create(producer.rpcLatencyHistogram, messages, cmd, messageMetadata.getSequenceId(), messageMetadata.getHighestSequenceId(), firstCallback, batchAllocatedSizeBytes); op.setNumMessagesInBatch(numMessagesInBatch); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index bdf00844c1cd2..81c196c731f70 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -20,6 +20,7 @@ import static java.lang.String.format; import io.netty.buffer.ByteBuf; +import io.opentelemetry.api.common.Attributes; import java.net.InetSocketAddress; import java.net.URI; import java.util.ArrayList; @@ -34,6 +35,7 @@ import org.apache.commons.lang3.mutable.MutableObject; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SchemaSerializationException; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace.Mode; import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse; import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse.LookupType; @@ -63,6 +65,11 @@ public class BinaryProtoLookupService implements LookupService { private final ConcurrentHashMap> partitionedMetadataInProgress = new ConcurrentHashMap<>(); + private final LatencyHistogram histoGetBroker; + private final LatencyHistogram histoGetTopicMetadata; + private final LatencyHistogram histoGetSchema; + private final LatencyHistogram histoListTopics; + public BinaryProtoLookupService(PulsarClientImpl client, String serviceUrl, boolean useTls, @@ -84,6 +91,15 @@ public BinaryProtoLookupService(PulsarClientImpl client, this.serviceNameResolver = new PulsarServiceNameResolver(); this.listenerName = listenerName; updateServiceUrl(serviceUrl); + + LatencyHistogram histo = client.instrumentProvider().newLatencyHistogram("pulsar.client.lookup.duration", + "Duration of lookup operations", null, + Attributes.builder().put("pulsar.lookup.transport-type", "binary").build()); + histoGetBroker = histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "topic").build()); + histoGetTopicMetadata = + histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "metadata").build()); + histoGetSchema = histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "schema").build()); + histoListTopics = histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "list-topics").build()); } @Override @@ -99,12 +115,20 @@ public void updateServiceUrl(String serviceUrl) throws PulsarClientException { * @return broker-socket-address that serves given topic */ public CompletableFuture getBroker(TopicName topicName) { + long startTime = System.nanoTime(); final MutableObject newFutureCreated = new MutableObject<>(); try { return lookupInProgress.computeIfAbsent(topicName, tpName -> { CompletableFuture newFuture = findBroker(serviceNameResolver.resolveHost(), false, topicName, 0); newFutureCreated.setValue(newFuture); + + newFuture.thenRun(() -> { + histoGetBroker.recordSuccess(System.nanoTime() - startTime); + }).exceptionally(x -> { + histoGetBroker.recordFailure(System.nanoTime() - startTime); + return null; + }); return newFuture; }); } finally { @@ -224,6 +248,7 @@ private CompletableFuture findBroker(InetSocketAddress socket private CompletableFuture getPartitionedTopicMetadata(InetSocketAddress socketAddress, TopicName topicName) { + long startTime = System.nanoTime(); CompletableFuture partitionFuture = new CompletableFuture<>(); client.getCnxPool().getConnection(socketAddress).thenAccept(clientCnx -> { @@ -231,11 +256,13 @@ private CompletableFuture getPartitionedTopicMetadata( ByteBuf request = Commands.newPartitionMetadataRequest(topicName.toString(), requestId); clientCnx.newLookup(request, requestId).whenComplete((r, t) -> { if (t != null) { + histoGetTopicMetadata.recordFailure(System.nanoTime() - startTime); log.warn("[{}] failed to get Partitioned metadata : {}", topicName, t.getMessage(), t); partitionFuture.completeExceptionally(t); } else { try { + histoGetTopicMetadata.recordSuccess(System.nanoTime() - startTime); partitionFuture.complete(new PartitionedTopicMetadata(r.partitions)); } catch (Exception e) { partitionFuture.completeExceptionally(new PulsarClientException.LookupException( @@ -263,6 +290,7 @@ public CompletableFuture> getSchema(TopicName topicName) { @Override public CompletableFuture> getSchema(TopicName topicName, byte[] version) { + long startTime = System.nanoTime(); CompletableFuture> schemaFuture = new CompletableFuture<>(); if (version != null && version.length == 0) { schemaFuture.completeExceptionally(new SchemaSerializationException("Empty schema version")); @@ -275,10 +303,12 @@ public CompletableFuture> getSchema(TopicName topicName, by Optional.ofNullable(BytesSchemaVersion.of(version))); clientCnx.sendGetSchema(request, requestId).whenComplete((r, t) -> { if (t != null) { + histoGetSchema.recordFailure(System.nanoTime() - startTime); log.warn("[{}] failed to get schema : {}", topicName, t.getMessage(), t); schemaFuture.completeExceptionally(t); } else { + histoGetSchema.recordSuccess(System.nanoTime() - startTime); schemaFuture.complete(r); } client.getCnxPool().releaseConnection(clientCnx); @@ -326,6 +356,8 @@ private void getTopicsUnderNamespace(InetSocketAddress socketAddress, Mode mode, String topicsPattern, String topicsHash) { + long startTime = System.nanoTime(); + client.getCnxPool().getConnection(socketAddress).thenAccept(clientCnx -> { long requestId = client.newRequestId(); ByteBuf request = Commands.newGetTopicsOfNamespaceRequest( @@ -333,8 +365,10 @@ private void getTopicsUnderNamespace(InetSocketAddress socketAddress, clientCnx.newGetTopicsOfNamespace(request, requestId).whenComplete((r, t) -> { if (t != null) { + histoListTopics.recordFailure(System.nanoTime() - startTime); getTopicsResultFuture.completeExceptionally(t); } else { + histoListTopics.recordSuccess(System.nanoTime() - startTime); if (log.isDebugEnabled()) { log.debug("[namespace: {}] Success get topics list in request: {}", namespace, requestId); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java index 9a86d81c93fab..2548a52aa95a8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import static com.google.common.base.Preconditions.checkArgument; +import io.opentelemetry.api.OpenTelemetry; import java.net.InetSocketAddress; import java.time.Clock; import java.util.List; @@ -121,6 +122,12 @@ public ClientBuilder authentication(Authentication authentication) { return this; } + @Override + public ClientBuilder openTelemetry(OpenTelemetry openTelemetry) { + conf.setOpenTelemetry(openTelemetry); + return this; + } + @Override public ClientBuilder authentication(String authPluginClassName, String authParamsString) throws UnsupportedAuthenticationException { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index 938a0b4d8f683..03e0f406dd2f2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -32,6 +32,7 @@ import io.netty.handler.codec.LengthFieldBasedFrameDecoder; import io.netty.handler.ssl.SslHandshakeCompletionEvent; import io.netty.util.concurrent.Promise; +import io.opentelemetry.api.common.Attributes; import java.net.InetSocketAddress; import java.net.SocketAddress; import java.net.URI; @@ -60,6 +61,9 @@ import org.apache.pulsar.client.api.PulsarClientException.TimeoutException; import org.apache.pulsar.client.impl.BinaryProtoLookupService.LookupDataResult; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.Counter; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.client.impl.metrics.Unit; import org.apache.pulsar.client.impl.schema.SchemaInfoUtil; import org.apache.pulsar.client.impl.transaction.TransactionBufferHandler; import org.apache.pulsar.client.util.TimedCompletableFuture; @@ -201,6 +205,9 @@ protected enum State { None, SentConnectFrame, Ready, Failed, Connecting } + private final Counter connectionsOpenedCounter; + private final Counter connectionsClosedCounter; + private static class RequestTime { private final long creationTimeNanos; final long requestId; @@ -236,12 +243,13 @@ String getDescription() { } } - - public ClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) { - this(conf, eventLoopGroup, Commands.getCurrentProtocolVersion()); + public ClientCnx(InstrumentProvider instrumentProvider, + ClientConfigurationData conf, EventLoopGroup eventLoopGroup) { + this(instrumentProvider, conf, eventLoopGroup, Commands.getCurrentProtocolVersion()); } - public ClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, int protocolVersion) { + public ClientCnx(InstrumentProvider instrumentProvider, ClientConfigurationData conf, EventLoopGroup eventLoopGroup, + int protocolVersion) { super(conf.getKeepAliveIntervalSeconds(), TimeUnit.SECONDS); checkArgument(conf.getMaxLookupRequest() > conf.getConcurrentLookupRequest()); this.pendingLookupRequestSemaphore = new Semaphore(conf.getConcurrentLookupRequest(), false); @@ -257,11 +265,19 @@ public ClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, in this.idleState = new ClientCnxIdleState(this); this.clientVersion = "Pulsar-Java-v" + PulsarVersion.getVersion() + (conf.getDescription() == null ? "" : ("-" + conf.getDescription())); + this.connectionsOpenedCounter = + instrumentProvider.newCounter("pulsar.client.connection.opened", Unit.Connections, + "The number of connections opened", null, Attributes.empty()); + this.connectionsClosedCounter = + instrumentProvider.newCounter("pulsar.client.connection.closed", Unit.Connections, + "The number of connections closed", null, Attributes.empty()); + } @Override public void channelActive(ChannelHandlerContext ctx) throws Exception { super.channelActive(ctx); + connectionsOpenedCounter.increment(); this.localAddress = ctx.channel().localAddress(); this.remoteAddress = ctx.channel().remoteAddress(); @@ -304,6 +320,7 @@ protected ByteBuf newConnectCommand() throws Exception { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { super.channelInactive(ctx); + connectionsClosedCounter.increment(); lastDisconnectedTimestamp = System.currentTimeMillis(); log.info("{} Disconnected", ctx.channel()); if (!connectionFuture.isDone()) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java index 850e805067d12..d5adbdd7098ed 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java @@ -32,6 +32,7 @@ import io.netty.resolver.dns.SequentialDnsServerAddressStreamProvider; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.ScheduledFuture; +import io.opentelemetry.api.common.Attributes; import java.net.InetSocketAddress; import java.net.URI; import java.net.URISyntaxException; @@ -54,6 +55,9 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.InvalidServiceURL; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.Counter; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.client.impl.metrics.Unit; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.netty.DnsResolverUtil; @@ -88,6 +92,8 @@ public class ConnectionPool implements AutoCloseable { /** Async release useless connections task. **/ private ScheduledFuture asyncReleaseUselessConnectionsTask; + private final Counter connectionsTcpFailureCounter; + private final Counter connectionsHandshakeFailureCounter; @Value private static class Key { @@ -96,16 +102,19 @@ private static class Key { int randomKey; } - public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) throws PulsarClientException { - this(conf, eventLoopGroup, () -> new ClientCnx(conf, eventLoopGroup)); + public ConnectionPool(InstrumentProvider instrumentProvider, + ClientConfigurationData conf, EventLoopGroup eventLoopGroup) throws PulsarClientException { + this(instrumentProvider, conf, eventLoopGroup, () -> new ClientCnx(instrumentProvider, conf, eventLoopGroup)); } - public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, + public ConnectionPool(InstrumentProvider instrumentProvider, + ClientConfigurationData conf, EventLoopGroup eventLoopGroup, Supplier clientCnxSupplier) throws PulsarClientException { - this(conf, eventLoopGroup, clientCnxSupplier, Optional.empty()); + this(instrumentProvider, conf, eventLoopGroup, clientCnxSupplier, Optional.empty()); } - public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, + public ConnectionPool(InstrumentProvider instrumentProvider, + ClientConfigurationData conf, EventLoopGroup eventLoopGroup, Supplier clientCnxSupplier, Optional> addressResolver) throws PulsarClientException { @@ -155,6 +164,14 @@ public ConnectionPool(ClientConfigurationData conf, EventLoopGroup eventLoopGrou } }, idleDetectionIntervalSeconds, idleDetectionIntervalSeconds, TimeUnit.SECONDS); } + + connectionsTcpFailureCounter = + instrumentProvider.newCounter("pulsar.client.connection.failed", Unit.Connections, + "The number of failed connection attempts", null, + Attributes.builder().put("pulsar.failure.type", "tcp-failed").build()); + connectionsHandshakeFailureCounter = instrumentProvider.newCounter("pulsar.client.connection.failed", + Unit.Connections, "The number of failed connection attempts", null, + Attributes.builder().put("pulsar.failure.type", "handshake").build()); } private static AddressResolver createAddressResolver(ClientConfigurationData conf, @@ -295,6 +312,7 @@ private CompletableFuture createConnection(Key key) { } cnxFuture.complete(cnx); }).exceptionally(exception -> { + connectionsHandshakeFailureCounter.increment(); log.warn("[{}] Connection handshake failed: {}", cnx.channel(), exception.getMessage()); cnxFuture.completeExceptionally(exception); // this cleanupConnection may happen before that the @@ -306,6 +324,7 @@ private CompletableFuture createConnection(Key key) { return null; }); }).exceptionally(exception -> { + connectionsTcpFailureCounter.increment(); eventLoopGroup.execute(() -> { log.warn("Failed to open connection to {} : {}", key.physicalAddress, exception.getMessage()); pool.remove(key, cnxFuture); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 5a0e5de330d31..f1e259086ec8a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -34,6 +34,7 @@ import io.netty.util.ReferenceCountUtil; import io.netty.util.Timeout; import io.netty.util.concurrent.FastThreadLocal; +import io.opentelemetry.api.common.Attributes; import java.io.IOException; import java.net.URI; import java.nio.ByteBuffer; @@ -91,6 +92,10 @@ import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.crypto.MessageCryptoBc; +import org.apache.pulsar.client.impl.metrics.Counter; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.client.impl.metrics.Unit; +import org.apache.pulsar.client.impl.metrics.UpDownCounter; import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.ExecutorProvider; @@ -216,6 +221,17 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private final boolean createTopicIfDoesNotExist; private final boolean poolMessages; + private final Counter messagesReceivedCounter; + private final Counter bytesReceivedCounter; + private final UpDownCounter messagesPrefetchedGauge; + private final UpDownCounter bytesPrefetchedGauge; + private final Counter consumersOpenedCounter; + private final Counter consumersClosedCounter; + private final Counter consumerAcksCounter; + private final Counter consumerNacksCounter; + + private final Counter consumerDlqMessagesCounter; + private final AtomicReference clientCnxUsedForConsumerRegistration = new AtomicReference<>(); private final List previousExceptions = new CopyOnWriteArrayList(); private volatile boolean hasSoughtByTimestamp = false; @@ -389,7 +405,30 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat topicNameWithoutPartition = topicName.getPartitionedTopicName(); + InstrumentProvider ip = client.instrumentProvider(); + Attributes attrs = Attributes.builder().put("pulsar.subscription", subscription).build(); + consumersOpenedCounter = ip.newCounter("pulsar.client.consumer.opened", Unit.Sessions, + "The number of consumer sessions opened", topic, attrs); + consumersClosedCounter = ip.newCounter("pulsar.client.consumer.closed", Unit.Sessions, + "The number of consumer sessions closed", topic, attrs); + messagesReceivedCounter = ip.newCounter("pulsar.client.consumer.message.received.count", Unit.Messages, + "The number of messages explicitly received by the consumer application", topic, attrs); + bytesReceivedCounter = ip.newCounter("pulsar.client.consumer.message.received.size", Unit.Bytes, + "The number of bytes explicitly received by the consumer application", topic, attrs); + messagesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.receive_queue.count", Unit.Messages, + "The number of messages currently sitting in the consumer receive queue", topic, attrs); + bytesPrefetchedGauge = ip.newUpDownCounter("pulsar.client.consumer.receive_queue.size", Unit.Bytes, + "The total size in bytes of messages currently sitting in the consumer receive queue", topic, attrs); + + consumerAcksCounter = ip.newCounter("pulsar.client.consumer.message.ack", Unit.Messages, + "The number of acknowledged messages", topic, attrs); + consumerNacksCounter = ip.newCounter("pulsar.client.consumer.message.nack", Unit.Messages, + "The number of negatively acknowledged messages", topic, attrs); + consumerDlqMessagesCounter = ip.newCounter("pulsar.client.consumer.message.dlq", Unit.Messages, + "The number of messages sent to DLQ", topic, attrs); grabCnx(); + + consumersOpenedCounter.increment(); } public ConnectionHandler getConnectionHandler() { @@ -552,6 +591,8 @@ protected CompletableFuture> internalBatchReceiveAsync() { protected CompletableFuture doAcknowledge(MessageId messageId, AckType ackType, Map properties, TransactionImpl txn) { + consumerAcksCounter.increment(); + if (getState() != State.Ready && getState() != State.Connecting) { stats.incrementNumAcksFailed(); PulsarClientException exception = new PulsarClientException("Consumer not ready. State: " + getState()); @@ -573,6 +614,8 @@ protected CompletableFuture doAcknowledge(MessageId messageId, AckType ack @Override protected CompletableFuture doAcknowledge(List messageIdList, AckType ackType, Map properties, TransactionImpl txn) { + consumerAcksCounter.increment(); + if (getState() != State.Ready && getState() != State.Connecting) { stats.incrementNumAcksFailed(); PulsarClientException exception = new PulsarClientException("Consumer not ready. State: " + getState()); @@ -668,6 +711,8 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a .value(retryMessage.getData()) .properties(propertiesMap); typedMessageBuilderNew.sendAsync().thenAccept(msgId -> { + consumerDlqMessagesCounter.increment(); + doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null).thenAccept(v -> { result.complete(null); }).exceptionally(ex -> { @@ -760,6 +805,7 @@ private MessageImpl getMessageImpl(Message message) { @Override public void negativeAcknowledge(MessageId messageId) { + consumerNacksCounter.increment(); negativeAcksTracker.add(messageId); // Ensure the message is not redelivered for ack-timeout, since we did receive an "ack" @@ -768,6 +814,7 @@ public void negativeAcknowledge(MessageId messageId) { @Override public void negativeAcknowledge(Message message) { + consumerNacksCounter.increment(); negativeAcksTracker.add(message); // Ensure the message is not redelivered for ack-timeout, since we did receive an "ack" @@ -1048,6 +1095,8 @@ public CompletableFuture closeAsync() { return closeFuture; } + consumersClosedCounter.increment(); + if (!isConnected()) { log.info("[{}] [{}] Closed Consumer (not connected)", topic, subscription); setState(State.Closed); @@ -1240,6 +1289,9 @@ protected MessageImpl newMessage(final MessageIdImpl messageId, } private void executeNotifyCallback(final MessageImpl message) { + messagesPrefetchedGauge.increment(); + bytesPrefetchedGauge.add(message.size()); + // Enqueue the message so that it can be retrieved when application calls receive() // if the conf.getReceiverQueueSize() is 0 then discard message if no one is waiting for it. // if asyncReceive is waiting then notify callback without adding to incomingMessages queue @@ -1732,6 +1784,12 @@ protected synchronized void messageProcessed(Message msg) { ClientCnx msgCnx = ((MessageImpl) msg).getCnx(); lastDequeuedMessageId = msg.getMessageId(); + messagesPrefetchedGauge.decrement(); + messagesReceivedCounter.increment(); + + bytesPrefetchedGauge.subtract(msg.size()); + bytesReceivedCounter.add(msg.size()); + if (msgCnx != currentCnx) { // The processed message did belong to the old queue that was cleared after reconnection. } else { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index 02d0d10626fa6..8158b6d979efd 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import io.netty.channel.EventLoopGroup; +import io.opentelemetry.api.common.Attributes; import java.net.InetSocketAddress; import java.net.URI; import java.nio.ByteBuffer; @@ -34,6 +35,8 @@ import org.apache.pulsar.client.api.PulsarClientException.NotFoundException; import org.apache.pulsar.client.api.SchemaSerializationException; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.apache.pulsar.client.impl.schema.SchemaInfoUtil; import org.apache.pulsar.client.impl.schema.SchemaUtils; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace.Mode; @@ -60,11 +63,26 @@ public class HttpLookupService implements LookupService { private static final String BasePathV1 = "lookup/v2/destination/"; private static final String BasePathV2 = "lookup/v2/topic/"; - public HttpLookupService(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) + private final LatencyHistogram histoGetBroker; + private final LatencyHistogram histoGetTopicMetadata; + private final LatencyHistogram histoGetSchema; + private final LatencyHistogram histoListTopics; + + public HttpLookupService(InstrumentProvider instrumentProvider, ClientConfigurationData conf, + EventLoopGroup eventLoopGroup) throws PulsarClientException { this.httpClient = new HttpClient(conf, eventLoopGroup); this.useTls = conf.isUseTls(); this.listenerName = conf.getListenerName(); + + LatencyHistogram histo = instrumentProvider.newLatencyHistogram("pulsar.client.lookup.duration", + "Duration of lookup operations", null, + Attributes.builder().put("pulsar.lookup.transport-type", "http").build()); + histoGetBroker = histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "topic").build()); + histoGetTopicMetadata = + histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "metadata").build()); + histoGetSchema = histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "schema").build()); + histoListTopics = histo.withAttributes(Attributes.builder().put("pulsar.lookup.type", "list-topics").build()); } @Override @@ -84,8 +102,18 @@ public CompletableFuture getBroker(TopicName topicName) { String basePath = topicName.isV2() ? BasePathV2 : BasePathV1; String path = basePath + topicName.getLookupName(); path = StringUtils.isBlank(listenerName) ? path : path + "?listenerName=" + Codec.encode(listenerName); - return httpClient.get(path, LookupData.class) - .thenCompose(lookupData -> { + + long startTime = System.nanoTime(); + CompletableFuture httpFuture = httpClient.get(path, LookupData.class); + + httpFuture.thenRun(() -> { + histoGetBroker.recordSuccess(System.nanoTime() - startTime); + }).exceptionally(x -> { + histoGetBroker.recordFailure(System.nanoTime() - startTime); + return null; + }); + + return httpFuture.thenCompose(lookupData -> { // Convert LookupData into as SocketAddress, handling exceptions URI uri = null; try { @@ -112,9 +140,21 @@ public CompletableFuture getBroker(TopicName topicName) { @Override public CompletableFuture getPartitionedTopicMetadata(TopicName topicName) { + long startTime = System.nanoTime(); + String format = topicName.isV2() ? "admin/v2/%s/partitions" : "admin/%s/partitions"; - return httpClient.get(String.format(format, topicName.getLookupName()) + "?checkAllowAutoCreation=true", + CompletableFuture httpFuture = httpClient.get( + String.format(format, topicName.getLookupName()) + "?checkAllowAutoCreation=true", PartitionedTopicMetadata.class); + + httpFuture.thenRun(() -> { + histoGetTopicMetadata.recordSuccess(System.nanoTime() - startTime); + }).exceptionally(x -> { + histoGetTopicMetadata.recordFailure(System.nanoTime() - startTime); + return null; + }); + + return httpFuture; } @Override @@ -130,6 +170,8 @@ public InetSocketAddress resolveHost() { @Override public CompletableFuture getTopicsUnderNamespace(NamespaceName namespace, Mode mode, String topicsPattern, String topicsHash) { + long startTime = System.nanoTime(); + CompletableFuture future = new CompletableFuture<>(); String format = namespace.isV2() @@ -152,6 +194,14 @@ public CompletableFuture getTopicsUnderNamespace(NamespaceName future.completeExceptionally(cause); return null; }); + + future.thenRun(() -> { + histoListTopics.recordSuccess(System.nanoTime() - startTime); + }).exceptionally(x -> { + histoListTopics.recordFailure(System.nanoTime() - startTime); + return null; + }); + return future; } @@ -162,6 +212,7 @@ public CompletableFuture> getSchema(TopicName topicName) { @Override public CompletableFuture> getSchema(TopicName topicName, byte[] version) { + long startTime = System.nanoTime(); CompletableFuture> future = new CompletableFuture<>(); String schemaName = topicName.getSchemaName(); @@ -201,6 +252,13 @@ public CompletableFuture> getSchema(TopicName topicName, by } return null; }); + + future.thenRun(() -> { + histoGetSchema.recordSuccess(System.nanoTime() - startTime); + }).exceptionally(x -> { + histoGetSchema.recordFailure(System.nanoTime() - startTime); + return null; + }); return future; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 880185f7a9781..dbd3aae426900 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -40,6 +40,7 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; import io.netty.util.concurrent.ScheduledFuture; +import io.opentelemetry.api.common.Attributes; import java.io.IOException; import java.net.URI; import java.nio.ByteBuffer; @@ -76,6 +77,11 @@ import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.client.impl.crypto.MessageCryptoBc; +import org.apache.pulsar.client.impl.metrics.Counter; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; +import org.apache.pulsar.client.impl.metrics.Unit; +import org.apache.pulsar.client.impl.metrics.UpDownCounter; import org.apache.pulsar.client.impl.schema.JSONSchema; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.MathUtils; @@ -171,6 +177,15 @@ public class ProducerImpl extends ProducerBase implements TimerTask, Conne private boolean errorState; + private final LatencyHistogram latencyHistogram; + final LatencyHistogram rpcLatencyHistogram; + private final Counter publishedBytesCounter; + private final UpDownCounter pendingMessagesUpDownCounter; + private final UpDownCounter pendingBytesUpDownCounter; + + private final Counter producersOpenedCounter; + private final Counter producersClosedCounter; + public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfigurationData conf, CompletableFuture> producerCreatedFuture, int partitionIndex, Schema schema, ProducerInterceptors interceptors, Optional overrideProducerName) { @@ -265,6 +280,26 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration metadata = Collections.unmodifiableMap(new HashMap<>(conf.getProperties())); } + InstrumentProvider ip = client.instrumentProvider(); + latencyHistogram = ip.newLatencyHistogram("pulsar.client.producer.message.send.duration", + "Publish latency experienced by the application, includes client batching time", topic, + Attributes.empty()); + rpcLatencyHistogram = ip.newLatencyHistogram("pulsar.client.producer.rpc.send.duration", + "Publish RPC latency experienced internally by the client when sending data to receiving an ack", topic, + Attributes.empty()); + publishedBytesCounter = ip.newCounter("pulsar.client.producer.message.send.size", + Unit.Bytes, "The number of bytes published", topic, Attributes.empty()); + pendingMessagesUpDownCounter = + ip.newUpDownCounter("pulsar.client.producer.message.pending.count", Unit.Messages, + "The number of messages in the producer internal send queue, waiting to be sent", topic, + Attributes.empty()); + pendingBytesUpDownCounter = ip.newUpDownCounter("pulsar.client.producer.message.pending.size", Unit.Bytes, + "The size of the messages in the producer internal queue, waiting to sent", topic, Attributes.empty()); + producersOpenedCounter = ip.newCounter("pulsar.client.producer.opened", Unit.Sessions, + "The number of producer sessions opened", topic, Attributes.empty()); + producersClosedCounter = ip.newCounter("pulsar.client.producer.closed", Unit.Sessions, + "The number of producer sessions closed", topic, Attributes.empty()); + this.connectionHandler = new ConnectionHandler(this, new BackoffBuilder() .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), TimeUnit.NANOSECONDS) @@ -274,6 +309,7 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration this); setChunkMaxMessageSize(); grabCnx(); + producersOpenedCounter.increment(); } private void setChunkMaxMessageSize() { @@ -337,6 +373,11 @@ CompletableFuture internalSendAsync(Message message) { if (interceptors != null) { interceptorMessage.getProperties(); } + + int msgSize = interceptorMessage.getDataBuffer().readableBytes(); + pendingMessagesUpDownCounter.increment(); + pendingBytesUpDownCounter.add(msgSize); + sendAsync(interceptorMessage, new SendCallback() { SendCallback nextCallback = null; MessageImpl nextMsg = null; @@ -359,15 +400,22 @@ public MessageImpl getNextMessage() { @Override public void sendComplete(Exception e) { + long latencyNanos = System.nanoTime() - createdAt; + pendingMessagesUpDownCounter.decrement(); + pendingBytesUpDownCounter.subtract(msgSize); + try { if (e != null) { + latencyHistogram.recordFailure(latencyNanos); stats.incrementSendFailed(); onSendAcknowledgement(interceptorMessage, null, e); future.completeExceptionally(e); } else { + latencyHistogram.recordSuccess(latencyNanos); + publishedBytesCounter.add(msgSize); onSendAcknowledgement(interceptorMessage, interceptorMessage.getMessageId(), null); future.complete(interceptorMessage.getMessageId()); - stats.incrementNumAcksReceived(System.nanoTime() - createdAt); + stats.incrementNumAcksReceived(latencyNanos); } } finally { interceptorMessage.getDataBuffer().release(); @@ -413,15 +461,16 @@ CompletableFuture internalSendWithTxnAsync(Message message, Transa } else { CompletableFuture completableFuture = new CompletableFuture<>(); if (!((TransactionImpl) txn).checkIfOpen(completableFuture)) { - return completableFuture; + return completableFuture; } return ((TransactionImpl) txn).registerProducedTopic(topic) - .thenCompose(ignored -> internalSendAsync(message)); + .thenCompose(ignored -> internalSendAsync(message)); } } /** * Compress the payload if compression is configured. + * * @param payload * @return a new payload */ @@ -473,9 +522,10 @@ public void sendAsync(Message message, SendCallback callback) { if (!msg.isReplicated() && msgMetadata.hasProducerName()) { PulsarClientException.InvalidMessageException invalidMessageException = - new PulsarClientException.InvalidMessageException( - format("The producer %s of the topic %s can not reuse the same message", producerName, topic), - msg.getSequenceId()); + new PulsarClientException.InvalidMessageException( + format("The producer %s of the topic %s can not reuse the same message", producerName, + topic), + msg.getSequenceId()); completeCallbackAndReleaseSemaphore(uncompressedSize, callback, invalidMessageException); compressedPayload.release(); return; @@ -645,8 +695,8 @@ private void serializeAndSendMessage(MessageImpl msg, msgMetadata.setUuid(uuid); } msgMetadata.setChunkId(chunkId) - .setNumChunksFromMsg(totalChunks) - .setTotalChunkMsgSize(compressedPayloadSize); + .setNumChunksFromMsg(totalChunks) + .setTotalChunkMsgSize(compressedPayloadSize); } if (canAddToBatch(msg) && totalChunks <= 1) { @@ -697,9 +747,9 @@ private void serializeAndSendMessage(MessageImpl msg, if (msg.getSchemaState() == MessageImpl.SchemaState.Ready) { ByteBufPair cmd = sendMessage(producerId, sequenceId, numMessages, messageId, msgMetadata, encryptedPayload); - op = OpSendMsg.create(msg, cmd, sequenceId, callback); + op = OpSendMsg.create(rpcLatencyHistogram, msg, cmd, sequenceId, callback); } else { - op = OpSendMsg.create(msg, null, sequenceId, callback); + op = OpSendMsg.create(rpcLatencyHistogram, msg, null, sequenceId, callback); final MessageMetadata finalMsgMetadata = msgMetadata; op.rePopulate = () -> { if (msgMetadata.hasChunkId()) { @@ -780,8 +830,8 @@ private void tryRegisterSchema(ClientCnx cnx, MessageImpl msg, SendCallback call } SchemaInfo schemaInfo = msg.hasReplicateFrom() ? msg.getSchemaInfoForReplicator() : msg.getSchemaInfo(); schemaInfo = Optional.ofNullable(schemaInfo) - .filter(si -> si.getType().getValue() > 0) - .orElse(Schema.BYTES.getSchemaInfo()); + .filter(si -> si.getType().getValue() > 0) + .orElse(Schema.BYTES.getSchemaInfo()); getOrCreateSchemaAsync(cnx, schemaInfo).handle((v, ex) -> { if (ex != null) { Throwable t = FutureUtil.unwrapCompletionException(ex); @@ -816,10 +866,10 @@ private void tryRegisterSchema(ClientCnx cnx, MessageImpl msg, SendCallback call private CompletableFuture getOrCreateSchemaAsync(ClientCnx cnx, SchemaInfo schemaInfo) { if (!Commands.peerSupportsGetOrCreateSchema(cnx.getRemoteEndpointProtocolVersion())) { return FutureUtil.failedFuture( - new PulsarClientException.NotSupportedException( - format("The command `GetOrCreateSchema` is not supported for the protocol version %d. " - + "The producer is %s, topic is %s", - cnx.getRemoteEndpointProtocolVersion(), producerName, topic))); + new PulsarClientException.NotSupportedException( + format("The command `GetOrCreateSchema` is not supported for the protocol version %d. " + + "The producer is %s, topic is %s", + cnx.getRemoteEndpointProtocolVersion(), producerName, topic))); } long requestId = client.newRequestId(); ByteBuf request = Commands.newGetOrCreateSchema(requestId, topic, schemaInfo); @@ -891,7 +941,7 @@ private boolean canAddToBatch(MessageImpl msg) { private boolean canAddToCurrentBatch(MessageImpl msg) { return batchMessageContainer.haveEnoughSpace(msg) - && (!isMultiSchemaEnabled(false) || batchMessageContainer.hasSameSchema(msg)) + && (!isMultiSchemaEnabled(false) || batchMessageContainer.hasSameSchema(msg)) && batchMessageContainer.hasSameTxn(msg); } @@ -920,30 +970,31 @@ private void doBatchSendAndAdd(MessageImpl msg, SendCallback callback, ByteBu private boolean isValidProducerState(SendCallback callback, long sequenceId) { switch (getState()) { - case Ready: - // OK - case Connecting: - // We are OK to queue the messages on the client, it will be sent to the broker once we get the connection - case RegisteringSchema: - // registering schema - return true; - case Closing: - case Closed: - callback.sendComplete( - new PulsarClientException.AlreadyClosedException("Producer already closed", sequenceId)); - return false; - case ProducerFenced: - callback.sendComplete(new PulsarClientException.ProducerFencedException("Producer was fenced")); - return false; - case Terminated: - callback.sendComplete( - new PulsarClientException.TopicTerminatedException("Topic was terminated", sequenceId)); - return false; - case Failed: - case Uninitialized: - default: - callback.sendComplete(new PulsarClientException.NotConnectedException(sequenceId)); - return false; + case Ready: + // OK + case Connecting: + // We are OK to queue the messages on the client, it will be sent to the broker once we get the + // connection + case RegisteringSchema: + // registering schema + return true; + case Closing: + case Closed: + callback.sendComplete( + new PulsarClientException.AlreadyClosedException("Producer already closed", sequenceId)); + return false; + case ProducerFenced: + callback.sendComplete(new PulsarClientException.ProducerFencedException("Producer was fenced")); + return false; + case Terminated: + callback.sendComplete( + new PulsarClientException.TopicTerminatedException("Topic was terminated", sequenceId)); + return false; + case Failed: + case Uninitialized: + default: + callback.sendComplete(new PulsarClientException.NotConnectedException(sequenceId)); + return false; } } @@ -1043,9 +1094,11 @@ private static final class LastSendFutureWrapper { private LastSendFutureWrapper(CompletableFuture lastSendFuture) { this.lastSendFuture = lastSendFuture; } + static LastSendFutureWrapper create(CompletableFuture lastSendFuture) { return new LastSendFutureWrapper(lastSendFuture); } + public CompletableFuture handleOnce() { return lastSendFuture.handle((ignore, t) -> { if (t != null && THROW_ONCE_UPDATER.compareAndSet(this, FALSE, TRUE)) { @@ -1070,6 +1123,7 @@ public CompletableFuture closeAsync() { return CompletableFuture.completedFuture(null); } + producersClosedCounter.increment(); closeProducerTasks(); ClientCnx cnx = cnx(); @@ -1276,9 +1330,10 @@ protected synchronized void recoverChecksumError(ClientCnx cnx, long sequenceId) releaseSemaphoreForSendOp(op); try { op.sendComplete( - new PulsarClientException.ChecksumException( - format("The checksum of the message which is produced by producer %s to the topic " - + "%s is corrupted", producerName, topic))); + new PulsarClientException.ChecksumException( + format("The checksum of the message which is produced by producer %s to the " + + "topic " + + "%s is corrupted", producerName, topic))); } catch (Throwable t) { log.warn("[{}] [{}] Got exception while completing the callback for msg {}:", topic, producerName, sequenceId, t); @@ -1326,7 +1381,7 @@ protected synchronized void recoverNotAllowedError(long sequenceId, String error * * @param op * @return returns true only if message is not modified and computed-checksum is same as previous checksum else - * return false that means that message is corrupted. Returns true if checksum is not present. + * return false that means that message is corrupted. Returns true if checksum is not present. */ protected boolean verifyLocalBufferIsNotCorrupted(OpSendMsg op) { ByteBufPair msg = op.cmd; @@ -1402,6 +1457,7 @@ public ReferenceCounted touch(Object hint) { } protected static final class OpSendMsg { + LatencyHistogram rpcLatencyHistogram; MessageImpl msg; List> msgs; ByteBufPair cmd; @@ -1421,6 +1477,7 @@ protected static final class OpSendMsg { int chunkId = -1; void initialize() { + rpcLatencyHistogram = null; msg = null; msgs = null; cmd = null; @@ -1440,9 +1497,11 @@ void initialize() { chunkedMessageCtx = null; } - static OpSendMsg create(MessageImpl msg, ByteBufPair cmd, long sequenceId, SendCallback callback) { + static OpSendMsg create(LatencyHistogram rpcLatencyHistogram, MessageImpl msg, ByteBufPair cmd, + long sequenceId, SendCallback callback) { OpSendMsg op = RECYCLER.get(); op.initialize(); + op.rpcLatencyHistogram = rpcLatencyHistogram; op.msg = msg; op.cmd = cmd; op.callback = callback; @@ -1452,10 +1511,11 @@ static OpSendMsg create(MessageImpl msg, ByteBufPair cmd, long sequenceId, Se return op; } - static OpSendMsg create(List> msgs, ByteBufPair cmd, long sequenceId, SendCallback callback, - int batchAllocatedSize) { + static OpSendMsg create(LatencyHistogram rpcLatencyHistogram, List> msgs, ByteBufPair cmd, + long sequenceId, SendCallback callback, int batchAllocatedSize) { OpSendMsg op = RECYCLER.get(); op.initialize(); + op.rpcLatencyHistogram = rpcLatencyHistogram; op.msgs = msgs; op.cmd = cmd; op.callback = callback; @@ -1469,10 +1529,12 @@ static OpSendMsg create(List> msgs, ByteBufPair cmd, long sequenc return op; } - static OpSendMsg create(List> msgs, ByteBufPair cmd, long lowestSequenceId, - long highestSequenceId, SendCallback callback, int batchAllocatedSize) { + static OpSendMsg create(LatencyHistogram rpcLatencyHistogram, List> msgs, ByteBufPair cmd, + long lowestSequenceId, + long highestSequenceId, SendCallback callback, int batchAllocatedSize) { OpSendMsg op = RECYCLER.get(); op.initialize(); + op.rpcLatencyHistogram = rpcLatencyHistogram; op.msgs = msgs; op.cmd = cmd; op.callback = callback; @@ -1497,30 +1559,38 @@ void updateSentTimestamp() { void sendComplete(final Exception e) { SendCallback callback = this.callback; + + long now = System.nanoTime(); if (null != callback) { Exception finalEx = e; if (finalEx instanceof TimeoutException) { TimeoutException te = (TimeoutException) e; long sequenceId = te.getSequenceId(); - long ns = System.nanoTime(); + //firstSentAt and lastSentAt maybe -1, it means that the message didn't flush to channel. String errMsg = String.format( - "%s : createdAt %s seconds ago, firstSentAt %s seconds ago, lastSentAt %s seconds ago, " - + "retryCount %s", - te.getMessage(), - RelativeTimeUtil.nsToSeconds(ns - this.createdAt), - RelativeTimeUtil.nsToSeconds(this.firstSentAt <= 0 - ? this.firstSentAt - : ns - this.firstSentAt), - RelativeTimeUtil.nsToSeconds(this.lastSentAt <= 0 - ? this.lastSentAt - : ns - this.lastSentAt), - retryCount + "%s : createdAt %s seconds ago, firstSentAt %s seconds ago, lastSentAt %s seconds ago, " + + "retryCount %s", + te.getMessage(), + RelativeTimeUtil.nsToSeconds(now - this.createdAt), + RelativeTimeUtil.nsToSeconds(this.firstSentAt <= 0 + ? this.firstSentAt + : now - this.firstSentAt), + RelativeTimeUtil.nsToSeconds(this.lastSentAt <= 0 + ? this.lastSentAt + : now - this.lastSentAt), + retryCount ); finalEx = new TimeoutException(errMsg, sequenceId); } + if (e == null) { + rpcLatencyHistogram.recordSuccess(now - this.lastSentAt); + } else { + rpcLatencyHistogram.recordFailure(now - this.lastSentAt); + } + callback.sendComplete(finalEx); } } @@ -1687,7 +1757,7 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { long requestId = client.newRequestId(); PRODUCER_DEADLINE_UPDATER - .compareAndSet(this, 0, System.currentTimeMillis() + client.getConfiguration().getOperationTimeoutMs()); + .compareAndSet(this, 0, System.currentTimeMillis() + client.getConfiguration().getOperationTimeoutMs()); SchemaInfo schemaInfo = null; if (schema != null) { @@ -1698,7 +1768,7 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { // but now we have standardized on every schema to generate an Avro based schema if (Commands.peerSupportJsonSchemaAvroFormat(cnx.getRemoteEndpointProtocolVersion())) { schemaInfo = schema.getSchemaInfo(); - } else if (schema instanceof JSONSchema){ + } else if (schema instanceof JSONSchema) { JSONSchema jsonSchema = (JSONSchema) schema; schemaInfo = jsonSchema.getBackwardsCompatibleJsonSchemaInfo(); } else { @@ -1721,146 +1791,148 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { conf.getAccessMode(), topicEpoch, client.conf.isEnableTransaction(), conf.getInitialSubscriptionName()), requestId).thenAccept(response -> { - String producerName = response.getProducerName(); - long lastSequenceId = response.getLastSequenceId(); - schemaVersion = Optional.ofNullable(response.getSchemaVersion()); - schemaVersion.ifPresent(v -> schemaCache.put(SchemaHash.of(schema), v)); - - // We are now reconnected to broker and clear to send messages. Re-send all pending messages and - // set the cnx pointer so that new messages will be sent immediately - synchronized (ProducerImpl.this) { - State state = getState(); - if (state == State.Closing || state == State.Closed) { - // Producer was closed while reconnecting, close the connection to make sure the broker - // drops the producer on its side - cnx.removeProducer(producerId); - cnx.channel().close(); - future.complete(null); - return; - } - resetBackoff(); + String producerName = response.getProducerName(); + long lastSequenceId = response.getLastSequenceId(); + schemaVersion = Optional.ofNullable(response.getSchemaVersion()); + schemaVersion.ifPresent(v -> schemaCache.put(SchemaHash.of(schema), v)); - log.info("[{}] [{}] Created producer on cnx {}", topic, producerName, cnx.ctx().channel()); - connectionId = cnx.ctx().channel().toString(); - connectedSince = DateFormatter.now(); - if (conf.getAccessMode() != ProducerAccessMode.Shared && !topicEpoch.isPresent()) { - log.info("[{}] [{}] Producer epoch is {}", topic, producerName, response.getTopicEpoch()); - } - topicEpoch = response.getTopicEpoch(); + // We are now reconnected to broker and clear to send messages. Re-send all pending messages and + // set the cnx pointer so that new messages will be sent immediately + synchronized (ProducerImpl.this) { + State state = getState(); + if (state == State.Closing || state == State.Closed) { + // Producer was closed while reconnecting, close the connection to make sure the broker + // drops the producer on its side + cnx.removeProducer(producerId); + cnx.channel().close(); + future.complete(null); + return; + } + resetBackoff(); - if (this.producerName == null) { - this.producerName = producerName; - } + log.info("[{}] [{}] Created producer on cnx {}", topic, producerName, cnx.ctx().channel()); + connectionId = cnx.ctx().channel().toString(); + connectedSince = DateFormatter.now(); + if (conf.getAccessMode() != ProducerAccessMode.Shared && !topicEpoch.isPresent()) { + log.info("[{}] [{}] Producer epoch is {}", topic, producerName, response.getTopicEpoch()); + } + topicEpoch = response.getTopicEpoch(); - if (this.msgIdGenerator == 0 && conf.getInitialSequenceId() == null) { - // Only update sequence id generator if it wasn't already modified. That means we only want - // to update the id generator the first time the producer gets established, and ignore the - // sequence id sent by broker in subsequent producer reconnects - this.lastSequenceIdPublished = lastSequenceId; - this.msgIdGenerator = lastSequenceId + 1; - } + if (this.producerName == null) { + this.producerName = producerName; + } - resendMessages(cnx, epoch); - } - future.complete(null); - }).exceptionally((e) -> { - Throwable cause = e.getCause(); - cnx.removeProducer(producerId); - State state = getState(); - if (state == State.Closing || state == State.Closed) { - // Producer was closed while reconnecting, close the connection to make sure the broker - // drops the producer on its side - cnx.channel().close(); - future.complete(null); - return null; - } + if (this.msgIdGenerator == 0 && conf.getInitialSequenceId() == null) { + // Only update sequence id generator if it wasn't already modified. That means we only want + // to update the id generator the first time the producer gets established, and ignore the + // sequence id sent by broker in subsequent producer reconnects + this.lastSequenceIdPublished = lastSequenceId; + this.msgIdGenerator = lastSequenceId + 1; + } - if (cause instanceof TimeoutException) { - // Creating the producer has timed out. We need to ensure the broker closes the producer - // in case it was indeed created, otherwise it might prevent new create producer operation, - // since we are not necessarily closing the connection. - long closeRequestId = client.newRequestId(); - ByteBuf cmd = Commands.newCloseProducer(producerId, closeRequestId); - cnx.sendRequestWithId(cmd, closeRequestId); - } + resendMessages(cnx, epoch); + } + future.complete(null); + }).exceptionally((e) -> { + Throwable cause = e.getCause(); + cnx.removeProducer(producerId); + State state = getState(); + if (state == State.Closing || state == State.Closed) { + // Producer was closed while reconnecting, close the connection to make sure the broker + // drops the producer on its side + cnx.channel().close(); + future.complete(null); + return null; + } - if (cause instanceof PulsarClientException.ProducerFencedException) { - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] Failed to create producer: {}", - topic, producerName, cause.getMessage()); - } - } else { - log.error("[{}] [{}] Failed to create producer: {}", topic, producerName, cause.getMessage()); + if (cause instanceof TimeoutException) { + // Creating the producer has timed out. We need to ensure the broker closes the producer + // in case it was indeed created, otherwise it might prevent new create producer operation, + // since we are not necessarily closing the connection. + long closeRequestId = client.newRequestId(); + ByteBuf cmd = Commands.newCloseProducer(producerId, closeRequestId); + cnx.sendRequestWithId(cmd, closeRequestId); + } + + if (cause instanceof PulsarClientException.ProducerFencedException) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Failed to create producer: {}", + topic, producerName, cause.getMessage()); + } + } else { + log.error("[{}] [{}] Failed to create producer: {}", topic, producerName, cause.getMessage()); + } + // Close the producer since topic does not exist. + if (cause instanceof PulsarClientException.TopicDoesNotExistException) { + closeAsync().whenComplete((v, ex) -> { + if (ex != null) { + log.error("Failed to close producer on TopicDoesNotExistException.", ex); } - // Close the producer since topic does not exist. - if (cause instanceof PulsarClientException.TopicDoesNotExistException) { - closeAsync().whenComplete((v, ex) -> { - if (ex != null) { - log.error("Failed to close producer on TopicDoesNotExistException.", ex); - } - producerCreatedFuture.completeExceptionally(cause); - }); - future.complete(null); - return null; + producerCreatedFuture.completeExceptionally(cause); + }); + future.complete(null); + return null; + } + if (cause instanceof PulsarClientException.ProducerBlockedQuotaExceededException) { + synchronized (this) { + log.warn("[{}] [{}] Topic backlog quota exceeded. Throwing Exception on producer.", topic, + producerName); + + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Pending messages: {}", topic, producerName, + pendingMessages.messagesCount()); } - if (cause instanceof PulsarClientException.ProducerBlockedQuotaExceededException) { - synchronized (this) { - log.warn("[{}] [{}] Topic backlog quota exceeded. Throwing Exception on producer.", topic, - producerName); - - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] Pending messages: {}", topic, producerName, - pendingMessages.messagesCount()); - } - - PulsarClientException bqe = new PulsarClientException.ProducerBlockedQuotaExceededException( - format("The backlog quota of the topic %s that the producer %s produces to is exceeded", + + PulsarClientException bqe = new PulsarClientException.ProducerBlockedQuotaExceededException( + format("The backlog quota of the topic %s that the producer %s produces to is exceeded", topic, producerName)); - failPendingMessages(cnx(), bqe); - } - } else if (cause instanceof PulsarClientException.ProducerBlockedQuotaExceededError) { - log.warn("[{}] [{}] Producer is blocked on creation because backlog exceeded on topic.", - producerName, topic); - } + failPendingMessages(cnx(), bqe); + } + } else if (cause instanceof PulsarClientException.ProducerBlockedQuotaExceededError) { + log.warn("[{}] [{}] Producer is blocked on creation because backlog exceeded on topic.", + producerName, topic); + } - if (cause instanceof PulsarClientException.TopicTerminatedException) { - setState(State.Terminated); - synchronized (this) { - failPendingMessages(cnx(), (PulsarClientException) cause); - } - producerCreatedFuture.completeExceptionally(cause); - closeProducerTasks(); - client.cleanupProducer(this); - } else if (cause instanceof PulsarClientException.ProducerFencedException) { - setState(State.ProducerFenced); - synchronized (this) { - failPendingMessages(cnx(), (PulsarClientException) cause); - } - producerCreatedFuture.completeExceptionally(cause); - closeProducerTasks(); - client.cleanupProducer(this); - } else if (producerCreatedFuture.isDone() - || (cause instanceof PulsarClientException && PulsarClientException.isRetriableError(cause) - && System.currentTimeMillis() < PRODUCER_DEADLINE_UPDATER.get(ProducerImpl.this))) { - // Either we had already created the producer once (producerCreatedFuture.isDone()) or we are - // still within the initial timeout budget and we are dealing with a retriable error - future.completeExceptionally(cause); - } else { - setState(State.Failed); - producerCreatedFuture.completeExceptionally(cause); - closeProducerTasks(); - client.cleanupProducer(this); - Timeout timeout = sendTimeout; - if (timeout != null) { - timeout.cancel(); - sendTimeout = null; - } - } - if (!future.isDone()) { - future.complete(null); - } - return null; - }); + if (cause instanceof PulsarClientException.TopicTerminatedException) { + setState(State.Terminated); + synchronized (this) { + failPendingMessages(cnx(), (PulsarClientException) cause); + } + producerCreatedFuture.completeExceptionally(cause); + closeProducerTasks(); + client.cleanupProducer(this); + } else if (cause instanceof PulsarClientException.ProducerFencedException) { + setState(State.ProducerFenced); + synchronized (this) { + failPendingMessages(cnx(), (PulsarClientException) cause); + } + producerCreatedFuture.completeExceptionally(cause); + closeProducerTasks(); + client.cleanupProducer(this); + } else if (producerCreatedFuture.isDone() || ( + cause instanceof PulsarClientException + && PulsarClientException.isRetriableError(cause) + && System.currentTimeMillis() < PRODUCER_DEADLINE_UPDATER.get(ProducerImpl.this) + )) { + // Either we had already created the producer once (producerCreatedFuture.isDone()) or we are + // still within the initial timeout budget and we are dealing with a retriable error + future.completeExceptionally(cause); + } else { + setState(State.Failed); + producerCreatedFuture.completeExceptionally(cause); + closeProducerTasks(); + client.cleanupProducer(this); + Timeout timeout = sendTimeout; + if (timeout != null) { + timeout.cancel(); + sendTimeout = null; + } + } + if (!future.isDone()) { + future.complete(null); + } + return null; + }); return future; } @@ -1966,7 +2038,7 @@ private void stripChecksum(OpSendMsg op) { headerFrame.setInt(0, newTotalFrameSizeLength); // rewrite new [total-size] ByteBuf metadata = headerFrame.slice(checksumMark, headerFrameSize - checksumMark); // sliced only - // metadata + // metadata headerFrame.writerIndex(headerSize); // set headerFrame write-index to overwrite metadata over checksum metadata.readBytes(headerFrame, metadata.readableBytes()); headerFrame.capacity(headerFrameSize - checksumSize); // reduce capacity by removed checksum bytes @@ -2078,6 +2150,7 @@ private void failPendingMessages(ClientCnx cnx, PulsarClientException ex) { log.warn("[{}] [{}] Got exception while completing the callback for msg {}:", topic, producerName, op.sequenceId, t); } + client.getMemoryLimitController().releaseMemory(op.uncompressedSize); ReferenceCountUtil.safeRelease(op.cmd); op.recycle(); @@ -2102,7 +2175,6 @@ private void failPendingMessages(ClientCnx cnx, PulsarClientException ex) { /** * fail any pending batch messages that were enqueued, however batch was not closed out. - * */ private void failPendingBatchMessages(PulsarClientException ex) { if (batchMessageContainer.isEmpty()) { @@ -2122,7 +2194,7 @@ public CompletableFuture flushAsync() { if (isBatchMessagingEnabled()) { batchMessageAndSend(false); } - CompletableFuture lastSendFuture = this.lastSendFuture; + CompletableFuture lastSendFuture = this.lastSendFuture; if (!(lastSendFuture == this.lastSendFutureWrapper.lastSendFuture)) { this.lastSendFutureWrapper = LastSendFutureWrapper.create(lastSendFuture); } @@ -2241,7 +2313,7 @@ protected void processOpSendMsg(OpSendMsg op) { } else { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Connection is not ready -- sequenceId {}", topic, producerName, - op.sequenceId); + op.sequenceId); } } } catch (Throwable t) { @@ -2257,7 +2329,8 @@ private void recoverProcessOpSendMsgFrom(ClientCnx cnx, MessageImpl from, long e // In this case, the cnx passed to this method is no longer the active connection. This method will get // called again once the new connection registers the producer with the broker. log.info("[{}][{}] Producer epoch mismatch or the current connection is null. Skip re-sending the " - + " {} pending messages since they will deliver using another connection.", topic, producerName, + + " {} pending messages since they will deliver using another connection.", topic, + producerName, pendingMessages.messagesCount()); return; } @@ -2298,7 +2371,7 @@ private void recoverProcessOpSendMsgFrom(ClientCnx cnx, MessageImpl from, long e op.cmd.retain(); if (log.isDebugEnabled()) { log.debug("[{}] [{}] Re-Sending message in cnx {}, sequenceId {}", topic, producerName, - cnx.channel(), op.sequenceId); + cnx.channel(), op.sequenceId); } cnx.ctx().write(op.cmd, cnx.ctx().voidPromise()); op.updateSentTimestamp(); @@ -2322,7 +2395,7 @@ private void recoverProcessOpSendMsgFrom(ClientCnx cnx, MessageImpl from, long e } /** - * Check if final message size for non-batch and non-chunked messages is larger than max message size. + * Check if final message size for non-batch and non-chunked messages is larger than max message size. */ private boolean isMessageSizeExceeded(OpSendMsg op) { if (op.msg != null && !conf.isChunkingEnabled()) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 179996f4ea9f1..a919eb19a7ff8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -70,6 +70,7 @@ import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.client.impl.conf.ReaderConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.client.impl.schema.AutoProduceBytesSchema; import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema; @@ -149,6 +150,8 @@ public SchemaInfoProvider load(String topicName) { private final Clock clientClock; + private final InstrumentProvider instrumentProvider; + @Getter private TransactionCoordinatorClientImpl tcClient; @@ -176,6 +179,7 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG Timer timer, ExecutorProvider externalExecutorProvider, ExecutorProvider internalExecutorProvider, ScheduledExecutorProvider scheduledExecutorProvider) throws PulsarClientException { + EventLoopGroup eventLoopGroupReference = null; ConnectionPool connectionPoolReference = null; try { @@ -193,10 +197,12 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG throw new PulsarClientException.InvalidConfigurationException("Invalid client configuration"); } this.conf = conf; + this.instrumentProvider = new InstrumentProvider(conf.getOpenTelemetry()); clientClock = conf.getClock(); conf.getAuthentication().start(); connectionPoolReference = - connectionPool != null ? connectionPool : new ConnectionPool(conf, this.eventLoopGroup); + connectionPool != null ? connectionPool : + new ConnectionPool(instrumentProvider, conf, this.eventLoopGroup); this.cnxPool = connectionPoolReference; this.externalExecutorProvider = externalExecutorProvider != null ? externalExecutorProvider : new ExecutorProvider(conf.getNumListenerThreads(), "pulsar-external-listener"); @@ -205,7 +211,7 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG this.scheduledExecutorProvider = scheduledExecutorProvider != null ? scheduledExecutorProvider : new ScheduledExecutorProvider(conf.getNumIoThreads(), "pulsar-client-scheduled"); if (conf.getServiceUrl().startsWith("http")) { - lookup = new HttpLookupService(conf, this.eventLoopGroup); + lookup = new HttpLookupService(instrumentProvider, conf, this.eventLoopGroup); } else { lookup = new BinaryProtoLookupService(this, conf.getServiceUrl(), conf.getListenerName(), conf.isUseTls(), this.scheduledExecutorProvider.getExecutor()); @@ -1053,7 +1059,7 @@ public void reloadLookUp() throws PulsarClientException { public LookupService createLookup(String url) throws PulsarClientException { if (url.startsWith("http")) { - return new HttpLookupService(conf, eventLoopGroup); + return new HttpLookupService(instrumentProvider, conf, eventLoopGroup); } else { return new BinaryProtoLookupService(this, url, conf.getListenerName(), conf.isUseTls(), externalExecutorProvider.getExecutor()); @@ -1231,6 +1237,11 @@ public ScheduledExecutorProvider getScheduledExecutorProvider() { return scheduledExecutorProvider; } + InstrumentProvider instrumentProvider() { + return instrumentProvider; + } + + // // Transaction related API // diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java index 20ec9c3d99af4..e755b6ba1ee6d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java @@ -22,6 +22,7 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; import io.netty.util.concurrent.FastThreadLocal; +import io.opentelemetry.api.common.Attributes; import java.io.Closeable; import java.util.ArrayDeque; import java.util.Collections; @@ -35,6 +36,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; +import org.apache.pulsar.client.impl.metrics.Counter; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.client.impl.metrics.Unit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,6 +56,8 @@ public class UnAckedMessageTracker implements Closeable { protected final long ackTimeoutMillis; protected final long tickDurationInMs; + private final Counter consumerAckTimeoutsCounter; + private static class UnAckedMessageTrackerDisabled extends UnAckedMessageTracker { @Override public void clear() { @@ -89,13 +95,14 @@ public void close() { protected Timeout timeout; - public UnAckedMessageTracker() { + private UnAckedMessageTracker() { readLock = null; writeLock = null; timePartitions = null; messageIdPartitionMap = null; this.ackTimeoutMillis = 0; this.tickDurationInMs = 0; + this.consumerAckTimeoutsCounter = null; } protected static final FastThreadLocal> TL_MESSAGE_IDS_SET = @@ -114,6 +121,14 @@ public UnAckedMessageTracker(PulsarClientImpl client, ConsumerBase consumerBa ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock(); this.readLock = readWriteLock.readLock(); this.writeLock = readWriteLock.writeLock(); + + InstrumentProvider ip = client.instrumentProvider(); + consumerAckTimeoutsCounter = ip.newCounter("pulsar.client.consumer.message.ack.timeout", Unit.Messages, + "The number of messages that were not acknowledged in the configured timeout period, hence, were " + + "requested by the client to be redelivered", + consumerBase.getTopic(), + Attributes.builder().put("pulsar.subscription", consumerBase.getSubscription()).build()); + if (conf.getAckTimeoutRedeliveryBackoff() == null) { this.messageIdPartitionMap = new HashMap<>(); this.timePartitions = new ArrayDeque<>(); @@ -136,6 +151,7 @@ public void run(Timeout t) throws Exception { try { HashSet headPartition = timePartitions.removeFirst(); if (!headPartition.isEmpty()) { + consumerAckTimeoutsCounter.add(headPartition.size()); log.info("[{}] {} messages will be re-delivered", consumerBase, headPartition.size()); headPartition.forEach(messageId -> { if (messageId instanceof ChunkMessageIdImpl) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java index 1dc1c2a8689c6..6dcea7dc46672 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.opentelemetry.api.OpenTelemetry; import io.swagger.annotations.ApiModelProperty; import java.io.Serializable; import java.net.InetSocketAddress; @@ -395,6 +396,8 @@ public class ClientConfigurationData implements Serializable, Cloneable { ) private String description; + private transient OpenTelemetry openTelemetry; + /** * Gets the authentication settings for the client. * diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java new file mode 100644 index 0000000000000..fffbab4217a86 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java @@ -0,0 +1,60 @@ +/* + * 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.impl.metrics; + +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getDefaultAggregationLabels; +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getTopicAttributes; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.LongCounterBuilder; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.extension.incubator.metrics.ExtendedLongCounterBuilder; + +public class Counter { + + private final LongCounter counter; + private final Attributes attributes; + + Counter(Meter meter, String name, Unit unit, String description, String topic, Attributes attributes) { + LongCounterBuilder builder = meter.counterBuilder(name) + .setDescription(description) + .setUnit(unit.toString()); + + if (topic != null) { + if (builder instanceof ExtendedLongCounterBuilder) { + ExtendedLongCounterBuilder eb = (ExtendedLongCounterBuilder) builder; + eb.setAttributesAdvice(getDefaultAggregationLabels(attributes)); + } + + attributes = getTopicAttributes(topic, attributes); + } + + this.counter = builder.build(); + this.attributes = attributes; + } + + public void increment() { + add(1); + } + + public void add(int delta) { + counter.add(delta, attributes); + } + +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java new file mode 100644 index 0000000000000..1e02af1fd37e1 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/InstrumentProvider.java @@ -0,0 +1,58 @@ +/* + * 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.impl.metrics; + +import io.opentelemetry.api.GlobalOpenTelemetry; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.Meter; +import org.apache.pulsar.PulsarVersion; + +public class InstrumentProvider { + + public static final InstrumentProvider NOOP = new InstrumentProvider(OpenTelemetry.noop()); + + private final Meter meter; + + public InstrumentProvider(OpenTelemetry otel) { + if (otel == null) { + // By default, metrics are disabled, unless the OTel java agent is configured. + // This allows to enable metrics without any code change. + otel = GlobalOpenTelemetry.get(); + } + this.meter = otel.getMeterProvider() + .meterBuilder("org.apache.pulsar.client") + .setInstrumentationVersion(PulsarVersion.getVersion()) + .build(); + } + + public Counter newCounter(String name, Unit unit, String description, String topic, Attributes attributes) { + return new Counter(meter, name, unit, description, topic, attributes); + } + + public UpDownCounter newUpDownCounter(String name, Unit unit, String description, String topic, + Attributes attributes) { + return new UpDownCounter(meter, name, unit, description, topic, attributes); + } + + public LatencyHistogram newLatencyHistogram(String name, String description, String topic, Attributes attributes) { + return new LatencyHistogram(meter, name, description, topic, attributes); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java new file mode 100644 index 0000000000000..ed04eff03b39d --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java @@ -0,0 +1,110 @@ +/* + * 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.impl.metrics; + +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getDefaultAggregationLabels; +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getTopicAttributes; +import com.google.common.collect.Lists; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.DoubleHistogram; +import io.opentelemetry.api.metrics.DoubleHistogramBuilder; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.extension.incubator.metrics.ExtendedDoubleHistogramBuilder; +import java.util.List; +import java.util.concurrent.TimeUnit; + +public class LatencyHistogram { + + // Used for tests + public static final LatencyHistogram NOOP = new LatencyHistogram() { + public void recordSuccess(long latencyNanos) { + } + + public void recordFailure(long latencyNanos) { + } + }; + + private static final List latencyHistogramBuckets = + Lists.newArrayList(.0005, .001, .0025, .005, .01, .025, .05, .1, .25, .5, 1.0, 2.5, 5.0, 10.0, 30.0, 60.0); + + private static final double NANOS = TimeUnit.SECONDS.toNanos(1); + + private final Attributes successAttributes; + + private final Attributes failedAttributes; + private final DoubleHistogram histogram; + + private LatencyHistogram() { + successAttributes = null; + failedAttributes = null; + histogram = null; + } + + LatencyHistogram(Meter meter, String name, String description, String topic, Attributes attributes) { + DoubleHistogramBuilder builder = meter.histogramBuilder(name) + .setDescription(description) + .setUnit(Unit.Seconds.toString()) + .setExplicitBucketBoundariesAdvice(latencyHistogramBuckets); + + if (topic != null) { + if (builder instanceof ExtendedDoubleHistogramBuilder) { + ExtendedDoubleHistogramBuilder eb = (ExtendedDoubleHistogramBuilder) builder; + eb.setAttributesAdvice( + getDefaultAggregationLabels( + attributes.toBuilder().put("pulsar.response.status", "success").build())); + } + attributes = getTopicAttributes(topic, attributes); + } + + successAttributes = attributes.toBuilder() + .put("pulsar.response.status", "success") + .build(); + failedAttributes = attributes.toBuilder() + .put("pulsar.response.status", "failed") + .build(); + this.histogram = builder.build(); + } + + private LatencyHistogram(DoubleHistogram histogram, Attributes successAttributes, Attributes failedAttributes) { + this.histogram = histogram; + this.successAttributes = successAttributes; + this.failedAttributes = failedAttributes; + } + + /** + * Create a new histograms that inherits the old histograms attributes and adds new ones. + */ + public LatencyHistogram withAttributes(Attributes attributes) { + return new LatencyHistogram( + histogram, + successAttributes.toBuilder().putAll(attributes).build(), + failedAttributes.toBuilder().putAll(attributes).build() + ); + } + + + public void recordSuccess(long latencyNanos) { + histogram.record(latencyNanos / NANOS, successAttributes); + } + + public void recordFailure(long latencyNanos) { + histogram.record(latencyNanos / NANOS, failedAttributes); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/MetricsUtil.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/MetricsUtil.java new file mode 100644 index 0000000000000..b9802f4f32b5f --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/MetricsUtil.java @@ -0,0 +1,59 @@ +/* + * 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.impl.metrics; + +import com.google.common.collect.Lists; +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.common.AttributesBuilder; +import java.util.ArrayList; +import java.util.List; +import lombok.experimental.UtilityClass; +import org.apache.pulsar.common.naming.TopicName; + +@UtilityClass +public class MetricsUtil { + + // By default, advice to use namespace level aggregation only + private static final List> DEFAULT_AGGREGATION_LABELS = Lists.newArrayList( + AttributeKey.stringKey("pulsar.tenant"), + AttributeKey.stringKey("pulsar.namespace") + ); + + static List> getDefaultAggregationLabels(Attributes attrs) { + List> res = new ArrayList<>(); + res.addAll(DEFAULT_AGGREGATION_LABELS); + res.addAll(attrs.asMap().keySet()); + return res; + } + + static Attributes getTopicAttributes(String topic, Attributes baseAttributes) { + TopicName tn = TopicName.get(topic); + + AttributesBuilder ab = baseAttributes.toBuilder(); + if (tn.isPartitioned()) { + ab.put("pulsar.partition", tn.getPartitionIndex()); + } + ab.put("pulsar.topic", tn.getPartitionedTopicName()); + ab.put("pulsar.namespace", tn.getNamespace()); + ab.put("pulsar.tenant", tn.getTenant()); + return ab.build(); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Unit.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Unit.java new file mode 100644 index 0000000000000..5204cc2f03eae --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Unit.java @@ -0,0 +1,59 @@ +/* + * 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.impl.metrics; + +public enum Unit { + Bytes, + + Messages, + + Seconds, + + Connections, + + Sessions, + + None, + + ; + + public String toString() { + switch (this) { + case Bytes: + return "By"; + + case Messages: + return "{message}"; + + case Seconds: + return "s"; + + case Connections: + return "{connection}"; + + case Sessions: + return "{session}"; + + case None: + default: + return "1"; + } + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java new file mode 100644 index 0000000000000..3df0c2bb42302 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java @@ -0,0 +1,68 @@ +/* + * 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.impl.metrics; + +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getDefaultAggregationLabels; +import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getTopicAttributes; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.LongUpDownCounter; +import io.opentelemetry.api.metrics.LongUpDownCounterBuilder; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.extension.incubator.metrics.ExtendedLongUpDownCounterBuilder; + +public class UpDownCounter { + + private final LongUpDownCounter counter; + private final Attributes attributes; + + UpDownCounter(Meter meter, String name, Unit unit, String description, String topic, Attributes attributes) { + LongUpDownCounterBuilder builder = meter.upDownCounterBuilder(name) + .setDescription(description) + .setUnit(unit.toString()); + + if (topic != null) { + if (builder instanceof ExtendedLongUpDownCounterBuilder) { + ExtendedLongUpDownCounterBuilder eb = (ExtendedLongUpDownCounterBuilder) builder; + eb.setAttributesAdvice(getDefaultAggregationLabels(attributes)); + } + + attributes = getTopicAttributes(topic, attributes); + } + + this.counter = builder.build(); + this.attributes = attributes; + } + + public void increment() { + add(1); + } + + public void decrement() { + add(-1); + } + + public void add(long delta) { + counter.add(delta, attributes); + } + + public void subtract(long diff) { + add(-diff); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/package-info.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/package-info.java new file mode 100644 index 0000000000000..ee99bb3332c26 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/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. + */ + +/** + * Pulsar Client OTel metrics utilities + */ +package org.apache.pulsar.client.impl.metrics; diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java index 1d1a6f85bfd41..514e3dde14070 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java @@ -41,6 +41,7 @@ import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.util.TimedCompletableFuture; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; @@ -70,7 +71,7 @@ public void setup() throws NoSuchFieldException, IllegalAccessException { doReturn(client).when(consumer).getClient(); doReturn(cnx).when(consumer).getClientCnx(); doReturn(new ConsumerStatsRecorderImpl()).when(consumer).getStats(); - doReturn(new UnAckedMessageTracker().UNACKED_MESSAGE_TRACKER_DISABLED) + doReturn(UnAckedMessageTracker.UNACKED_MESSAGE_TRACKER_DISABLED) .when(consumer).getUnAckedMessageTracker(); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); when(cnx.ctx()).thenReturn(ctx); @@ -423,7 +424,7 @@ public void testDoIndividualBatchAckAsync() throws Exception{ public class ClientCnxTest extends ClientCnx { public ClientCnxTest(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) { - super(conf, eventLoopGroup); + super(InstrumentProvider.NOOP, conf, eventLoopGroup); } @Override diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java index 87188255b20b8..983cd21a7a9d8 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java @@ -36,6 +36,7 @@ import org.apache.pulsar.client.api.PulsarClientException.LookupException; import org.apache.pulsar.client.impl.BinaryProtoLookupService.LookupDataResult; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.naming.TopicName; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -65,6 +66,7 @@ public void setup() throws Exception { doReturn(0).when(clientConfig).getMaxLookupRedirects(); PulsarClientImpl client = mock(PulsarClientImpl.class); + doReturn(InstrumentProvider.NOOP).when(client).instrumentProvider(); doReturn(cnxPool).when(client).getCnxPool(); doReturn(clientConfig).when(client).getConfiguration(); doReturn(1L).when(client).newRequestId(); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxRequestTimeoutQueueTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxRequestTimeoutQueueTest.java index ca6114d2ed823..d573229fddefa 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxRequestTimeoutQueueTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxRequestTimeoutQueueTest.java @@ -26,6 +26,7 @@ import io.netty.util.concurrent.DefaultThreadFactory; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.util.TimedCompletableFuture; import org.apache.pulsar.common.util.netty.EventLoopUtil; @@ -60,7 +61,7 @@ void setupClientCnx() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setKeepAliveIntervalSeconds(0); conf.setOperationTimeoutMs(1); - cnx = new ClientCnx(conf, eventLoop); + cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java index 4f657da82b289..bc1d940c76bbf 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java @@ -42,6 +42,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.BrokerMetadataException; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.CommandCloseConsumer; import org.apache.pulsar.common.api.proto.CommandCloseProducer; import org.apache.pulsar.common.api.proto.CommandConnected; @@ -63,7 +64,7 @@ public void testClientCnxTimeout() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setOperationTimeoutMs(10); conf.setKeepAliveIntervalSeconds(0); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -89,7 +90,7 @@ public void testPendingLookupRequestSemaphore() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setOperationTimeoutMs(10_000); conf.setKeepAliveIntervalSeconds(0); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -127,7 +128,7 @@ public void testPendingLookupRequestSemaphoreServiceNotReady() throws Exception ClientConfigurationData conf = new ClientConfigurationData(); conf.setOperationTimeoutMs(10_000); conf.setKeepAliveIntervalSeconds(0); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -170,7 +171,7 @@ public void testPendingWaitingLookupRequestSemaphore() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setOperationTimeoutMs(10_000); conf.setKeepAliveIntervalSeconds(0); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -196,7 +197,7 @@ public void testReceiveErrorAtSendConnectFrameState() throws Exception { EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, threadFactory); ClientConfigurationData conf = new ClientConfigurationData(); conf.setOperationTimeoutMs(10); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -230,7 +231,7 @@ public void testGetLastMessageIdWithError() throws Exception { ThreadFactory threadFactory = new DefaultThreadFactory("testReceiveErrorAtSendConnectFrameState"); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, threadFactory); ClientConfigurationData conf = new ClientConfigurationData(); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); @@ -276,7 +277,7 @@ public void testHandleCloseConsumer() { ThreadFactory threadFactory = new DefaultThreadFactory("testHandleCloseConsumer"); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, threadFactory); ClientConfigurationData conf = new ClientConfigurationData(); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); long consumerId = 1; PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); @@ -300,7 +301,7 @@ public void testHandleCloseProducer() { ThreadFactory threadFactory = new DefaultThreadFactory("testHandleCloseProducer"); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, threadFactory); ClientConfigurationData conf = new ClientConfigurationData(); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); long producerId = 1; PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); @@ -393,7 +394,7 @@ private void withConnection(String testName, Consumer test) { try { ClientConfigurationData conf = new ClientConfigurationData(); - ClientCnx cnx = new ClientCnx(conf, eventLoop); + ClientCnx cnx = new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); Channel channel = mock(Channel.class); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/OpSendMsgQueueTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/OpSendMsgQueueTest.java index 2db23782640eb..efcc06bede3e1 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/OpSendMsgQueueTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/OpSendMsgQueueTest.java @@ -23,6 +23,7 @@ import static org.testng.Assert.assertEquals; import com.google.common.collect.Lists; import java.util.Arrays; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -39,7 +40,7 @@ public void createMockMessage() { } private ProducerImpl.OpSendMsg createDummyOpSendMsg() { - return ProducerImpl.OpSendMsg.create(message, null, 0L, null); + return ProducerImpl.OpSendMsg.create(LatencyHistogram.NOOP, message, null, 0L, null); } @Test diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PartitionedProducerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PartitionedProducerImplTest.java index b96f6a344a3dc..f96d2e2e0b0e9 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PartitionedProducerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PartitionedProducerImplTest.java @@ -47,6 +47,7 @@ import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.client.impl.customroute.PartialRoundRobinMessageRouterImpl; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.assertj.core.util.Sets; @@ -78,6 +79,7 @@ public void setup() { producerBuilderImpl = new ProducerBuilderImpl(client, Schema.BYTES); + when(client.instrumentProvider()).thenReturn(InstrumentProvider.NOOP); when(client.getConfiguration()).thenReturn(clientConfigurationData); when(client.timer()).thenReturn(timer); when(client.newProducer()).thenReturn(producerBuilderImpl); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java index 6fcedc3f94de7..f9df63759394a 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerImplTest.java @@ -27,6 +27,7 @@ import static org.testng.Assert.assertTrue; import java.nio.ByteBuffer; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.metrics.LatencyHistogram; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.mockito.Mockito; import org.testng.annotations.Test; @@ -42,6 +43,7 @@ public void testChunkedMessageCtxDeallocate() { for (int i = 0; i < totalChunks; i++) { ProducerImpl.OpSendMsg opSendMsg = ProducerImpl.OpSendMsg.create( + LatencyHistogram.NOOP, MessageImpl.create(new MessageMetadata(), ByteBuffer.allocate(0), Schema.STRING, null), null, 0, null); opSendMsg.chunkedMessageCtx = ctx; diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java index c96443c1e2f9f..274b9b4f2d572 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java @@ -53,6 +53,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.util.ExecutorProvider; import org.apache.pulsar.client.util.ScheduledExecutorProvider; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; @@ -180,7 +181,7 @@ public void testInitializeWithTimer() throws PulsarClientException { ClientConfigurationData conf = new ClientConfigurationData(); @Cleanup("shutdownGracefully") EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, new DefaultThreadFactory("test")); - ConnectionPool pool = Mockito.spy(new ConnectionPool(conf, eventLoop)); + ConnectionPool pool = Mockito.spy(new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoop)); conf.setServiceUrl("pulsar://localhost:6650"); HashedWheelTimer timer = new HashedWheelTimer(); @@ -205,7 +206,7 @@ public void testResourceCleanup() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setServiceUrl(""); initializeEventLoopGroup(conf); - try (ConnectionPool connectionPool = new ConnectionPool(conf, eventLoopGroup)) { + try (ConnectionPool connectionPool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoopGroup)) { assertThrows(() -> new PulsarClientImpl(conf, eventLoopGroup, connectionPool)); } finally { // Externally passed eventLoopGroup should not be shutdown. diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java index 91ad321048226..b01fbcb879f80 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java @@ -36,6 +36,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.testng.annotations.Test; @@ -46,6 +47,7 @@ public class UnAckedMessageTrackerTest { public void testAddAndRemove() { PulsarClientImpl client = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.instrumentProvider()).thenReturn(InstrumentProvider.NOOP); when(client.getCnxPool()).thenReturn(connectionPool); Timer timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-timer", Thread.currentThread().isDaemon()), 1, TimeUnit.MILLISECONDS); @@ -86,6 +88,7 @@ public void testAddAndRemove() { public void testTrackChunkedMessageId() { PulsarClientImpl client = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.instrumentProvider()).thenReturn(InstrumentProvider.NOOP); when(client.getCnxPool()).thenReturn(connectionPool); Timer timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-timer", Thread.currentThread().isDaemon()), 1, TimeUnit.MILLISECONDS); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ClientConfigurationDataTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ClientConfigurationDataTest.java index 5856395566a67..27f521ef1ff73 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ClientConfigurationDataTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/conf/ClientConfigurationDataTest.java @@ -22,7 +22,14 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; +import io.opentelemetry.api.OpenTelemetry; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import lombok.Cleanup; import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.testng.Assert; import org.testng.annotations.Test; /** @@ -36,10 +43,35 @@ public void testDoNotPrintSensitiveInfo() throws JsonProcessingException { clientConfigurationData.setTlsTrustStorePassword("xxxx"); clientConfigurationData.setSocks5ProxyPassword("yyyy"); clientConfigurationData.setAuthentication(new AuthenticationToken("zzzz")); + clientConfigurationData.setOpenTelemetry(OpenTelemetry.noop()); ObjectMapper objectMapper = new ObjectMapper(); objectMapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false); String serializedConf = objectMapper.writeValueAsString(clientConfigurationData); assertThat(serializedConf).doesNotContain("xxxx", "yyyy", "zzzz"); } + @Test + public void testSerializable() throws Exception { + ClientConfigurationData conf = new ClientConfigurationData(); + conf.setConnectionsPerBroker(3); + conf.setTlsTrustStorePassword("xxxx"); + conf.setOpenTelemetry(OpenTelemetry.noop()); + + @Cleanup + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + @Cleanup + ObjectOutputStream oos = new ObjectOutputStream(bos); + oos.writeObject(conf); + byte[] serialized = bos.toByteArray(); + + // Deserialize + @Cleanup + ByteArrayInputStream bis = new ByteArrayInputStream(serialized); + @Cleanup + ObjectInputStream ois = new ObjectInputStream(bis); + Object object = ois.readObject(); + + Assert.assertEquals(object.getClass(), ClientConfigurationData.class); + Assert.assertEquals(object, conf); + } } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java index 782454022b1ed..d15d48b9209d0 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java @@ -26,6 +26,7 @@ import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.client.impl.ClientCnx; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.AuthData; import org.apache.pulsar.common.api.proto.CommandAuthChallenge; import org.apache.pulsar.common.protocol.Commands; @@ -47,7 +48,7 @@ public class ProxyClientCnx extends ClientCnx { public ProxyClientCnx(ClientConfigurationData conf, EventLoopGroup eventLoopGroup, String clientAuthRole, String clientAuthMethod, int protocolVersion, boolean forwardClientAuthData, ProxyConnection proxyConnection) { - super(conf, eventLoopGroup, protocolVersion); + super(InstrumentProvider.NOOP, conf, eventLoopGroup, protocolVersion); this.clientAuthRole = clientAuthRole; this.clientAuthMethod = clientAuthMethod; this.forwardClientAuthData = forwardClientAuthData; diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java index ba9247a085dff..594d6cbc3bb59 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java @@ -59,6 +59,7 @@ import org.apache.pulsar.client.impl.PulsarChannelInitializer; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ConfigurationDataUtils; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.client.internal.PropertiesUtils; import org.apache.pulsar.common.api.AuthData; import org.apache.pulsar.common.api.proto.CommandAuthResponse; @@ -383,11 +384,12 @@ private synchronized void completeConnect() throws PulsarClientException { service.getConfiguration().isForwardAuthorizationCredentials(), this); } else { clientCnxSupplier = - () -> new ClientCnx(clientConf, service.getWorkerGroup(), protocolVersionToAdvertise); + () -> new ClientCnx(InstrumentProvider.NOOP, clientConf, service.getWorkerGroup(), + protocolVersionToAdvertise); } if (this.connectionPool == null) { - this.connectionPool = new ConnectionPool(clientConf, service.getWorkerGroup(), + this.connectionPool = new ConnectionPool(InstrumentProvider.NOOP, clientConf, service.getWorkerGroup(), clientCnxSupplier, Optional.of(dnsAddressResolverGroup.getResolver(service.getWorkerGroup().next()))); } else { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java index 3f58250e6d68a..1a9459619ebe9 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java @@ -42,6 +42,7 @@ import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.CommandActiveConsumerChange; import org.apache.pulsar.common.api.proto.ProtocolVersion; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; @@ -251,8 +252,8 @@ private static PulsarClient getClientActiveConsumerChangeNotSupported(ClientConf final EventLoopGroup eventLoopGroup) throws Exception { - ConnectionPool cnxPool = new ConnectionPool(conf, eventLoopGroup, () -> { - return new ClientCnx(conf, eventLoopGroup, ProtocolVersion.v11_VALUE) { + ConnectionPool cnxPool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoopGroup, () -> { + return new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup, ProtocolVersion.v11_VALUE) { @Override protected void handleActiveConsumerChange(CommandActiveConsumerChange change) { throw new UnsupportedOperationException(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java index 9bc12dcc6fcb2..e1e49f9e8c5f2 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java @@ -50,6 +50,7 @@ import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.CommandActiveConsumerChange; import org.apache.pulsar.common.api.proto.ProtocolVersion; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; @@ -374,8 +375,8 @@ private PulsarClient getClientActiveConsumerChangeNotSupported(ClientConfigurati EventLoopGroup eventLoopGroup = EventLoopUtil.newEventLoopGroup(conf.getNumIoThreads(), false, threadFactory); registerCloseable(() -> eventLoopGroup.shutdownNow()); - ConnectionPool cnxPool = new ConnectionPool(conf, eventLoopGroup, () -> { - return new ClientCnx(conf, eventLoopGroup, ProtocolVersion.v11_VALUE) { + ConnectionPool cnxPool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoopGroup, () -> { + return new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup, ProtocolVersion.v11_VALUE) { @Override protected void handleActiveConsumerChange(CommandActiveConsumerChange change) { throw new UnsupportedOperationException(); diff --git a/pulsar-testclient/pom.xml b/pulsar-testclient/pom.xml index ecc12b2e563d5..ec32b57be15f4 100644 --- a/pulsar-testclient/pom.xml +++ b/pulsar-testclient/pom.xml @@ -112,6 +112,23 @@ jackson-databind + + io.opentelemetry + opentelemetry-exporter-prometheus + + + io.opentelemetry + opentelemetry-exporter-otlp + + + io.opentelemetry + opentelemetry-sdk + + + io.opentelemetry + opentelemetry-sdk-extension-autoconfigure + + org.awaitility awaitility diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java index 3b44023ef503e..b6b3d805edc75 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java @@ -19,6 +19,7 @@ package org.apache.pulsar.testclient; import static org.apache.commons.lang3.StringUtils.isNotBlank; +import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import java.lang.management.ManagementFactory; import java.util.Objects; import java.util.concurrent.TimeUnit; @@ -76,7 +77,9 @@ public static ClientBuilder createClientBuilderFromArguments(PerformanceBaseArgu .listenerThreads(arguments.listenerThreads) .tlsTrustCertsFilePath(arguments.tlsTrustCertsFilePath) .maxLookupRequests(arguments.maxLookupRequest) - .proxyServiceUrl(arguments.proxyServiceURL, arguments.proxyProtocol); + .proxyServiceUrl(arguments.proxyServiceURL, arguments.proxyProtocol) + .openTelemetry(AutoConfiguredOpenTelemetrySdk.builder() + .build().getOpenTelemetrySdk()); if (isNotBlank(arguments.authPluginClassName)) { clientBuilder.authentication(arguments.authPluginClassName, arguments.authParams); From 8fc30df37e2691156e299d95c4f40efafb64e678 Mon Sep 17 00:00:00 2001 From: Chris Bono Date: Thu, 28 Mar 2024 13:57:08 -0500 Subject: [PATCH 025/580] [feat][ci] Add Trivy container scan Github workflow (#22063) --- .../workflows/ci-trivy-container-scan.yaml | 66 +++++++++++++++++++ 1 file changed, 66 insertions(+) create mode 100644 .github/workflows/ci-trivy-container-scan.yaml diff --git a/.github/workflows/ci-trivy-container-scan.yaml b/.github/workflows/ci-trivy-container-scan.yaml new file mode 100644 index 0000000000000..47ebe654369d5 --- /dev/null +++ b/.github/workflows/ci-trivy-container-scan.yaml @@ -0,0 +1,66 @@ +# +# 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. +# + +name: CI - Trivy Container Scan +on: + schedule: + - cron: '0 8 * * *' # Every day at 8am UTC + workflow_dispatch: + inputs: + severity: + description: "Severities to include (comma-separated or 'ALL' to include all)" + required: false + default: 'CRITICAL,HIGH' + +jobs: + container_scan: + if: ${{ github.repository == 'apache/pulsar' }} + name: Trivy Docker image vulnerability scan + runs-on: ubuntu-latest + strategy: + fail-fast: false + matrix: + docker-image: + - 'apachepulsar/pulsar' + docker-tag: + - 'latest' + env: + IMAGE_REF: '${{ matrix.docker-image }}:${{ matrix.docker-tag }}' + steps: + - id: prepare-vars + shell: bash + run: | + IMAGE_REF_CLEAN="$(echo $IMAGE_REF | sed 's/-/_/g; s/\./_/g; s/:/_/g; s/\//_/g')" + echo "image_ref_clean=$IMAGE_REF_CLEAN" >> "$GITHUB_OUTPUT" + echo "report_filename=trivy-scan-$IMAGE_REF_CLEAN.${{ inputs.report-format }}" >> "$GITHUB_OUTPUT" + - name: Run Trivy container scan + uses: aquasecurity/trivy-action@master + with: + image-ref: ${{ env.IMAGE_REF }} + scanners: vuln + severity: ${{ inputs.severity != 'ALL' && inputs.severity || 'UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL' }} + limit-severities-for-sarif: true + format: 'sarif' + output: ${{ steps.prepare-vars.outputs.report_filename }} + exit-code: 1 + - name: Upload Trivy scan results to GitHub Security tab + uses: github/codeql-action/upload-sarif@v2 + if: ${{ failure() }} + with: + sarif_file: '${{ github.workspace }}/${{ steps.prepare-vars.outputs.report_filename }}' From a3bf4e8a42c84a0ee5b4c45b50d48daed0b3de0c Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Fri, 29 Mar 2024 08:33:27 +0800 Subject: [PATCH 026/580] [improve][io]: Add validation for JDBC sink not supporting primitive schema (#22376) --- pulsar-io/jdbc/core/pom.xml | 7 +++++ .../io/jdbc/BaseJdbcAutoSchemaSink.java | 5 +++ .../io/jdbc/BaseJdbcAutoSchemaSinkTest.java | 25 +++++++++++++++ .../pulsar/io/jdbc/SqliteJdbcSinkTest.java | 31 +++++++++++++------ 4 files changed, 59 insertions(+), 9 deletions(-) diff --git a/pulsar-io/jdbc/core/pom.xml b/pulsar-io/jdbc/core/pom.xml index 586307e8b8675..0232a6996805b 100644 --- a/pulsar-io/jdbc/core/pom.xml +++ b/pulsar-io/jdbc/core/pom.xml @@ -71,6 +71,13 @@ provided + + ${project.groupId} + pulsar-client-original + ${project.version} + test + + \ No newline at end of file diff --git a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java index 36c3674091932..3655688c0f3ad 100644 --- a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java +++ b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java @@ -34,6 +34,7 @@ import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.client.api.schema.KeyValueSchema; import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.jdbc.JdbcUtils.ColumnId; @@ -137,6 +138,10 @@ public Mutation createMutation(Record message) { } recordValueGetter = (k) -> data.get(k); } else { + SchemaType schemaType = message.getSchema().getSchemaInfo().getType(); + if (schemaType.isPrimitive()) { + throw new UnsupportedOperationException("Primitive schema is not supported: " + schemaType); + } recordValueGetter = (key) -> ((GenericRecord) record).getField(key); } String action = message.getProperties().get(ACTION_PROPERTY); diff --git a/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java b/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java index b15eb832242c7..c088dd3c42c32 100644 --- a/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java +++ b/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java @@ -22,6 +22,10 @@ import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; import org.apache.avro.util.Utf8; +import org.apache.pulsar.client.api.schema.GenericObject; +import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; +import org.apache.pulsar.functions.api.Record; import org.testng.Assert; import org.testng.annotations.Test; @@ -143,5 +147,26 @@ private Schema createFieldAndGetSchema(Function record = new Record() { + @Override + public org.apache.pulsar.client.api.Schema getSchema() { + return autoConsumeSchema; + } + + @Override + public GenericRecord getValue() { + return null; + } + }; + baseJdbcAutoSchemaSink.createMutation((Record) record); + } + } \ No newline at end of file diff --git a/pulsar-io/jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java b/pulsar-io/jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java index d9ed4cbd442bf..ca01615bef193 100644 --- a/pulsar-io/jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java +++ b/pulsar-io/jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java @@ -48,6 +48,7 @@ import org.apache.pulsar.client.api.schema.RecordSchemaBuilder; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.impl.MessageImpl; +import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.client.impl.schema.AvroSchema; import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema; import org.apache.pulsar.common.schema.KeyValue; @@ -282,9 +283,12 @@ public void TestUnknownAction() throws Exception { } @Test + @SuppressWarnings("unchecked") public void TestUpdateAction() throws Exception { AvroSchema schema = AvroSchema.of(SchemaDefinition.builder().withPojo(Foo.class).build()); + AutoConsumeSchema autoConsumeSchema = new AutoConsumeSchema(); + autoConsumeSchema.setSchema(schema); Foo updateObj = new Foo(); updateObj.setField1("ValueOfField3"); @@ -292,10 +296,11 @@ public void TestUpdateAction() throws Exception { updateObj.setField3(4); byte[] updateBytes = schema.encode(updateObj); - Message updateMessage = mock(MessageImpl.class); + Message updateMessage = mock(MessageImpl.class); CompletableFuture future = new CompletableFuture<>(); - Record updateRecord = PulsarRecord.builder() + Record updateRecord = PulsarRecord.builder() .message(updateMessage) + .schema(autoConsumeSchema) .topicName("fake_topic_name") .ackFunction(() -> future.complete(null)) .build(); @@ -312,7 +317,7 @@ public void TestUpdateAction() throws Exception { updateMessage.getValue().toString(), updateRecord.getValue().toString()); - jdbcSink.write(updateRecord); + jdbcSink.write((Record) updateRecord); future.get(1, TimeUnit.SECONDS); // value has been written to db, read it out and verify. @@ -325,18 +330,22 @@ public void TestUpdateAction() throws Exception { } @Test + @SuppressWarnings("unchecked") public void TestDeleteAction() throws Exception { AvroSchema schema = AvroSchema.of(SchemaDefinition.builder().withPojo(Foo.class).build()); + AutoConsumeSchema autoConsumeSchema = new AutoConsumeSchema(); + autoConsumeSchema.setSchema(schema); Foo deleteObj = new Foo(); deleteObj.setField3(5); byte[] deleteBytes = schema.encode(deleteObj); - Message deleteMessage = mock(MessageImpl.class); + Message deleteMessage = mock(MessageImpl.class); CompletableFuture future = new CompletableFuture<>(); - Record deleteRecord = PulsarRecord.builder() + Record deleteRecord = PulsarRecord.builder() .message(deleteMessage) + .schema(autoConsumeSchema) .topicName("fake_topic_name") .ackFunction(() -> future.complete(null)) .build(); @@ -352,7 +361,7 @@ public void TestDeleteAction() throws Exception { deleteMessage.getValue().toString(), deleteRecord.getValue().toString()); - jdbcSink.write(deleteRecord); + jdbcSink.write((Record) deleteRecord); future.get(1, TimeUnit.SECONDS); // value has been written to db, read it out and verify. @@ -848,17 +857,21 @@ public void testNullValueAction(NullValueActionTestConfig config) throws Excepti } } + @SuppressWarnings("unchecked") private Record createMockFooRecord(Foo record, Map actionProperties, CompletableFuture future) { - Message insertMessage = mock(MessageImpl.class); + Message insertMessage = mock(MessageImpl.class); GenericSchema genericAvroSchema; AvroSchema schema = AvroSchema.of(SchemaDefinition.builder().withPojo(Foo.class).withAlwaysAllowNull(true).build()); + AutoConsumeSchema autoConsumeSchema = new AutoConsumeSchema(); + autoConsumeSchema.setSchema(schema); byte[] insertBytes = schema.encode(record); - Record insertRecord = PulsarRecord.builder() + Record insertRecord = PulsarRecord.builder() .message(insertMessage) .topicName("fake_topic_name") + .schema(autoConsumeSchema) .ackFunction(() -> future.complete(true)) .failFunction(() -> future.complete(false)) .build(); @@ -866,7 +879,7 @@ private Record createMockFooRecord(Foo record, Map) insertRecord; } } From e34ea626a65da4c8e1578010f857aa961a7b5c55 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 29 Mar 2024 12:06:26 +0800 Subject: [PATCH 027/580] [improve] [broker] Avoid repeated Read-and-discard when using Key_Shared mode (#22245) --- .../MessageRedeliveryController.java | 8 + ...PersistentDispatcherMultipleConsumers.java | 47 +++- ...tStickyKeyDispatcherMultipleConsumers.java | 104 ++++++- .../client/api/KeySharedSubscriptionTest.java | 266 ++++++++++++++++++ .../client/api/ProducerConsumerBase.java | 66 +++++ ...SubscriptionPauseOnAckStatPersistTest.java | 78 +---- 6 files changed, 477 insertions(+), 92 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java index 5bf3f5506fa81..6380317724207 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java @@ -95,6 +95,14 @@ private void removeFromHashBlocker(long ledgerId, long entryId) { } } + public Long getHash(long ledgerId, long entryId) { + LongPair value = hashesToBeBlocked.get(ledgerId, entryId); + if (value == null) { + return null; + } + return value.first; + } + public void removeAllUpTo(long markDeleteLedgerId, long markDeleteEntryId) { if (!allowOutOfOrderDelivery) { List keysToRemove = new ArrayList<>(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 039104fe0221a..b441400dae11f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -334,24 +334,25 @@ public synchronized void readMoreEntries() { } NavigableSet messagesToReplayNow = getMessagesToReplayNow(messagesToRead); - - if (!messagesToReplayNow.isEmpty()) { + NavigableSet messagesToReplayFiltered = filterOutEntriesWillBeDiscarded(messagesToReplayNow); + if (!messagesToReplayFiltered.isEmpty()) { if (log.isDebugEnabled()) { - log.debug("[{}] Schedule replay of {} messages for {} consumers", name, messagesToReplayNow.size(), - consumerList.size()); + log.debug("[{}] Schedule replay of {} messages for {} consumers", name, + messagesToReplayFiltered.size(), consumerList.size()); } havePendingReplayRead = true; minReplayedPosition = messagesToReplayNow.first(); Set deletedMessages = topic.isDelayedDeliveryEnabled() - ? asyncReplayEntriesInOrder(messagesToReplayNow) : asyncReplayEntries(messagesToReplayNow); + ? asyncReplayEntriesInOrder(messagesToReplayFiltered) + : asyncReplayEntries(messagesToReplayFiltered); // clear already acked positions from replay bucket deletedMessages.forEach(position -> redeliveryMessages.remove(((PositionImpl) position).getLedgerId(), ((PositionImpl) position).getEntryId())); // if all the entries are acked-entries and cleared up from redeliveryMessages, try to read // next entries as readCompletedEntries-callback was never called - if ((messagesToReplayNow.size() - deletedMessages.size()) == 0) { + if ((messagesToReplayFiltered.size() - deletedMessages.size()) == 0) { havePendingReplayRead = false; readMoreEntriesAsync(); } @@ -360,7 +361,7 @@ public synchronized void readMoreEntries() { log.debug("[{}] Dispatcher read is blocked due to unackMessages {} reached to max {}", name, totalUnackedMessages, topic.getMaxUnackedMessagesOnSubscription()); } - } else if (!havePendingRead) { + } else if (!havePendingRead && hasConsumersNeededNormalRead()) { if (shouldPauseOnAckStatePersist(ReadType.Normal)) { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Skipping read for the topic, Due to blocked on ack state persistent.", @@ -396,7 +397,16 @@ public synchronized void readMoreEntries() { topic.getMaxReadPosition()); } } else { - log.debug("[{}] Cannot schedule next read until previous one is done", name); + if (log.isDebugEnabled()) { + if (!messagesToReplayNow.isEmpty()) { + log.debug("[{}] [{}] Skipping read for the topic: because all entries in replay queue were" + + " filtered out due to the mechanism of Key_Shared mode, and the left consumers have" + + " no permits now", + topic.getName(), getSubscriptionName()); + } else { + log.debug("[{}] Cannot schedule next read until previous one is done", name); + } + } } } else { if (log.isDebugEnabled()) { @@ -1179,6 +1189,27 @@ protected synchronized NavigableSet getMessagesToReplayNow(int max } } + /** + * This is a mode method designed for Key_Shared mode. + * Filter out the entries that will be discarded due to the order guarantee mechanism of Key_Shared mode. + * This method is in order to avoid the scenario below: + * - Get positions from the Replay queue. + * - Read entries from BK. + * - The order guarantee mechanism of Key_Shared mode filtered out all the entries. + * - Delivery non entry to the client, but we did a BK read. + */ + protected NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + return src; + } + + /** + * This is a mode method designed for Key_Shared mode, to avoid unnecessary stuck. + * See detail {@link PersistentStickyKeyDispatcherMultipleConsumers#hasConsumersNeededNormalRead}. + */ + protected boolean hasConsumersNeededNormalRead() { + return true; + } + protected synchronized boolean shouldPauseDeliveryForDelayTracker() { return delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().shouldPauseAllDeliveries(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 8f05530f58bfa..ee2ebd7ca867e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -30,13 +30,16 @@ import java.util.Map; import java.util.NavigableSet; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.commons.collections4.MapUtils; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.ConsistentHashingStickyKeyConsumerSelector; @@ -165,6 +168,14 @@ protected Map> initialValue() throws Exception { } }; + private static final FastThreadLocal>> localGroupedPositions = + new FastThreadLocal>>() { + @Override + protected Map> initialValue() throws Exception { + return new HashMap<>(); + } + }; + @Override protected synchronized boolean trySendMessagesToConsumers(ReadType readType, List entries) { long totalMessagesSent = 0; @@ -248,15 +259,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis assert consumer != null; // checked when added to groupedEntries List entriesWithSameKey = current.getValue(); int entriesWithSameKeyCount = entriesWithSameKey.size(); - int availablePermits = Math.max(consumer.getAvailablePermits(), 0); - if (consumer.getMaxUnackedMessages() > 0) { - int remainUnAckedMessages = - // Avoid negative number - Math.max(consumer.getMaxUnackedMessages() - consumer.getUnackedMessages(), 0); - availablePermits = Math.min(availablePermits, remainUnAckedMessages); - } - int maxMessagesForC = Math.min(entriesWithSameKeyCount, availablePermits); - int messagesForC = getRestrictedMaxEntriesForConsumer(consumer, entriesWithSameKey, maxMessagesForC, + int availablePermits = getAvailablePermits(consumer); + int messagesForC = getRestrictedMaxEntriesForConsumer(consumer, + entriesWithSameKey.stream().map(Entry::getPosition).collect(Collectors.toList()), availablePermits, readType, consumerStickyKeyHashesMap.get(consumer)); if (log.isDebugEnabled()) { log.debug("[{}] select consumer {} with messages num {}, read type is {}", @@ -289,7 +294,6 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(messagesForC); totalEntries += filterEntriesForConsumer(entriesWithSameKey, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); - consumer.sendMessages(entriesWithSameKey, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), @@ -332,8 +336,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis return false; } - private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List entries, int maxMessages, - ReadType readType, Set stickyKeyHashes) { + private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List entries, + int availablePermits, ReadType readType, Set stickyKeyHashes) { + int maxMessages = Math.min(entries.size(), availablePermits); if (maxMessages == 0) { return 0; } @@ -378,7 +383,7 @@ private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List en // Here, the consumer is one that has recently joined, so we can only send messages that were // published before it has joined. for (int i = 0; i < maxMessages; i++) { - if (((PositionImpl) entries.get(i).getPosition()).compareTo(maxReadPosition) >= 0) { + if (((PositionImpl) entries.get(i)).compareTo(maxReadPosition) >= 0) { // We have already crossed the divider line. All messages in the list are now // newer than what we can currently dispatch to this consumer return i; @@ -405,6 +410,9 @@ && removeConsumersFromRecentJoinedConsumers()) { } private boolean removeConsumersFromRecentJoinedConsumers() { + if (MapUtils.isEmpty(recentlyJoinedConsumers)) { + return false; + } Iterator> itr = recentlyJoinedConsumers.entrySet().iterator(); boolean hasConsumerRemovedFromTheRecentJoinedConsumers = false; PositionImpl mdp = (PositionImpl) cursor.getMarkDeletedPosition(); @@ -437,6 +445,76 @@ protected synchronized NavigableSet getMessagesToReplayNow(int max } } + private int getAvailablePermits(Consumer c) { + int availablePermits = Math.max(c.getAvailablePermits(), 0); + if (c.getMaxUnackedMessages() > 0) { + // Avoid negative number + int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0); + availablePermits = Math.min(availablePermits, remainUnAckedMessages); + } + return availablePermits; + } + + @Override + protected synchronized NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + if (src.isEmpty()) { + return src; + } + NavigableSet res = new TreeSet<>(); + // Group positions. + final Map> groupedPositions = localGroupedPositions.get(); + groupedPositions.clear(); + for (PositionImpl pos : src) { + Long stickyKeyHash = redeliveryMessages.getHash(pos.getLedgerId(), pos.getEntryId()); + if (stickyKeyHash == null) { + res.add(pos); + continue; + } + Consumer c = selector.select(stickyKeyHash.intValue()); + if (c == null) { + // Maybe using HashRangeExclusiveStickyKeyConsumerSelector. + continue; + } + groupedPositions.computeIfAbsent(c, k -> new ArrayList<>()).add(pos); + } + // Filter positions by the Recently Joined Position rule. + for (Map.Entry> item : groupedPositions.entrySet()) { + int availablePermits = getAvailablePermits(item.getKey()); + if (availablePermits == 0) { + continue; + } + int posCountToRead = getRestrictedMaxEntriesForConsumer(item.getKey(), item.getValue(), availablePermits, + ReadType.Replay, null); + if (posCountToRead > 0) { + res.addAll(item.getValue().subList(0, posCountToRead)); + } + } + return res; + } + + /** + * In Key_Shared mode, the consumer will not receive any entries from a normal reading if it is included in + * {@link #recentlyJoinedConsumers}, they can only receive entries from replay reads. + * If all entries in {@link #redeliveryMessages} have been filtered out due to the order guarantee mechanism, + * Broker need a normal read to make the consumers not included in @link #recentlyJoinedConsumers} will not be + * stuck. See https://github.com/apache/pulsar/pull/7105. + */ + @Override + protected boolean hasConsumersNeededNormalRead() { + for (Consumer consumer : consumerList) { + if (consumer == null || consumer.isBlocked()) { + continue; + } + if (recentlyJoinedConsumers.containsKey(consumer)) { + continue; + } + if (consumer.getAvailablePermits() > 0) { + return true; + } + } + return false; + } + @Override public SubType getType() { return SubType.Key_Shared; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index 18fb141be3178..7219555050839 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -38,6 +38,7 @@ import java.util.Optional; import java.util.Random; import java.util.Set; +import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentSkipListSet; @@ -48,12 +49,17 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import lombok.Cleanup; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.common.api.proto.KeySharedMode; import org.apache.pulsar.common.naming.TopicDomain; @@ -61,6 +67,7 @@ import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.util.Murmur3_32Hash; import org.awaitility.Awaitility; +import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -1630,4 +1637,263 @@ public void testContinueDispatchMessagesWhenMessageDelayed() throws Exception { log.info("Got {} other messages...", sum); Assert.assertEquals(sum, delayedMessages + messages); } + + private AtomicInteger injectReplayReadCounter(String topicName, String cursorName) throws Exception { + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get(); + ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedCursorImpl cursor = (ManagedCursorImpl) managedLedger.openCursor(cursorName); + managedLedger.getCursors().removeCursor(cursor.getName()); + managedLedger.getActiveCursors().removeCursor(cursor.getName()); + ManagedCursorImpl spyCursor = Mockito.spy(cursor); + managedLedger.getCursors().add(spyCursor, PositionImpl.EARLIEST); + managedLedger.getActiveCursors().add(spyCursor, PositionImpl.EARLIEST); + AtomicInteger replyReadCounter = new AtomicInteger(); + Mockito.doAnswer(invocation -> { + if (!String.valueOf(invocation.getArguments()[2]).equals("Normal")) { + replyReadCounter.incrementAndGet(); + } + return invocation.callRealMethod(); + }).when(spyCursor).asyncReplayEntries(Mockito.anySet(), Mockito.any(), Mockito.any()); + Mockito.doAnswer(invocation -> { + if (!String.valueOf(invocation.getArguments()[2]).equals("Normal")) { + replyReadCounter.incrementAndGet(); + } + return invocation.callRealMethod(); + }).when(spyCursor).asyncReplayEntries(Mockito.anySet(), Mockito.any(), Mockito.any(), Mockito.anyBoolean()); + admin.topics().createSubscription(topicName, cursorName, MessageId.earliest); + return replyReadCounter; + } + + @Test + public void testNoRepeatedReadAndDiscard() throws Exception { + int delayedMessages = 100; + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subName = "my-sub"; + admin.topics().createNonPartitionedTopic(topic); + AtomicInteger replyReadCounter = injectReplayReadCounter(topic, subName); + + // Send messages. + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32).topic(topic).enableBatching(false).create(); + for (int i = 0; i < delayedMessages; i++) { + MessageId messageId = producer.newMessage() + .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .value(100 + i) + .send(); + log.info("Published message :{}", messageId); + } + producer.close(); + + // Make ack holes. + Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + List msgList1 = new ArrayList<>(); + List msgList2 = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + Message msg1 = consumer1.receive(1, TimeUnit.SECONDS); + if (msg1 != null) { + msgList1.add(msg1); + } + Message msg2 = consumer2.receive(1, TimeUnit.SECONDS); + if (msg2 != null) { + msgList2.add(msg2); + } + } + Consumer redeliverConsumer = null; + if (!msgList1.isEmpty()) { + msgList1.forEach(msg -> consumer1.acknowledgeAsync(msg)); + redeliverConsumer = consumer2; + } else { + msgList2.forEach(msg -> consumer2.acknowledgeAsync(msg)); + redeliverConsumer = consumer1; + } + + // consumer3 will be added to the "recentJoinedConsumers". + Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(1000) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + redeliverConsumer.close(); + + // Verify: no repeated Read-and-discard. + Thread.sleep(5 * 1000); + int maxReplayCount = delayedMessages * 2; + log.info("Reply read count: {}", replyReadCounter.get()); + assertTrue(replyReadCounter.get() < maxReplayCount); + + // cleanup. + consumer1.close(); + consumer2.close(); + consumer3.close(); + admin.topics().delete(topic, false); + } + + /** + * This test is in order to guarantee the feature added by https://github.com/apache/pulsar/pull/7105. + * 1. Start 3 consumers: + * - consumer1 will be closed and trigger a messages redeliver. + * - consumer2 will not ack any messages to make the new consumer joined late will be stuck due + * to the mechanism "recentlyJoinedConsumers". + * - consumer3 will always receive and ack messages. + * 2. Add consumer4 after consumer1 was close, and consumer4 will be stuck due to the mechanism + * "recentlyJoinedConsumers". + * 3. Verify: + * - (Main purpose) consumer3 can still receive messages util the cursor.readerPosition is larger than LAC. + * - no repeated Read-and-discard. + * - at last, all messages will be received. + */ + @Test(timeOut = 180 * 1000) // the test will be finished in 60s. + public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { + final int messagesSentPerTime = 100; + final Set totalReceivedMessages = new TreeSet<>(); + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subName = "my-sub"; + admin.topics().createNonPartitionedTopic(topic); + AtomicInteger replyReadCounter = injectReplayReadCounter(topic, subName); + + // Send messages. + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32).topic(topic).enableBatching(false).create(); + for (int i = 0; i < messagesSentPerTime; i++) { + MessageId messageId = producer.newMessage() + .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .value(100 + i) + .send(); + log.info("Published message :{}", messageId); + } + + // 1. Start 3 consumers and make ack holes. + // - one consumer will be closed and trigger a messages redeliver. + // - one consumer will not ack any messages to make the new consumer joined late will be stuck due to the + // mechanism "recentlyJoinedConsumers". + // - one consumer will always receive and ack messages. + Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + List msgList1 = new ArrayList<>(); + List msgList2 = new ArrayList<>(); + List msgList3 = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + Message msg1 = consumer1.receive(1, TimeUnit.SECONDS); + if (msg1 != null) { + totalReceivedMessages.add(msg1.getValue()); + msgList1.add(msg1); + } + Message msg2 = consumer2.receive(1, TimeUnit.SECONDS); + if (msg2 != null) { + totalReceivedMessages.add(msg2.getValue()); + msgList2.add(msg2); + } + Message msg3 = consumer3.receive(1, TimeUnit.SECONDS); + if (msg2 != null) { + totalReceivedMessages.add(msg3.getValue()); + msgList3.add(msg3); + } + } + Consumer consumerWillBeClose = null; + Consumer consumerAlwaysAck = null; + Consumer consumerStuck = null; + if (!msgList1.isEmpty()) { + msgList1.forEach(msg -> consumer1.acknowledgeAsync(msg)); + consumerAlwaysAck = consumer1; + consumerWillBeClose = consumer2; + consumerStuck = consumer3; + } else if (!msgList2.isEmpty()){ + msgList2.forEach(msg -> consumer2.acknowledgeAsync(msg)); + consumerAlwaysAck = consumer2; + consumerWillBeClose = consumer3; + consumerStuck = consumer1; + } else { + msgList3.forEach(msg -> consumer3.acknowledgeAsync(msg)); + consumerAlwaysAck = consumer3; + consumerWillBeClose = consumer1; + consumerStuck = consumer2; + } + + // 2. Add consumer4 after "consumerWillBeClose" was close, and consumer4 will be stuck due to the mechanism + // "recentlyJoinedConsumers". + Consumer consumer4 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(1000) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + consumerWillBeClose.close(); + + Thread.sleep(2000); + + for (int i = messagesSentPerTime; i < messagesSentPerTime * 2; i++) { + MessageId messageId = producer.newMessage() + .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .value(100 + i) + .send(); + log.info("Published message :{}", messageId); + } + + // Send messages again. + // Verify: "consumerAlwaysAck" can receive messages util the cursor.readerPosition is larger than LAC. + while (true) { + Message msg = consumerAlwaysAck.receive(2, TimeUnit.SECONDS); + if (msg == null) { + break; + } + totalReceivedMessages.add(msg.getValue()); + consumerAlwaysAck.acknowledge(msg); + } + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedCursorImpl cursor = (ManagedCursorImpl) managedLedger.openCursor(subName); + log.info("cursor_readPosition {}, LAC {}", cursor.getReadPosition(), managedLedger.getLastConfirmedEntry()); + assertTrue(((PositionImpl) cursor.getReadPosition()) + .compareTo((PositionImpl) managedLedger.getLastConfirmedEntry()) > 0); + + // Make all consumers to start to read and acknowledge messages. + // Verify: no repeated Read-and-discard. + Thread.sleep(5 * 1000); + int maxReplayCount = messagesSentPerTime * 2; + log.info("Reply read count: {}", replyReadCounter.get()); + assertTrue(replyReadCounter.get() < maxReplayCount); + // Verify: at last, all messages will be received. + ReceivedMessages receivedMessages = ackAllMessages(consumerAlwaysAck, consumerStuck, consumer4); + totalReceivedMessages.addAll(receivedMessages.messagesReceived.stream().map(p -> p.getRight()).collect( + Collectors.toList())); + assertEquals(totalReceivedMessages.size(), messagesSentPerTime * 2); + + // cleanup. + consumer1.close(); + consumer2.close(); + consumer3.close(); + consumer4.close(); + producer.close(); + admin.topics().delete(topic, false); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java index f58c1fa26afc7..ef070250ca1aa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java @@ -21,9 +21,14 @@ import com.google.common.collect.Sets; import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; import java.util.Random; import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.function.BiFunction; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -69,4 +74,65 @@ protected String newTopicName() { return "my-property/my-ns/topic-" + Long.toHexString(random.nextLong()); } + protected ReceivedMessages receiveAndAckMessages( + BiFunction ackPredicate, + Consumer...consumers) throws Exception { + ReceivedMessages receivedMessages = new ReceivedMessages(); + while (true) { + int receivedMsgCount = 0; + for (int i = 0; i < consumers.length; i++) { + Consumer consumer = consumers[i]; + while (true) { + Message msg = consumer.receive(2, TimeUnit.SECONDS); + if (msg != null) { + receivedMsgCount++; + T v = msg.getValue(); + MessageId messageId = msg.getMessageId(); + receivedMessages.messagesReceived.add(Pair.of(msg.getMessageId(), v)); + if (ackPredicate.apply(messageId, v)) { + consumer.acknowledge(msg); + receivedMessages.messagesAcked.add(Pair.of(msg.getMessageId(), v)); + } + } else { + break; + } + } + } + // Because of the possibility of consumers getting stuck with each other, only jump out of the loop if all + // consumers could not receive messages. + if (receivedMsgCount == 0) { + break; + } + } + return receivedMessages; + } + + protected ReceivedMessages ackAllMessages(Consumer...consumers) throws Exception { + return receiveAndAckMessages((msgId, msgV) -> true, consumers); + } + + protected static class ReceivedMessages { + + List> messagesReceived = new ArrayList<>(); + + List> messagesAcked = new ArrayList<>(); + + public boolean hasReceivedMessage(T v) { + for (Pair pair : messagesReceived) { + if (pair.getRight().equals(v)) { + return true; + } + } + return false; + } + + public boolean hasAckedMessage(T v) { + for (Pair pair : messagesAcked) { + if (pair.getRight().equals(v)) { + return true; + } + } + return false; + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java index 22edbc36f6ce0..9a4de8ecf21cc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java @@ -21,13 +21,10 @@ import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Arrays; -import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; -import java.util.function.BiFunction; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; -import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService; @@ -148,71 +145,10 @@ private enum SkipType{ RESET_CURSOR; } - private ReceivedMessages receiveAndAckMessages(BiFunction ackPredicate, - Consumer...consumers) throws Exception { - ReceivedMessages receivedMessages = new ReceivedMessages(); - while (true) { - int receivedMsgCount = 0; - for (int i = 0; i < consumers.length; i++) { - Consumer consumer = consumers[i]; - while (true) { - Message msg = consumer.receive(2, TimeUnit.SECONDS); - if (msg != null) { - receivedMsgCount++; - String v = msg.getValue(); - MessageId messageId = msg.getMessageId(); - receivedMessages.messagesReceived.add(Pair.of(msg.getMessageId(), v)); - if (ackPredicate.apply(messageId, v)) { - consumer.acknowledge(msg); - receivedMessages.messagesAcked.add(Pair.of(msg.getMessageId(), v)); - } - } else { - break; - } - } - } - // Because of the possibility of consumers getting stuck with each other, only jump out of the loop if all - // consumers could not receive messages. - if (receivedMsgCount == 0) { - break; - } - } - return receivedMessages; - } - - private ReceivedMessages ackAllMessages(Consumer...consumers) throws Exception { - return receiveAndAckMessages((msgId, msgV) -> true, consumers); - } - - private ReceivedMessages ackOddMessagesOnly(Consumer...consumers) throws Exception { + private ReceivedMessages ackOddMessagesOnly(Consumer...consumers) throws Exception { return receiveAndAckMessages((msgId, msgV) -> Integer.valueOf(msgV) % 2 == 1, consumers); } - private static class ReceivedMessages { - - List> messagesReceived = new ArrayList<>(); - - List> messagesAcked = new ArrayList<>(); - - public boolean hasReceivedMessage(String v) { - for (Pair pair : messagesReceived) { - if (pair.getRight().equals(v)) { - return true; - } - } - return false; - } - - public boolean hasAckedMessage(String v) { - for (Pair pair : messagesAcked) { - if (pair.getRight().equals(v)) { - return true; - } - } - return false; - } - } - @DataProvider(name = "typesOfSetDispatcherPauseOnAckStatePersistent") public Object[][] typesOfSetDispatcherPauseOnAckStatePersistent() { return new Object[][]{ @@ -367,7 +303,7 @@ public void testPauseOnAckStatPersist(SubscriptionType subscriptionType) throws // Verify: after ack messages, will unpause the dispatcher. c1.acknowledge(messageIdsSent); - ReceivedMessages receivedMessagesAfterPause = ackAllMessages(c1); + ReceivedMessages receivedMessagesAfterPause = ackAllMessages(c1); Assert.assertTrue(receivedMessagesAfterPause.hasReceivedMessage(specifiedMessage)); Assert.assertTrue(receivedMessagesAfterPause.hasAckedMessage(specifiedMessage)); @@ -417,7 +353,7 @@ public void testUnPauseOnSkipEntries(SkipType skipType) throws Exception { final String specifiedMessage2 = "9876543211"; p1.send(specifiedMessage2); - ReceivedMessages receivedMessagesAfterPause = ackAllMessages(c1); + ReceivedMessages receivedMessagesAfterPause = ackAllMessages(c1); Assert.assertTrue(receivedMessagesAfterPause.hasReceivedMessage(specifiedMessage2)); Assert.assertTrue(receivedMessagesAfterPause.hasAckedMessage(specifiedMessage2)); @@ -520,7 +456,7 @@ public void testPauseOnAckStatPersistNotAffectReplayRead(SubscriptionType subscr messageIdsSent.add(messageId); } // Make ack holes. - ReceivedMessages receivedMessagesC1 = ackOddMessagesOnly(c1); + ReceivedMessages receivedMessagesC1 = ackOddMessagesOnly(c1); verifyAckHolesIsMuchThanLimit(tpName, subscription); cancelPendingRead(tpName, subscription); @@ -540,7 +476,7 @@ public void testPauseOnAckStatPersistNotAffectReplayRead(SubscriptionType subscr // Verify: close the previous consumer, the new one could receive all messages. c1.close(); - ReceivedMessages receivedMessagesC2 = ackAllMessages(c2); + ReceivedMessages receivedMessagesC2 = ackAllMessages(c2); int messageCountAckedByC1 = receivedMessagesC1.messagesAcked.size(); int messageCountAckedByC2 = receivedMessagesC2.messagesAcked.size(); Assert.assertEquals(messageCountAckedByC2, msgSendCount - messageCountAckedByC1 + specifiedMessageCount); @@ -577,7 +513,7 @@ public void testMultiConsumersPauseOnAckStatPersistNotAffectReplayRead(Subscript messageIdsSent.add(messageId); } // Make ack holes. - ReceivedMessages receivedMessagesC1AndC2 = ackOddMessagesOnly(c1, c2); + ReceivedMessages receivedMessagesC1AndC2 = ackOddMessagesOnly(c1, c2); verifyAckHolesIsMuchThanLimit(tpName, subscription); cancelPendingRead(tpName, subscription); @@ -601,7 +537,7 @@ public void testMultiConsumersPauseOnAckStatPersistNotAffectReplayRead(Subscript // Verify: close the previous consumer, the new one could receive all messages. c1.close(); c2.close(); - ReceivedMessages receivedMessagesC3AndC4 = ackAllMessages(c3, c4); + ReceivedMessages receivedMessagesC3AndC4 = ackAllMessages(c3, c4); int messageCountAckedByC1AndC2 = receivedMessagesC1AndC2.messagesAcked.size(); int messageCountAckedByC3AndC4 = receivedMessagesC3AndC4.messagesAcked.size(); Assert.assertEquals(messageCountAckedByC3AndC4, From 0701d7eedcef6aae750b5067139caf8e73434818 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Fri, 29 Mar 2024 12:43:11 +0800 Subject: [PATCH 028/580] [fix][sec] implicit narrowing conversion in compound assignment (#22074) --- .../apache/pulsar/common/policies/data/SubscriptionStats.java | 2 +- .../common/policies/data/stats/SubscriptionStatsImpl.java | 2 +- .../pulsar/policies/data/loadbalancer/LocalBrokerData.java | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index 9ff94a2952ea3..d4850adaa6f22 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -46,7 +46,7 @@ public interface SubscriptionStats { double getMessageAckRate(); /** Chunked message dispatch rate. */ - int getChunkedMessageRate(); + double getChunkedMessageRate(); /** Number of entries in the subscription backlog. */ long getMsgBacklog(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java index bed8aabf27d8d..a8ea0060629a0 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java @@ -54,7 +54,7 @@ public class SubscriptionStatsImpl implements SubscriptionStats { public double messageAckRate; /** Chunked message dispatch rate. */ - public int chunkedMessageRate; + public double chunkedMessageRate; /** Number of entries in the subscription backlog. */ public long msgBacklog; diff --git a/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerData.java b/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerData.java index 60ade64e68871..8c27323694598 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerData.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerData.java @@ -66,7 +66,7 @@ public class LocalBrokerData implements LoadManagerReport { // The stats given in the most recent invocation of update. private Map lastStats; - private int numTopics; + private long numTopics; private int numBundles; private int numConsumers; private int numProducers; @@ -202,7 +202,7 @@ private void updateBundleData(final Map bundleStat msgRateOut = 0; msgThroughputIn = 0; msgThroughputOut = 0; - int totalNumTopics = 0; + long totalNumTopics = 0; int totalNumBundles = 0; int totalNumConsumers = 0; int totalNumProducers = 0; From 7315aeb6258b7adc9d874268d50acb95ffc0cf2b Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Fri, 29 Mar 2024 13:47:29 +0800 Subject: [PATCH 029/580] [improve][fn] Pass FunctionDetails to Go instance (#22350) --- pulsar-function-go/conf/conf.go | 2 + pulsar-function-go/pf/instanceConf.go | 11 + pulsar-function-go/pf/instanceConf_test.go | 207 ++++++++++++++++++ .../instance/go/GoInstanceConfig.java | 2 + .../functions/runtime/RuntimeUtils.java | 6 + .../kubernetes/KubernetesRuntimeTest.java | 8 + 6 files changed, 236 insertions(+) diff --git a/pulsar-function-go/conf/conf.go b/pulsar-function-go/conf/conf.go index 03513648fac3b..1442a0f865f4a 100644 --- a/pulsar-function-go/conf/conf.go +++ b/pulsar-function-go/conf/conf.go @@ -91,6 +91,8 @@ type Conf struct { UserConfig string `json:"userConfig" yaml:"userConfig"` //metrics config MetricsPort int `json:"metricsPort" yaml:"metricsPort"` + // FunctionDetails + FunctionDetails string `json:"functionDetails" yaml:"functionDetails"` } var ( diff --git a/pulsar-function-go/pf/instanceConf.go b/pulsar-function-go/pf/instanceConf.go index 4cb60dd258ad9..844a2bc9b89a3 100644 --- a/pulsar-function-go/pf/instanceConf.go +++ b/pulsar-function-go/pf/instanceConf.go @@ -25,7 +25,9 @@ import ( "time" "github.com/apache/pulsar/pulsar-function-go/conf" + log "github.com/apache/pulsar/pulsar-function-go/logutil" pb "github.com/apache/pulsar/pulsar-function-go/pb" + "google.golang.org/protobuf/encoding/protojson" ) // This is the config passed to the Golang Instance. Contains all the information @@ -122,6 +124,15 @@ func newInstanceConfWithConf(cfg *conf.Conf) *instanceConf { tlsAllowInsecure: cfg.TLSAllowInsecureConnection, tlsHostnameVerification: cfg.TLSHostnameVerificationEnable, } + // parse the raw function details and ignore the unmarshal error(fallback to original way) + if cfg.FunctionDetails != "" { + functionDetails := pb.FunctionDetails{} + if err := protojson.Unmarshal([]byte(cfg.FunctionDetails), &functionDetails); err != nil { + log.Errorf("Failed to unmarshal function details: %v", err) + } else { + instanceConf.funcDetails = functionDetails + } + } if instanceConf.funcDetails.ProcessingGuarantees == pb.ProcessingGuarantees_EFFECTIVELY_ONCE { panic("Go instance current not support EFFECTIVELY_ONCE processing guarantees.") diff --git a/pulsar-function-go/pf/instanceConf_test.go b/pulsar-function-go/pf/instanceConf_test.go index 02aef913ebc97..cc5f46e2fe12b 100644 --- a/pulsar-function-go/pf/instanceConf_test.go +++ b/pulsar-function-go/pf/instanceConf_test.go @@ -20,6 +20,7 @@ package pf import ( + "fmt" "testing" cfg "github.com/apache/pulsar/pulsar-function-go/conf" @@ -113,3 +114,209 @@ func TestInstanceConf_Fail(t *testing.T) { newInstanceConfWithConf(&cfg.Conf{ProcessingGuarantees: 3}) }, "Should have a panic") } + +func TestInstanceConf_WithDetails(t *testing.T) { + cfg := &cfg.Conf{ + FunctionDetails: `{"tenant":"public","namespace":"default","name":"test-function","className":"process", +"logTopic":"test-logs","userConfig":"{\"key1\":\"value1\"}","runtime":"GO","autoAck":true,"parallelism":1, +"source":{"configs":"{\"username\":\"admin\"}","typeClassName":"string","timeoutMs":"15000", +"subscriptionName":"test-subscription","inputSpecs":{"input":{"schemaType":"avro","receiverQueueSize":{"value":1000}, +"schemaProperties":{"schema_prop1":"schema1"},"consumerProperties":{"consumer_prop1":"consumer1"},"cryptoSpec": +{"cryptoKeyReaderClassName":"key-reader","producerCryptoFailureAction":"SEND","consumerCryptoFailureAction":"CONSUME"}}} +,"negativeAckRedeliveryDelayMs":"15000"},"sink":{"configs":"{\"password\":\"admin\"}","topic":"test-output", +"typeClassName":"string","schemaType":"avro","producerSpec":{"maxPendingMessages":2000,"useThreadLocalProducers":true, +"cryptoSpec":{"cryptoKeyReaderClassName":"key-reader","producerCryptoFailureAction":"DISCARD"}, +"batchBuilder":"DEFAULT"}},"resources":{"cpu":2.0,"ram":"1024","disk":"1024"},"packageUrl":"/path/to/package", +"retryDetails":{"maxMessageRetries":3,"deadLetterTopic":"test-dead-letter-topic"},"secretsMap": +"{\"secret1\":\"secret-value1\"}","runtimeFlags":"flags","componentType":"FUNCTION","customRuntimeOptions":"options", +"retainOrdering":true,"retainKeyOrdering":true,"subscriptionPosition":"EARLIEST"}`, + } + instanceConf := newInstanceConfWithConf(cfg) + assert.Equal(t, "public", instanceConf.funcDetails.Tenant) + assert.Equal(t, "default", instanceConf.funcDetails.Namespace) + assert.Equal(t, "test-function", instanceConf.funcDetails.Name) + assert.Equal(t, "process", instanceConf.funcDetails.ClassName) + assert.Equal(t, "test-logs", instanceConf.funcDetails.LogTopic) + assert.Equal(t, pb.ProcessingGuarantees_ATLEAST_ONCE, instanceConf.funcDetails.ProcessingGuarantees) + assert.Equal(t, `{"key1":"value1"}`, instanceConf.funcDetails.UserConfig) + assert.Equal(t, `{"secret1":"secret-value1"}`, instanceConf.funcDetails.SecretsMap) + assert.Equal(t, pb.FunctionDetails_GO, instanceConf.funcDetails.Runtime) + + assert.Equal(t, true, instanceConf.funcDetails.AutoAck) + assert.Equal(t, int32(1), instanceConf.funcDetails.Parallelism) + + sourceSpec := pb.SourceSpec{ + TypeClassName: "string", + TimeoutMs: 15000, + Configs: `{"username":"admin"}`, + SubscriptionName: "test-subscription", + SubscriptionType: pb.SubscriptionType_SHARED, + NegativeAckRedeliveryDelayMs: 15000, + InputSpecs: map[string]*pb.ConsumerSpec{ + "input": { + SchemaType: "avro", + SchemaProperties: map[string]string{ + "schema_prop1": "schema1", + }, + ConsumerProperties: map[string]string{ + "consumer_prop1": "consumer1", + }, + ReceiverQueueSize: &pb.ConsumerSpec_ReceiverQueueSize{ + Value: 1000, + }, + CryptoSpec: &pb.CryptoSpec{ + CryptoKeyReaderClassName: "key-reader", + ProducerCryptoFailureAction: pb.CryptoSpec_SEND, + ConsumerCryptoFailureAction: pb.CryptoSpec_CONSUME, + }, + }, + }, + } + assert.Equal(t, sourceSpec.String(), instanceConf.funcDetails.Source.String()) + + sinkSpec := pb.SinkSpec{ + TypeClassName: "string", + Topic: "test-output", + Configs: `{"password":"admin"}`, + SchemaType: "avro", + ProducerSpec: &pb.ProducerSpec{ + MaxPendingMessages: 2000, + UseThreadLocalProducers: true, + CryptoSpec: &pb.CryptoSpec{ + CryptoKeyReaderClassName: "key-reader", + ProducerCryptoFailureAction: pb.CryptoSpec_DISCARD, + ConsumerCryptoFailureAction: pb.CryptoSpec_FAIL, + }, + BatchBuilder: "DEFAULT", + }, + } + assert.Equal(t, sinkSpec.String(), instanceConf.funcDetails.Sink.String()) + + resource := pb.Resources{ + Cpu: 2.0, + Ram: 1024, + Disk: 1024, + } + assert.Equal(t, resource.String(), instanceConf.funcDetails.Resources.String()) + assert.Equal(t, "/path/to/package", instanceConf.funcDetails.PackageUrl) + + retryDetails := pb.RetryDetails{ + MaxMessageRetries: 3, + DeadLetterTopic: "test-dead-letter-topic", + } + assert.Equal(t, retryDetails.String(), instanceConf.funcDetails.RetryDetails.String()) + + assert.Equal(t, "flags", instanceConf.funcDetails.RuntimeFlags) + assert.Equal(t, pb.FunctionDetails_FUNCTION, instanceConf.funcDetails.ComponentType) + assert.Equal(t, "options", instanceConf.funcDetails.CustomRuntimeOptions) + assert.Equal(t, "", instanceConf.funcDetails.Builtin) + assert.Equal(t, true, instanceConf.funcDetails.RetainOrdering) + assert.Equal(t, true, instanceConf.funcDetails.RetainKeyOrdering) + assert.Equal(t, pb.SubscriptionPosition_EARLIEST, instanceConf.funcDetails.SubscriptionPosition) +} + +func TestInstanceConf_WithEmptyOrInvalidDetails(t *testing.T) { + testCases := []struct { + name string + details string + }{ + { + name: "empty details", + details: "", + }, + { + name: "invalid details", + details: "error", + }, + } + + for i, testCase := range testCases { + + t.Run(fmt.Sprintf("testCase[%d] %s", i, testCase.name), func(t *testing.T) { + cfg := &cfg.Conf{ + FunctionDetails: testCase.details, + Tenant: "public", + NameSpace: "default", + Name: "test-function", + LogTopic: "test-logs", + ProcessingGuarantees: 0, + UserConfig: `{"key1":"value1"}`, + SecretsMap: `{"secret1":"secret-value1"}`, + Runtime: 3, + AutoACK: true, + Parallelism: 1, + SubscriptionType: 1, + TimeoutMs: 15000, + SubscriptionName: "test-subscription", + CleanupSubscription: false, + SubscriptionPosition: 0, + SinkSpecTopic: "test-output", + SinkSchemaType: "avro", + Cpu: 2.0, + Ram: 1024, + Disk: 1024, + MaxMessageRetries: 3, + DeadLetterTopic: "test-dead-letter-topic", + SourceInputSpecs: map[string]string{ + "input": `{"schemaType":"avro","receiverQueueSize":{"value":1000},"schemaProperties": +{"schema_prop1":"schema1"},"consumerProperties":{"consumer_prop1":"consumer1"}}`, + }, + } + instanceConf := newInstanceConfWithConf(cfg) + + assert.Equal(t, "public", instanceConf.funcDetails.Tenant) + assert.Equal(t, "default", instanceConf.funcDetails.Namespace) + assert.Equal(t, "test-function", instanceConf.funcDetails.Name) + assert.Equal(t, "test-logs", instanceConf.funcDetails.LogTopic) + assert.Equal(t, pb.ProcessingGuarantees_ATLEAST_ONCE, instanceConf.funcDetails.ProcessingGuarantees) + assert.Equal(t, `{"key1":"value1"}`, instanceConf.funcDetails.UserConfig) + assert.Equal(t, `{"secret1":"secret-value1"}`, instanceConf.funcDetails.SecretsMap) + assert.Equal(t, pb.FunctionDetails_GO, instanceConf.funcDetails.Runtime) + + assert.Equal(t, true, instanceConf.funcDetails.AutoAck) + assert.Equal(t, int32(1), instanceConf.funcDetails.Parallelism) + + sourceSpec := pb.SourceSpec{ + SubscriptionType: pb.SubscriptionType_FAILOVER, + TimeoutMs: 15000, + SubscriptionName: "test-subscription", + CleanupSubscription: false, + SubscriptionPosition: pb.SubscriptionPosition_LATEST, + InputSpecs: map[string]*pb.ConsumerSpec{ + "input": { + SchemaType: "avro", + SchemaProperties: map[string]string{ + "schema_prop1": "schema1", + }, + ConsumerProperties: map[string]string{ + "consumer_prop1": "consumer1", + }, + ReceiverQueueSize: &pb.ConsumerSpec_ReceiverQueueSize{ + Value: 1000, + }, + }, + }, + } + assert.Equal(t, sourceSpec.String(), instanceConf.funcDetails.Source.String()) + + sinkSpec := pb.SinkSpec{ + Topic: "test-output", + SchemaType: "avro", + } + assert.Equal(t, sinkSpec.String(), instanceConf.funcDetails.Sink.String()) + + resource := pb.Resources{ + Cpu: 2.0, + Ram: 1024, + Disk: 1024, + } + assert.Equal(t, resource.String(), instanceConf.funcDetails.Resources.String()) + + retryDetails := pb.RetryDetails{ + MaxMessageRetries: 3, + DeadLetterTopic: "test-dead-letter-topic", + } + assert.Equal(t, retryDetails.String(), instanceConf.funcDetails.RetryDetails.String()) + }) + } +} diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/go/GoInstanceConfig.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/go/GoInstanceConfig.java index 599b6ed8f4fdf..467ec74921330 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/go/GoInstanceConfig.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/go/GoInstanceConfig.java @@ -83,4 +83,6 @@ public class GoInstanceConfig { private String deadLetterTopic = ""; private int metricsPort; + + private String functionDetails = ""; } diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/RuntimeUtils.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/RuntimeUtils.java index 0214b18fb2326..6160626c958ef 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/RuntimeUtils.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/RuntimeUtils.java @@ -139,6 +139,12 @@ public static List getGoInstanceCmd(InstanceConfig instanceConfig, final List args = new LinkedList<>(); GoInstanceConfig goInstanceConfig = new GoInstanceConfig(); + // pass the raw functino details directly so that we don't need to assemble the `instanceConf.funcDetails` + // manually in Go instance + String functionDetails = + JsonFormat.printer().omittingInsignificantWhitespace().print(instanceConfig.getFunctionDetails()); + goInstanceConfig.setFunctionDetails(functionDetails); + if (instanceConfig.getClusterName() != null) { goInstanceConfig.setClusterName(instanceConfig.getClusterName()); } diff --git a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java index 02f3c0d23fb17..980f763f7c303 100644 --- a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java +++ b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java @@ -1006,6 +1006,14 @@ private void verifyGolangInstance(InstanceConfig config) throws Exception { assertEquals(goInstanceConfig.get("expectedHealthCheckInterval"), 0); assertEquals(goInstanceConfig.get("deadLetterTopic"), ""); assertEquals(goInstanceConfig.get("metricsPort"), 4331); + assertEquals(goInstanceConfig.get("functionDetails"), "{\"tenant\":\"tenant\",\"namespace\":\"namespace\"," + + "\"name\":\"container\",\"className\":\"org.apache.pulsar.functions.utils.functioncache" + + ".AddFunction\",\"logTopic\":\"container-log\",\"runtime\":\"GO\",\"source\":{\"className\":\"org" + + ".pulsar.pulsar.TestSource\",\"subscriptionType\":\"FAILOVER\",\"typeClassName\":\"java.lang" + + ".String\",\"inputSpecs\":{\"test_src\":{}}},\"sink\":{\"className\":\"org.pulsar.pulsar" + + ".TestSink\",\"topic\":\"container-output\",\"serDeClassName\":\"org.apache.pulsar.functions" + + ".runtime.serde.Utf8Serializer\",\"typeClassName\":\"java.lang.String\"},\"resources\":{\"cpu\":1" + + ".0,\"ram\":\"1000\",\"disk\":\"10000\"}}"); // check padding and xmx V1Container containerSpec = container.getFunctionContainer(Collections.emptyList(), RESOURCES); From 9529738efe2faabe5283f74921f780d8589fb437 Mon Sep 17 00:00:00 2001 From: houxiaoyu Date: Sat, 30 Mar 2024 21:38:55 +0800 Subject: [PATCH 030/580] [fix][ml] No rollover inactive ledgers when metadata service invalid (#22284) ### Motivation We should not rollover inactive ledgers when metadata service is invailable. ### Modifications Checking metadata service is vailable when schedule `checkInactiveLedgerAndRollOver` --- .../mledger/impl/ManagedLedgerImpl.java | 7 +++--- .../mledger/impl/ManagedLedgerTest.java | 24 +++++++++++++++++++ 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 0f089ef4a8573..3a12cb2ad6c74 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -4478,9 +4478,10 @@ private void cancelScheduledTasks() { @Override public boolean checkInactiveLedgerAndRollOver() { - long currentTimeMs = System.currentTimeMillis(); - if (currentLedgerEntries > 0 && inactiveLedgerRollOverTimeMs > 0 && currentTimeMs > (lastAddEntryTimeMs - + inactiveLedgerRollOverTimeMs)) { + if (factory.isMetadataServiceAvailable() + && currentLedgerEntries > 0 + && inactiveLedgerRollOverTimeMs > 0 + && System.currentTimeMillis() > (lastAddEntryTimeMs + inactiveLedgerRollOverTimeMs)) { log.info("[{}] Closing inactive ledger, last-add entry {}", name, lastAddEntryTimeMs); if (STATE_UPDATER.compareAndSet(this, State.LedgerOpened, State.ClosingLedger)) { LedgerHandle currentLedger = this.currentLedger; diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 0baafa7e1b01c..6b409babcb461 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -3946,6 +3946,30 @@ public void testDontRollOverEmptyInactiveLedgers() throws Exception { ledger.close(); } + @Test + public void testDontRollOverInactiveLedgersWhenMetadataServiceInvalid() throws Exception { + int inactiveLedgerRollOverTimeMs = 5; + @Cleanup("shutdown") + ManagedLedgerFactoryImpl factory = spy(new ManagedLedgerFactoryImpl(metadataStore, bkc)); + // mock metadata service invalid + when(factory.isMetadataServiceAvailable()).thenReturn(false); + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setInactiveLedgerRollOverTime(inactiveLedgerRollOverTimeMs, TimeUnit.MILLISECONDS); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("rollover_inactive", config); + + long ledgerId = ledger.currentLedger.getId(); + + Thread.sleep(inactiveLedgerRollOverTimeMs * 5); + ledger.checkInactiveLedgerAndRollOver(); + + Thread.sleep(inactiveLedgerRollOverTimeMs * 5); + ledger.checkInactiveLedgerAndRollOver(); + + assertEquals(ledger.currentLedger.getId(), ledgerId); + + ledger.close(); + } + @Test public void testOffloadTaskCancelled() throws Exception { @Cleanup("shutdown") From 3eb3b1cd23d2cc11424bf882e244d3bc2e92bf27 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Mon, 1 Apr 2024 01:52:21 -0700 Subject: [PATCH 031/580] [fix][broker] Skip topic.close during unloading if the topic future fails with ownership check, and fix isBundleOwnedByAnyBroker to use ns.checkOwnershipPresentAsync for ExtensibleLoadBalancer (#22379) --- .../extensions/manager/UnloadManager.java | 14 +++++- .../pulsar/broker/service/BrokerService.java | 11 ++++- .../pulsar/broker/web/PulsarWebResource.java | 5 +++ .../ExtensibleLoadManagerImplBaseTest.java | 8 ---- .../ExtensibleLoadManagerImplTest.java | 27 ++++++++++++ .../extensions/manager/UnloadManagerTest.java | 44 +++++++++++-------- 6 files changed, 80 insertions(+), 29 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java index b210dedbfe8f4..ffae9475243da 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.manager; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigning; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.models.UnloadDecision.Label.Failure; import static org.apache.pulsar.broker.loadbalance.extensions.models.UnloadDecision.Reason.Unknown; import com.google.common.annotations.VisibleForTesting; @@ -28,6 +30,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; import org.apache.pulsar.broker.loadbalance.extensions.models.UnloadCounter; @@ -170,6 +173,15 @@ public void beforeEvent(String serviceUnit, ServiceUnitStateData data) { @Override public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable t) { + ServiceUnitState state = ServiceUnitStateData.state(data); + + if (StringUtils.isBlank(data.sourceBroker()) && (state == Owned || state == Assigning)) { + if (log.isDebugEnabled()) { + log.debug("Skipping {} for service unit {} from the assignment command.", data, serviceUnit); + } + return; + } + if (t != null) { if (log.isDebugEnabled()) { log.debug("Handling {} for service unit {} with exception.", data, serviceUnit, t); @@ -181,7 +193,7 @@ public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable if (log.isDebugEnabled()) { log.debug("Handling {} for service unit {}", data, serviceUnit); } - ServiceUnitState state = ServiceUnitStateData.state(data); + switch (state) { case Free, Owned -> complete(serviceUnit, t); case Releasing -> LatencyMetric.RELEASE.endMeasurement(serviceUnit); 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 98a0ed95b1a45..549dfef896cd0 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 @@ -2244,9 +2244,18 @@ private CompletableFuture unloadServiceUnit(NamespaceBundle serviceUnit closeFutures.add(topicFuture .thenCompose(t -> t.isPresent() ? t.get().close( disconnectClients, closeWithoutWaitingClientDisconnect) - : CompletableFuture.completedFuture(null))); + : CompletableFuture.completedFuture(null)) + .exceptionally(e -> { + if (e.getCause() instanceof BrokerServiceException.ServiceUnitNotReadyException + && e.getMessage().contains("Please redo the lookup")) { + log.warn("[{}] Topic ownership check failed. Skipping it", topicName); + return null; + } + throw FutureUtil.wrapToCompletionException(e); + })); } }); + if (getPulsar().getConfig().isTransactionCoordinatorEnabled() && serviceUnit.getNamespaceObject().equals(NamespaceName.SYSTEM_NAMESPACE)) { TransactionMetadataStoreService metadataStoreService = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java index e23286ae4492e..07c33107d4b22 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java @@ -610,11 +610,16 @@ protected CompletableFuture isBundleOwnedByAnyBroker(NamespaceName fqnn NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, bundles, bundleRange); NamespaceService nsService = pulsar().getNamespaceService(); + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { + return nsService.checkOwnershipPresentAsync(nsBundle); + } + LookupOptions options = LookupOptions.builder() .authoritative(false) .requestHttps(isRequestHttps()) .readOnly(true) .loadTopicsInBundle(false).build(); + return nsService.getWebServiceUrlAsync(nsBundle, options).thenApply(Optional::isPresent); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java index 9e20fccff6d93..651a544a04e82 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java @@ -37,7 +37,6 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.common.policies.data.TopicType; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; @@ -65,18 +64,11 @@ protected ExtensibleLoadManagerImplBaseTest(String defaultTestNamespace) { } protected ServiceConfiguration initConfig(ServiceConfiguration conf) { - // Set the inflight state waiting time and ownership monitor delay time to 5 seconds to avoid - // stuck when doing unload. - conf.setLoadBalancerInFlightServiceUnitStateWaitingTimeInMillis(5 * 1000); - conf.setLoadBalancerServiceUnitStateMonitorIntervalInSeconds(1); conf.setForceDeleteNamespaceAllowed(true); - conf.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); - conf.setAllowAutoTopicCreation(true); conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); conf.setLoadBalancerSheddingEnabled(false); conf.setLoadBalancerDebugModeEnabled(true); - conf.setTopicLevelPoliciesEnabled(true); return conf; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index 308a755235c6d..aee57f9d26093 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -67,6 +67,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; @@ -264,6 +265,32 @@ public CompletableFuture> filterAsync(Map { + future1.completeExceptionally(new CompletionException( + new BrokerServiceException.ServiceUnitNotReadyException("Please redo the lookup"))); + future2.completeExceptionally(new CompletionException( + new BrokerServiceException.ServiceUnitNotReadyException("Please redo the lookup"))); + }); + admin.namespaces().unloadNamespaceBundle(bundle.getNamespaceObject().toString(), bundle.getBundleRange()); + } finally { + pulsar1.getBrokerService().getTopics().remove(topicName.toString()); + pulsar2.getBrokerService().getTopics().remove(topicName.toString()); + } + } + + @Test(timeOut = 30 * 1000) public void testUnloadAdminAPI() throws Exception { Pair topicAndBundle = getBundleIsNotOwnByChangeEventTopic("test-unload"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java index 56c28966ac235..5d0abea33577b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java @@ -93,53 +93,59 @@ public void testTimeout() throws IllegalAccessException { public void testSuccess() throws IllegalAccessException, ExecutionException, InterruptedException { UnloadCounter counter = new UnloadCounter(); UnloadManager manager = new UnloadManager(counter, "mockBrokerId"); + String dstBroker = "broker-2"; + String srcBroker = "broker-1"; + String bundle = "bundle-1"; var unloadDecision = - new UnloadDecision(new Unload("broker-1", "bundle-1"), Success, Admin); + new UnloadDecision(new Unload(srcBroker, bundle), Success, Admin); CompletableFuture future = manager.waitAsync(CompletableFuture.completedFuture(null), - "bundle-1", unloadDecision, 5, TimeUnit.SECONDS); + bundle, unloadDecision, 5, TimeUnit.SECONDS); Map> inFlightUnloadRequestMap = getInFlightUnloadRequestMap(manager); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Assigning, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Assigning, null, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Deleted, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Deleted, null, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Splitting, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Splitting, null, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Releasing, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Releasing, null, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Init, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Init, null, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Free, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Free, null, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 0); future.get(); assertEquals(counter.getBreakdownCounters().get(Success).get(Admin).get(), 1); // Success with Owned state. future = manager.waitAsync(CompletableFuture.completedFuture(null), - "bundle-1", unloadDecision, 5, TimeUnit.SECONDS); + bundle, unloadDecision, 5, TimeUnit.SECONDS); inFlightUnloadRequestMap = getInFlightUnloadRequestMap(manager); + assertEquals(inFlightUnloadRequestMap.size(), 1); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Owned, dstBroker, null, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Owned, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Owned, dstBroker, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 0); - future.get(); + future.get(); assertEquals(counter.getBreakdownCounters().get(Success).get(Admin).get(), 2); } @@ -157,7 +163,7 @@ public void testFailedStage() throws IllegalAccessException { assertEquals(inFlightUnloadRequestMap.size(), 1); manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Owned, "broker-1", VERSION_ID_INIT), + new ServiceUnitStateData(ServiceUnitState.Owned, null, "broker-1", VERSION_ID_INIT), new IllegalStateException("Failed stage.")); try { From ce4ecd2a134ecb2da18b27abc667c1d846a26d4c Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 1 Apr 2024 19:07:42 +0800 Subject: [PATCH 032/580] [improve][misc] Upgrade log4j2 to 2.23.1 (#22327) Signed-off-by: Zixuan Liu --- buildtools/pom.xml | 2 +- conf/log4j2.yaml | 2 +- distribution/server/src/assemble/LICENSE.bin.txt | 8 ++++---- distribution/shell/src/assemble/LICENSE.bin.txt | 8 ++++---- pom.xml | 2 +- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index d1e6e5f5ce42c..cd4d02af3d7b4 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -40,7 +40,7 @@ 1.8 1.8 3.1.0 - 2.18.0 + 2.23.1 1.7.32 7.7.1 3.11 diff --git a/conf/log4j2.yaml b/conf/log4j2.yaml index 9c261a6b89a50..0e49503581c48 100644 --- a/conf/log4j2.yaml +++ b/conf/log4j2.yaml @@ -19,7 +19,7 @@ Configuration: - status: INFO + status: ERROR monitorInterval: 30 name: pulsar packages: io.prometheus.client.log4j2 diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index cab23db279aca..518f92313753f 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -348,10 +348,10 @@ The Apache Software License, Version 2.0 - jakarta.validation-jakarta.validation-api-2.0.2.jar - javax.validation-validation-api-1.1.0.Final.jar * Log4J - - org.apache.logging.log4j-log4j-api-2.18.0.jar - - org.apache.logging.log4j-log4j-core-2.18.0.jar - - org.apache.logging.log4j-log4j-slf4j-impl-2.18.0.jar - - org.apache.logging.log4j-log4j-web-2.18.0.jar + - org.apache.logging.log4j-log4j-api-2.23.1.jar + - org.apache.logging.log4j-log4j-core-2.23.1.jar + - org.apache.logging.log4j-log4j-slf4j-impl-2.23.1.jar + - org.apache.logging.log4j-log4j-web-2.23.1.jar * Java Native Access JNA - net.java.dev.jna-jna-jpms-5.12.1.jar - net.java.dev.jna-jna-platform-jpms-5.12.1.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 9042257f34c67..b5036b67751f0 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -383,10 +383,10 @@ The Apache Software License, Version 2.0 - simpleclient_tracer_otel-0.16.0.jar - simpleclient_tracer_otel_agent-0.16.0.jar * Log4J - - log4j-api-2.18.0.jar - - log4j-core-2.18.0.jar - - log4j-slf4j-impl-2.18.0.jar - - log4j-web-2.18.0.jar + - log4j-api-2.23.1.jar + - log4j-core-2.23.1.jar + - log4j-slf4j-impl-2.23.1.jar + - log4j-web-2.23.1.jar * OpenTelemetry - opentelemetry-api-1.34.1.jar - opentelemetry-context-1.34.1.jar diff --git a/pom.xml b/pom.xml index 86a5be07c2a8f..7c19afef73a12 100644 --- a/pom.xml +++ b/pom.xml @@ -155,7 +155,7 @@ flexible messaging model and an intuitive client API. 7.9.2 1.7.32 4.4 - 2.18.0 + 2.23.1 1.75 1.0.6 1.0.2.4 From 50121e7f7be541f45bb6dc976f51e30658b1cb8d Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Mon, 1 Apr 2024 20:46:18 +0800 Subject: [PATCH 033/580] [improve][admin] Align the auth and check it at the first place for topic related API (#22342) --- .../pulsar/broker/admin/AdminResource.java | 4 +- .../admin/impl/PersistentTopicsBase.java | 1284 ++++++++--------- .../broker/admin/v2/PersistentTopics.java | 3 +- .../pulsar/broker/admin/TopicAuthZTest.java | 759 ++++++++++ 4 files changed, 1388 insertions(+), 662 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index 618c4ca73e17a..a1bfeb2142ffc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -480,9 +480,9 @@ protected CompletableFuture getPartitionedTopicMetadat // validates global-namespace contains local/peer cluster: if peer/local cluster present then lookup can // serve/redirect request else fail partitioned-metadata-request so, client fails while creating // producer/consumer - return validateClusterOwnershipAsync(topicName.getCluster()) + return validateTopicOperationAsync(topicName, TopicOperation.LOOKUP) + .thenCompose(__ -> validateClusterOwnershipAsync(topicName.getCluster())) .thenCompose(__ -> validateGlobalNamespaceOwnershipAsync(topicName.getNamespaceObject())) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.LOOKUP)) .thenCompose(__ -> { if (checkAllowAutoCreation) { return pulsar().getBrokerService() diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 86993f749b5fe..16d088756f57b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -553,8 +553,8 @@ protected CompletableFuture internalGetPartitionedMeta } protected CompletableFuture> internalGetPropertiesAsync(boolean authoritative) { - return validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.GET_METADATA)) + return validateTopicOperationAsync(topicName, TopicOperation.GET_METADATA) + .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> { if (topicName.isPartitioned()) { return getPropertiesAsync(); @@ -586,27 +586,27 @@ protected CompletableFuture internalUpdatePropertiesAsync(boolean authorit log.warn("[{}] [{}] properties is empty, ignore update", clientAppId(), topicName); return CompletableFuture.completedFuture(null); } - return validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.UPDATE_METADATA)) - .thenCompose(__ -> { - if (topicName.isPartitioned()) { - return internalUpdateNonPartitionedTopicProperties(properties); - } else { - return pulsar().getBrokerService().fetchPartitionedTopicMetadataAsync(topicName) - .thenCompose(metadata -> { - if (metadata.partitions == 0) { - return internalUpdateNonPartitionedTopicProperties(properties); - } - return namespaceResources() - .getPartitionedTopicResources().updatePartitionedTopicAsync(topicName, - p -> new PartitionedTopicMetadata(p.partitions, - p.properties == null ? properties - : MapUtils.putAll(p.properties, properties.entrySet().toArray()))); - }); - } - }).thenAccept(__ -> - log.info("[{}] [{}] update properties success with properties {}", - clientAppId(), topicName, properties)); + return validateTopicOperationAsync(topicName, TopicOperation.UPDATE_METADATA) + .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> { + if (topicName.isPartitioned()) { + return internalUpdateNonPartitionedTopicProperties(properties); + } else { + return pulsar().getBrokerService().fetchPartitionedTopicMetadataAsync(topicName) + .thenCompose(metadata -> { + if (metadata.partitions == 0) { + return internalUpdateNonPartitionedTopicProperties(properties); + } + return namespaceResources() + .getPartitionedTopicResources().updatePartitionedTopicAsync(topicName, + p -> new PartitionedTopicMetadata(p.partitions, + p.properties == null ? properties + : MapUtils.putAll(p.properties, properties.entrySet().toArray()))); + }); + } + }).thenAccept(__ -> + log.info("[{}] [{}] update properties success with properties {}", clientAppId(), + topicName, properties)); } private CompletableFuture internalUpdateNonPartitionedTopicProperties(Map properties) { @@ -640,8 +640,8 @@ public void updatePropertiesFailed(ManagedLedgerException exception, Object ctx) } protected CompletableFuture internalRemovePropertiesAsync(boolean authoritative, String key) { - return validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.DELETE_METADATA)) + return validateTopicOperationAsync(topicName, TopicOperation.DELETE_METADATA) + .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> { if (topicName.isPartitioned()) { return internalRemoveNonPartitionedTopicProperties(key); @@ -703,9 +703,8 @@ protected CompletableFuture internalCheckTopicExists(TopicName topicName) protected void internalDeletePartitionedTopic(AsyncResponse asyncResponse, boolean authoritative, boolean force) { - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateNamespaceOperationAsync(topicName.getNamespaceObject(), - NamespaceOperation.DELETE_TOPIC)) + validateNamespaceOperationAsync(topicName.getNamespaceObject(), NamespaceOperation.DELETE_TOPIC) + .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> pulsar().getBrokerService() .fetchPartitionedTopicMetadataAsync(topicName) .thenCompose(partitionedMeta -> { @@ -1111,98 +1110,89 @@ private boolean isUnexpectedTopicName(PartitionedTopicMetadata topicMetadata) { } protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - future.thenCompose(__ -> - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(unused -> validateTopicOperationAsync(topicName, TopicOperation.GET_SUBSCRIPTIONS)) - .thenAccept(unused1 -> { - // If the topic name is a partition name, no need to get partition topic metadata again - if (topicName.isPartitioned()) { - internalGetSubscriptionsForNonPartitionedTopic(asyncResponse); - } else { - getPartitionedTopicMetadataAsync(topicName, authoritative, false) - .thenAccept(partitionMetadata -> { - if (partitionMetadata.partitions > 0 && !isUnexpectedTopicName(partitionMetadata)) { - try { - final Set subscriptions = - Collections.newSetFromMap( - new ConcurrentHashMap<>(partitionMetadata.partitions)); - final List> subscriptionFutures = new ArrayList<>(); - if (topicName.getDomain() == TopicDomain.persistent) { - final Map> existsFutures = - new ConcurrentHashMap<>(partitionMetadata.partitions); - for (int i = 0; i < partitionMetadata.partitions; i++) { - existsFutures.put(i, - topicResources().persistentTopicExists(topicName.getPartition(i))); - } - FutureUtil.waitForAll(new ArrayList<>(existsFutures.values())) - .thenApply(unused2 -> - existsFutures.entrySet().stream().filter(e -> e.getValue().join()) - .map(item -> topicName.getPartition(item.getKey()).toString()) - .collect(Collectors.toList()) - ).thenAccept(topics -> { - if (log.isDebugEnabled()) { - log.debug("activeTopics : {}", topics); - } - topics.forEach(topic -> { - try { - CompletableFuture> subscriptionsAsync = pulsar() - .getAdminClient() - .topics().getSubscriptionsAsync(topic); - subscriptionFutures.add(subscriptionsAsync - .thenApply(subscriptions::addAll)); - } catch (PulsarServerException e) { - throw new RestException(e); - } - }); - }).thenAccept(unused3 -> resumeAsyncResponse(asyncResponse, - subscriptions, subscriptionFutures)); - } else { - for (int i = 0; i < partitionMetadata.partitions; i++) { + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.GET_SUBSCRIPTIONS); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenAccept(__ -> { + // If the topic name is a partition name, no need to get partition topic metadata again + if (topicName.isPartitioned()) { + internalGetSubscriptionsForNonPartitionedTopic(asyncResponse); + } else { + getPartitionedTopicMetadataAsync(topicName, authoritative, false) + .thenAccept(partitionMetadata -> { + if (partitionMetadata.partitions > 0 && !isUnexpectedTopicName(partitionMetadata)) { + try { + final Set subscriptions = + Collections.newSetFromMap( + new ConcurrentHashMap<>(partitionMetadata.partitions)); + final List> subscriptionFutures = new ArrayList<>(); + if (topicName.getDomain() == TopicDomain.persistent) { + final Map> existsFutures = + new ConcurrentHashMap<>(partitionMetadata.partitions); + for (int i = 0; i < partitionMetadata.partitions; i++) { + existsFutures.put(i, + topicResources().persistentTopicExists(topicName.getPartition(i))); + } + FutureUtil.waitForAll(new ArrayList<>(existsFutures.values())) + .thenApply(unused2 -> + existsFutures.entrySet().stream().filter(e -> e.getValue().join()) + .map(item -> topicName.getPartition(item.getKey()).toString()) + .collect(Collectors.toList()) + ).thenAccept(topics -> { + if (log.isDebugEnabled()) { + log.debug("activeTopics : {}", topics); + } + topics.forEach(topic -> { + try { CompletableFuture> subscriptionsAsync = pulsar() - .getAdminClient().topics() - .getSubscriptionsAsync(topicName.getPartition(i).toString()); + .getAdminClient() + .topics().getSubscriptionsAsync(topic); subscriptionFutures.add(subscriptionsAsync .thenApply(subscriptions::addAll)); + } catch (PulsarServerException e) { + throw new RestException(e); } - resumeAsyncResponse(asyncResponse, subscriptions, subscriptionFutures); - } - } catch (Exception e) { - log.error("[{}] Failed to get list of subscriptions for {}", - clientAppId(), topicName, e); - asyncResponse.resume(e); - } + }); + }).thenAccept(unused3 -> resumeAsyncResponse(asyncResponse, + subscriptions, subscriptionFutures)); } else { - internalGetSubscriptionsForNonPartitionedTopic(asyncResponse); - } - }).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { - log.error("[{}] Failed to get partitioned topic metadata while get" - + " subscriptions for topic {}", clientAppId(), topicName, ex); + for (int i = 0; i < partitionMetadata.partitions; i++) { + CompletableFuture> subscriptionsAsync = pulsar() + .getAdminClient().topics() + .getSubscriptionsAsync(topicName.getPartition(i).toString()); + subscriptionFutures.add(subscriptionsAsync + .thenApply(subscriptions::addAll)); + } + resumeAsyncResponse(asyncResponse, subscriptions, subscriptionFutures); } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - }); + } catch (Exception e) { + log.error("[{}] Failed to get list of subscriptions for {}", + clientAppId(), topicName, e); + asyncResponse.resume(e); + } + } else { + internalGetSubscriptionsForNonPartitionedTopic(asyncResponse); } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. if (!isNot307And404Exception(ex)) { - log.error("[{}] Failed to validate the global namespace/topic ownership while get subscriptions" - + " for topic {}", clientAppId(), topicName, ex); + log.error("[{}] Failed to get partitioned topic metadata while get" + + " subscriptions for topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; - }) - ).exceptionally(ex -> { + }); + } + }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. if (!isNot307And404Exception(ex)) { - log.error("[{}] Failed to get subscriptions for {}", clientAppId(), topicName, ex); + log.error("[{}] Failed to validate the global namespace/topic ownership while get subscriptions" + + " for topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; @@ -1250,38 +1240,36 @@ private void internalGetSubscriptionsForNonPartitionedTopic(AsyncResponse asyncR protected CompletableFuture internalGetStatsAsync(boolean authoritative, GetStatsOptions getStatsOptions) { - CompletableFuture future; - - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - - return future.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenComposeAsync(__ -> validateTopicOperationAsync(topicName, TopicOperation.GET_STATS)) - .thenCompose(__ -> getTopicReferenceAsync(topicName)) - .thenCompose(topic -> topic.asyncGetStats(getStatsOptions)); + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.GET_STATS); + return future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> getTopicReferenceAsync(topicName)) + .thenCompose(topic -> topic.asyncGetStats(getStatsOptions)); } protected CompletableFuture internalGetInternalStatsAsync(boolean authoritative, boolean metadata) { - CompletableFuture ret; - if (topicName.isGlobal()) { - ret = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - ret = CompletableFuture.completedFuture(null); - } - return ret.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.GET_STATS)) - .thenCompose(__ -> { - if (metadata) { - return validateTopicOperationAsync(topicName, TopicOperation.GET_METADATA); - } - return CompletableFuture.completedFuture(null); - }) - .thenCompose(__ -> getTopicReferenceAsync(topicName)) - .thenCompose(topic -> topic.getInternalStats(metadata)); + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.GET_STATS); + return future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> { + if (metadata) { + return validateTopicOperationAsync(topicName, TopicOperation.GET_METADATA); + } + return CompletableFuture.completedFuture(null); + }) + .thenCompose(__ -> getTopicReferenceAsync(topicName)) + .thenCompose(topic -> topic.getInternalStats(metadata)); } protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse, boolean authoritative) { @@ -1545,16 +1533,14 @@ protected void internalGetPartitionedStatsInternal(AsyncResponse asyncResponse, protected CompletableFuture internalDeleteSubscriptionAsync(String subName, boolean authoritative, boolean force) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - - return future - .thenCompose((__) -> validateTopicOperationAsync(topicName, TopicOperation.UNSUBSCRIBE, subName)) - .thenCompose(__ -> { + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.UNSUBSCRIBE, subName); + return future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> { if (topicName.isPartitioned()) { return internalDeleteSubscriptionForNonPartitionedTopicAsync(subName, authoritative, force); } else { @@ -1701,7 +1687,6 @@ private void internalGetSubscriptionPropertiesForNonPartitionedTopic(AsyncRespon String subName, boolean authoritative) { validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.CONSUME, subName)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenApply((Topic topic) -> { Subscription sub = topic.getSubscription(subName); @@ -1828,72 +1813,71 @@ private void internalDeleteSubscriptionForNonPartitionedTopicForcefully(AsyncRes } protected void internalSkipAllMessages(AsyncResponse asyncResponse, String subName, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - - future.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.SKIP, subName)) - .thenCompose(__ -> { - // If the topic name is a partition name, no need to get partition topic metadata again - if (topicName.isPartitioned()) { - return internalSkipAllMessagesForNonPartitionedTopicAsync(asyncResponse, subName); - } else { - return getPartitionedTopicMetadataAsync(topicName, - authoritative, false).thenCompose(partitionMetadata -> { - if (partitionMetadata.partitions > 0) { - final List> futures = new ArrayList<>(); + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.SKIP, subName); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> { + // If the topic name is a partition name, no need to get partition topic metadata again + if (topicName.isPartitioned()) { + return internalSkipAllMessagesForNonPartitionedTopicAsync(asyncResponse, subName); + } else { + return getPartitionedTopicMetadataAsync(topicName, + authoritative, false).thenCompose(partitionMetadata -> { + if (partitionMetadata.partitions > 0) { + final List> futures = new ArrayList<>(); - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - futures.add(pulsar() - .getAdminClient() - .topics() - .skipAllMessagesAsync(topicNamePartition.toString(), - subName)); - } catch (Exception e) { - log.error("[{}] Failed to skip all messages {} {}", - clientAppId(), topicNamePartition, subName, e); - asyncResponse.resume(new RestException(e)); - return CompletableFuture.completedFuture(null); - } + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + futures.add(pulsar() + .getAdminClient() + .topics() + .skipAllMessagesAsync(topicNamePartition.toString(), + subName)); + } catch (Exception e) { + log.error("[{}] Failed to skip all messages {} {}", + clientAppId(), topicNamePartition, subName, e); + asyncResponse.resume(new RestException(e)); + return CompletableFuture.completedFuture(null); } + } - return FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = exception.getCause(); - if (t instanceof NotFoundException) { - asyncResponse.resume( - new RestException(Status.NOT_FOUND, - getSubNotFoundErrorMessage(topicName.toString(), subName))); - } else { - log.error("[{}] Failed to skip all messages {} {}", - clientAppId(), topicName, subName, t); - asyncResponse.resume(new RestException(t)); - } - return null; + return FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = exception.getCause(); + if (t instanceof NotFoundException) { + asyncResponse.resume( + new RestException(Status.NOT_FOUND, + getSubNotFoundErrorMessage(topicName.toString(), subName))); + } else { + log.error("[{}] Failed to skip all messages {} {}", + clientAppId(), topicName, subName, t); + asyncResponse.resume(new RestException(t)); } - asyncResponse.resume(Response.noContent().build()); return null; - }); - } else { - return internalSkipAllMessagesForNonPartitionedTopicAsync(asyncResponse, subName); - } - }); - } - }).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { - log.error("[{}] Failed to skip all messages for subscription {} on topic {}", - clientAppId(), subName, topicName, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - }); + } + asyncResponse.resume(Response.noContent().build()); + return null; + }); + } else { + return internalSkipAllMessagesForNonPartitionedTopicAsync(asyncResponse, subName); + } + }); + } + }).exceptionally(ex -> { + // If the exception is not redirect exception we need to log it. + if (!isNot307And404Exception(ex)) { + log.error("[{}] Failed to skip all messages for subscription {} on topic {}", + clientAppId(), subName, topicName, ex); + } + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } private CompletableFuture internalSkipAllMessagesForNonPartitionedTopicAsync(AsyncResponse asyncResponse, @@ -1942,127 +1926,126 @@ private CompletableFuture internalSkipAllMessagesForNonPartitionedTopicAsy protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, int numMessages, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - future.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.SKIP, subName)) - .thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false) - .thenCompose(partitionMetadata -> { - if (partitionMetadata.partitions > 0) { - String msg = "Skip messages on a partitioned topic is not allowed"; - log.warn("[{}] {} {} {}", clientAppId(), msg, topicName, subName); - throw new RestException(Status.METHOD_NOT_ALLOWED, msg); + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.SKIP, subName); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) + .thenCompose(partitionMetadata -> { + if (partitionMetadata.partitions > 0) { + String msg = "Skip messages on a partitioned topic is not allowed"; + log.warn("[{}] {} {} {}", clientAppId(), msg, topicName, subName); + throw new RestException(Status.METHOD_NOT_ALLOWED, msg); + } + return getTopicReferenceAsync(topicName).thenCompose(t -> { + PersistentTopic topic = (PersistentTopic) t; + if (topic == null) { + throw new RestException(new RestException(Status.NOT_FOUND, + getTopicNotFoundErrorMessage(topicName.toString()))); + } + if (subName.startsWith(topic.getReplicatorPrefix())) { + String remoteCluster = PersistentReplicator.getRemoteCluster(subName); + PersistentReplicator repl = + (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); + if (repl == null) { + return FutureUtil.failedFuture( + new RestException(Status.NOT_FOUND, "Replicator not found")); + } + return repl.skipMessages(numMessages).thenAccept(unused -> { + log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, + topicName, subName); + asyncResponse.resume(Response.noContent().build()); } - return getTopicReferenceAsync(topicName).thenCompose(t -> { - PersistentTopic topic = (PersistentTopic) t; - if (topic == null) { - throw new RestException(new RestException(Status.NOT_FOUND, - getTopicNotFoundErrorMessage(topicName.toString()))); - } - if (subName.startsWith(topic.getReplicatorPrefix())) { - String remoteCluster = PersistentReplicator.getRemoteCluster(subName); - PersistentReplicator repl = - (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); - if (repl == null) { - return FutureUtil.failedFuture( - new RestException(Status.NOT_FOUND, "Replicator not found")); - } - return repl.skipMessages(numMessages).thenAccept(unused -> { - log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, - topicName, subName); - asyncResponse.resume(Response.noContent().build()); - } - ); - } else { - PersistentSubscription sub = topic.getSubscription(subName); - if (sub == null) { - return FutureUtil.failedFuture( - new RestException(Status.NOT_FOUND, - getSubNotFoundErrorMessage(topicName.toString(), subName))); - } - return sub.skipMessages(numMessages).thenAccept(unused -> { - log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, - topicName, subName); - asyncResponse.resume(Response.noContent().build()); - } - ); - } - }); - }) - ).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { - log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, topicName, - subName, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - }); + ); + } else { + PersistentSubscription sub = topic.getSubscription(subName); + if (sub == null) { + return FutureUtil.failedFuture( + new RestException(Status.NOT_FOUND, + getSubNotFoundErrorMessage(topicName.toString(), subName))); + } + return sub.skipMessages(numMessages).thenAccept(unused -> { + log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, + topicName, subName); + asyncResponse.resume(Response.noContent().build()); + } + ); + } + }); + } + ).exceptionally(ex -> { + // If the exception is not redirect exception we need to log it. + if (!isNot307And404Exception(ex)) { + log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, topicName, + subName, ex); + } + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } protected void internalExpireMessagesForAllSubscriptions(AsyncResponse asyncResponse, int expireTimeInSeconds, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - future.thenCompose(__ -> - getPartitionedTopicMetadataAsync(topicName, authoritative, false) - .thenAccept(partitionMetadata -> { - if (topicName.isPartitioned()) { - internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(asyncResponse, - partitionMetadata, expireTimeInSeconds, authoritative); - } else { - if (partitionMetadata.partitions > 0) { - final List> futures = new ArrayList<>(partitionMetadata.partitions); - - // expire messages for each partition topic - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - futures.add(pulsar() - .getAdminClient() - .topics() - .expireMessagesForAllSubscriptionsAsync( - topicNamePartition.toString(), expireTimeInSeconds)); - } catch (Exception e) { - log.error("[{}] Failed to expire messages up to {} on {}", - clientAppId(), expireTimeInSeconds, - topicNamePartition, e); - asyncResponse.resume(new RestException(e)); - return; - } - } + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.EXPIRE_MESSAGES); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) + .thenAccept(partitionMetadata -> { + if (topicName.isPartitioned()) { + internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(asyncResponse, + partitionMetadata, expireTimeInSeconds, authoritative); + } else { + if (partitionMetadata.partitions > 0) { + final List> futures = new ArrayList<>(partitionMetadata.partitions); - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = FutureUtil.unwrapCompletionException(exception); - if (t instanceof PulsarAdminException) { - log.warn("[{}] Failed to expire messages up to {} on {}: {}", clientAppId(), - expireTimeInSeconds, topicName, t.toString()); - } else { - log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), - expireTimeInSeconds, topicName, t); - } - resumeAsyncResponseExceptionally(asyncResponse, t); - return null; - } - asyncResponse.resume(Response.noContent().build()); - return null; - }); - } else { - internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(asyncResponse, - partitionMetadata, expireTimeInSeconds, authoritative); + // expire messages for each partition topic + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + futures.add(pulsar() + .getAdminClient() + .topics() + .expireMessagesForAllSubscriptionsAsync( + topicNamePartition.toString(), expireTimeInSeconds)); + } catch (Exception e) { + log.error("[{}] Failed to expire messages up to {} on {}", + clientAppId(), expireTimeInSeconds, + topicNamePartition, e); + asyncResponse.resume(new RestException(e)); + return; } } + + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = FutureUtil.unwrapCompletionException(exception); + if (t instanceof PulsarAdminException) { + log.warn("[{}] Failed to expire messages up to {} on {}: {}", clientAppId(), + expireTimeInSeconds, topicName, t.toString()); + } else { + log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), + expireTimeInSeconds, topicName, t); + } + resumeAsyncResponseExceptionally(asyncResponse, t); + return null; + } + asyncResponse.resume(Response.noContent().build()); + return null; + }); + } else { + internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(asyncResponse, + partitionMetadata, expireTimeInSeconds, authoritative); } - ) + } + } ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. if (!isNot307And404Exception(ex)) { @@ -2082,7 +2065,6 @@ private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(Asy boolean authoritative) { // validate ownership and redirect if current broker is not owner validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.EXPIRE_MESSAGES)) .thenCompose(__ -> getTopicReferenceAsync(topicName).thenAccept(t -> { if (t == null) { resumeAsyncResponseExceptionally(asyncResponse, new RestException(Status.NOT_FOUND, @@ -2143,23 +2125,22 @@ private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(Asy protected CompletableFuture internalResetCursorAsync(String subName, long timestamp, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - return future - .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.RESET_CURSOR, subName)) - .thenCompose(__ -> { - // If the topic name is a partition name, no need to get partition topic metadata again - if (topicName.isPartitioned()) { - return internalResetCursorForNonPartitionedTopic(subName, timestamp, authoritative); - } else { - return internalResetCursorForPartitionedTopic(subName, timestamp, authoritative); - } - }); + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.RESET_CURSOR, subName); + return future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> { + // If the topic name is a partition name, no need to get partition topic metadata again + if (topicName.isPartitioned()) { + return internalResetCursorForNonPartitionedTopic(subName, timestamp, authoritative); + } else { + return internalResetCursorForPartitionedTopic(subName, timestamp, authoritative); + } + }); } private CompletableFuture internalResetCursorForPartitionedTopic(String subName, long timestamp, @@ -2614,88 +2595,77 @@ protected void internalGetSubscriptionProperties(AsyncResponse asyncResponse, St protected void internalResetCursorOnPosition(AsyncResponse asyncResponse, String subName, boolean authoritative, MessageIdImpl messageId, boolean isExcluded, int batchIndex) { - CompletableFuture ret; - // If the topic name is a partition name, no need to get partition topic metadata again - if (!topicName.isPartitioned()) { - ret = getPartitionedTopicMetadataAsync(topicName, authoritative, false) - .thenCompose(topicMetadata -> { - if (topicMetadata.partitions > 0) { - log.warn("[{}] Not supported operation on partitioned-topic {} {}", - clientAppId(), topicName, subName); - throw new CompletionException(new RestException(Status.METHOD_NOT_ALLOWED, - "Reset-cursor at position is not allowed for partitioned-topic")); - } - return CompletableFuture.completedFuture(null); - }); - } else { - ret = CompletableFuture.completedFuture(null); - } - - CompletableFuture future; - if (topicName.isGlobal()) { - future = ret.thenCompose(__ -> validateGlobalNamespaceOwnershipAsync(namespaceName)); - } else { - future = CompletableFuture.completedFuture(null); - } - future.thenAccept(__ -> { + CompletableFuture ret = validateTopicOperationAsync(topicName, TopicOperation.RESET_CURSOR, subName); + ret.thenCompose(__ -> { + // If the topic name is a partition name, no need to get partition topic metadata again + if (!topicName.isPartitioned()) { + return getPartitionedTopicMetadataAsync(topicName, authoritative, false) + .thenCompose(topicMetadata -> { + if (topicMetadata.partitions > 0) { + log.warn("[{}] Not supported operation on partitioned-topic {} {}", + clientAppId(), topicName, subName); + throw new CompletionException(new RestException(Status.METHOD_NOT_ALLOWED, + "Reset-cursor at position is not allowed for partitioned-topic")); + } + return CompletableFuture.completedFuture(null); + }); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> { log.info("[{}][{}] received reset cursor on subscription {} to position {}", clientAppId(), topicName, subName, messageId); - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(ignore -> - validateTopicOperationAsync(topicName, TopicOperation.RESET_CURSOR, subName)) - .thenCompose(ignore -> getTopicReferenceAsync(topicName)) - .thenAccept(topic -> { - if (topic == null) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, - getTopicNotFoundErrorMessage(topicName.toString()))); - return; - } - PersistentSubscription sub = ((PersistentTopic) topic).getSubscription(subName); - if (sub == null) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, - getSubNotFoundErrorMessage(topicName.toString(), subName))); - return; - } - CompletableFuture batchSizeFuture = new CompletableFuture<>(); - getEntryBatchSize(batchSizeFuture, (PersistentTopic) topic, messageId, batchIndex); - batchSizeFuture.thenAccept(bi -> { - PositionImpl seekPosition = calculatePositionAckSet(isExcluded, bi, batchIndex, - messageId); - sub.resetCursor(seekPosition).thenRun(() -> { - log.info("[{}][{}] successfully reset cursor on subscription {}" - + " to position {}", clientAppId(), - topicName, subName, messageId); - asyncResponse.resume(Response.noContent().build()); - }).exceptionally(ex -> { - Throwable t = (ex instanceof CompletionException ? ex.getCause() : ex); - log.warn("[{}][{}] Failed to reset cursor on subscription {}" - + " to position {}", clientAppId(), - topicName, subName, messageId, t); - if (t instanceof SubscriptionInvalidCursorPosition) { - asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, - "Unable to find position for position specified: " - + t.getMessage())); - } else if (t instanceof SubscriptionBusyException) { - asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, - "Failed for Subscription Busy: " + t.getMessage())); - } else { - resumeAsyncResponseExceptionally(asyncResponse, t); - } - return null; - }); - }).exceptionally(e -> { - asyncResponse.resume(e); - return null; - }); - }).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { - log.warn("[{}][{}] Failed to reset cursor on subscription {} to position {}", - clientAppId(), topicName, subName, messageId, ex.getCause()); - } - resumeAsyncResponseExceptionally(asyncResponse, ex.getCause()); - return null; - }); + return validateTopicOwnershipAsync(topicName, authoritative); + }).thenCompose(ignore -> getTopicReferenceAsync(topicName)) + .thenAccept(topic -> { + if (topic == null) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, + getTopicNotFoundErrorMessage(topicName.toString()))); + return; + } + PersistentSubscription sub = ((PersistentTopic) topic).getSubscription(subName); + if (sub == null) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, + getSubNotFoundErrorMessage(topicName.toString(), subName))); + return; + } + CompletableFuture batchSizeFuture = new CompletableFuture<>(); + getEntryBatchSize(batchSizeFuture, (PersistentTopic) topic, messageId, batchIndex); + batchSizeFuture.thenAccept(bi -> { + PositionImpl seekPosition = calculatePositionAckSet(isExcluded, bi, batchIndex, + messageId); + sub.resetCursor(seekPosition).thenRun(() -> { + log.info("[{}][{}] successfully reset cursor on subscription {}" + + " to position {}", clientAppId(), + topicName, subName, messageId); + asyncResponse.resume(Response.noContent().build()); + }).exceptionally(ex -> { + Throwable t = (ex instanceof CompletionException ? ex.getCause() : ex); + log.warn("[{}][{}] Failed to reset cursor on subscription {}" + + " to position {}", clientAppId(), + topicName, subName, messageId, t); + if (t instanceof SubscriptionInvalidCursorPosition) { + asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, + "Unable to find position for position specified: " + + t.getMessage())); + } else if (t instanceof SubscriptionBusyException) { + asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, + "Failed for Subscription Busy: " + t.getMessage())); + } else { + resumeAsyncResponseExceptionally(asyncResponse, t); + } + return null; + }); + }).exceptionally(e -> { + asyncResponse.resume(e); + return null; + }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. if (!isNot307And404Exception(ex)) { @@ -2797,13 +2767,14 @@ private PositionImpl calculatePositionAckSet(boolean isExcluded, int batchSize, } protected CompletableFuture internalGetMessageById(long ledgerId, long entryId, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES); return future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> { if (topicName.isPartitioned()) { return CompletableFuture.completedFuture(null); } else { @@ -2816,11 +2787,8 @@ protected CompletableFuture internalGetMessageById(long ledgerId, long "GetMessageById is not allowed on partitioned-topic"); } }); - } - }) - .thenCompose(ignore -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES)) + }).thenCompose(ignore -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenCompose(topic -> { CompletableFuture results = new CompletableFuture<>(); @@ -2948,139 +2916,141 @@ private CompletableFuture findMessageIdByPublishTime(long timestamp, protected CompletableFuture internalPeekNthMessageAsync(String subName, int messagePosition, boolean authoritative) { - CompletableFuture ret; - // If the topic name is a partition name, no need to get partition topic metadata again - if (!topicName.isPartitioned()) { - ret = getPartitionedTopicMetadataAsync(topicName, authoritative, false) - .thenCompose(topicMetadata -> { - if (topicMetadata.partitions > 0) { - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Peek messages on a partitioned topic is not allowed"); - } - return CompletableFuture.completedFuture(null); - }); - } else { - ret = CompletableFuture.completedFuture(null); - } - return ret.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES, subName)) - .thenCompose(__ -> getTopicReferenceAsync(topicName)) - .thenCompose(topic -> { - CompletableFuture entry; - if (!(topic instanceof PersistentTopic)) { - log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), - topicName, subName); - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Peek messages on a non-persistent topic is not allowed"); - } else { - if (subName.startsWith(((PersistentTopic) topic).getReplicatorPrefix())) { - PersistentReplicator repl = getReplicatorReference(subName, (PersistentTopic) topic); - entry = repl.peekNthMessage(messagePosition); - } else { - PersistentSubscription sub = - (PersistentSubscription) getSubscriptionReference(subName, (PersistentTopic) topic); - entry = sub.peekNthMessage(messagePosition); - } - } - return entry; - }).thenCompose(entry -> { - try { - Response response = generateResponseWithEntry(entry); - return CompletableFuture.completedFuture(response); - } catch (NullPointerException npe) { - throw new RestException(Status.NOT_FOUND, "Message not found"); - } catch (Exception exception) { - log.error("[{}] Failed to peek message at position {} from {} {}", clientAppId(), - messagePosition, topicName, subName, exception); - throw new RestException(exception); - } finally { - if (entry != null) { - entry.release(); - } - } - }); + CompletableFuture ret = validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES, subName); + return ret.thenCompose(__ -> { + // If the topic name is a partition name, no need to get partition topic metadata again + if (!topicName.isPartitioned()) { + return getPartitionedTopicMetadataAsync(topicName, authoritative, false) + .thenCompose(topicMetadata -> { + if (topicMetadata.partitions > 0) { + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Peek messages on a partitioned topic is not allowed"); + } + return CompletableFuture.completedFuture(null); + }); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> getTopicReferenceAsync(topicName)) + .thenCompose(topic -> { + CompletableFuture entry; + if (!(topic instanceof PersistentTopic)) { + log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), + topicName, subName); + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Peek messages on a non-persistent topic is not allowed"); + } else { + if (subName.startsWith(((PersistentTopic) topic).getReplicatorPrefix())) { + PersistentReplicator repl = getReplicatorReference(subName, (PersistentTopic) topic); + entry = repl.peekNthMessage(messagePosition); + } else { + PersistentSubscription sub = + (PersistentSubscription) getSubscriptionReference(subName, (PersistentTopic) topic); + entry = sub.peekNthMessage(messagePosition); + } + } + return entry; + }).thenCompose(entry -> { + try { + Response response = generateResponseWithEntry(entry); + return CompletableFuture.completedFuture(response); + } catch (NullPointerException npe) { + throw new RestException(Status.NOT_FOUND, "Message not found"); + } catch (Exception exception) { + log.error("[{}] Failed to peek message at position {} from {} {}", clientAppId(), + messagePosition, topicName, subName, exception); + throw new RestException(exception); + } finally { + if (entry != null) { + entry.release(); + } + } + }); } protected CompletableFuture internalExamineMessageAsync(String initialPosition, long messagePosition, boolean authoritative) { - CompletableFuture ret; - if (topicName.isGlobal()) { - ret = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - ret = CompletableFuture.completedFuture(null); - } - - ret = ret.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)); long messagePositionLocal = messagePosition < 1 ? 1 : messagePosition; String initialPositionLocal = initialPosition == null ? "latest" : initialPosition; - if (!topicName.isPartitioned()) { - ret = ret.thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) - .thenCompose(partitionedTopicMetadata -> { - if (partitionedTopicMetadata.partitions > 0) { - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Examine messages on a partitioned topic is not allowed, " - + "please try examine message on specific topic partition"); - } else { - return CompletableFuture.completedFuture(null); - } - }); - } - return ret.thenCompose(__ -> getTopicReferenceAsync(topicName)) - .thenCompose(topic -> { - if (!(topic instanceof PersistentTopic)) { - log.error("[{}] Not supported operation of non-persistent topic {} ", clientAppId(), topicName); - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Examine messages on a non-persistent topic is not allowed"); - } - try { - PersistentTopic persistentTopic = (PersistentTopic) topic; - long totalMessage = persistentTopic.getNumberOfEntries(); - if (totalMessage <= 0) { - throw new RestException(Status.PRECONDITION_FAILED, - "Could not examine messages due to the total message is zero"); - } - PositionImpl startPosition = persistentTopic.getFirstPosition(); - - long messageToSkip = initialPositionLocal.equals("earliest") ? messagePositionLocal : - totalMessage - messagePositionLocal + 1; - CompletableFuture future = new CompletableFuture<>(); - PositionImpl readPosition = persistentTopic.getPositionAfterN(startPosition, messageToSkip); - persistentTopic.asyncReadEntry(readPosition, new AsyncCallbacks.ReadEntryCallback() { - @Override - public void readEntryComplete(Entry entry, Object ctx) { - future.complete(entry); + CompletableFuture ret = validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES); + return ret.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> { + if (!topicName.isPartitioned()) { + return getPartitionedTopicMetadataAsync(topicName, authoritative, false) + .thenCompose(partitionedTopicMetadata -> { + if (partitionedTopicMetadata.partitions > 0) { + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Examine messages on a partitioned topic is not allowed, " + + "please try examine message on specific topic partition"); + } else { + return CompletableFuture.completedFuture(null); } + }); + } + return CompletableFuture.completedFuture(null); + }).thenCompose(__ -> getTopicReferenceAsync(topicName)) + .thenCompose(topic -> { + if (!(topic instanceof PersistentTopic)) { + log.error("[{}] Not supported operation of non-persistent topic {} ", clientAppId(), topicName); + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Examine messages on a non-persistent topic is not allowed"); + } + try { + PersistentTopic persistentTopic = (PersistentTopic) topic; + long totalMessage = persistentTopic.getNumberOfEntries(); + if (totalMessage <= 0) { + throw new RestException(Status.PRECONDITION_FAILED, + "Could not examine messages due to the total message is zero"); + } + PositionImpl startPosition = persistentTopic.getFirstPosition(); - @Override - public void readEntryFailed(ManagedLedgerException exception, Object ctx) { - future.completeExceptionally(exception); - } + long messageToSkip = initialPositionLocal.equals("earliest") ? messagePositionLocal : + totalMessage - messagePositionLocal + 1; + CompletableFuture future = new CompletableFuture<>(); + PositionImpl readPosition = persistentTopic.getPositionAfterN(startPosition, messageToSkip); + persistentTopic.asyncReadEntry(readPosition, new AsyncCallbacks.ReadEntryCallback() { + @Override + public void readEntryComplete(Entry entry, Object ctx) { + future.complete(entry); + } - @Override - public String toString() { - return String.format("Topic [%s] internal examine message async", - PersistentTopicsBase.this.topicName); - } - }, null); - return future; - } catch (ManagedLedgerException exception) { - log.error("[{}] Failed to examine message at position {} from {} due to {}", clientAppId(), - messagePosition, - topicName, exception); - throw new RestException(exception); + @Override + public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); } - }).thenApply(entry -> { - try { - return generateResponseWithEntry(entry); - } catch (IOException exception) { - throw new RestException(exception); - } finally { - if (entry != null) { - entry.release(); - } + @Override + public String toString() { + return String.format("Topic [%s] internal examine message async", + PersistentTopicsBase.this.topicName); } - }); + }, null); + return future; + } catch (ManagedLedgerException exception) { + log.error("[{}] Failed to examine message at position {} from {} due to {}", clientAppId(), + messagePosition, + topicName, exception); + throw new RestException(exception); + } + + }).thenApply(entry -> { + try { + return generateResponseWithEntry(entry); + } catch (IOException exception) { + throw new RestException(exception); + } finally { + if (entry != null) { + entry.release(); + } + } + }); } private Response generateResponseWithEntry(Entry entry) throws IOException { @@ -3926,83 +3896,82 @@ protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, bo protected void internalExpireMessagesByTimestamp(AsyncResponse asyncResponse, String subName, int expireTimeInSeconds, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - future.thenCompose(__ -> - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(unused -> validateTopicOperationAsync(topicName, TopicOperation.EXPIRE_MESSAGES, subName)) - .thenCompose(unused2 -> - // If the topic name is a partition name, no need to get partition topic metadata again - getPartitionedTopicMetadataAsync(topicName, authoritative, false) - .thenCompose(partitionMetadata -> { - if (topicName.isPartitioned()) { - return internalExpireMessagesByTimestampForSinglePartitionAsync - (partitionMetadata, subName, expireTimeInSeconds) - .thenAccept(unused3 -> - asyncResponse.resume(Response.noContent().build())); - } else { - if (partitionMetadata.partitions > 0) { - return CompletableFuture.completedFuture(null).thenAccept(unused -> { - final List> futures = new ArrayList<>(); - - // expire messages for each partition topic - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - futures.add(pulsar() - .getAdminClient() - .topics() - .expireMessagesAsync(topicNamePartition.toString(), - subName, expireTimeInSeconds)); - } catch (Exception e) { - log.error("[{}] Failed to expire messages up to {} on {}", - clientAppId(), - expireTimeInSeconds, topicNamePartition, e); - asyncResponse.resume(new RestException(e)); - return; - } - } - - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = FutureUtil.unwrapCompletionException(exception); - if (t instanceof NotFoundException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, - getSubNotFoundErrorMessage(topicName.toString(), - subName))); - return null; - } else { - if (t instanceof PulsarAdminException) { - log.warn("[{}] Failed to expire messages up " - + "to {} on {}: {}", clientAppId(), - expireTimeInSeconds, topicName, - t.toString()); - } else { - log.error("[{}] Failed to expire messages up " - + "to {} on {}", clientAppId(), - expireTimeInSeconds, topicName, t); - } - resumeAsyncResponseExceptionally(asyncResponse, t); - return null; - } - } - asyncResponse.resume(Response.noContent().build()); - return null; - }); - }); - } else { - return internalExpireMessagesByTimestampForSinglePartitionAsync - (partitionMetadata, subName, expireTimeInSeconds) - .thenAccept(unused -> - asyncResponse.resume(Response.noContent().build())); + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.EXPIRE_MESSAGES, + subName); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> + // If the topic name is a partition name, no need to get partition topic metadata again + getPartitionedTopicMetadataAsync(topicName, authoritative, false) + .thenCompose(partitionMetadata -> { + if (topicName.isPartitioned()) { + return internalExpireMessagesByTimestampForSinglePartitionAsync + (partitionMetadata, subName, expireTimeInSeconds) + .thenAccept(unused3 -> + asyncResponse.resume(Response.noContent().build())); + } else { + if (partitionMetadata.partitions > 0) { + return CompletableFuture.completedFuture(null).thenAccept(unused -> { + final List> futures = new ArrayList<>(); + + // expire messages for each partition topic + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + futures.add(pulsar() + .getAdminClient() + .topics() + .expireMessagesAsync(topicNamePartition.toString(), + subName, expireTimeInSeconds)); + } catch (Exception e) { + log.error("[{}] Failed to expire messages up to {} on {}", + clientAppId(), + expireTimeInSeconds, topicNamePartition, e); + asyncResponse.resume(new RestException(e)); + return; } } - })) + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = FutureUtil.unwrapCompletionException(exception); + if (t instanceof NotFoundException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, + getSubNotFoundErrorMessage(topicName.toString(), + subName))); + return null; + } else { + if (t instanceof PulsarAdminException) { + log.warn("[{}] Failed to expire messages up " + + "to {} on {}: {}", clientAppId(), + expireTimeInSeconds, topicName, + t.toString()); + } else { + log.error("[{}] Failed to expire messages up " + + "to {} on {}", clientAppId(), + expireTimeInSeconds, topicName, t); + } + resumeAsyncResponseExceptionally(asyncResponse, t); + return null; + } + } + asyncResponse.resume(Response.noContent().build()); + return null; + }); + }); + } else { + return internalExpireMessagesByTimestampForSinglePartitionAsync + (partitionMetadata, subName, expireTimeInSeconds) + .thenAccept(unused -> + asyncResponse.resume(Response.noContent().build())); + } + } + }) ).exceptionally(ex -> { Throwable cause = FutureUtil.unwrapCompletionException(ex); // If the exception is not redirect exception we need to log it. @@ -4090,44 +4059,43 @@ protected void internalExpireMessagesByPosition(AsyncResponse asyncResponse, Str asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, msg)); return; } - CompletableFuture ret; - // If the topic name is a partition name, no need to get partition topic metadata again - if (!topicName.isPartitioned()) { - ret = getPartitionedTopicMetadataAsync(topicName, authoritative, false) - .thenCompose(topicMetadata -> { - if (topicMetadata.partitions > 0) { - String msg = "Expire message at position is not supported for partitioned-topic"; - log.warn("[{}] {} {}({}) {}", clientAppId(), msg, topicName, messageId, subName); - asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, msg)); - } - return CompletableFuture.completedFuture(null); - }); - } else { - ret = CompletableFuture.completedFuture(null); - } - CompletableFuture future; - if (topicName.isGlobal()) { - future = ret.thenCompose(__ -> validateGlobalNamespaceOwnershipAsync(namespaceName)); - } else { - future = ret; - } - - future.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.EXPIRE_MESSAGES, subName)) - .thenCompose(__ -> { - log.info("[{}][{}] Received expire messages on subscription {} to position {}", clientAppId(), - topicName, subName, messageId); - return internalExpireMessagesNonPartitionedTopicByPosition(asyncResponse, subName, - messageId, isExcluded, batchIndex); - }).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { - log.error("[{}] Failed to expire messages up to {} on subscription {} to position {}", - clientAppId(), topicName, subName, messageId, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - }); + CompletableFuture ret = validateTopicOperationAsync(topicName, TopicOperation.EXPIRE_MESSAGES, subName); + ret.thenCompose(__ -> { + // If the topic name is a partition name, no need to get partition topic metadata again + if (!topicName.isPartitioned()) { + return getPartitionedTopicMetadataAsync(topicName, authoritative, false) + .thenCompose(topicMetadata -> { + if (topicMetadata.partitions > 0) { + String msg = "Expire message at position is not supported for partitioned-topic"; + log.warn("[{}] {} {}({}) {}", clientAppId(), msg, topicName, messageId, subName); + asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, msg)); + } + return CompletableFuture.completedFuture(null); + }); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> { + log.info("[{}][{}] Received expire messages on subscription {} to position {}", clientAppId(), + topicName, subName, messageId); + return internalExpireMessagesNonPartitionedTopicByPosition(asyncResponse, subName, + messageId, isExcluded, batchIndex); + }).exceptionally(ex -> { + // If the exception is not redirect exception we need to log it. + if (!isNot307And404Exception(ex)) { + log.error("[{}] Failed to expire messages up to {} on subscription {} to position {}", + clientAppId(), topicName, subName, messageId, ex); + } + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } private CompletableFuture internalExpireMessagesNonPartitionedTopicByPosition(AsyncResponse asyncResponse, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index d2cbaa5428a74..90f0208c81cd6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -1946,8 +1946,7 @@ public void examineMessage( @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES) - .thenCompose(__ -> internalExamineMessageAsync(initialPosition, messagePosition, authoritative)) + internalExamineMessageAsync(initialPosition, messagePosition, authoritative) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { if (isNot307And404Exception(ex)) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index e23f9bbaf9b30..2ff03732fae27 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -25,6 +25,10 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; @@ -33,6 +37,7 @@ import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.HashMap; import java.util.Map; @@ -84,6 +89,14 @@ public void after() { close(); } + @DataProvider(name = "partitioned") + public static Object[][] partitioned() { + return new Object[][] { + {true}, + {false} + }; + } + @SneakyThrows @Test @@ -342,4 +355,750 @@ public void testCreateMissingPartition() { } superUserAdmin.topics().deletePartitionedTopic(topic, true); } + + @Test(dataProvider = "partitioned") + @SneakyThrows + public void testPartitionedTopicMetadata(boolean partitioned) { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, partitioned); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + superUserAdmin.topics().getPartitionedTopicMetadata(topic); + + // test tenant manager + tenantManagerAdmin.topics().getPartitionedTopicMetadata(topic); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPartitionedTopicMetadata(topic)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.produce == action || AuthAction.consume == action) { + subAdmin.topics().getPartitionedTopicMetadata(topic); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPartitionedTopicMetadata(topic)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, partitioned); + } + + @Test(dataProvider = "partitioned") + @SneakyThrows + public void testGetProperties(boolean partitioned) { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, partitioned); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + Map properties = new HashMap<>(); + properties.put("key1", "value1"); + superUserAdmin.topics().getProperties(topic); + + // test tenant manager + tenantManagerAdmin.topics().getProperties(topic); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getProperties(topic)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.produce == action || AuthAction.consume == action) { + subAdmin.topics().getPartitionedTopicMetadata(topic); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPartitionedTopicMetadata(topic)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, partitioned); + } + + @Test(dataProvider = "partitioned") + @SneakyThrows + public void testUpdateProperties(boolean partitioned) { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, partitioned); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + Map properties = new HashMap<>(); + properties.put("key1", "value1"); + superUserAdmin.topics().updateProperties(topic, properties); + + // test tenant manager + tenantManagerAdmin.topics().updateProperties(topic, properties); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().updateProperties(topic, properties)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().updateProperties(topic, properties)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, partitioned); + } + + @Test(dataProvider = "partitioned") + @SneakyThrows + public void testRemoveProperties(boolean partitioned) { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, partitioned); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + superUserAdmin.topics().removeProperties(topic, "key1"); + + // test tenant manager + tenantManagerAdmin.topics().removeProperties(topic, "key1"); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().removeProperties(topic, "key1")); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().removeProperties(topic, "key1")); + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, partitioned); + } + + @Test + @SneakyThrows + public void testDeletePartitionedTopic() { + final String random = UUID.randomUUID().toString(); + String ns = "public/default/"; + final String topic = "persistent://" + ns + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + createTopic(topic , true); + superUserAdmin.topics().deletePartitionedTopic(topic); + + // test tenant manager + createTopic(topic, true); + tenantManagerAdmin.topics().deletePartitionedTopic(topic); + + createTopic(topic, true); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().deletePartitionedTopic(topic)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(ns, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().deletePartitionedTopic(topic)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(ns, subject); + } + deleteTopic(topic, true); + } + + @Test(dataProvider = "partitioned") + @SneakyThrows + public void testGetSubscription(boolean partitioned) { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, partitioned); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + superUserAdmin.topics().getSubscriptions(topic); + + // test tenant manager + tenantManagerAdmin.topics().getSubscriptions(topic); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getSubscriptions(topic)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.topics().getSubscriptions(topic); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getSubscriptions(topic)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, partitioned); + } + + @Test(dataProvider = "partitioned") + @SneakyThrows + public void testGetInternalStats(boolean partitioned) { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, partitioned); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + if (partitioned) { + superUserAdmin.topics().getPartitionedInternalStats(topic); + } else { + superUserAdmin.topics().getInternalStats(topic); + } + + // test tenant manager + if (partitioned) { + tenantManagerAdmin.topics().getPartitionedInternalStats(topic); + } else { + tenantManagerAdmin.topics().getInternalStats(topic); + + } + + if (partitioned) { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPartitionedInternalStats(topic)); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getInternalStats(topic)); + } + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.produce == action || AuthAction.consume == action) { + if (partitioned) { + subAdmin.topics().getPartitionedInternalStats(topic); + } else { + subAdmin.topics().getInternalStats(topic); + } + } else { + if (partitioned) { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPartitionedInternalStats(topic)); + + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getInternalStats(topic)); + } + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, partitioned); + } + + @Test(dataProvider = "partitioned") + @SneakyThrows + public void testDeleteSubscription(boolean partitioned) { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, partitioned); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + String subName = "test-sub"; + superUserAdmin.topics().createSubscription(topic, subName, MessageId.latest); + superUserAdmin.topics().deleteSubscription(topic, subName); + + // test tenant manager + superUserAdmin.topics().createSubscription(topic, subName, MessageId.latest); + tenantManagerAdmin.topics().deleteSubscription(topic, subName); + + superUserAdmin.topics().createSubscription(topic, subName, MessageId.latest); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().deleteSubscription(topic, subName)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.topics().deleteSubscription(topic, "test-sub"); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().deleteSubscription(topic, "test-sub")); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, partitioned); + } + + @Test(dataProvider = "partitioned") + @SneakyThrows + public void testSkipAllMessage(boolean partitioned) { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, partitioned); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + String subName = "test-sub"; + superUserAdmin.topics().createSubscription(topic, subName, MessageId.latest); + superUserAdmin.topics().skipAllMessages(topic, subName); + + // test tenant manager + tenantManagerAdmin.topics().skipAllMessages(topic, subName); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().skipAllMessages(topic, subName)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.topics().skipAllMessages(topic,subName); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().skipAllMessages(topic, subName)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, partitioned); + } + + @Test + @SneakyThrows + public void testSkipMessage() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + String subName = "test-sub"; + superUserAdmin.topics().createSubscription(topic, subName, MessageId.latest); + superUserAdmin.topics().skipMessages(topic, subName, 1); + + // test tenant manager + tenantManagerAdmin.topics().skipMessages(topic, subName, 1); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().skipMessages(topic, subName, 1)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.topics().skipMessages(topic, subName, 1); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().skipMessages(topic, subName, 1)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + } + + @Test(dataProvider = "partitioned") + @SneakyThrows + public void testExpireMessagesForAllSubscriptions(boolean partitioned) { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, partitioned); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + superUserAdmin.topics().expireMessagesForAllSubscriptions(topic, 1); + + // test tenant manager + tenantManagerAdmin.topics().expireMessagesForAllSubscriptions(topic, 1); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().expireMessagesForAllSubscriptions(topic, 1)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.topics().expireMessagesForAllSubscriptions(topic, 1); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().expireMessagesForAllSubscriptions(topic, 1)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, partitioned); + } + + @Test(dataProvider = "partitioned") + @SneakyThrows + public void testResetCursor(boolean partitioned) { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, partitioned); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + String subName = "test-sub"; + superUserAdmin.topics().createSubscription(topic, subName, MessageId.latest); + superUserAdmin.topics().resetCursor(topic, subName, System.currentTimeMillis()); + + // test tenant manager + tenantManagerAdmin.topics().resetCursor(topic, subName, System.currentTimeMillis()); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().resetCursor(topic, subName, System.currentTimeMillis())); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.topics().resetCursor(topic, subName, System.currentTimeMillis()); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().resetCursor(topic, subName, System.currentTimeMillis())); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, partitioned); + } + + @Test + @SneakyThrows + public void testResetCursorOnPosition() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + String subName = "test-sub"; + superUserAdmin.topics().createSubscription(topic, subName, MessageId.latest); + superUserAdmin.topics().resetCursor(topic, subName, MessageId.latest); + + // test tenant manager + tenantManagerAdmin.topics().resetCursor(topic, subName, MessageId.latest); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().resetCursor(topic, subName, MessageId.latest)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.topics().resetCursor(topic, subName, MessageId.latest); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().resetCursor(topic, subName, MessageId.latest)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testGetMessageById() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + @Cleanup + final PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(getPulsarService().getBrokerServiceUrl()) + .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) + .build(); + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + final MessageIdImpl messageId = (MessageIdImpl) producer.send("test"); + superUserAdmin.topics().getMessagesById(topic, messageId.getLedgerId(), messageId.getEntryId()); + + // test tenant manager + tenantManagerAdmin.topics().getMessagesById(topic, messageId.getLedgerId(), messageId.getEntryId()); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getMessagesById(topic, messageId.getLedgerId(), messageId.getEntryId())); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.topics().getMessagesById(topic, messageId.getLedgerId(), messageId.getEntryId()); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getMessagesById(topic, messageId.getLedgerId(), messageId.getEntryId())); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testPeekNthMessage() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + @Cleanup + final PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(getPulsarService().getBrokerServiceUrl()) + .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) + .build(); + String subName = "test-sub"; + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + producer.send("test"); + superUserAdmin.topics().peekMessages(topic, subName, 1); + + // test tenant manager + tenantManagerAdmin.topics().peekMessages(topic, subName, 1); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().peekMessages(topic, subName, 1)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.topics().peekMessages(topic, subName, 1); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().peekMessages(topic, subName, 1)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testExamineMessage() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + @Cleanup + final PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(getPulsarService().getBrokerServiceUrl()) + .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) + .build(); + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + producer.send("test"); + superUserAdmin.topics().examineMessage(topic, "latest", 1); + + // test tenant manager + tenantManagerAdmin.topics().examineMessage(topic, "latest", 1); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().examineMessage(topic, "latest", 1)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.topics().examineMessage(topic, "latest", 1); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().examineMessage(topic, "latest", 1)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + } + + @Test(dataProvider = "partitioned") + @SneakyThrows + public void testExpireMessage(boolean partitioned) { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, partitioned); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + @Cleanup + final PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(getPulsarService().getBrokerServiceUrl()) + .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) + .build(); + String subName = "test-sub"; + superUserAdmin.topics().createSubscription(topic, subName, MessageId.latest); + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + producer.send("test1"); + producer.send("test2"); + producer.send("test3"); + producer.send("test4"); + superUserAdmin.topics().expireMessages(topic, subName, 1); + + // test tenant manager + tenantManagerAdmin.topics().expireMessages(topic, subName, 1); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().expireMessages(topic, subName, 1)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.topics().expireMessages(topic, subName, 1); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().expireMessages(topic, subName, 1)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, partitioned); + } + + @Test + @SneakyThrows + public void testExpireMessageByPosition() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + @Cleanup + final PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(getPulsarService().getBrokerServiceUrl()) + .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) + .build(); + String subName = "test-sub"; + superUserAdmin.topics().createSubscription(topic, subName, MessageId.latest); + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + producer.send("test1"); + producer.send("test2"); + producer.send("test3"); + producer.send("test4"); + superUserAdmin.topics().expireMessages(topic, subName, MessageId.earliest, false); + + // test tenant manager + tenantManagerAdmin.topics().expireMessages(topic, subName, MessageId.earliest, false); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().expireMessages(topic, subName, MessageId.earliest, false)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.topics().expireMessages(topic, subName, MessageId.earliest, false); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().expireMessages(topic, subName, MessageId.earliest, false)); + } + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + } + + private void createTopic(String topic, boolean partitioned) throws Exception { + if (partitioned) { + superUserAdmin.topics().createPartitionedTopic(topic, 2); + } else { + superUserAdmin.topics().createNonPartitionedTopic(topic); + } + } + + private void deleteTopic(String topic, boolean partitioned) throws Exception { + if (partitioned) { + superUserAdmin.topics().deletePartitionedTopic(topic, true); + } else { + superUserAdmin.topics().delete(topic, true); + } + } } From ad28a7c1ef717aafa1c457762f43101152665572 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 2 Apr 2024 08:17:26 -0700 Subject: [PATCH 034/580] [improve][broker] Don't log brokerClientAuthenticationParameters and bookkeeperClientAuthenticationParameters by default (#22395) --- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 2 ++ 1 file changed, 2 insertions(+) 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 e088f50a05c88..80dfcaf4b0b20 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 @@ -1689,6 +1689,7 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece category = CATEGORY_STORAGE_BK, doc = "Parameters for bookkeeper auth plugin" ) + @ToString.Exclude private String bookkeeperClientAuthenticationParameters; @FieldContext( @@ -3303,6 +3304,7 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "Authentication parameters of the authentication plugin the broker is using to connect " + "to other brokers" ) + @ToString.Exclude private String brokerClientAuthenticationParameters = ""; @FieldContext( category = CATEGORY_REPLICATION, From d7d54522933b63f6a74ec7139c6dedebe8ad9149 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Tue, 2 Apr 2024 18:05:37 -0700 Subject: [PATCH 035/580] [fix][broker] Update TransferShedder underloaded broker check to consider max loaded broker's msgThroughputEMA and update IsExtensibleLoadBalancerImpl check (#22321) --- .../apache/pulsar/broker/PulsarService.java | 8 ++--- .../broker/admin/impl/NamespacesBase.java | 4 +-- .../extensions/ExtensibleLoadManagerImpl.java | 6 +--- .../extensions/scheduler/TransferShedder.java | 22 +++++++++----- .../broker/namespace/NamespaceService.java | 30 +++++++++---------- .../pulsar/broker/web/PulsarWebResource.java | 4 +-- .../scheduler/TransferShedderTest.java | 13 ++++---- 7 files changed, 46 insertions(+), 41 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index c1137bcfc25b7..9f7b40cc38334 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -399,7 +399,7 @@ public void closeMetadataServiceSession() throws Exception { } private void closeLeaderElectionService() throws Exception { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(this)) { ExtensibleLoadManagerImpl.get(loadManager.get()).getLeaderElectionService().close(); } else { if (this.leaderElectionService != null) { @@ -1156,7 +1156,7 @@ protected void closeLocalMetadataStore() throws Exception { } protected void startLeaderElectionService() { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(this)) { LOG.info("The load manager extension is enabled. Skipping PulsarService LeaderElectionService."); return; } @@ -1271,7 +1271,7 @@ protected void startLoadManagementService() throws PulsarServerException { LOG.info("Starting load management service ..."); this.loadManager.get().start(); - if (config.isLoadBalancerEnabled() && !ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (config.isLoadBalancerEnabled() && !ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(this)) { LOG.info("Starting load balancer"); if (this.loadReportTask == null) { long loadReportMinInterval = config.getLoadBalancerReportUpdateMinIntervalMillis(); @@ -1358,7 +1358,7 @@ public boolean isRunning() { * @return a reference of the current LeaderElectionService instance. */ public LeaderElectionService getLeaderElectionService() { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(this)) { return ExtensibleLoadManagerImpl.get(loadManager.get()).getLeaderElectionService(); } else { return this.leaderElectionService; 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 9d01530c60121..bbadc7bb3316d 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 @@ -969,13 +969,13 @@ public CompletableFuture setNamespaceBundleAffinityAsync(String bundleRang return CompletableFuture.completedFuture(null); }) .thenCompose(__ -> { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config())) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar())) { return CompletableFuture.completedFuture(null); } return validateLeaderBrokerAsync(); }) .thenAccept(__ -> { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config())) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar())) { return; } // For ExtensibleLoadManager, this operation will be ignored. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 3210578d8290a..c35dc11d7efc7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -270,10 +270,6 @@ public ExtensibleLoadManagerImpl() { this.brokerSelectionStrategy = new LeastResourceUsageWithWeight(); } - public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) { - return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName()); - } - public static boolean isLoadManagerExtensionEnabled(PulsarService pulsar) { return pulsar.getLoadManager().get() instanceof ExtensibleLoadManagerWrapper; } @@ -346,7 +342,7 @@ private static boolean configureSystemTopics(PulsarService pulsar) { public static CompletableFuture> getAssignedBrokerLookupData(PulsarService pulsar, String topic) { var config = pulsar.getConfig(); - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config) + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar) && config.isLoadBalancerMultiPhaseBundleUnload()) { var topicName = TopicName.get(topic); try { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedder.java index 3564b4e9e3b94..7126ccb034196 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedder.java @@ -362,7 +362,7 @@ public Set findBundlesForUnloading(LoadManagerContext context, final double targetStd = conf.getLoadBalancerBrokerLoadTargetStd(); boolean transfer = conf.isLoadBalancerTransferEnabled(); if (stats.std() > targetStd - || isUnderLoaded(context, stats.peekMinBroker(), stats.avg) + || isUnderLoaded(context, stats.peekMinBroker(), stats) || isOverLoaded(context, stats.peekMaxBroker(), stats.avg)) { unloadConditionHitCount++; } else { @@ -390,7 +390,7 @@ public Set findBundlesForUnloading(LoadManagerContext context, UnloadDecision.Reason reason; if (stats.std() > targetStd) { reason = Overloaded; - } else if (isUnderLoaded(context, stats.peekMinBroker(), stats.avg)) { + } else if (isUnderLoaded(context, stats.peekMinBroker(), stats)) { reason = Underloaded; if (debugMode) { log.info(String.format("broker:%s is underloaded:%s although " @@ -669,19 +669,27 @@ public Set findBundlesForUnloading(LoadManagerContext context, } - private boolean isUnderLoaded(LoadManagerContext context, String broker, double avgLoad) { + private boolean isUnderLoaded(LoadManagerContext context, String broker, LoadStats stats) { var brokerLoadDataOptional = context.brokerLoadDataStore().get(broker); if (brokerLoadDataOptional.isEmpty()) { return false; } var brokerLoadData = brokerLoadDataOptional.get(); - if (brokerLoadData.getMsgThroughputEMA() < 1) { + + var underLoadedMultiplier = + Math.min(0.5, Math.max(0.0, context.brokerConfiguration().getLoadBalancerBrokerLoadTargetStd() / 2.0)); + + if (brokerLoadData.getWeightedMaxEMA() < stats.avg * underLoadedMultiplier) { return true; } - return brokerLoadData.getWeightedMaxEMA() - < avgLoad * Math.min(0.5, Math.max(0.0, - context.brokerConfiguration().getLoadBalancerBrokerLoadTargetStd() / 2)); + var maxBrokerLoadDataOptional = context.brokerLoadDataStore().get(stats.peekMaxBroker()); + if (maxBrokerLoadDataOptional.isEmpty()) { + return false; + } + + return brokerLoadData.getMsgThroughputEMA() + < maxBrokerLoadDataOptional.get().getMsgThroughputEMA() * underLoadedMultiplier; } private boolean isOverLoaded(LoadManagerContext context, String broker, double avgLoad) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index dec8b098dddac..6228703f03ab9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -225,7 +225,7 @@ public CompletableFuture> getBrokerServiceUrlAsync(TopicN pulsar.getBrokerId(), optResult.get(), topic); return CompletableFuture.completedFuture(optResult); } - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return loadManager.get().findBrokerServiceUrl(Optional.of(topic), bundle); } else { // TODO: Add unit tests cover it. @@ -351,7 +351,7 @@ private CompletableFuture> internalGetWebServiceUrl(@Nullable Serv return CompletableFuture.completedFuture(Optional.empty()); } CompletableFuture> future = - ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config) + ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar) ? loadManager.get().findBrokerServiceUrl(Optional.ofNullable(topic), bundle) : findBrokerServiceUrl(bundle, options); @@ -415,7 +415,7 @@ public boolean registerNamespace(NamespaceName nsname, boolean ensureOwned) thro NamespaceBundle nsFullBundle = bundleFactory.getFullBundle(nsname); // v2 namespace will always use full bundle object final NamespaceEphemeralData otherData; - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl loadManager = ExtensibleLoadManagerImpl.get(this.loadManager.get()); otherData = loadManager.tryAcquiringOwnership(nsFullBundle).get(); } else { @@ -821,7 +821,7 @@ public CompletableFuture unloadNamespaceBundle(NamespaceBundle bundle, long timeout, TimeUnit timeoutUnit, boolean closeWithoutWaitingClientDisconnect) { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return ExtensibleLoadManagerImpl.get(loadManager.get()) .unloadNamespaceBundleAsync(bundle, destinationBroker); } @@ -843,7 +843,7 @@ public CompletableFuture> getOwnedNameSpac .getIsolationDataPoliciesAsync(pulsar.getConfiguration().getClusterName()) .thenApply(nsIsolationPoliciesOpt -> nsIsolationPoliciesOpt.orElseGet(NamespaceIsolationPolicies::new)) .thenCompose(namespaceIsolationPolicies -> { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); var statusMap = extensibleLoadManager.getOwnedServiceUnits().stream() @@ -923,7 +923,7 @@ public boolean isNamespaceBundleDisabled(NamespaceBundle bundle) throws Exceptio public CompletableFuture splitAndOwnBundle(NamespaceBundle bundle, boolean unload, NamespaceBundleSplitAlgorithm splitAlgorithm, List boundaries) { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return ExtensibleLoadManagerImpl.get(loadManager.get()) .splitNamespaceBundleAsync(bundle, splitAlgorithm, boundaries); } @@ -1166,7 +1166,7 @@ public OwnershipCache getOwnershipCache() { } public Set getOwnedServiceUnits() { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); return extensibleLoadManager.getOwnedServiceUnits(); } @@ -1188,7 +1188,7 @@ public CompletableFuture isServiceUnitOwnedAsync(ServiceUnitId suName) } if (suName instanceof NamespaceBundle) { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return loadManager.get().checkOwnershipAsync(Optional.empty(), suName); } // TODO: Add unit tests cover it. @@ -1216,7 +1216,7 @@ public boolean isServiceUnitActive(TopicName topicName) { public CompletableFuture isServiceUnitActiveAsync(TopicName topicName) { // TODO: Add unit tests cover it. - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return getBundleAsync(topicName) .thenCompose(bundle -> loadManager.get().checkOwnershipAsync(Optional.of(topicName), bundle)); } @@ -1231,7 +1231,7 @@ public CompletableFuture isServiceUnitActiveAsync(TopicName topicName) private CompletableFuture isNamespaceOwnedAsync(NamespaceName fqnn) { // TODO: Add unit tests cover it. - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return getFullBundleAsync(fqnn) .thenCompose(bundle -> loadManager.get().checkOwnershipAsync(Optional.empty(), bundle)); } @@ -1241,7 +1241,7 @@ private CompletableFuture isNamespaceOwnedAsync(NamespaceName fqnn) { private CompletableFuture isTopicOwnedAsync(TopicName topic) { // TODO: Add unit tests cover it. - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return getBundleAsync(topic) .thenCompose(bundle -> loadManager.get().checkOwnershipAsync(Optional.of(topic), bundle)); } @@ -1250,7 +1250,7 @@ private CompletableFuture isTopicOwnedAsync(TopicName topic) { public CompletableFuture checkTopicOwnership(TopicName topicName) { // TODO: Add unit tests cover it. - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return getBundleAsync(topicName) .thenCompose(bundle -> loadManager.get().checkOwnershipAsync(Optional.of(topicName), bundle)); } @@ -1260,7 +1260,7 @@ public CompletableFuture checkTopicOwnership(TopicName topicName) { public CompletableFuture removeOwnedServiceUnitAsync(NamespaceBundle nsBundle) { CompletableFuture future; - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); future = extensibleLoadManager.unloadNamespaceBundleAsync(nsBundle, Optional.empty()); } else { @@ -1566,7 +1566,7 @@ public PulsarClientImpl getNamespaceClient(ClusterDataImpl cluster) { } public CompletableFuture> getOwnerAsync(NamespaceBundle bundle) { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); return extensibleLoadManager.getOwnershipWithLookupDataAsync(bundle) .thenCompose(lookupData -> lookupData @@ -1583,7 +1583,7 @@ public boolean checkOwnershipPresent(NamespaceBundle bundle) throws Exception { } public CompletableFuture checkOwnershipPresentAsync(NamespaceBundle bundle) { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); return extensibleLoadManager.getOwnershipAsync(Optional.empty(), bundle) .thenApply(Optional::isPresent); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java index 07c33107d4b22..99f0a30d1a5f2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java @@ -733,7 +733,7 @@ public CompletableFuture validateBundleOwnershipAsync(NamespaceBundle bund .host(webUrl.get().getHost()) .port(webUrl.get().getPort()) .replaceQueryParam("authoritative", newAuthoritative); - if (!ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config())) { + if (!ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { uriBuilder.replaceQueryParam("destinationBroker", null); } URI redirect = uriBuilder.build(); @@ -1006,7 +1006,7 @@ protected boolean isLeaderBroker() { protected static boolean isLeaderBroker(PulsarService pulsar) { // For extensible load manager, it doesn't have leader election service on pulsar broker. - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar.getConfig())) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return true; } return pulsar.getLeaderElectionService().isLeader(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java index 0ff64616973d9..efca2880949f2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java @@ -1104,16 +1104,17 @@ public void testUnloadBundlesGreaterThanTargetThroughputAfterSplit() throws Ille assertEquals(stats.std(), 2.5809568279517847E-8); } - @Test - public void testMinBrokerWithZeroTraffic() throws IllegalAccessException { + public void testMinBrokerWithLowTraffic() throws IllegalAccessException { UnloadCounter counter = new UnloadCounter(); TransferShedder transferShedder = new TransferShedder(counter); var ctx = setupContext(); var brokerLoadDataStore = ctx.brokerLoadDataStore(); - var load = getCpuLoad(ctx, 4, "broker2:8080"); - FieldUtils.writeDeclaredField(load,"msgThroughputEMA", 0, true); + var load = getCpuLoad(ctx, 4, "broker2:8080"); + FieldUtils.writeDeclaredField(load, "msgThroughputEMA", 10, true); + + brokerLoadDataStore.pushAsync("broker2:8080", load); brokerLoadDataStore.pushAsync("broker4:8080", getCpuLoad(ctx, 55, "broker4:8080")); brokerLoadDataStore.pushAsync("broker5:8080", getCpuLoad(ctx, 65, "broker5:8080")); @@ -1268,10 +1269,10 @@ public void testOverloadOutlier() { Assertions.assertThat(res).isIn( Set.of(new UnloadDecision( new Unload("broker99:8080", "my-tenant/my-namespace99/0x00000000_0x0FFFFFFF", - Optional.of("broker52:8080")), Success, Overloaded)), + Optional.of("broker52:8080")), Success, Underloaded)), Set.of(new UnloadDecision( new Unload("broker99:8080", "my-tenant/my-namespace99/0x00000000_0x0FFFFFFF", - Optional.of("broker83:8080")), Success, Overloaded)) + Optional.of("broker83:8080")), Success, Underloaded)) ); assertEquals(counter.getLoadAvg(), 0.019900000000000008, 0.00001); assertEquals(counter.getLoadStd(), 0.09850375627355534, 0.00001); From 7e93d34ee5f9255703ba53a53063a814e2f8e68f Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 3 Apr 2024 17:28:10 +0800 Subject: [PATCH 036/580] [fix][cli] Fix help option (#22408) Signed-off-by: Zixuan Liu --- .../src/main/java/org/apache/pulsar/PulsarBrokerStarter.java | 2 +- .../main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java | 2 +- .../src/main/java/org/apache/pulsar/PulsarVersionStarter.java | 2 +- .../main/java/org/apache/pulsar/compaction/CompactorTool.java | 2 +- .../apache/pulsar/functions/worker/FunctionWorkerStarter.java | 2 +- .../pulsar/websocket/service/WebSocketServiceStarter.java | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarBrokerStarter.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarBrokerStarter.java index bd5b5399b0091..2d031cc8a74f6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarBrokerStarter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarBrokerStarter.java @@ -99,7 +99,7 @@ private static class StarterArguments { @Option(names = {"-fwc", "--functions-worker-conf"}, description = "Configuration file for Functions Worker") private String fnWorkerConfigFile = "conf/functions_worker.yml"; - @Option(names = {"-h", "--help"}, description = "Show this help message") + @Option(names = {"-h", "--help"}, usageHelp = true, description = "Show this help message") private boolean help = false; @Option(names = {"-g", "--generate-docs"}, description = "Generate docs") diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java index 854a5179161a4..e8efeabcdd37c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java @@ -161,7 +161,7 @@ private static class Arguments { "--proxy-url"}, description = "Proxy-server URL to which to connect.", required = false) private String clusterProxyUrl; - @Option(names = {"-h", "--help"}, description = "Show this help message") + @Option(names = {"-h", "--help"}, usageHelp = true, description = "Show this help message") private boolean help = false; @Option(names = {"-g", "--generate-docs"}, description = "Generate docs") diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarVersionStarter.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarVersionStarter.java index 556b3ebfd84b6..32876b6481c8a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarVersionStarter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarVersionStarter.java @@ -31,7 +31,7 @@ public class PulsarVersionStarter { @Command(name = "version", showDefaultValues = true, scope = ScopeType.INHERIT) private static class Arguments { - @Option(names = {"-h", "--help"}, description = "Show this help message") + @Option(names = {"-h", "--help"}, usageHelp = true, description = "Show this help message") private boolean help = false; @Option(names = {"-g", "--generate-docs"}, description = "Generate docs") diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java index f8cc95e6ac0ba..3225f7294d5a0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java @@ -61,7 +61,7 @@ private static class Arguments { @Option(names = {"-t", "--topic"}, description = "Topic to compact", required = true) private String topic; - @Option(names = {"-h", "--help"}, description = "Show this help message") + @Option(names = {"-h", "--help"}, usageHelp = true, description = "Show this help message") private boolean help = false; @Option(names = {"-g", "--generate-docs"}, description = "Generate docs") diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionWorkerStarter.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionWorkerStarter.java index c5fb552d9cc80..768b8e169786c 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionWorkerStarter.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionWorkerStarter.java @@ -40,7 +40,7 @@ private static class WorkerArguments { description = "Configuration File for Function Worker") private String configFile; - @Option(names = {"-h", "--help"}, description = "Show this help message") + @Option(names = {"-h", "--help"}, usageHelp = true, description = "Show this help message") private boolean help = false; @Option(names = {"-g", "--generate-docs"}, description = "Generate docs") diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketServiceStarter.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketServiceStarter.java index ed1a99b813331..0a445aebe3a00 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketServiceStarter.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketServiceStarter.java @@ -47,7 +47,7 @@ private static class Arguments { @Parameters(description = "config file", arity = "0..1") private String configFile = ""; - @Option(names = {"-h", "--help"}, description = "Show this help message") + @Option(names = {"-h", "--help"}, usageHelp = true, description = "Show this help message") private boolean help = false; @Option(names = {"-g", "--generate-docs"}, description = "Generate docs") From cd6f53baee7dbe0651e6581cb9bd3570017348c7 Mon Sep 17 00:00:00 2001 From: Teet Vaher <89580420+teet-vaher-sympower@users.noreply.github.com> Date: Wed, 3 Apr 2024 14:26:24 +0300 Subject: [PATCH 037/580] [fix][broker] Fix invalid condition in logging exceptions (#22412) --- .../admin/impl/PersistentTopicsBase.java | 80 +++++++++---------- .../broker/admin/v2/NonPersistentTopics.java | 6 +- .../pulsar/broker/admin/v3/Transactions.java | 12 +-- 3 files changed, 49 insertions(+), 49 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 16d088756f57b..424c081d9877c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -879,7 +879,7 @@ protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authorit } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned metadata while unloading topic {}", clientAppId(), topicName, ex); } @@ -889,7 +889,7 @@ protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authorit } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to validate the global namespace ownership while unloading topic {}", clientAppId(), topicName, ex); } @@ -1059,7 +1059,7 @@ private void internalUnloadNonPartitionedTopicAsync(AsyncResponse asyncResponse, }) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to unload topic {}, {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1079,7 +1079,7 @@ private void internalUnloadTransactionCoordinatorAsync(AsyncResponse asyncRespon }) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to unload tc {},{}", clientAppId(), topicName.getPartitionIndex(), ex); } @@ -1180,7 +1180,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned topic metadata while get" + " subscriptions for topic {}", clientAppId(), topicName, ex); } @@ -1190,7 +1190,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to validate the global namespace/topic ownership while get subscriptions" + " for topic {}", clientAppId(), topicName, ex); } @@ -1230,7 +1230,7 @@ private void internalGetSubscriptionsForNonPartitionedTopic(AsyncResponse asyncR .thenAccept(topic -> asyncResponse.resume(new ArrayList<>(topic.getSubscriptions().keys()))) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1337,7 +1337,7 @@ protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse, boolean } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned metadata while get managed info for {}", clientAppId(), topicName, ex); } @@ -1347,7 +1347,7 @@ protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse, boolean } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to validate the global namespace ownership while get managed info for {}", clientAppId(), topicName, ex); } @@ -1466,7 +1466,7 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned internal stats for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1522,7 +1522,7 @@ protected void internalGetPartitionedStatsInternal(AsyncResponse asyncResponse, }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned internal stats for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1648,7 +1648,7 @@ private void internalAnalyzeSubscriptionBacklogForNonPartitionedTopic(AsyncRespo }).exceptionally(ex -> { Throwable cause = ex.getCause(); // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to analyze subscription backlog {} {}", clientAppId(), topicName, subName, cause); } @@ -1675,7 +1675,7 @@ private void internalUpdateSubscriptionPropertiesForNonPartitionedTopic(AsyncRes }).exceptionally(ex -> { Throwable cause = ex.getCause(); // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} {}", clientAppId(), topicName, subName, cause); } asyncResponse.resume(new RestException(cause)); @@ -1703,7 +1703,7 @@ private void internalGetSubscriptionPropertiesForNonPartitionedTopic(AsyncRespon }).exceptionally(ex -> { Throwable cause = ex.getCause(); // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} {}", clientAppId(), topicName, subName, cause); } asyncResponse.resume(new RestException(cause)); @@ -1871,7 +1871,7 @@ protected void internalSkipAllMessages(AsyncResponse asyncResponse, String subNa } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to skip all messages for subscription {} on topic {}", clientAppId(), subName, topicName, ex); } @@ -1915,7 +1915,7 @@ private CompletableFuture internalSkipAllMessagesForNonPartitionedTopicAsy } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to skip all messages for subscription {} on topic {}", clientAppId(), subName, topicName, ex); } @@ -1979,7 +1979,7 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, } ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, topicName, subName, ex); } @@ -2048,7 +2048,7 @@ protected void internalExpireMessagesForAllSubscriptions(AsyncResponse asyncResp } ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages for all subscription on topic {}", clientAppId(), topicName, ex); } @@ -2114,7 +2114,7 @@ private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(Asy }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages for all subscription up to {} on {}", clientAppId(), expireTimeInSeconds, topicName, ex); } @@ -2321,7 +2321,7 @@ protected void internalCreateSubscription(AsyncResponse asyncResponse, String su })).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to create subscription {} on topic {}", clientAppId(), subscriptionName, topicName, ex); } @@ -2331,7 +2331,7 @@ protected void internalCreateSubscription(AsyncResponse asyncResponse, String su } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to create subscription {} on topic {}", clientAppId(), subscriptionName, topicName, ex); } @@ -2461,7 +2461,7 @@ protected void internalUpdateSubscriptionProperties(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} from topic {}", clientAppId(), subName, topicName, ex); } @@ -2500,7 +2500,7 @@ protected void internalAnalyzeSubscriptionBacklog(AsyncResponse asyncResponse, S }) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to analyze back log of subscription {} from topic {}", clientAppId(), subName, topicName, ex); } @@ -2584,7 +2584,7 @@ protected void internalGetSubscriptionProperties(AsyncResponse asyncResponse, St } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} from topic {}", clientAppId(), subName, topicName, ex); } @@ -2668,7 +2668,7 @@ protected void internalResetCursorOnPosition(AsyncResponse asyncResponse, String }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.warn("[{}][{}] Failed to reset cursor on subscription {} to position {}", clientAppId(), topicName, subName, messageId, ex.getCause()); } @@ -3304,7 +3304,7 @@ protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get backlog size for topic {}", clientAppId(), topicName, ex); } @@ -3312,7 +3312,7 @@ protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, return null; })).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to validate global namespace ownership " + "to get backlog size for topic {}", clientAppId(), topicName, ex); } @@ -3878,7 +3878,7 @@ protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, bo } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3886,7 +3886,7 @@ protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, bo }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3975,7 +3975,7 @@ protected void internalExpireMessagesByTimestamp(AsyncResponse asyncResponse, St ).exceptionally(ex -> { Throwable cause = FutureUtil.unwrapCompletionException(ex); // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(cause)) { + if (isNot307And404Exception(cause)) { if (cause instanceof RestException) { log.warn("[{}] Failed to expire messages up to {} on {}: {}", clientAppId(), expireTimeInSeconds, topicName, cause.toString()); @@ -4089,7 +4089,7 @@ protected void internalExpireMessagesByPosition(AsyncResponse asyncResponse, Str messageId, isExcluded, batchIndex); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages up to {} on subscription {} to position {}", clientAppId(), topicName, subName, messageId, ex); } @@ -4240,7 +4240,7 @@ protected void internalTriggerCompaction(AsyncResponse asyncResponse, boolean au } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to trigger compaction on topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4249,7 +4249,7 @@ protected void internalTriggerCompaction(AsyncResponse asyncResponse, boolean au } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to validate global namespace ownership to trigger compaction on topic {}", clientAppId(), topicName, ex); } @@ -4278,7 +4278,7 @@ protected void internalTriggerCompactionNonPartitionedTopic(AsyncResponse asyncR } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to trigger compaction for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4314,7 +4314,7 @@ protected void internalTriggerOffload(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to trigger offload for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4331,7 +4331,7 @@ protected void internalOffloadStatus(AsyncResponse asyncResponse, boolean author asyncResponse.resume(offloadProcessStatus); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to offload status on topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4608,7 +4608,7 @@ protected void internalGetLastMessageId(AsyncResponse asyncResponse, boolean aut }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get last messageId {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -5113,7 +5113,7 @@ protected void internalSetReplicatedSubscriptionStatus(AsyncResponse asyncRespon resultFuture.exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.warn("[{}] Failed to change replicated subscription status to {} - {} {}", clientAppId(), enabled, topicName, subName, ex); } @@ -5160,7 +5160,7 @@ private void internalSetReplicatedSubscriptionStatusForNonPartitionedTopic( } ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to set replicated subscription status on {} {}", clientAppId(), topicName, subName, ex); } @@ -5261,7 +5261,7 @@ protected void internalGetReplicatedSubscriptionStatus(AsyncResponse asyncRespon } resultFuture.exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get replicated subscription status on {} {}", clientAppId(), topicName, subName, ex); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index a22ad4b242f57..7de7d7363c0b1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -132,7 +132,7 @@ public void getInternalStats( }) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get internal stats for topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -479,7 +479,7 @@ public void getListFromBundle( } asyncResponse.resume(topicList); }).exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to list topics on namespace bundle {}/{}", clientAppId(), namespaceName, bundleRange, ex); } @@ -488,7 +488,7 @@ public void getListFromBundle( }); } }).exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to list topics on namespace bundle {}/{}", clientAppId(), namespaceName, bundleRange, ex); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java index c2a54987ea2d2..19a93db0b5146 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java @@ -105,7 +105,7 @@ public void getTransactionInBufferStats(@Suspended final AsyncResponse asyncResp Long.parseLong(leastSigBits)) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction state in transaction buffer {}", clientAppId(), topicName, ex); } @@ -143,7 +143,7 @@ public void getTransactionInPendingAckStats(@Suspended final AsyncResponse async Long.parseLong(leastSigBits), subName) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction state in pending ack {}", clientAppId(), topicName, ex); } @@ -181,7 +181,7 @@ public void getTransactionBufferStats(@Suspended final AsyncResponse asyncRespon internalGetTransactionBufferStats(authoritative, lowWaterMarks, segmentStats) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction buffer stats in topic {}", clientAppId(), topicName, ex); } @@ -217,7 +217,7 @@ public void getPendingAckStats(@Suspended final AsyncResponse asyncResponse, internalGetPendingAckStats(authoritative, subName, lowWaterMarks) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction pending ack stats in topic {}", clientAppId(), topicName, ex); } @@ -314,7 +314,7 @@ public void getPendingAckInternalStats(@Suspended final AsyncResponse asyncRespo internalGetPendingAckInternalStats(authoritative, subName, metadata) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get pending ack internal stats {}", clientAppId(), topicName, ex); } @@ -365,7 +365,7 @@ public void getTransactionBufferInternalStats(@Suspended final AsyncResponse asy internalGetTransactionBufferInternalStats(authoritative, metadata) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction buffer internal stats {}", clientAppId(), topicName, ex); } From a1970ae0996b2ccaad1251a8ef692faee24b83b8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 4 Apr 2024 00:27:25 -0700 Subject: [PATCH 038/580] [fix][broker] Support OIDC providers with JWK without alg field set in keys (#22421) --- .../oidc/AuthenticationProviderOpenID.java | 3 ++- ...enticationProviderOpenIDIntegrationTest.java | 17 ++++++++++++++++- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java b/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java index 59ad071a2cd50..a9d812c10b06a 100644 --- a/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java +++ b/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java @@ -304,7 +304,8 @@ private CompletableFuture authenticateToken(String token) { return verifyIssuerAndGetJwk(jwt) .thenCompose(jwk -> { try { - if (!jwt.getAlgorithm().equals(jwk.getAlgorithm())) { + // verify the algorithm, if it is set ("alg" is optional in the JWK spec) + if (jwk.getAlgorithm() != null && !jwt.getAlgorithm().equals(jwk.getAlgorithm())) { incrementFailureMetric(AuthenticationExceptionCode.ALGORITHM_MISMATCH); return CompletableFuture.failedFuture( new AuthenticationException("JWK's alg [" + jwk.getAlgorithm() diff --git a/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java b/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java index 32cde11960204..e11fd8395a5bf 100644 --- a/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java +++ b/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java @@ -75,6 +75,7 @@ public class AuthenticationProviderOpenIDIntegrationTest { // These are the kid values for JWKs in the /keys endpoint String validJwk = "valid"; String invalidJwk = "invalid"; + String validJwkWithoutAlg = "valid_without_alg"; // The valid issuer String issuer; @@ -188,10 +189,16 @@ void beforeClass() throws IOException { "kty":"RSA", "n":"invalid-key", "e":"AQAB" + }, + { + "kid":"%s", + "kty":"RSA", + "n":"%s", + "e":"%s" } ] } - """.formatted(validJwk, n, e, invalidJwk)))); + """.formatted(validJwk, n, e, invalidJwk, validJwkWithoutAlg, n, e)))); server.stubFor( get(urlEqualTo("/missing-kid/.well-known/openid-configuration")) @@ -275,6 +282,14 @@ public void testTokenWithValidJWK() throws Exception { assertEquals(role, provider.authenticateAsync(new AuthenticationDataCommand(token)).get()); } + @Test + public void testTokenWithValidJWKWithoutAlg() throws Exception { + String role = "superuser"; + // test with a key in JWK that does not have an "alg" field. "alg" is optional in the JWK spec + String token = generateToken(validJwkWithoutAlg, issuer, role, "allowed-audience", 0L, 0L, 10000L); + assertEquals(role, provider.authenticateAsync(new AuthenticationDataCommand(token)).get()); + } + @Test public void testTokenWithTrailingSlashAndValidJWK() throws Exception { String role = "superuser"; From f4235580e6477f0c2f846419866b70c1b057e372 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 4 Apr 2024 01:15:44 -0700 Subject: [PATCH 039/580] [fix][misc] Rename all shaded Netty native libraries (#22415) --- src/rename-netty-native-libs.cmd | 22 +++++++++++++++++++--- src/rename-netty-native-libs.sh | 6 ++++++ 2 files changed, 25 insertions(+), 3 deletions(-) diff --git a/src/rename-netty-native-libs.cmd b/src/rename-netty-native-libs.cmd index 9003f6d0ef499..e3108746fcbfe 100644 --- a/src/rename-netty-native-libs.cmd +++ b/src/rename-netty-native-libs.cmd @@ -42,11 +42,27 @@ call %UNZIP_CMD% cd /d %TMP_DIR%/%FILE_PREFIX% :: Loop through the number of groups -SET Obj_Length=2 +SET Obj_Length=10 SET Obj[0].FROM=libnetty_transport_native_epoll_x86_64.so SET Obj[0].TO=liborg_apache_pulsar_shade_netty_transport_native_epoll_x86_64.so -SET Obj[1].FROM=libnetty_tcnative_linux_x86_64.so -SET Obj[1].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so +SET Obj[1].FROM=libnetty_transport_native_epoll_aarch_64.so +SET Obj[1].TO=liborg_apache_pulsar_shade_netty_transport_native_epoll_aarch_64.so +SET Obj[2].FROM=libnetty_tcnative_linux_x86_64.so +SET Obj[2].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so +SET Obj[3].FROM=libnetty_tcnative_linux_aarch_64.so +SET Obj[3].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_aarch_64.so +SET Obj[4].FROM=libnetty_tcnative_osx_x86_64.jnilib +SET Obj[4].TO=liborg_apache_pulsar_shade_netty_tcnative_osx_x86_64.jnilib +SET Obj[5].FROM=libnetty_tcnative_osx_aarch_64.jnilib +SET Obj[5].TO=liborg_apache_pulsar_shade_netty_tcnative_osx_aarch_64.jnilib +SET Obj[6].FROM=libnetty_transport_native_io_uring_x86_64.so +SET Obj[6].TO=liborg_apache_pulsar_shade_netty_transport_native_io_uring_x86_64.so +SET Obj[7].FROM=libnetty_transport_native_io_uring_aarch_64.so +SET Obj[7].TO=liborg_apache_pulsar_shade_netty_transport_native_io_uring_aarch_64.so +SET Obj[8].FROM=libnetty_resolver_dns_native_macos_aarch_64.jnilib +SET Obj[8].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_aarch_64.jnilib +SET Obj[9].FROM=libnetty_resolver_dns_native_macos_x86_64.jnilib +SET Obj[9].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_x86_64.jnilibSET Obj_Index=0 SET Obj_Index=0 :LoopStart diff --git a/src/rename-netty-native-libs.sh b/src/rename-netty-native-libs.sh index 44b971a02c912..ea2a4c0e2421e 100755 --- a/src/rename-netty-native-libs.sh +++ b/src/rename-netty-native-libs.sh @@ -27,7 +27,13 @@ FILE_PREFIX='META-INF/native' FILES_TO_RENAME=( 'libnetty_transport_native_epoll_x86_64.so liborg_apache_pulsar_shade_netty_transport_native_epoll_x86_64.so' + 'libnetty_transport_native_epoll_aarch_64.so liborg_apache_pulsar_shade_netty_transport_native_epoll_aarch_64.so' 'libnetty_tcnative_linux_x86_64.so liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so' + 'libnetty_tcnative_linux_aarch_64.so liborg_apache_pulsar_shade_netty_tcnative_linux_aarch_64.so' + 'libnetty_tcnative_osx_x86_64.jnilib liborg_apache_pulsar_shade_netty_tcnative_osx_x86_64.jnilib' + 'libnetty_tcnative_osx_aarch_64.jnilib liborg_apache_pulsar_shade_netty_tcnative_osx_aarch_64.jnilib' + 'libnetty_transport_native_io_uring_x86_64.so liborg_apache_pulsar_shade_netty_transport_native_io_uring_x86_64.so' + 'libnetty_transport_native_io_uring_aarch_64.so liborg_apache_pulsar_shade_netty_transport_native_io_uring_aarch_64.so' 'libnetty_resolver_dns_native_macos_aarch_64.jnilib liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_aarch_64.jnilib' 'libnetty_resolver_dns_native_macos_x86_64.jnilib liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_x86_64.jnilib' ) From 5b6f91bc0f839c467bdc1af35c8eac7b14aa8822 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 4 Apr 2024 06:39:53 -0700 Subject: [PATCH 040/580] [improve][build] Upgrade Lombok to 1.18.32 for Java 22 support (#22425) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7c19afef73a12..6c3d8d276b1ca 100644 --- a/pom.xml +++ b/pom.xml @@ -227,7 +227,7 @@ flexible messaging model and an intuitive client API. 0.9.1 2.1.0 3.24.2 - 1.18.30 + 1.18.32 1.3.2 2.3.1 1.2.0 From 706b588860c93d2a8a5f54bd3db0d10c004699db Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Thu, 4 Apr 2024 23:01:00 +0800 Subject: [PATCH 041/580] [cleanup][admin] Remove unused methods in PersistentTopicsBase (#22424) --- .../admin/impl/PersistentTopicsBase.java | 101 ------------------ 1 file changed, 101 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 424c081d9877c..c5e280c5577d9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -1711,107 +1711,6 @@ private void internalGetSubscriptionPropertiesForNonPartitionedTopic(AsyncRespon }); } - protected void internalDeleteSubscriptionForcefully(AsyncResponse asyncResponse, - String subName, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - - future.thenAccept(__ -> { - if (topicName.isPartitioned()) { - internalDeleteSubscriptionForNonPartitionedTopicForcefully(asyncResponse, subName, authoritative); - } else { - getPartitionedTopicMetadataAsync(topicName, - authoritative, false).thenAccept(partitionMetadata -> { - if (partitionMetadata.partitions > 0) { - final List> futures = new ArrayList<>(); - - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - futures.add(pulsar().getAdminClient().topics() - .deleteSubscriptionAsync(topicNamePartition.toString(), subName, true)); - } catch (Exception e) { - log.error("[{}] Failed to delete subscription forcefully {} {}", - clientAppId(), topicNamePartition, subName, - e); - asyncResponse.resume(new RestException(e)); - return; - } - } - - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = exception.getCause(); - if (t instanceof NotFoundException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, - getSubNotFoundErrorMessage(topicName.toString(), subName))); - return null; - } else { - log.error("[{}] Failed to delete subscription forcefully {} {}", - clientAppId(), topicName, subName, t); - asyncResponse.resume(new RestException(t)); - return null; - } - } - - asyncResponse.resume(Response.noContent().build()); - return null; - }); - } else { - internalDeleteSubscriptionForNonPartitionedTopicForcefully(asyncResponse, subName, - authoritative); - } - }).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { - log.error("[{}] Failed to delete subscription forcefully {} from topic {}", - clientAppId(), subName, topicName, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - }); - } - }).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { - log.error("[{}] Failed to delete subscription {} from topic {}", - clientAppId(), subName, topicName, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - }); - } - - private void internalDeleteSubscriptionForNonPartitionedTopicForcefully(AsyncResponse asyncResponse, - String subName, boolean authoritative) { - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.UNSUBSCRIBE, subName)) - .thenCompose(__ -> getTopicReferenceAsync(topicName)) - .thenCompose(topic -> { - Subscription sub = topic.getSubscription(subName); - if (sub == null) { - throw new RestException(Status.NOT_FOUND, - getSubNotFoundErrorMessage(topicName.toString(), subName)); - } - return sub.deleteForcefully(); - }).thenRun(() -> { - log.info("[{}][{}] Deleted subscription forcefully {}", clientAppId(), topicName, subName); - asyncResponse.resume(Response.noContent().build()); - }).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { - log.error("[{}] Failed to delete subscription forcefully {} {}", - clientAppId(), topicName, subName, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - }); - } - protected void internalSkipAllMessages(AsyncResponse asyncResponse, String subName, boolean authoritative) { CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.SKIP, subName); future.thenCompose(__ -> { From ba8e8f5e218f01d42f39bc7f62bfc0bcdff99085 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Thu, 4 Apr 2024 23:08:45 +0800 Subject: [PATCH 042/580] [admin][broker] Fix force delete subscription not working (#22423) --- .../admin/impl/PersistentTopicsBase.java | 5 ++-- .../broker/admin/PersistentTopicsTest.java | 30 +++++++++++++++++++ 2 files changed, 32 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index c5e280c5577d9..c9c29271b6afe 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -1557,7 +1557,7 @@ protected CompletableFuture internalDeleteSubscriptionAsync(String subName for (int i = 0; i < partitionMetadata.partitions; i++) { TopicName topicNamePartition = topicName.getPartition(i); futures.add(adminClient.topics() - .deleteSubscriptionAsync(topicNamePartition.toString(), subName, false)); + .deleteSubscriptionAsync(topicNamePartition.toString(), subName, force)); } return FutureUtil.waitForAll(futures).handle((result, exception) -> { @@ -1576,8 +1576,7 @@ protected CompletableFuture internalDeleteSubscriptionAsync(String subName return null; }); } - return internalDeleteSubscriptionForNonPartitionedTopicAsync(subName, authoritative, - force); + return internalDeleteSubscriptionForNonPartitionedTopicAsync(subName, authoritative, force); }); } }); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index 9a292175caa59..f37b53bb0dc75 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -74,10 +74,12 @@ import org.apache.pulsar.client.admin.Topics; import org.apache.pulsar.client.admin.internal.TopicsImpl; import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.interceptor.ProducerInterceptor; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; @@ -1786,4 +1788,32 @@ public void testCreateMissingPartitions() throws Exception { String topicName = "persistent://" + testTenant + "/" + testNamespaceLocal + "/testCreateMissingPartitions"; assertThrows(PulsarAdminException.NotFoundException.class, () -> admin.topics().createMissedPartitions(topicName)); } + + @Test + public void testForceDeleteSubscription() throws Exception { + try { + pulsar.getConfiguration().setAllowAutoSubscriptionCreation(false); + String topicName = "persistent://" + testTenant + "/" + testNamespaceLocal + "/testForceDeleteSubscription"; + String subName = "sub1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subName, MessageId.latest); + + @Cleanup + Consumer c0 = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + @Cleanup + Consumer c1 = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + admin.topics().deleteSubscription(topicName, subName, true); + } finally { + pulsar.getConfiguration().setAllowAutoSubscriptionCreation(true); + } + } } From bdb3d6922f990f85552f22029c222f632350c771 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 4 Apr 2024 08:19:56 -0700 Subject: [PATCH 043/580] [fix][ci] Fix labels for flaky test GitHub issue template (#22434) --- .github/ISSUE_TEMPLATE/flaky-test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/ISSUE_TEMPLATE/flaky-test.yml b/.github/ISSUE_TEMPLATE/flaky-test.yml index 44ff64197822c..e7b57e1aeda87 100644 --- a/.github/ISSUE_TEMPLATE/flaky-test.yml +++ b/.github/ISSUE_TEMPLATE/flaky-test.yml @@ -18,7 +18,7 @@ name: Flaky test title: "Flaky-test: test_class.test_method" description: Report a flaky test failure -labels: [ "component/test", "flaky-tests" ] +labels: [ "area/test", "type/flaky-tests" ] body: - type: markdown attributes: From 5390ef2f633c7145798b20a53f4a96667b74bd14 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 4 Apr 2024 08:37:44 -0700 Subject: [PATCH 044/580] [fix][build] Fix typo in rename script for windows cmd (#22426) --- src/rename-netty-native-libs.cmd | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/rename-netty-native-libs.cmd b/src/rename-netty-native-libs.cmd index e3108746fcbfe..bfaa16de0812c 100644 --- a/src/rename-netty-native-libs.cmd +++ b/src/rename-netty-native-libs.cmd @@ -62,7 +62,7 @@ SET Obj[7].TO=liborg_apache_pulsar_shade_netty_transport_native_io_uring_aarch_6 SET Obj[8].FROM=libnetty_resolver_dns_native_macos_aarch_64.jnilib SET Obj[8].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_aarch_64.jnilib SET Obj[9].FROM=libnetty_resolver_dns_native_macos_x86_64.jnilib -SET Obj[9].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_x86_64.jnilibSET Obj_Index=0 +SET Obj[9].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_x86_64.jnilib SET Obj_Index=0 :LoopStart From 5f31ec383bb7526eca24b95002f6cd498057fee7 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 4 Apr 2024 11:25:16 -0700 Subject: [PATCH 045/580] [improve][test] Move most flaky tests to flaky group (#22433) - also add solution for running test methods added to flaky group since that was missing --- build/run_unit_group.sh | 14 +++++++++--- .../pulsar/tests/AnnotationListener.java | 22 +++++++++++++++++++ .../mledger/impl/ManagedLedgerTest.java | 4 ++-- pom.xml | 2 +- .../admin/AdminApiMultiBrokersTest.java | 2 +- .../pulsar/broker/admin/TopicAuthZTest.java | 6 ++--- .../service/PersistentMessageFinderTest.java | 6 ++--- .../impl/ProducerConsumerInternalTest.java | 2 +- .../client/impl/TransactionEndToEndTest.java | 4 ++-- ...ctionEndToEndWithoutBatchIndexAckTest.java | 2 +- 10 files changed, 47 insertions(+), 17 deletions(-) diff --git a/build/run_unit_group.sh b/build/run_unit_group.sh index 76931f4a44370..351477aed1c92 100755 --- a/build/run_unit_group.sh +++ b/build/run_unit_group.sh @@ -139,13 +139,21 @@ function print_testng_failures() { function test_group_broker_flaky() { echo "::endgroup::" echo "::group::Running quarantined tests" - mvn_test --no-fail-fast -pl pulsar-broker -Dgroups='quarantine' -DexcludedGroups='' -DfailIfNoTests=false \ + mvn_test --no-fail-fast -pl pulsar-broker -Dgroups='quarantine' -DexcludedGroups='flaky' -DfailIfNoTests=false \ -DtestForkCount=2 || print_testng_failures pulsar-broker/target/surefire-reports/testng-failed.xml "Quarantined test failure in" "Quarantined test failures" echo "::endgroup::" echo "::group::Running flaky tests" - mvn_test --no-fail-fast -pl pulsar-broker -Dgroups='flaky' -DtestForkCount=2 + mvn_test --no-fail-fast -pl pulsar-broker -Dgroups='flaky' -DexcludedGroups='quarantine' -DtestForkCount=2 echo "::endgroup::" + local modules_with_flaky_tests=$(git grep -l '@Test.*"flaky"' | grep '/src/test/java/' | \ + awk -F '/src/test/java/' '{ print $1 }' | grep -v -E 'pulsar-broker' | sort | uniq | \ + perl -0777 -p -e 's/\n(\S)/,$1/g') + if [ -n "${modules_with_flaky_tests}" ]; then + echo "::group::Running flaky tests in modules '${modules_with_flaky_tests}'" + mvn_test --no-fail-fast -pl "${modules_with_flaky_tests}" -Dgroups='flaky' -DexcludedGroups='quarantine' -DfailIfNoTests=false + echo "::endgroup::" + fi } function test_group_proxy() { @@ -179,7 +187,7 @@ function test_group_other() { perl -0777 -p -e 's/\n(\S)/,$1/g') if [ -n "${modules_with_quarantined_tests}" ]; then echo "::group::Running quarantined tests outside of pulsar-broker & pulsar-proxy (if any)" - mvn_test --no-fail-fast -pl "${modules_with_quarantined_tests}" test -Dgroups='quarantine' -DexcludedGroups='' \ + mvn_test --no-fail-fast -pl "${modules_with_quarantined_tests}" test -Dgroups='quarantine' -DexcludedGroups='flaky' \ -DfailIfNoTests=false || \ echo "::warning::There were test failures in the 'quarantine' test group." echo "::endgroup::" diff --git a/buildtools/src/main/java/org/apache/pulsar/tests/AnnotationListener.java b/buildtools/src/main/java/org/apache/pulsar/tests/AnnotationListener.java index 38cd2a1747a63..0c464fd97a970 100644 --- a/buildtools/src/main/java/org/apache/pulsar/tests/AnnotationListener.java +++ b/buildtools/src/main/java/org/apache/pulsar/tests/AnnotationListener.java @@ -32,6 +32,10 @@ public class AnnotationListener implements IAnnotationTransformer { private static final long DEFAULT_TEST_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5); private static final String OTHER_GROUP = "other"; + private static final String FLAKY_GROUP = "flaky"; + + private static final String QUARANTINE_GROUP = "quarantine"; + public AnnotationListener() { System.out.println("Created annotation listener"); } @@ -51,9 +55,27 @@ public void transform(ITestAnnotation annotation, annotation.setTimeOut(DEFAULT_TEST_TIMEOUT_MILLIS); } + replaceGroupsIfFlakyOrQuarantineGroupIsIncluded(annotation); addToOtherGroupIfNoGroupsSpecified(annotation); } + // A test method will inherit the test groups from the class level and this solution ensures that a test method + // added to the flaky or quarantine group will not be executed as part of other groups. + private void replaceGroupsIfFlakyOrQuarantineGroupIsIncluded(ITestAnnotation annotation) { + if (annotation.getGroups() != null && annotation.getGroups().length > 1) { + for (String group : annotation.getGroups()) { + if (group.equals(QUARANTINE_GROUP)) { + annotation.setGroups(new String[]{QUARANTINE_GROUP}); + return; + } + if (group.equals(FLAKY_GROUP)) { + annotation.setGroups(new String[]{FLAKY_GROUP}); + return; + } + } + } + } + private void addToOtherGroupIfNoGroupsSpecified(ITestOrConfiguration annotation) { // Add test to "other" group if there's no specified group if (annotation.getGroups() == null || annotation.getGroups().length == 0) { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 6b409babcb461..22cf4d8b7a7ca 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -2446,7 +2446,7 @@ public void testRetentionSize() throws Exception { }); } - @Test + @Test(groups = "flaky") public void testTimestampOnWorkingLedger() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); conf.setMaxEntriesPerLedger(1); @@ -3525,7 +3525,7 @@ public void testLedgerReachMaximumRolloverTime() throws Exception { .until(() -> firstLedgerId != ml.addEntry("test".getBytes()).getLedgerId()); } - @Test + @Test(groups = "flaky") public void testLedgerNotRolloverWithoutOpenState() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(2); diff --git a/pom.xml b/pom.xml index 6c3d8d276b1ca..835bd28f7f25b 100644 --- a/pom.xml +++ b/pom.xml @@ -92,7 +92,7 @@ flexible messaging model and an intuitive client API. **/Test*.java,**/*Test.java,**/*Tests.java,**/*TestCase.java - quarantine + quarantine,flaky UTF-8 UTF-8 diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMultiBrokersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMultiBrokersTest.java index 7c9154a27ff69..46b24abd6d4e2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMultiBrokersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMultiBrokersTest.java @@ -132,7 +132,7 @@ public void testTopicLookup(TopicDomain topicDomain, boolean isPartition) throws Assert.assertEquals(lookupResultSet.size(), 1); } - @Test + @Test(groups = "flaky") public void testForceDeletePartitionedTopicWithSub() throws Exception { final int numPartitions = 10; TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of("role1", "role2"), Set.of("test")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index 2ff03732fae27..2e75b59ec8582 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -58,7 +58,7 @@ public class TopicAuthZTest extends MockedPulsarStandalone { .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); @SneakyThrows - @BeforeClass + @BeforeClass(alwaysRun = true) public void before() { configureTokenAuthentication(); configureDefaultAuthorization(); @@ -78,7 +78,7 @@ public void before() { @SneakyThrows - @AfterClass + @AfterClass(alwaysRun = true) public void after() { if (superUserAdmin != null) { superUserAdmin.close(); @@ -988,7 +988,7 @@ public void testExamineMessage() { deleteTopic(topic, false); } - @Test(dataProvider = "partitioned") + @Test(dataProvider = "partitioned", groups = "flaky") @SneakyThrows public void testExpireMessage(boolean partitioned) { final String random = UUID.randomUUID().toString(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java index 6883c0467e481..6965ac28068c1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java @@ -266,7 +266,7 @@ void testPersistentMessageFinderWhenLastMessageDelete() throws Exception { ledger.addEntry(createMessageWrittenToLedger("msg2")); ledger.addEntry(createMessageWrittenToLedger("msg3")); Position lastPosition = ledger.addEntry(createMessageWrittenToLedger("last-message")); - + long endTimestamp = System.currentTimeMillis() + 1000; Result result = new Result(); @@ -383,7 +383,7 @@ public static Set getBrokerEntryMetadataIntercep * * @throws Exception */ - @Test + @Test(groups = "flaky") void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { final String ledgerAndCursorName = "testPersistentMessageExpiryWithNonRecoverableLedgers"; @@ -440,7 +440,7 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { } - @Test + @Test(groups = "flaky") public void testIncorrectClientClock() throws Exception { final String ledgerAndCursorName = "testIncorrectClientClock"; int maxTTLSeconds = 1; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerConsumerInternalTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerConsumerInternalTest.java index 9cf457fd9d0e1..a06085d3d4626 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerConsumerInternalTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerConsumerInternalTest.java @@ -120,7 +120,7 @@ private void removeServiceProducerMaintainedByServerCnx(ServiceProducer serviceP }); } - @Test + @Test(groups = "flaky") public void testExclusiveConsumerWillAlwaysRetryEvenIfReceivedConsumerBusyError() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp_"); final String subscriptionName = "subscription1"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java index 75b551516c3f5..4fa86c49914a4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java @@ -98,7 +98,7 @@ public class TransactionEndToEndTest extends TransactionTestBase { protected static final int NUM_PARTITIONS = 16; private static final int waitTimeForCanReceiveMsgInSec = 5; private static final int waitTimeForCannotReceiveMsgInSec = 5; - @BeforeClass + @BeforeClass(alwaysRun = true) protected void setup() throws Exception { conf.setAcknowledgmentAtBatchIndexLevelEnabled(true); setUpBase(1, NUM_PARTITIONS, TOPIC_OUTPUT, TOPIC_PARTITION); @@ -1626,7 +1626,7 @@ public void testSendTxnAckBatchMessageToDLQ() throws Exception { admin.topics().delete(topic, true); } - @Test + @Test(groups = "flaky") public void testDelayedTransactionMessages() throws Exception { String topic = NAMESPACE1 + "/testDelayedTransactionMessages"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndWithoutBatchIndexAckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndWithoutBatchIndexAckTest.java index 52faae2f8ea1f..df4ad32b6a8ae 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndWithoutBatchIndexAckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndWithoutBatchIndexAckTest.java @@ -30,7 +30,7 @@ @Test(groups = "broker-impl") public class TransactionEndToEndWithoutBatchIndexAckTest extends TransactionEndToEndTest { - @BeforeClass + @BeforeClass(alwaysRun = true) protected void setup() throws Exception { conf.setAcknowledgmentAtBatchIndexLevelEnabled(false); setUpBase(1, NUM_PARTITIONS, TOPIC_OUTPUT, TOPIC_PARTITION); From 902728ef6590233b87c14d2528590ad7e6fdcc12 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Fri, 5 Apr 2024 15:42:40 +0800 Subject: [PATCH 046/580] [fix][broker][admin] Fix cannot update properties on NonDurable subscription. (#22411) --- .../mledger/impl/ManagedCursorImpl.java | 10 +++-- .../broker/admin/PersistentTopicsTest.java | 38 +++++++++++++++++++ 2 files changed, 45 insertions(+), 3 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index b253da72fa92b..4daa06cad576a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -355,15 +355,19 @@ private CompletableFuture computeCursorProperties( final Function, Map> updateFunction) { CompletableFuture updateCursorPropertiesResult = new CompletableFuture<>(); - final Stat lastCursorLedgerStat = ManagedCursorImpl.this.cursorLedgerStat; - Map newProperties = updateFunction.apply(ManagedCursorImpl.this.cursorProperties); + if (!isDurable()) { + this.cursorProperties = Collections.unmodifiableMap(newProperties); + updateCursorPropertiesResult.complete(null); + return updateCursorPropertiesResult; + } + ManagedCursorInfo copy = ManagedCursorInfo .newBuilder(ManagedCursorImpl.this.managedCursorInfo) .clearCursorProperties() .addAllCursorProperties(buildStringPropertiesMap(newProperties)) .build(); - + final Stat lastCursorLedgerStat = ManagedCursorImpl.this.cursorLedgerStat; ledger.getStore().asyncUpdateCursorInfo(ledger.getName(), name, copy, lastCursorLedgerStat, new MetaStoreCallback<>() { @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index f37b53bb0dc75..8e1375303ce4c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -32,6 +32,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertSame; import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; @@ -54,6 +55,8 @@ import javax.ws.rs.core.UriInfo; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.commons.collections4.MapUtils; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.admin.v2.ExtPersistentTopics; import org.apache.pulsar.broker.admin.v2.NonPersistentTopics; @@ -66,6 +69,8 @@ import org.apache.pulsar.broker.resources.TopicResources; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.client.admin.LongRunningProcessStatus; @@ -78,6 +83,7 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.interceptor.ProducerInterceptor; @@ -1816,4 +1822,36 @@ public void testForceDeleteSubscription() throws Exception { pulsar.getConfiguration().setAllowAutoSubscriptionCreation(true); } } + + @Test + public void testUpdatePropertiesOnNonDurableSub() throws Exception { + String topic = "persistent://" + testTenant + "/" + testNamespaceLocal + "/testUpdatePropertiesOnNonDurableSub"; + String subscription = "sub"; + admin.topics().createNonPartitionedTopic(topic); + + @Cleanup + Reader __ = pulsarClient.newReader(Schema.STRING) + .startMessageId(MessageId.earliest) + .subscriptionName(subscription) + .topic(topic) + .create(); + + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).get().get(); + PersistentSubscription subscription1 = persistentTopic.getSubscriptions().get(subscription); + assertNotNull(subscription1); + ManagedCursor cursor = subscription1.getCursor(); + + Map properties = admin.topics().getSubscriptionProperties(topic, subscription); + assertEquals(properties.size(), 0); + assertTrue(MapUtils.isEmpty(cursor.getCursorProperties())); + + admin.topics().updateSubscriptionProperties(topic, subscription, Map.of("foo", "bar")); + properties = admin.topics().getSubscriptionProperties(topic, subscription); + assertEquals(properties.size(), 1); + assertEquals(properties.get("foo"), "bar"); + + assertEquals(cursor.getCursorProperties().size(), 1); + assertEquals(cursor.getCursorProperties().get("foo"), "bar"); + } } From 11b3c168e932a00999864690465fc97e676e2d83 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 5 Apr 2024 09:25:12 -0700 Subject: [PATCH 047/580] [improve][misc] Specify /pulsar/data as the home dir for the user in the Alpine based image (#22447) --- docker/pulsar/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 3b0b6322734c6..1ca6edb2e323c 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -118,5 +118,5 @@ ENV PATH=$PATH:$JAVA_HOME/bin:/pulsar/bin # The UID must be non-zero. Otherwise, it is arbitrary. No logic should rely on its specific value. ARG DEFAULT_USERNAME=pulsar -RUN adduser ${DEFAULT_USERNAME} -u 10000 -G root -D +RUN adduser ${DEFAULT_USERNAME} -u 10000 -G root -D -H -h /pulsar/data USER 10000 From 60ab060dd2922a51e66f151bddac0637e06897fe Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 5 Apr 2024 12:21:38 -0700 Subject: [PATCH 048/580] [improve][ci] revisit tune-runner-vm action, drop tuning for docker (#22448) --- .github/actions/tune-runner-vm/action.yml | 6 ------ .github/workflows/pulsar-ci.yaml | 8 ++++---- README.md | 6 ++++++ build/build_java_test_image.sh | 8 +------- 4 files changed, 11 insertions(+), 17 deletions(-) diff --git a/.github/actions/tune-runner-vm/action.yml b/.github/actions/tune-runner-vm/action.yml index 7e5f77f9a83fe..ab0f65767a62d 100644 --- a/.github/actions/tune-runner-vm/action.yml +++ b/.github/actions/tune-runner-vm/action.yml @@ -77,12 +77,6 @@ runs: # stop Azure Linux agent to save RAM sudo systemctl stop walinuxagent.service || true - # enable docker experimental mode which is - # required for using "docker build --squash" / "-Ddocker.squash=true" - daemon_json="$(sudo cat /etc/docker/daemon.json | jq '.experimental = true')" - echo "$daemon_json" | sudo tee /etc/docker/daemon.json - # restart docker daemon - sudo systemctl restart docker echo '::endgroup::' # show memory diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index db1c175bc6221..22d061ac58094 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -512,8 +512,8 @@ jobs: - name: Build java-test-image docker image run: | # build docker image - mvn -B -am -pl docker/pulsar,tests/docker-images/java-test-image install -Pcore-modules,-main,integrationTests,docker \ - -Dmaven.test.skip=true -Ddocker.squash=true -DskipSourceReleaseAssembly=true \ + DOCKER_CLI_EXPERIMENTAL=enabled mvn -B -am -pl docker/pulsar,tests/docker-images/java-test-image install -Pcore-modules,-main,integrationTests,docker \ + -Dmaven.test.skip=true -DskipSourceReleaseAssembly=true \ -Dspotbugs.skip=true -Dlicense.skip=true -Dcheckstyle.skip=true -Drat.skip=true - name: save docker image apachepulsar/java-test-image:latest to Github artifact cache @@ -868,8 +868,8 @@ jobs: run: | # build docker image # include building of Connectors, Offloaders and server distros - mvn -B -am -pl distribution/io,distribution/offloaders,distribution/server,distribution/shell,tests/docker-images/latest-version-image install \ - -Pmain,docker -Dmaven.test.skip=true -Ddocker.squash=true \ + DOCKER_CLI_EXPERIMENTAL=enabled mvn -B -am -pl distribution/io,distribution/offloaders,distribution/server,distribution/shell,tests/docker-images/latest-version-image install \ + -Pmain,docker -Dmaven.test.skip=true \ -Dspotbugs.skip=true -Dlicense.skip=true -Dcheckstyle.skip=true -Drat.skip=true # check full build artifacts licenses diff --git a/README.md b/README.md index e84265f91a717..3eae0ae29c334 100644 --- a/README.md +++ b/README.md @@ -192,6 +192,10 @@ Check https://pulsar.apache.org for documentation and examples. ## Build custom docker images +The commands used in the Apache Pulsar release process can be found in the [release process documentation](https://pulsar.apache.org/contribute/release-process/#stage-docker-images). + +Here are some general instructions for building custom docker images: + * Docker images must be built with Java 8 for `branch-2.7` or previous branches because of [ISSUE-8445](https://github.com/apache/pulsar/issues/8445). * Java 11 is the recommended JDK version in `branch-2.8`, `branch-2.9` and `branch-2.10`. * Java 17 is the recommended JDK version in `master`. @@ -200,6 +204,8 @@ The following command builds the docker images `apachepulsar/pulsar-all:latest` ```bash mvn clean install -DskipTests +# setting DOCKER_CLI_EXPERIMENTAL=enabled is required in some environments with older docker versions +export DOCKER_CLI_EXPERIMENTAL=enabled mvn package -Pdocker,-main -am -pl docker/pulsar-all -DskipTests ``` diff --git a/build/build_java_test_image.sh b/build/build_java_test_image.sh index 0747e6dacb82a..3869b6688051f 100755 --- a/build/build_java_test_image.sh +++ b/build/build_java_test_image.sh @@ -20,12 +20,6 @@ SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" cd "$SCRIPT_DIR/.." -SQUASH_PARAM="" -# check if docker experimental mode is enabled which is required for -# using "docker build --squash" for squashing all intermediate layers of the build to a single layer -if [[ "$(docker version -f '{{.Server.Experimental}}' 2>/dev/null)" == "true" ]]; then - SQUASH_PARAM="-Ddocker.squash=true" -fi mvn -am -pl tests/docker-images/java-test-image -Pcore-modules,-main,integrationTests,docker \ - -Dmaven.test.skip=true -DskipSourceReleaseAssembly=true -Dspotbugs.skip=true -Dlicense.skip=true $SQUASH_PARAM \ + -Dmaven.test.skip=true -DskipSourceReleaseAssembly=true -Dspotbugs.skip=true -Dlicense.skip=true \ "$@" install \ No newline at end of file From 2469b97b7e4de10fec64cc7ff1f4f46a410ad125 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sun, 7 Apr 2024 09:28:04 +0800 Subject: [PATCH 049/580] [fix][client] Fix client side memory leak when call MessageImpl.create and fix imprecise client-side metrics: pendingMessagesUpDownCounter, pendingBytesUpDownCounter, latencyHistogram (#22393) --- .../api/SimpleProducerConsumerTest.java | 144 ++++++++++++++++++ .../pulsar/client/impl/ProducerImpl.java | 135 ++++++++-------- 2 files changed, 213 insertions(+), 66 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index 4c106d39e7ad7..7552b84a1c553 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -51,6 +51,7 @@ import java.time.Instant; import java.time.ZoneId; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -69,6 +70,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -92,6 +94,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.ClientBuilderImpl; import org.apache.pulsar.client.impl.ConsumerBase; import org.apache.pulsar.client.impl.ConsumerImpl; @@ -99,11 +102,13 @@ import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; import org.apache.pulsar.client.impl.PartitionedProducerImpl; +import org.apache.pulsar.client.impl.ProducerBase; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.TopicMessageImpl; import org.apache.pulsar.client.impl.TypedMessageBuilderImpl; import org.apache.pulsar.client.impl.crypto.MessageCryptoBc; import org.apache.pulsar.client.impl.schema.writer.AvroWriter; +import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.EncryptionContext; import org.apache.pulsar.common.api.EncryptionContext.EncryptionKey; import org.apache.pulsar.common.api.proto.MessageMetadata; @@ -4692,4 +4697,143 @@ public void flush(ChannelHandlerContext ctx) throws Exception { consumer.close(); admin.topics().delete(topic, false); } + + @DataProvider(name = "enableBatchSend") + public Object[][] enableBatchSend() { + return new Object[][]{ + {true}, + {false} + }; + } + + @Test(dataProvider = "enableBatchSend") + public void testPublishWithCreateMessageManually(boolean enableBatchSend) throws Exception { + final int messageCount = 10; + final List messageArrayBeforeSend = Collections.synchronizedList(new ArrayList<>()); + final List messageArrayOnSendAcknowledgement = Collections.synchronizedList(new ArrayList<>()); + // Create an interceptor to verify the ref count of Message.payload is as expected. + AtomicBoolean payloadWasReleasedWhenIntercept = new AtomicBoolean(false); + ProducerInterceptor interceptor = new ProducerInterceptor(){ + + @Override + public void close() { + + } + @Override + public Message beforeSend(Producer producer, Message message) { + MessageImpl msgImpl = (MessageImpl) message; + log.info("payload.refCnf before send: {}", msgImpl.getDataBuffer().refCnt()); + if (msgImpl.getDataBuffer().refCnt() < 1) { + payloadWasReleasedWhenIntercept.set(true); + } + messageArrayBeforeSend.add(msgImpl); + return message; + } + + @Override + public void onSendAcknowledgement(Producer producer, Message message, MessageId msgId, + Throwable exception) { + MessageImpl msgImpl = (MessageImpl) message; + log.info("payload.refCnf on send acknowledgement: {}", msgImpl.getDataBuffer().refCnt()); + if (msgImpl.getDataBuffer().refCnt() < 1) { + payloadWasReleasedWhenIntercept.set(true); + } + messageArrayOnSendAcknowledgement.add(msgImpl); + } + }; + + final String topic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp"); + admin.topics().createNonPartitionedTopic(topic); + ProducerBase producerBase = (ProducerBase) pulsarClient.newProducer().topic(topic).intercept(interceptor) + .enableBatching(enableBatchSend).create(); + + // Publish message. + // Note: "ProducerBase.sendAsync" is not equals to "Producer.sendAsync". + final MessageImpl[] messageArraySent = new MessageImpl[messageCount]; + final ByteBuf[] payloads = new ByteBuf[messageCount]; + List> sendFutureList = new ArrayList<>(); + List releaseFutureList = new ArrayList<>(); + for (int i = 0; i < messageCount; i++) { + // Create message payload, refCnf = 1 now. + ByteBuf payload = PulsarByteBufAllocator.DEFAULT.heapBuffer(1); + payloads[i] = payload; + log.info("payload_{}.refCnf 1st: {}", i, payload.refCnt()); + payload.writeByte(i); + // refCnf = 2 now. + payload.retain(); + log.info("payload_{}.refCnf 2nd: {}", i, payload.refCnt()); + MessageMetadata messageMetadata = new MessageMetadata(); + messageMetadata.setUncompressedSize(1); + MessageImpl message1 = MessageImpl.create(topic, null, messageMetadata, payload, Optional.empty(), + null, Schema.BYTES, 0, true, 0); + messageArraySent[i] = message1; + // Release ByteBuf the first time, refCnf = 1 now. + CompletableFuture future = producerBase.sendAsync(message1); + sendFutureList.add(future); + final int indexForLog = i; + future.whenComplete((v, ex) -> { + message1.release(); + log.info("payload_{}.refCnf 3rd after_complete_refCnf: {}, ex: {}", indexForLog, payload.refCnt(), + ex == null ? "null" : ex.getMessage()); + }); + } + sendFutureList.get(messageCount - 1).join(); + + // Left 2 seconds to wait the code in the finally-block, which is using to avoid this test to be flaky. + Thread.sleep(1000 * 2); + + // Verify: payload's refCnf. + for (int i = 0; i < messageCount; i++) { + log.info("payload_{}.refCnf 4th: {}", i, payloads[i].refCnt()); + assertEquals(payloads[i].refCnt(), 1); + } + + // Verify: the messages has not been released when calling interceptor. + assertFalse(payloadWasReleasedWhenIntercept.get()); + + // Verify: the order of send complete event. + MessageIdImpl messageIdPreviousOne = null; + for (int i = 0; i < messageCount; i++) { + MessageIdImpl messageId = (MessageIdImpl) sendFutureList.get(i).get(); + if (messageIdPreviousOne != null) { + assertTrue(compareMessageIds(messageIdPreviousOne, messageId) > 0); + } + messageIdPreviousOne = messageId; + } + + // Verify: the order of interceptor events. + for (int i = 0; i < messageCount; i++) { + assertTrue(messageArraySent[i] == messageArrayBeforeSend.get(i)); + assertTrue(messageArraySent[i] == messageArrayOnSendAcknowledgement.get(i)); + } + + // cleanup. + for (int i = 0; i < messageCount; i++) { + payloads[i].release(); + } + producerBase.close(); + admin.topics().delete(topic, false); + } + + private int compareMessageIds(MessageIdImpl messageId1, MessageIdImpl messageId2) { + if (messageId2.getLedgerId() < messageId1.getLedgerId()) { + return -1; + } + if (messageId2.getLedgerId() > messageId1.getLedgerId()) { + return 1; + } + if (messageId2.getEntryId() < messageId1.getEntryId()) { + return -1; + } + if (messageId2.getEntryId() > messageId1.getEntryId()) { + return 1; + } + if (messageId2 instanceof BatchMessageIdImpl && messageId1 instanceof BatchMessageIdImpl) { + BatchMessageIdImpl batchMessageId1 = (BatchMessageIdImpl) messageId1; + BatchMessageIdImpl batchMessageId2 = (BatchMessageIdImpl) messageId2; + return batchMessageId2.getBatchIndex() - batchMessageId1.getBatchIndex(); + } else { + return 0; + } + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index dbd3aae426900..b8def7e3042bd 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -378,80 +378,83 @@ CompletableFuture internalSendAsync(Message message) { pendingMessagesUpDownCounter.increment(); pendingBytesUpDownCounter.add(msgSize); - sendAsync(interceptorMessage, new SendCallback() { - SendCallback nextCallback = null; - MessageImpl nextMsg = null; - long createdAt = System.nanoTime(); + sendAsync(interceptorMessage, new DefaultSendMessageCallback(future, interceptorMessage, msgSize)); + return future; + } - @Override - public CompletableFuture getFuture() { - return future; - } + private class DefaultSendMessageCallback implements SendCallback { - @Override - public SendCallback getNextSendCallback() { - return nextCallback; - } + CompletableFuture sendFuture; + MessageImpl currentMsg; + int msgSize; + long createdAt = System.nanoTime(); + SendCallback nextCallback = null; + MessageImpl nextMsg = null; - @Override - public MessageImpl getNextMessage() { - return nextMsg; - } + DefaultSendMessageCallback(CompletableFuture sendFuture, MessageImpl currentMsg, int msgSize) { + this.sendFuture = sendFuture; + this.currentMsg = currentMsg; + this.msgSize = msgSize; + } - @Override - public void sendComplete(Exception e) { - long latencyNanos = System.nanoTime() - createdAt; - pendingMessagesUpDownCounter.decrement(); - pendingBytesUpDownCounter.subtract(msgSize); + @Override + public CompletableFuture getFuture() { + return sendFuture; + } - try { - if (e != null) { - latencyHistogram.recordFailure(latencyNanos); - stats.incrementSendFailed(); - onSendAcknowledgement(interceptorMessage, null, e); - future.completeExceptionally(e); - } else { - latencyHistogram.recordSuccess(latencyNanos); - publishedBytesCounter.add(msgSize); - onSendAcknowledgement(interceptorMessage, interceptorMessage.getMessageId(), null); - future.complete(interceptorMessage.getMessageId()); - stats.incrementNumAcksReceived(latencyNanos); - } - } finally { - interceptorMessage.getDataBuffer().release(); - } + @Override + public SendCallback getNextSendCallback() { + return nextCallback; + } - while (nextCallback != null) { - SendCallback sendCallback = nextCallback; - MessageImpl msg = nextMsg; - // Retain the buffer used by interceptors callback to get message. Buffer will release after - // complete interceptors. - try { - msg.getDataBuffer().retain(); - if (e != null) { - stats.incrementSendFailed(); - onSendAcknowledgement(msg, null, e); - sendCallback.getFuture().completeExceptionally(e); - } else { - onSendAcknowledgement(msg, msg.getMessageId(), null); - sendCallback.getFuture().complete(msg.getMessageId()); - stats.incrementNumAcksReceived(System.nanoTime() - createdAt); - } - nextMsg = nextCallback.getNextMessage(); - nextCallback = nextCallback.getNextSendCallback(); - } finally { - msg.getDataBuffer().release(); - } - } - } + @Override + public MessageImpl getNextMessage() { + return nextMsg; + } - @Override - public void addCallback(MessageImpl msg, SendCallback scb) { - nextMsg = msg; - nextCallback = scb; + @Override + public void sendComplete(Exception e) { + SendCallback loopingCallback = this; + MessageImpl loopingMsg = currentMsg; + while (loopingCallback != null) { + onSendComplete(e, loopingCallback, loopingMsg); + loopingMsg = loopingCallback.getNextMessage(); + loopingCallback = loopingCallback.getNextSendCallback(); + } + } + + private void onSendComplete(Exception e, SendCallback sendCallback, MessageImpl msg) { + long createdAt = (sendCallback instanceof ProducerImpl.DefaultSendMessageCallback) + ? ((DefaultSendMessageCallback) sendCallback).createdAt : this.createdAt; + long latencyNanos = System.nanoTime() - createdAt; + pendingMessagesUpDownCounter.decrement(); + pendingBytesUpDownCounter.subtract(msgSize); + ByteBuf payload = msg.getDataBuffer(); + if (payload == null) { + log.error("[{}] [{}] Payload is null when calling onSendComplete, which is not expected.", + topic, producerName); + } else { + ReferenceCountUtil.safeRelease(payload); } - }); - return future; + if (e != null) { + latencyHistogram.recordFailure(latencyNanos); + stats.incrementSendFailed(); + onSendAcknowledgement(msg, null, e); + sendCallback.getFuture().completeExceptionally(e); + } else { + latencyHistogram.recordSuccess(latencyNanos); + publishedBytesCounter.add(msgSize); + stats.incrementNumAcksReceived(latencyNanos); + onSendAcknowledgement(msg, msg.getMessageId(), null); + sendCallback.getFuture().complete(msg.getMessageId()); + } + } + + @Override + public void addCallback(MessageImpl msg, SendCallback scb) { + nextMsg = msg; + nextCallback = scb; + } } @Override From a51bbdd1480b453d72891b3d68b4afd068ec374a Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 8 Apr 2024 10:34:10 +0800 Subject: [PATCH 050/580] [improve][broker] Deprecate unused enableNamespaceIsolationUpdateOnTime config (#22449) --- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 5 ++--- .../java/org/apache/pulsar/broker/admin/AdminApi2Test.java | 3 ++- 2 files changed, 4 insertions(+), 4 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 80dfcaf4b0b20..324a4c9a8dc01 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 @@ -1453,11 +1453,10 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "Enable or disable exposing broker entry metadata to client.") private boolean exposingBrokerEntryMetadataToClientEnabled = false; + @Deprecated @FieldContext( category = CATEGORY_SERVER, - doc = "Enable namespaceIsolation policy update take effect ontime or not," - + " if set to ture, then the related namespaces will be unloaded after reset policy to make it " - + "take effect." + doc = "This config never takes effect and will be removed in the next release" ) private boolean enableNamespaceIsolationUpdateOnTime = false; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java index 3899338870451..6bc64f4dd65d0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java @@ -199,7 +199,6 @@ protected ServiceConfiguration getDefaultConf() { void configureDefaults(ServiceConfiguration conf) { conf.setForceDeleteNamespaceAllowed(true); conf.setLoadBalancerEnabled(true); - conf.setEnableNamespaceIsolationUpdateOnTime(true); conf.setAllowOverrideEntryFilters(true); conf.setEntryFilterNames(List.of()); conf.setMaxNumPartitionsPerPartitionedTopic(0); @@ -1394,6 +1393,7 @@ public void brokerNamespaceIsolationPoliciesUpdateOnTime() throws Exception { try { admin.lookups().lookupTopic(ns1Name + "/topic3"); + fail(); } catch (Exception e) { // expected lookup fail, because no brokers matched the policy. log.info(" 2 expected fail lookup"); @@ -1401,6 +1401,7 @@ public void brokerNamespaceIsolationPoliciesUpdateOnTime() throws Exception { try { admin.lookups().lookupTopic(ns1Name + "/topic1"); + fail(); } catch (Exception e) { // expected lookup fail, because no brokers matched the policy. log.info(" 22 expected fail lookup"); From 57a616eaa79096af5b49db89c99cd39ccc94ec00 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Mon, 8 Apr 2024 18:22:05 +0800 Subject: [PATCH 051/580] [fix][broker] Fix consumer stops receiving messages when with large backlogs processing (#22454) --- .../mledger/impl/ManagedCursorImpl.java | 5 ++ .../mledger/impl/ManagedLedgerImpl.java | 9 +-- .../persistent/PersistentSubscription.java | 4 +- .../persistent/PersistentTopicTest.java | 56 ++++++++++++++++++- 4 files changed, 65 insertions(+), 9 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 4daa06cad576a..69b130a98c869 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -990,6 +990,11 @@ private void checkForNewEntries(OpReadEntry op, ReadEntriesCallback callback, Ob log.debug("[{}] [{}] Re-trying the read at position {}", ledger.getName(), name, op.readPosition); } + if (isClosed()) { + callback.readEntriesFailed(new CursorAlreadyClosedException("Cursor was already closed"), ctx); + return; + } + if (!hasMoreEntries()) { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Still no entries available. Register for notification", ledger.getName(), diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 3a12cb2ad6c74..698563ed7a1f2 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -1032,6 +1032,7 @@ public synchronized void asyncDeleteCursor(final String consumerName, final Dele + consumerName), ctx); return; } else if (!cursor.isDurable()) { + cursor.setState(ManagedCursorImpl.State.Closed); cursors.removeCursor(consumerName); deactivateCursorByName(consumerName); callback.deleteCursorComplete(ctx); @@ -3814,13 +3815,7 @@ public void removeWaitingCursor(ManagedCursor cursor) { } public void addWaitingCursor(ManagedCursorImpl cursor) { - if (cursor instanceof NonDurableCursorImpl) { - if (cursor.isActive()) { - this.waitingCursors.add(cursor); - } - } else { - this.waitingCursors.add(cursor); - } + this.waitingCursors.add(cursor); } public boolean isCursorActive(ManagedCursor cursor) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 6e8e94baeae23..dbbf92aa76dce 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -308,7 +308,6 @@ public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor if (dispatcher != null && dispatcher.getConsumers().isEmpty()) { deactivateCursor(); - topic.getManagedLedger().removeWaitingCursor(cursor); if (!cursor.isDurable()) { // If cursor is not durable, we need to clean up the subscription as well. No need to check for active @@ -338,11 +337,14 @@ public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor if (!isResetCursor) { try { topic.getManagedLedger().deleteCursor(cursor.getName()); + topic.getManagedLedger().removeWaitingCursor(cursor); } catch (InterruptedException | ManagedLedgerException e) { log.warn("[{}] [{}] Failed to remove non durable cursor", topic.getName(), subName, e); } } }); + } else { + topic.getManagedLedger().removeWaitingCursor(cursor); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index d42b1d92007aa..c214634e6ed32 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -80,6 +80,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.NamespaceBundle; @@ -113,6 +114,11 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + @Override protected void doInitConf() throws Exception { + super.doInitConf(); + this.conf.setManagedLedgerCursorBackloggedThreshold(10); + } + /** * Test validates that broker cleans up topic which failed to unload while bundle unloading. * @@ -681,7 +687,7 @@ public void testAddWaitingCursorsForNonDurable() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl)persistentTopic.getManagedLedger(); final ManagedCursor spyCursor= spy(ledger.newNonDurableCursor(PositionImpl.LATEST, "sub-2")); doAnswer((invocation) -> { - Thread.sleep(10_000); + Thread.sleep(5_000); invocation.callRealMethod(); return null; }).when(spyCursor).asyncReadEntriesOrWait(any(int.class), any(long.class), @@ -708,4 +714,52 @@ public void testAddWaitingCursorsForNonDurable() throws Exception { assertEquals(ledger.getWaitingCursorsCount(), 0); }); } + + @Test + public void testAddWaitingCursorsForNonDurable2() throws Exception { + final String ns = "prop/ns-test"; + admin.namespaces().createNamespace(ns, 2); + final String topicName = "persistent://prop/ns-test/testAddWaitingCursors2"; + admin.topics().createNonPartitionedTopic(topicName); + pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionMode(SubscriptionMode.Durable) + .subscriptionType(SubscriptionType.Shared) + .subscriptionName("sub-1").subscribe().close(); + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.STRING).enableBatching(false).topic(topicName).create(); + for (int i = 0; i < 100; i ++) { + producer.sendAsync("test-" + i); + } + @Cleanup + final Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionMode(SubscriptionMode.NonDurable) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionType(SubscriptionType.Exclusive) + .subscriptionName("sub-2").subscribe(); + int count = 0; + while(true) { + final Message msg = consumer.receive(3, TimeUnit.SECONDS); + if (msg != null) { + consumer.acknowledge(msg); + count++; + } else { + break; + } + } + Assert.assertEquals(count, 100); + Thread.sleep(3_000); + for (int i = 0; i < 100; i ++) { + producer.sendAsync("test-" + i); + } + while(true) { + final Message msg = consumer.receive(5, TimeUnit.SECONDS); + if (msg != null) { + consumer.acknowledge(msg); + count++; + } else { + break; + } + } + Assert.assertEquals(count, 200); + } } From b162d46906961623db09c97df3f06b7876cddb5e Mon Sep 17 00:00:00 2001 From: thetumbled <52550727+thetumbled@users.noreply.github.com> Date: Tue, 9 Apr 2024 13:14:25 +0800 Subject: [PATCH 052/580] [cleanup][broker] remove useless code comment (#22459) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 1650e449a3fd6..b21cd165402e4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -3875,9 +3875,6 @@ private void unfenceTopicToResume() { @Override public void publishTxnMessage(TxnID txnID, ByteBuf headersAndPayload, PublishContext publishContext) { pendingWriteOps.incrementAndGet(); - // in order to avoid the opAddEntry retain - - // in order to promise the publish txn message orderly, we should change the transactionCompletableFuture if (isFenced) { publishContext.completed(new TopicFencedException("fenced"), -1, -1); From 3bdb30c7a6151ec97c25865fbbd0bb24613ab991 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 9 Apr 2024 14:59:13 +0800 Subject: [PATCH 053/580] [fix][build] Fix ps command (#22451) Signed-off-by: Zixuan Liu --- docker/pulsar/Dockerfile | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 1ca6edb2e323c..f586a9dd4f9d7 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -97,7 +97,8 @@ RUN apk add --no-cache \ python3 \ py3-pip \ gcompat \ - ca-certificates + ca-certificates \ + procps # Install GLibc compatibility library COPY --from=glibc /root/packages /root/packages @@ -106,6 +107,9 @@ RUN apk add --allow-untrusted --force-overwrite /root/packages/glibc-*.apk COPY --from=jvm /opt/jvm /opt/jvm ENV JAVA_HOME=/opt/jvm +# The default is /pulsat/bin and cannot be written. +ENV PULSAR_PID_DIR=/pulsar/logs + # Copy Python depedencies from the other stage COPY --from=python-deps /usr/lib/python3.11/site-packages /usr/lib/python3.11/site-packages From 6de711d4008338a875c5c145e856c90dcb041f8f Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Tue, 9 Apr 2024 16:38:18 +0800 Subject: [PATCH 054/580] [improve][test] Add operation authentication test for namespace API (#22398) --- .../broker/admin/NamespaceAuthZTest.java | 882 +++++++++++++++++- 1 file changed, 875 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java index ce0b925614c55..d5a0468f340c9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java @@ -19,23 +19,47 @@ package org.apache.pulsar.broker.admin; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import io.jsonwebtoken.Jwts; +import java.io.File; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; import lombok.Cleanup; import lombok.SneakyThrows; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.BookieAffinityGroupData; +import org.apache.pulsar.common.policies.data.BundlesData; +import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.packages.management.core.MockedPackagesStorageProvider; +import org.apache.pulsar.packages.management.core.common.PackageMetadata; import org.apache.pulsar.security.MockedPulsarStandalone; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.UUID; @Test(groups = "broker-admin") public class NamespaceAuthZTest extends MockedPulsarStandalone { @@ -44,17 +68,27 @@ public class NamespaceAuthZTest extends MockedPulsarStandalone { private PulsarAdmin tenantManagerAdmin; + private PulsarClient pulsarClient; + + private AuthorizationService authorizationService; + + private AuthorizationService orignalAuthorizationService; + private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); private static final String TENANT_ADMIN_TOKEN = Jwts.builder() .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); @SneakyThrows @BeforeClass - public void before() { + public void setup() { + getServiceConfiguration().setEnablePackagesManagement(true); + getServiceConfiguration().setPackagesManagementStorageProvider(MockedPackagesStorageProvider.class.getName()); + getServiceConfiguration().setDefaultNumberOfNamespaceBundles(1); + getServiceConfiguration().setForceDeleteNamespaceAllowed(true); configureTokenAuthentication(); configureDefaultAuthorization(); start(); - this.superUserAdmin =PulsarAdmin.builder() + this.superUserAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) .build(); @@ -65,12 +99,13 @@ public void before() { .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) .build(); + this.pulsarClient = super.getPulsarService().getClient(); } @SneakyThrows @AfterClass - public void after() { + public void cleanup() { if (superUserAdmin != null) { superUserAdmin.close(); } @@ -80,6 +115,33 @@ public void after() { close(); } + @BeforeMethod + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + authorizationService = Mockito.spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + authorizationService, true); + } + + @AfterMethod + public void after() throws IllegalAccessException, PulsarAdminException { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + orignalAuthorizationService, true); + superUserAdmin.namespaces().deleteNamespace("public/default", true); + superUserAdmin.namespaces().createNamespace("public/default"); + } + + private void setAuthorizationOperationChecker(String role, NamespaceOperation operation) { + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(2); + if (role.equals(role_)) { + NamespaceOperation operation_ = invocationOnMock.getArgument(1); + Assert.assertEquals(operation_, operation); + } + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowNamespaceOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any()); + } @SneakyThrows @Test @@ -160,4 +222,810 @@ public void testProperties() { } superUserAdmin.topics().delete(topic, true); } + + @Test + public void testTopics() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + // test super admin + superUserAdmin.namespaces().getTopics(namespace); + + // test tenant manager + tenantManagerAdmin.namespaces().getTopics(namespace); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getTopics(namespace)); + + setAuthorizationOperationChecker(subject, NamespaceOperation.GET_TOPICS); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + if (AuthAction.consume == action || AuthAction.produce == action) { + subAdmin.namespaces().getTopics(namespace); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getTopics(namespace)); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testBookieAffinityGroup() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + // test super admin + BookieAffinityGroupData bookieAffinityGroupData = BookieAffinityGroupData.builder() + .bookkeeperAffinityGroupPrimary("aaa") + .bookkeeperAffinityGroupSecondary("bbb") + .build(); + superUserAdmin.namespaces().setBookieAffinityGroup(namespace, bookieAffinityGroupData); + BookieAffinityGroupData bookieAffinityGroup = superUserAdmin.namespaces().getBookieAffinityGroup(namespace); + Assert.assertEquals(bookieAffinityGroupData, bookieAffinityGroup); + superUserAdmin.namespaces().deleteBookieAffinityGroup(namespace); + bookieAffinityGroup = superUserAdmin.namespaces().getBookieAffinityGroup(namespace); + Assert.assertNull(bookieAffinityGroup); + + // test tenant manager + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> tenantManagerAdmin.namespaces().setBookieAffinityGroup(namespace, bookieAffinityGroupData)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> tenantManagerAdmin.namespaces().getBookieAffinityGroup(namespace)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> tenantManagerAdmin.namespaces().deleteBookieAffinityGroup(namespace)); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setBookieAffinityGroup(namespace, bookieAffinityGroupData)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getBookieAffinityGroup(namespace)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().deleteBookieAffinityGroup(namespace)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setBookieAffinityGroup(namespace, bookieAffinityGroupData)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getBookieAffinityGroup(namespace)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().deleteBookieAffinityGroup(namespace)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + + @Test + public void testGetBundles() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + producer.send("message".getBytes()); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test super admin + superUserAdmin.namespaces().getBundles(namespace); + + // test tenant manager + tenantManagerAdmin.namespaces().getBundles(namespace); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getBundles(namespace)); + + setAuthorizationOperationChecker(subject, NamespaceOperation.GET_BUNDLE); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + if (AuthAction.consume == action || AuthAction.produce == action) { + subAdmin.namespaces().getBundles(namespace); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getBundles(namespace)); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testUnloadBundles() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + producer.send("message".getBytes()); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + final String defaultBundle = "0x00000000_0xffffffff"; + + // test super admin + superUserAdmin.namespaces().unloadNamespaceBundle(namespace, defaultBundle); + + // test tenant manager + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> tenantManagerAdmin.namespaces().unloadNamespaceBundle(namespace, defaultBundle)); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().unloadNamespaceBundle(namespace, defaultBundle)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().unloadNamespaceBundle(namespace, defaultBundle)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testSplitBundles() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + producer.send("message".getBytes()); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + final String defaultBundle = "0x00000000_0xffffffff"; + + // test super admin + superUserAdmin.namespaces().splitNamespaceBundle(namespace, defaultBundle, false, null); + + // test tenant manager + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> tenantManagerAdmin.namespaces().splitNamespaceBundle(namespace, defaultBundle, false, null)); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().splitNamespaceBundle(namespace, defaultBundle, false, null)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().splitNamespaceBundle(namespace, defaultBundle, false, null)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testDeleteBundles() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + producer.send("message".getBytes()); + + for (int i = 0; i < 3; i++) { + superUserAdmin.namespaces().splitNamespaceBundle(namespace, Policies.BundleType.LARGEST.toString(), false, null); + } + + BundlesData bundles = superUserAdmin.namespaces().getBundles(namespace); + Assert.assertEquals(bundles.getNumBundles(), 4); + List boundaries = bundles.getBoundaries(); + Assert.assertEquals(boundaries.size(), 5); + + List bundleRanges = new ArrayList<>(); + for (int i = 0; i < boundaries.size() - 1; i++) { + String bundleRange = boundaries.get(i) + "_" + boundaries.get(i + 1); + List allTopicsFromNamespaceBundle = getPulsarService().getBrokerService() + .getAllTopicsFromNamespaceBundle(namespace, namespace + "/" + bundleRange); + System.out.println(StringUtils.join(allTopicsFromNamespaceBundle)); + if (allTopicsFromNamespaceBundle.isEmpty()) { + bundleRanges.add(bundleRange); + } + } + + // test super admin + superUserAdmin.namespaces().deleteNamespaceBundle(namespace, bundleRanges.get(0)); + + // test tenant manager + tenantManagerAdmin.namespaces().deleteNamespaceBundle(namespace, bundleRanges.get(1)); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().deleteNamespaceBundle(namespace, bundleRanges.get(1))); + + setAuthorizationOperationChecker(subject, NamespaceOperation.DELETE_BUNDLE); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().deleteNamespaceBundle(namespace, bundleRanges.get(1))); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + } + + @Test + public void testPermission() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + final String role = "sub"; + final AuthAction testAction = AuthAction.consume; + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + + // test super admin + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, role, Set.of(testAction)); + Map> permissions = superUserAdmin.namespaces().getPermissions(namespace); + Assert.assertEquals(permissions.get(role), Set.of(testAction)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, role); + permissions = superUserAdmin.namespaces().getPermissions(namespace); + Assert.assertTrue(permissions.isEmpty()); + + // test tenant manager + tenantManagerAdmin.namespaces().grantPermissionOnNamespace(namespace, role, Set.of(testAction)); + permissions = tenantManagerAdmin.namespaces().getPermissions(namespace); + Assert.assertEquals(permissions.get(role), Set.of(testAction)); + tenantManagerAdmin.namespaces().revokePermissionsOnNamespace(namespace, role); + permissions = tenantManagerAdmin.namespaces().getPermissions(namespace); + Assert.assertTrue(permissions.isEmpty()); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().grantPermissionOnNamespace(namespace, role, Set.of(testAction))); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getPermissions(namespace)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().revokePermissionsOnNamespace(namespace, role)); + + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + setAuthorizationOperationChecker(subject, NamespaceOperation.GRANT_PERMISSION); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().grantPermissionOnNamespace(namespace, role, Set.of(testAction))); + setAuthorizationOperationChecker(subject, NamespaceOperation.GET_PERMISSION); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getPermissions(namespace)); + setAuthorizationOperationChecker(subject, NamespaceOperation.REVOKE_PERMISSION); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().revokePermissionsOnNamespace(namespace, role)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testPermissionOnSubscription() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + final String subscription = "my-sub"; + final String role = "sub"; + pulsarClient.newConsumer().topic(topic) + .subscriptionName(subscription) + .subscribe().close(); + + + // test super admin + superUserAdmin.namespaces().grantPermissionOnSubscription(namespace, subscription, Set.of(role)); + Map> permissionOnSubscription = superUserAdmin.namespaces().getPermissionOnSubscription(namespace); + Assert.assertEquals(permissionOnSubscription.get(subscription), Set.of(role)); + superUserAdmin.namespaces().revokePermissionOnSubscription(namespace, subscription, role); + permissionOnSubscription = superUserAdmin.namespaces().getPermissionOnSubscription(namespace); + Assert.assertTrue(permissionOnSubscription.isEmpty()); + + // test tenant manager + tenantManagerAdmin.namespaces().grantPermissionOnSubscription(namespace, subscription, Set.of(role)); + permissionOnSubscription = tenantManagerAdmin.namespaces().getPermissionOnSubscription(namespace); + Assert.assertEquals(permissionOnSubscription.get(subscription), Set.of(role)); + tenantManagerAdmin.namespaces().revokePermissionOnSubscription(namespace, subscription, role); + permissionOnSubscription = tenantManagerAdmin.namespaces().getPermissionOnSubscription(namespace); + Assert.assertTrue(permissionOnSubscription.isEmpty()); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().grantPermissionOnSubscription(namespace, subscription, Set.of(role))); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getPermissionOnSubscription(namespace)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().revokePermissionOnSubscription(namespace, subscription, role)); + + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + setAuthorizationOperationChecker(subject, NamespaceOperation.GRANT_PERMISSION); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().grantPermissionOnSubscription(namespace, subscription, Set.of(role))); + setAuthorizationOperationChecker(subject, NamespaceOperation.GET_PERMISSION); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getPermissionOnSubscription(namespace)); + setAuthorizationOperationChecker(subject, NamespaceOperation.REVOKE_PERMISSION); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().revokePermissionOnSubscription(namespace, subscription, role)); + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testClearBacklog() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test super admin + superUserAdmin.namespaces().clearNamespaceBacklog(namespace); + + // test tenant manager + tenantManagerAdmin.namespaces().clearNamespaceBacklog(namespace); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().clearNamespaceBacklog(namespace)); + + setAuthorizationOperationChecker(subject, NamespaceOperation.CLEAR_BACKLOG); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.namespaces().clearNamespaceBacklog(namespace); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().clearNamespaceBacklog(namespace)); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testClearNamespaceBundleBacklog() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + @Cleanup + Producer batchProducer = pulsarClient.newProducer().topic(topic) + .enableBatching(false) + .create(); + + final String defaultBundle = "0x00000000_0xffffffff"; + + // test super admin + superUserAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle); + + // test tenant manager + tenantManagerAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle)); + + setAuthorizationOperationChecker(subject, NamespaceOperation.CLEAR_BACKLOG); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle)); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testUnsubscribeNamespace() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + @Cleanup + Producer batchProducer = pulsarClient.newProducer().topic(topic) + .enableBatching(false) + .create(); + + pulsarClient.newConsumer().topic(topic) + .subscriptionName("sub") + .subscribe().close(); + + // test super admin + superUserAdmin.namespaces().unsubscribeNamespace(namespace, "sub"); + + // test tenant manager + tenantManagerAdmin.namespaces().unsubscribeNamespace(namespace, "sub"); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().unsubscribeNamespace(namespace, "sub")); + + setAuthorizationOperationChecker(subject, NamespaceOperation.UNSUBSCRIBE); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.namespaces().unsubscribeNamespace(namespace, "sub"); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().unsubscribeNamespace(namespace, "sub")); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testUnsubscribeNamespaceBundle() throws Exception { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + @Cleanup + Producer batchProducer = pulsarClient.newProducer().topic(topic) + .enableBatching(false) + .create(); + + pulsarClient.newConsumer().topic(topic) + .subscriptionName("sub") + .subscribe().close(); + + final String defaultBundle = "0x00000000_0xffffffff"; + + // test super admin + superUserAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub"); + + // test tenant manager + tenantManagerAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub"); + + // test nobody + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub")); + + setAuthorizationOperationChecker(subject, NamespaceOperation.UNSUBSCRIBE); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + if (AuthAction.consume == action) { + subAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub"); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub")); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + + superUserAdmin.topics().delete(topic, true); + } + + @Test + public void testPackageAPI() throws Exception { + final String namespace = "public/default"; + + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + + File file = File.createTempFile("package-api-test", ".package"); + + // testing upload api + String packageName = "function://public/default/test@v1"; + PackageMetadata originalMetadata = PackageMetadata.builder().description("test").build(); + superUserAdmin.packages().upload(originalMetadata, packageName, file.getPath()); + + // testing download api + String downloadPath = new File(file.getParentFile(), "package-api-test-download.package").getPath(); + superUserAdmin.packages().download(packageName, downloadPath); + File downloadFile = new File(downloadPath); + assertTrue(downloadFile.exists()); + downloadFile.delete(); + + // testing list packages api + List packages = superUserAdmin.packages().listPackages("function", "public/default"); + assertEquals(packages.size(), 1); + assertEquals(packages.get(0), "test"); + + // testing list versions api + List versions = superUserAdmin.packages().listPackageVersions(packageName); + assertEquals(versions.size(), 1); + assertEquals(versions.get(0), "v1"); + + // testing get packages api + PackageMetadata metadata = superUserAdmin.packages().getMetadata(packageName); + assertEquals(metadata.getDescription(), originalMetadata.getDescription()); + assertNull(metadata.getContact()); + assertTrue(metadata.getModificationTime() > 0); + assertTrue(metadata.getCreateTime() > 0); + assertNull(metadata.getProperties()); + + // testing update package metadata api + PackageMetadata updatedMetadata = originalMetadata; + updatedMetadata.setContact("test@apache.org"); + updatedMetadata.setProperties(Collections.singletonMap("key", "value")); + superUserAdmin.packages().updateMetadata(packageName, updatedMetadata); + + superUserAdmin.packages().getMetadata(packageName); + + // ---- test tenant manager --- + + file = File.createTempFile("package-api-test", ".package"); + + // test tenant manager + packageName = "function://public/default/test@v2"; + originalMetadata = PackageMetadata.builder().description("test").build(); + tenantManagerAdmin.packages().upload(originalMetadata, packageName, file.getPath()); + + // testing download api + downloadPath = new File(file.getParentFile(), "package-api-test-download.package").getPath(); + tenantManagerAdmin.packages().download(packageName, downloadPath); + downloadFile = new File(downloadPath); + assertTrue(downloadFile.exists()); + downloadFile.delete(); + + // testing list packages api + packages = tenantManagerAdmin.packages().listPackages("function", "public/default"); + assertEquals(packages.size(), 1); + assertEquals(packages.get(0), "test"); + + // testing list versions api + tenantManagerAdmin.packages().listPackageVersions(packageName); + + // testing get packages api + tenantManagerAdmin.packages().getMetadata(packageName); + + // testing update package metadata api + updatedMetadata = originalMetadata; + updatedMetadata.setContact("test@apache.org"); + updatedMetadata.setProperties(Collections.singletonMap("key", "value")); + tenantManagerAdmin.packages().updateMetadata(packageName, updatedMetadata); + + // ---- test nobody --- + + File file3 = File.createTempFile("package-api-test", ".package"); + + // test tenant manager + String packageName3 = "function://public/default/test@v3"; + PackageMetadata originalMetadata3 = PackageMetadata.builder().description("test").build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().upload(originalMetadata3, packageName3, file3.getPath())); + + + // testing download api + String downloadPath3 = new File(file3.getParentFile(), "package-api-test-download.package").getPath(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().download(packageName3, downloadPath3)); + + // testing list packages api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().listPackages("function", "public/default")); + + // testing list versions api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().listPackageVersions(packageName3)); + + // testing get packages api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().getMetadata(packageName3)); + + // testing update package metadata api + PackageMetadata updatedMetadata3 = originalMetadata; + updatedMetadata3.setContact("test@apache.org"); + updatedMetadata3.setProperties(Collections.singletonMap("key", "value")); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().updateMetadata(packageName3, updatedMetadata3)); + + + setAuthorizationOperationChecker(subject, NamespaceOperation.PACKAGES); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); + File file4 = File.createTempFile("package-api-test", ".package"); + String packageName4 = "function://public/default/test@v4"; + PackageMetadata originalMetadata4 = PackageMetadata.builder().description("test").build(); + String downloadPath4 = new File(file3.getParentFile(), "package-api-test-download.package").getPath(); + if (AuthAction.packages == action) { + subAdmin.packages().upload(originalMetadata4, packageName4, file.getPath()); + + // testing download api + subAdmin.packages().download(packageName4, downloadPath4); + downloadFile = new File(downloadPath4); + assertTrue(downloadFile.exists()); + downloadFile.delete(); + + // testing list packages api + packages = subAdmin.packages().listPackages("function", "public/default"); + assertEquals(packages.size(), 1); + assertEquals(packages.get(0), "test"); + + // testing list versions api + subAdmin.packages().listPackageVersions(packageName4); + + // testing get packages api + subAdmin.packages().getMetadata(packageName4); + + // testing update package metadata api + PackageMetadata updatedMetadata4 = originalMetadata; + updatedMetadata4.setContact("test@apache.org"); + updatedMetadata4.setProperties(Collections.singletonMap("key", "value")); + subAdmin.packages().updateMetadata(packageName, updatedMetadata4); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().upload(originalMetadata4, packageName4, file4.getPath())); + + // testing download api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().download(packageName4, downloadPath4)); + + // testing list packages api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().listPackages("function", "public/default")); + + // testing list versions api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().listPackageVersions(packageName4)); + + // testing get packages api + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().getMetadata(packageName4)); + + // testing update package metadata api + PackageMetadata updatedMetadata4 = originalMetadata; + updatedMetadata4.setContact("test@apache.org"); + updatedMetadata4.setProperties(Collections.singletonMap("key", "value")); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.packages().updateMetadata(packageName4, updatedMetadata4)); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); + } + } } From 9555504ee2c7adf9febddc585a699a1fdb724013 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Tue, 9 Apr 2024 22:43:35 +0800 Subject: [PATCH 055/580] [improve][admin] Add authorization test for schema and align auth for transaction (#22399) --- .../pulsar/broker/admin/TopicAuthZTest.java | 249 ++++++++++++++++++ .../security/MockedPulsarStandalone.java | 4 +- 2 files changed, 252 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index 2e75b59ec8582..d09bc0a3ffde1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -20,19 +20,27 @@ package org.apache.pulsar.broker.admin; import io.jsonwebtoken.Jwts; +import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.SneakyThrows; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.security.MockedPulsarStandalone; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -62,7 +70,9 @@ public class TopicAuthZTest extends MockedPulsarStandalone { public void before() { configureTokenAuthentication(); configureDefaultAuthorization(); + enableTransaction(); start(); + createTransactionCoordinatorAssign(16); this.superUserAdmin =PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) @@ -74,8 +84,18 @@ public void before() { .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) .build(); + + superUserAdmin.tenants().createTenant("pulsar", tenantInfo); + superUserAdmin.namespaces().createNamespace("pulsar/system"); } + protected void createTransactionCoordinatorAssign(int numPartitionsOfTC) throws MetadataStoreException { + getPulsarService().getPulsarResources() + .getNamespaceResources() + .getPartitionedTopicResources() + .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, + new PartitionedTopicMetadata(numPartitionsOfTC)); + } @SneakyThrows @AfterClass(alwaysRun = true) @@ -1086,6 +1106,235 @@ public void testExpireMessageByPosition() { deleteTopic(topic, false); } + public enum OperationAuthType { + Lookup, + Produce, + Consume, + AdminOrSuperUser, + NOAuth + } + + private final String testTopic = "persistent://public/default/" + UUID.randomUUID().toString(); + @FunctionalInterface + public interface ThrowingBiConsumer { + void accept(T t) throws PulsarAdminException; + } + + @DataProvider(name = "authFunction") + public Object[][] authFunction () throws Exception { + String sub = "my-sub"; + createTopic(testTopic, false); + @Cleanup final PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(getPulsarService().getBrokerServiceUrl()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .enableTransaction(true) + .build(); + @Cleanup final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(testTopic).create(); + + @Cleanup final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(testTopic) + .subscriptionName(sub) + .subscribe(); + + Transaction transaction = pulsarClient.newTransaction().withTransactionTimeout(5, TimeUnit.MINUTES) + .build().get(); + MessageIdImpl messageId = (MessageIdImpl) producer.newMessage().value("test message").send(); + + consumer.acknowledgeAsync(messageId, transaction).get(); + + return new Object[][]{ + // SCHEMA + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo(testTopic), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo( + testTopic, 0), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getAllSchemas( + testTopic), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().createSchema(testTopic, + SchemaInfo.builder().type(SchemaType.STRING).build()), + OperationAuthType.Produce + }, + // TODO: improve the authorization check for testCompatibility and deleteSchema + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().testCompatibility( + testTopic, SchemaInfo.builder().type(SchemaType.STRING).build()), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().deleteSchema( + testTopic), + OperationAuthType.AdminOrSuperUser + }, + + // TRANSACTION + + // Modify transaction coordinator + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .abortTransaction(transaction.getTxnID()), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .scaleTransactionCoordinators(17), + OperationAuthType.AdminOrSuperUser + }, + // TODO: fix authorization check of check transaction coordinator stats. + // Check transaction coordinator stats + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getCoordinatorInternalStats(1, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getCoordinatorStats(), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getSlowTransactionsByCoordinatorId(1, 5, TimeUnit.SECONDS), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionMetadata(transaction.getTxnID()), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .listTransactionCoordinators(), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getSlowTransactions(5, TimeUnit.SECONDS), + OperationAuthType.AdminOrSuperUser + }, + + // TODO: Check the authorization of the topic when get stats of TB or TP + // Check stats related to transaction buffer and transaction pending ack + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPendingAckInternalStats(testTopic, sub, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPendingAckStats(testTopic, sub, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPositionStatsInPendingAck(testTopic, sub, messageId.getLedgerId(), + messageId.getEntryId(), null), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferInternalStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInBufferStats(transaction.getTxnID(), testTopic), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInBufferStats(transaction.getTxnID(), testTopic), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInPendingAckStats(transaction.getTxnID(), testTopic, sub), + OperationAuthType.NOAuth + }, + }; + } + + @Test(dataProvider = "authFunction") + public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer adminConsumer, OperationAuthType topicOpType) + throws Exception { + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test tenant manager + if (topicOpType != OperationAuthType.AdminOrSuperUser) { + adminConsumer.accept(tenantManagerAdmin); + } + + if (topicOpType != OperationAuthType.NOAuth) { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> adminConsumer.accept(subAdmin)); + } + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(testTopic, subject, Set.of(action)); + + if (authActionMatchOperation(topicOpType, action)) { + adminConsumer.accept(subAdmin); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> adminConsumer.accept(subAdmin)); + } + superUserAdmin.topics().revokePermissions(testTopic, subject); + } + } + + + private boolean authActionMatchOperation(OperationAuthType operationAuthType, AuthAction action) { + switch (operationAuthType) { + case Lookup -> { + if (AuthAction.consume == action || AuthAction.produce == action) { + return true; + } + } + case Consume -> { + if (AuthAction.consume == action) { + return true; + } + } + case Produce -> { + if (AuthAction.produce == action) { + return true; + } + } + case AdminOrSuperUser -> { + return false; + } + case NOAuth -> { + return true; + } + } + return false; + } + private void createTopic(String topic, boolean partitioned) throws Exception { if (partitioned) { superUserAdmin.topics().createPartitionedTopic(topic, 2); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java b/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java index 421727c0ed7f4..b82f3b584065d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java @@ -105,7 +105,9 @@ protected void configureTokenAuthentication() { } - + protected void enableTransaction() { + serviceConfiguration.setTransactionCoordinatorEnabled(true); + } protected void configureDefaultAuthorization() { serviceConfiguration.setAuthorizationEnabled(true); From f3d14a6b0b15f6d3c17509b21b28a586a22e5d89 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 9 Apr 2024 07:48:57 -0700 Subject: [PATCH 056/580] [improve][test] Replace usage of curl in Java test and fix stream leaks (#22463) --- .../pulsar/broker/web/WebServiceTest.java | 69 +++++++++---------- 1 file changed, 33 insertions(+), 36 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index 72437fe33743e..d2b59ed0e4997 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -23,18 +23,17 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; - import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import com.google.common.io.CharStreams; import com.google.common.io.Closeables; import io.netty.handler.ssl.util.InsecureTrustManagerFactory; - import java.io.BufferedReader; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.net.HttpURLConnection; import java.net.URL; import java.security.KeyStore; import java.security.PrivateKey; @@ -361,68 +360,66 @@ public void testBrokerReady() throws Exception { @Test public void testCompressOutputMetricsInPrometheus() throws Exception { - setupEnv(true, false, false, false, -1, false); String metricsUrl = pulsar.getWebServiceAddress() + "/metrics/"; - String[] command = {"curl", "-H", "Accept-Encoding: gzip", metricsUrl}; + URL url = new URL(metricsUrl); + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod("GET"); + connection.setRequestProperty("Accept-Encoding", "gzip"); - ProcessBuilder processBuilder = new ProcessBuilder(command); - Process process = processBuilder.start(); - - InputStream inputStream = process.getInputStream(); - - try { - GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream); + StringBuilder content = new StringBuilder(); - // Process the decompressed content - StringBuilder content = new StringBuilder(); - int data; - while ((data = gzipInputStream.read()) != -1) { - content.append((char) data); + try (InputStream inputStream = connection.getInputStream()) { + try (GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream)) { + // Process the decompressed content + int data; + while ((data = gzipInputStream.read()) != -1) { + content.append((char) data); + } } - log.info("Response Content: {}", content); - process.waitFor(); + log.info("Response Content: {}", content); assertTrue(content.toString().contains("process_cpu_seconds_total")); } catch (IOException e) { log.error("Failed to decompress the content, likely the content is not compressed ", e); fail(); + } finally { + connection.disconnect(); } } @Test public void testUnCompressOutputMetricsInPrometheus() throws Exception { - setupEnv(true, false, false, false, -1, false); String metricsUrl = pulsar.getWebServiceAddress() + "/metrics/"; - String[] command = {"curl", metricsUrl}; + URL url = new URL(metricsUrl); + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod("GET"); - ProcessBuilder processBuilder = new ProcessBuilder(command); - Process process = processBuilder.start(); + StringBuilder content = new StringBuilder(); - InputStream inputStream = process.getInputStream(); - try { - GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream); - fail(); - } catch (IOException e) { - log.error("Failed to decompress the content, likely the content is not compressed ", e); - assertTrue(e instanceof ZipException); - } + try (InputStream inputStream = connection.getInputStream()) { + try (GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream)) { + fail(); + } catch (IOException e) { + assertTrue(e instanceof ZipException); + } - BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream)); - StringBuilder content = new StringBuilder(); - String line; - while ((line = reader.readLine()) != null) { - content.append(line + "\n"); + BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream)); + String line; + while ((line = reader.readLine()) != null) { + content.append(line + "\n"); + } + } finally { + connection.disconnect(); } log.info("Response Content: {}", content); - process.waitFor(); assertTrue(content.toString().contains("process_cpu_seconds_total")); } From fb5caeb2cd3353db0499e32e9ec79390741b809c Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 10 Apr 2024 04:27:22 +0800 Subject: [PATCH 057/580] [fix][broker] Update topic partition failed when config maxNumPartitionsPerPartitionedTopic<0 (#22397) --- .../pulsar/broker/ServiceConfiguration.java | 3 +- .../admin/impl/PersistentTopicsBase.java | 2 +- .../broker/admin/PersistentTopicsTest.java | 45 +++++++++++++++++++ 3 files changed, 48 insertions(+), 2 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 324a4c9a8dc01..a7deda752fdde 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 @@ -1344,7 +1344,8 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece category = CATEGORY_SERVER, dynamic = true, doc = "The number of partitions per partitioned topic.\n" - + "If try to create or update partitioned topics by exceeded number of partitions, then fail." + + "If try to create or update partitioned topics by exceeded number of partitions, then fail.\n" + + "Use 0 or negative number to disable the check." ) private int maxNumPartitionsPerPartitionedTopic = 0; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index c9c29271b6afe..ab74b1e2bcc0e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -348,7 +348,7 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean } int brokerMaximumPartitionsPerTopic = pulsarService.getConfiguration() .getMaxNumPartitionsPerPartitionedTopic(); - if (brokerMaximumPartitionsPerTopic != 0 && expectPartitions > brokerMaximumPartitionsPerTopic) { + if (brokerMaximumPartitionsPerTopic > 0 && expectPartitions > brokerMaximumPartitionsPerTopic) { throw new RestException(422 /* Unprocessable entity*/, String.format("Desired partitions %s can't be greater than the maximum partitions per" + " topic %s.", diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index 8e1375303ce4c..c588051a0feff 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -1742,6 +1742,51 @@ public void testUpdatePartitionedTopic() verify(response, timeout(5000).times(1)).resume(metaCaptor.capture()); partitionedTopicMetadata = metaCaptor.getValue(); Assert.assertEquals(partitionedTopicMetadata.partitions, 4); + + // test for configuration maxNumPartitionsPerPartitionedTopic + conf.setMaxNumPartitionsPerPartitionedTopic(4); + response = mock(AsyncResponse.class); + throwableCaptor = ArgumentCaptor.forClass(Throwable.class); + persistentTopics.updatePartitionedTopic(response, testTenant, testNamespaceLocal, topicName, false, true, + true, 5); + verify(response, timeout(5000).times(1)).resume(throwableCaptor.capture()); + Assert.assertEquals(throwableCaptor.getValue().getMessage(), + "Desired partitions 5 can't be greater than the maximum partitions per topic 4."); + + response = mock(AsyncResponse.class); + metaCaptor = ArgumentCaptor.forClass(PartitionedTopicMetadata.class); + persistentTopics.getPartitionedMetadata(response, testTenant, testNamespaceLocal, topicName, true, false); + verify(response, timeout(5000).times(1)).resume(metaCaptor.capture()); + partitionedTopicMetadata = metaCaptor.getValue(); + Assert.assertEquals(partitionedTopicMetadata.partitions, 4); + + conf.setMaxNumPartitionsPerPartitionedTopic(-1); + response = mock(AsyncResponse.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.updatePartitionedTopic(response, testTenant, testNamespaceLocal, topicName, false, true, + true, 5); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + + response = mock(AsyncResponse.class); + metaCaptor = ArgumentCaptor.forClass(PartitionedTopicMetadata.class); + persistentTopics.getPartitionedMetadata(response, testTenant, testNamespaceLocal, topicName, true, false); + verify(response, timeout(5000).times(1)).resume(metaCaptor.capture()); + partitionedTopicMetadata = metaCaptor.getValue(); + Assert.assertEquals(partitionedTopicMetadata.partitions, 5); + + conf.setMaxNumPartitionsPerPartitionedTopic(0); + response = mock(AsyncResponse.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.updatePartitionedTopic(response, testTenant, testNamespaceLocal, topicName, false, true, + true, 6); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + + response = mock(AsyncResponse.class); + metaCaptor = ArgumentCaptor.forClass(PartitionedTopicMetadata.class); + persistentTopics.getPartitionedMetadata(response, testTenant, testNamespaceLocal, topicName, true, false); + verify(response, timeout(5000).times(1)).resume(metaCaptor.capture()); + partitionedTopicMetadata = metaCaptor.getValue(); + Assert.assertEquals(partitionedTopicMetadata.partitions, 6); } @Test From cea1a9ba9b576bf43f0a45ff8d65369b0f2bbb36 Mon Sep 17 00:00:00 2001 From: zhangqian <503837557@qq.com> Date: Wed, 10 Apr 2024 16:51:26 +0800 Subject: [PATCH 058/580] [fix][broker] Fix message drop record in producer stat (#22458) Co-authored-by: ceceezhang --- .../main/java/org/apache/pulsar/broker/service/Producer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java index 7e4459505a523..9cfde67802bb0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java @@ -749,7 +749,7 @@ public void updateRates() { } if (this.isNonPersistentTopic) { msgDrop.calculateRate(); - ((NonPersistentPublisherStatsImpl) stats).msgDropRate = msgDrop.getRate(); + ((NonPersistentPublisherStatsImpl) stats).msgDropRate = msgDrop.getValueRate(); } } From b42d94121c0209c197339f1fe6ad702e9880c5f9 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 11 Apr 2024 10:10:31 -0700 Subject: [PATCH 059/580] [improve][broker] Recover susbcription creation on the broken schema ledger topic (#22469) --- .../pulsar/broker/service/ServerCnx.java | 4 +- .../schema/BookkeeperSchemaStorage.java | 2 + .../org/apache/pulsar/schema/SchemaTest.java | 76 +++++++++++++++++++ 3 files changed, 81 insertions(+), 1 deletion(-) 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 716f3a1a04c25..4ee6ac43465f4 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 @@ -26,6 +26,7 @@ import static org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.unsafeGetPartitionedTopicMetadataAsync; import static org.apache.pulsar.broker.lookup.TopicLookupBase.lookupTopicAsync; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.getMigratedClusterUrl; +import static org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage.ignoreUnrecoverableBKException; import static org.apache.pulsar.common.api.proto.ProtocolVersion.v5; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; import static org.apache.pulsar.common.protocol.Commands.newCloseConsumer; @@ -1291,7 +1292,8 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { .schemaType(schema == null ? null : schema.getType()) .build(); if (schema != null && schema.getType() != SchemaType.AUTO_CONSUME) { - return topic.addSchemaIfIdleOrCheckCompatible(schema) + return ignoreUnrecoverableBKException + (topic.addSchemaIfIdleOrCheckCompatible(schema)) .thenCompose(v -> topic.subscribe(option)); } else { return topic.subscribe(option); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java index c509764bf6710..acdd906f6b8af 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java @@ -52,6 +52,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; import org.apache.pulsar.broker.service.schema.exceptions.SchemaException; import org.apache.pulsar.common.protocol.schema.SchemaStorage; import org.apache.pulsar.common.protocol.schema.SchemaVersion; @@ -716,6 +717,7 @@ public static CompletableFuture ignoreUnrecoverableBKException(Completabl return source.exceptionally(t -> { if (t.getCause() != null && (t.getCause() instanceof SchemaException) + && !(t.getCause() instanceof IncompatibleSchemaException) && !((SchemaException) t.getCause()).isRecoverable()) { // Meeting NoSuchLedgerExistsException, NoSuchEntryException or // NoSuchLedgerExistsOnMetadataServerException when reading schemas in diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java index aa47c378fc38c..d21e853ba0982 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java @@ -32,6 +32,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.Sets; import java.io.ByteArrayInputStream; +import java.io.IOException; import java.io.Serializable; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -58,6 +59,8 @@ import org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage; import org.apache.pulsar.broker.service.schema.SchemaRegistry; import org.apache.pulsar.broker.service.schema.SchemaRegistryServiceImpl; +import org.apache.pulsar.broker.service.schema.SchemaStorageFormat; +import org.apache.pulsar.broker.service.schema.SchemaStorageFormat.SchemaLocator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -87,6 +90,9 @@ import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataCache; +import org.apache.pulsar.metadata.api.MetadataSerde; +import org.apache.pulsar.metadata.api.Stat; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -1410,4 +1416,74 @@ public User(String name) { } } + /** + * This test validates that consumer/producers should recover on topic whose + * schema ledgers are not able to open due to non-recoverable error. + * + * @throws Exception + */ + @Test + public void testDeletedSchemaLedgerRecovery() throws Exception { + final String tenant = PUBLIC_TENANT; + final String namespace = "test-namespace-" + randomName(16); + final String topicOne = "test-multi-version-schema-one"; + final String subName = "test"; + final String topicName = TopicName.get(TopicDomain.persistent.value(), tenant, namespace, topicOne).toString(); + + admin.namespaces().createNamespace(tenant + "/" + namespace, Sets.newHashSet(CLUSTER_NAME)); + + // (1) create schema + Producer producer = pulsarClient + .newProducer(Schema.AVRO(SchemaDefinition. builder().withAlwaysAllowNull(false) + .withSupportSchemaVersioning(true).withPojo(Schemas.PersonTwo.class).build())) + .topic(topicName).create(); + + Schemas.PersonTwo personTwo = new Schemas.PersonTwo(); + personTwo.setId(1); + personTwo.setName("Tom"); + + Consumer consumer = pulsarClient + .newConsumer(Schema.AVRO(SchemaDefinition. builder().withAlwaysAllowNull(false) + .withSupportSchemaVersioning(true).withPojo(Schemas.PersonTwo.class).build())) + .subscriptionName(subName).topic(topicName).subscribe(); + + producer.send(personTwo); + producer.close(); + consumer.close(); + + // (2) Delete schema ledger + MetadataCache locatorEntryCache = pulsar.getLocalMetadataStore() + .getMetadataCache(new MetadataSerde() { + @Override + public byte[] serialize(String path, SchemaStorageFormat.SchemaLocator value) { + return value.toByteArray(); + } + + @Override + public SchemaStorageFormat.SchemaLocator deserialize(String path, byte[] content, Stat stat) + throws IOException { + return SchemaStorageFormat.SchemaLocator.parseFrom(content); + } + }); + String path = "/schemas/public/" + namespace + "/test-multi-version-schema-one"; + SchemaLocator schema = locatorEntryCache.get(path).get().get(); + schema = locatorEntryCache.get(path).get().get(); + long ledgerId = schema.getInfo().getPosition().getLedgerId(); + pulsar.getBookKeeperClient().deleteLedger(ledgerId); + + // (3) Topic should recover from deleted schema and should allow to create consumer and producer + consumer = pulsarClient + .newConsumer(Schema.AVRO(SchemaDefinition. builder().withAlwaysAllowNull(false) + .withSupportSchemaVersioning(true).withPojo(Schemas.PersonTwo.class).build())) + .subscriptionName(subName).topic(topicName).subscribe(); + + producer = pulsarClient + .newProducer(Schema.AVRO(SchemaDefinition. builder().withAlwaysAllowNull(false) + .withSupportSchemaVersioning(true).withPojo(Schemas.PersonTwo.class).build())) + .topic(topicName).create(); + assertNotNull(consumer); + assertNotNull(producer); + consumer.close(); + producer.close(); + } } From 094742d5fa6f07b5ceed581876c45564fa0379bd Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Thu, 11 Apr 2024 16:21:45 -0700 Subject: [PATCH 060/580] [fix][broker] Do not migrate internal topics during Blue-Green Migration when ExtensibleLoadBalancer is used (#22478) --- .../pulsar/PulsarClusterMetadataSetup.java | 4 +- .../extensions/ExtensibleLoadManagerImpl.java | 4 +- .../channel/ServiceUnitStateChannelImpl.java | 4 ++ .../pulsar/broker/service/BrokerService.java | 3 ++ .../nonpersistent/NonPersistentTopic.java | 4 ++ .../service/persistent/PersistentTopic.java | 4 ++ .../broker/service/ClusterMigrationTest.java | 47 +++++++++++++++---- 7 files changed, 57 insertions(+), 13 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java index e8efeabcdd37c..d5b8df43a4737 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java @@ -358,8 +358,8 @@ private static void initializeCluster(Arguments arguments, int bundleNumberForDe log.info("Cluster metadata for '{}' setup correctly", arguments.cluster); } - static void createTenantIfAbsent(PulsarResources resources, String tenant, String cluster) throws IOException, - InterruptedException, ExecutionException { + public static void createTenantIfAbsent(PulsarResources resources, String tenant, String cluster) + throws IOException, InterruptedException, ExecutionException { TenantResources tenantResources = resources.getTenantResources(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index c35dc11d7efc7..0c9448ab69c38 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -825,11 +825,11 @@ synchronized void playLeader() { boolean becameFollower = false; while (!Thread.currentThread().isInterrupted()) { try { + initWaiter.await(); if (!serviceUnitStateChannel.isChannelOwner()) { becameFollower = true; break; } - initWaiter.await(); // Confirm the system topics have been created or create them if they do not exist. // If the leader has changed, the new leader need to reset // the local brokerService.topics (by this topic creations). @@ -875,11 +875,11 @@ synchronized void playFollower() { boolean becameLeader = false; while (!Thread.currentThread().isInterrupted()) { try { + initWaiter.await(); if (serviceUnitStateChannel.isChannelOwner()) { becameLeader = true; break; } - initWaiter.await(); unloadScheduler.close(); serviceUnitStateChannel.cancelOwnershipMonitor(); brokerLoadDataStore.init(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index 71ddb3acb28b7..68b38080e73a1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -293,6 +293,10 @@ public synchronized void start() throws PulsarServerException { log.info("Closed the channel producer."); } } + + PulsarClusterMetadataSetup.createTenantIfAbsent + (pulsar.getPulsarResources(), SYSTEM_NAMESPACE.getTenant(), config.getClusterName()); + PulsarClusterMetadataSetup.createNamespaceIfAbsent (pulsar.getPulsarResources(), SYSTEM_NAMESPACE, config.getClusterName()); 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 549dfef896cd0..b4d0f38b4a4dc 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 @@ -1785,6 +1785,9 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { } private CompletableFuture checkTopicAlreadyMigrated(TopicName topicName) { + if (ExtensibleLoadManagerImpl.isInternalTopic(topicName.toString())) { + return CompletableFuture.completedFuture(null); + } CompletableFuture result = new CompletableFuture<>(); AbstractTopic.isClusterMigrationEnabled(pulsar, topicName.toString()).handle((isMigrated, ex) -> { if (isMigrated) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 88f8c69895002..0ac06d6883ff1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -996,6 +996,10 @@ public boolean isActive() { @Override public CompletableFuture checkClusterMigration() { + if (ExtensibleLoadManagerImpl.isInternalTopic(topic)) { + return CompletableFuture.completedFuture(null); + } + Optional url = getMigratedClusterUrl(); if (url.isPresent()) { this.migrated = true; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index b21cd165402e4..3ceecd7f4aa20 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -2740,6 +2740,10 @@ private boolean hasBacklogs() { @Override public CompletableFuture checkClusterMigration() { + if (ExtensibleLoadManagerImpl.isInternalTopic(topic)) { + return CompletableFuture.completedFuture(null); + } + Optional clusterUrl = getMigratedClusterUrl(); if (!clusterUrl.isPresent()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java index 7bd82cdd840ee..20e13023cacfb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java @@ -35,6 +35,8 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; @@ -53,6 +55,7 @@ import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "cluster-migration") @@ -86,6 +89,8 @@ public class ClusterMigrationTest { PulsarService pulsar4; PulsarAdmin admin4; + String loadManagerClassName; + @DataProvider(name="NamespaceMigrationTopicSubscriptionTypes") public Object[][] namespaceMigrationSubscriptionTypes() { return new Object[][] { @@ -95,15 +100,28 @@ public Object[][] namespaceMigrationSubscriptionTypes() { }; } + @DataProvider(name = "loadManagerClassName") + public static Object[][] loadManagerClassName() { + return new Object[][]{ + {ModularLoadManagerImpl.class.getName()}, + {ExtensibleLoadManagerImpl.class.getName()} + }; + } + + @Factory(dataProvider = "loadManagerClassName") + public ClusterMigrationTest(String loadManagerClassName) { + this.loadManagerClassName = loadManagerClassName; + } + @BeforeMethod(alwaysRun = true, timeOut = 300000) public void setup() throws Exception { log.info("--- Starting ReplicatorTestBase::setup ---"); - broker1 = new TestBroker("r1"); - broker2 = new TestBroker("r2"); - broker3 = new TestBroker("r3"); - broker4 = new TestBroker("r4"); + broker1 = new TestBroker("r1", loadManagerClassName); + broker2 = new TestBroker("r2", loadManagerClassName); + broker3 = new TestBroker("r3", loadManagerClassName); + broker4 = new TestBroker("r4", loadManagerClassName); pulsar1 = broker1.getPulsarService(); url1 = new URL(pulsar1.getWebServiceAddress()); @@ -163,9 +181,9 @@ public void setup() throws Exception { .brokerServiceUrlTls(pulsar4.getBrokerServiceUrlTls()).build()); // Setting r3 as replication cluster for r1 - admin1.tenants().createTenant("pulsar", + updateTenantInfo(admin1, "pulsar", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r1", "r3"))); - admin3.tenants().createTenant("pulsar", + updateTenantInfo(admin3, "pulsar", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r1", "r3"))); admin1.namespaces().createNamespace(namespace, Sets.newHashSet("r1", "r3")); admin3.namespaces().createNamespace(namespace); @@ -175,9 +193,9 @@ public void setup() throws Exception { admin1.namespaces().setNamespaceReplicationClusters(namespaceNotToMigrate, Sets.newHashSet("r1", "r3")); // Setting r4 as replication cluster for r2 - admin2.tenants().createTenant("pulsar", + updateTenantInfo(admin2, "pulsar", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r2", "r4"))); - admin4.tenants().createTenant("pulsar", + updateTenantInfo(admin4,"pulsar", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r2", "r4"))); admin2.namespaces().createNamespace(namespace, Sets.newHashSet("r2", "r4")); admin4.namespaces().createNamespace(namespace); @@ -200,6 +218,14 @@ public void setup() throws Exception { } + protected void updateTenantInfo(PulsarAdmin admin, String tenant, TenantInfoImpl tenantInfo) throws Exception { + if (!admin.tenants().getTenants().contains(tenant)) { + admin.tenants().createTenant(tenant, tenantInfo); + } else { + admin.tenants().updateTenant(tenant, tenantInfo); + } + } + @AfterMethod(alwaysRun = true, timeOut = 300000) protected void cleanup() throws Exception { log.info("--- Shutting down ---"); @@ -1059,9 +1085,11 @@ public void testNamespaceMigrationWithReplicationBacklog(SubscriptionType subTyp static class TestBroker extends MockedPulsarServiceBaseTest { private String clusterName; + private String loadManagerClassName; - public TestBroker(String clusterName) throws Exception { + public TestBroker(String clusterName, String loadManagerClassName) throws Exception { this.clusterName = clusterName; + this.loadManagerClassName = loadManagerClassName; setup(); } @@ -1073,6 +1101,7 @@ protected void setup() throws Exception { @Override protected void doInitConf() throws Exception { super.doInitConf(); + this.conf.setLoadManagerClassName(loadManagerClassName); this.conf.setWebServicePortTls(Optional.of(0)); this.conf.setBrokerServicePortTls(Optional.of(0)); } From 4a5400f0c66dab2c3fbb0050c8f537952fef1951 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 00:15:33 -0700 Subject: [PATCH 061/580] [improve][misc] Upgrade to Bookkeeper 4.16.5 (#22484) --- .../server/src/assemble/LICENSE.bin.txt | 56 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 6 +- pom.xml | 2 +- 3 files changed, 32 insertions(+), 32 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 518f92313753f..a409ad07ed1b4 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -356,34 +356,34 @@ The Apache Software License, Version 2.0 - net.java.dev.jna-jna-jpms-5.12.1.jar - net.java.dev.jna-jna-platform-jpms-5.12.1.jar * BookKeeper - - org.apache.bookkeeper-bookkeeper-common-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-proto-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-server-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.4.jar - - org.apache.bookkeeper-circe-checksum-4.16.4.jar - - org.apache.bookkeeper-cpu-affinity-4.16.4.jar - - org.apache.bookkeeper-statelib-4.16.4.jar - - org.apache.bookkeeper-stream-storage-api-4.16.4.jar - - org.apache.bookkeeper-stream-storage-common-4.16.4.jar - - org.apache.bookkeeper-stream-storage-java-client-4.16.4.jar - - org.apache.bookkeeper-stream-storage-java-client-base-4.16.4.jar - - org.apache.bookkeeper-stream-storage-proto-4.16.4.jar - - org.apache.bookkeeper-stream-storage-server-4.16.4.jar - - org.apache.bookkeeper-stream-storage-service-api-4.16.4.jar - - org.apache.bookkeeper-stream-storage-service-impl-4.16.4.jar - - org.apache.bookkeeper.http-http-server-4.16.4.jar - - org.apache.bookkeeper.http-vertx-http-server-4.16.4.jar - - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.4.jar - - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.4.jar - - org.apache.distributedlog-distributedlog-common-4.16.4.jar - - org.apache.distributedlog-distributedlog-core-4.16.4-tests.jar - - org.apache.distributedlog-distributedlog-core-4.16.4.jar - - org.apache.distributedlog-distributedlog-protocol-4.16.4.jar - - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.4.jar - - org.apache.bookkeeper-native-io-4.16.4.jar + - org.apache.bookkeeper-bookkeeper-common-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-proto-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-server-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.5.jar + - org.apache.bookkeeper-circe-checksum-4.16.5.jar + - org.apache.bookkeeper-cpu-affinity-4.16.5.jar + - org.apache.bookkeeper-statelib-4.16.5.jar + - org.apache.bookkeeper-stream-storage-api-4.16.5.jar + - org.apache.bookkeeper-stream-storage-common-4.16.5.jar + - org.apache.bookkeeper-stream-storage-java-client-4.16.5.jar + - org.apache.bookkeeper-stream-storage-java-client-base-4.16.5.jar + - org.apache.bookkeeper-stream-storage-proto-4.16.5.jar + - org.apache.bookkeeper-stream-storage-server-4.16.5.jar + - org.apache.bookkeeper-stream-storage-service-api-4.16.5.jar + - org.apache.bookkeeper-stream-storage-service-impl-4.16.5.jar + - org.apache.bookkeeper.http-http-server-4.16.5.jar + - org.apache.bookkeeper.http-vertx-http-server-4.16.5.jar + - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.5.jar + - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.5.jar + - org.apache.distributedlog-distributedlog-common-4.16.5.jar + - org.apache.distributedlog-distributedlog-core-4.16.5-tests.jar + - org.apache.distributedlog-distributedlog-core-4.16.5.jar + - org.apache.distributedlog-distributedlog-protocol-4.16.5.jar + - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.5.jar + - org.apache.bookkeeper-native-io-4.16.5.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index b5036b67751f0..3ac489fa49a68 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -393,9 +393,9 @@ The Apache Software License, Version 2.0 - opentelemetry-extension-incubator-1.34.1-alpha.jar * BookKeeper - - bookkeeper-common-allocator-4.16.4.jar - - cpu-affinity-4.16.4.jar - - circe-checksum-4.16.4.jar + - bookkeeper-common-allocator-4.16.5.jar + - cpu-affinity-4.16.5.jar + - circe-checksum-4.16.5.jar * AirCompressor - aircompressor-0.20.jar * AsyncHttpClient diff --git a/pom.xml b/pom.xml index 835bd28f7f25b..47ac21b62bfed 100644 --- a/pom.xml +++ b/pom.xml @@ -137,7 +137,7 @@ flexible messaging model and an intuitive client API. 1.26.0 - 4.16.4 + 4.16.5 3.9.2 1.5.0 1.10.0 From 7984cc2f93f8dc85b598ded1167508eae4ee06ec Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 05:56:55 -0700 Subject: [PATCH 062/580] [feat][admin] Enable Gzip Compression by Default in Admin Client (#22464) --- .../client/admin/PulsarAdminBuilder.java | 11 +- pulsar-client-admin/pom.xml | 7 + .../internal/PulsarAdminBuilderImpl.java | 22 +++- .../admin/internal/PulsarAdminImpl.java | 8 +- .../internal/http/AsyncHttpConnector.java | 14 +- .../http/AsyncHttpConnectorProvider.java | 9 +- .../admin/internal/PulsarAdminGzipTest.java | 122 ++++++++++++++++++ 7 files changed, 183 insertions(+), 10 deletions(-) create mode 100644 pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminGzipTest.java diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java index 1260555a7c43f..1b025a752d9f3 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java @@ -327,4 +327,13 @@ PulsarAdminBuilder authentication(String authPluginClassName, Maphamcrest test + + + com.github.tomakehurst + wiremock-jre8 + ${wiremock.version} + test + diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java index 009fa67fbaa29..f7b1695f5f37b 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java @@ -38,10 +38,11 @@ public class PulsarAdminBuilderImpl implements PulsarAdminBuilder { protected ClientConfigurationData conf; private ClassLoader clientBuilderClassLoader = null; + private boolean acceptGzipCompression = true; @Override public PulsarAdmin build() throws PulsarClientException { - return new PulsarAdminImpl(conf.getServiceUrl(), conf, clientBuilderClassLoader); + return new PulsarAdminImpl(conf.getServiceUrl(), conf, clientBuilderClassLoader, acceptGzipCompression); } public PulsarAdminBuilderImpl() { @@ -54,13 +55,24 @@ private PulsarAdminBuilderImpl(ClientConfigurationData conf) { @Override public PulsarAdminBuilder clone() { - return new PulsarAdminBuilderImpl(conf.clone()); + PulsarAdminBuilderImpl pulsarAdminBuilder = new PulsarAdminBuilderImpl(conf.clone()); + pulsarAdminBuilder.clientBuilderClassLoader = clientBuilderClassLoader; + pulsarAdminBuilder.acceptGzipCompression = acceptGzipCompression; + return pulsarAdminBuilder; } @Override public PulsarAdminBuilder loadConf(Map config) { conf = ConfigurationDataUtils.loadData(config, conf, ClientConfigurationData.class); setAuthenticationFromPropsIfAvailable(conf); + if (config.containsKey("acceptGzipCompression")) { + Object acceptGzipCompressionObj = config.get("acceptGzipCompression"); + if (acceptGzipCompressionObj instanceof Boolean) { + acceptGzipCompression = (Boolean) acceptGzipCompressionObj; + } else { + acceptGzipCompression = Boolean.parseBoolean(acceptGzipCompressionObj.toString()); + } + } return this; } @@ -227,4 +239,10 @@ public PulsarAdminBuilder setContextClassLoader(ClassLoader clientBuilderClassLo this.clientBuilderClassLoader = clientBuilderClassLoader; return this; } + + @Override + public PulsarAdminBuilder acceptGzipCompression(boolean acceptGzipCompression) { + this.acceptGzipCompression = acceptGzipCompression; + return this; + } } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java index 259ca90cc08b7..39347850cf69c 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java @@ -106,6 +106,12 @@ public class PulsarAdminImpl implements PulsarAdmin { public PulsarAdminImpl(String serviceUrl, ClientConfigurationData clientConfigData, ClassLoader clientBuilderClassLoader) throws PulsarClientException { + this(serviceUrl, clientConfigData, clientBuilderClassLoader, true); + } + + public PulsarAdminImpl(String serviceUrl, ClientConfigurationData clientConfigData, + ClassLoader clientBuilderClassLoader, boolean acceptGzipCompression) + throws PulsarClientException { checkArgument(StringUtils.isNotBlank(serviceUrl), "Service URL needs to be specified"); this.clientConfigData = clientConfigData; @@ -119,7 +125,7 @@ public PulsarAdminImpl(String serviceUrl, ClientConfigurationData clientConfigDa } AsyncHttpConnectorProvider asyncConnectorProvider = new AsyncHttpConnectorProvider(clientConfigData, - clientConfigData.getAutoCertRefreshSeconds()); + clientConfigData.getAutoCertRefreshSeconds(), acceptGzipCompression); ClientConfig httpConfig = new ClientConfig(); httpConfig.property(ClientProperties.FOLLOW_REDIRECTS, true); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java index 9ed2b8564f2ae..9ad0ce5029c47 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java @@ -83,19 +83,23 @@ public class AsyncHttpConnector implements Connector { private final PulsarServiceNameResolver serviceNameResolver; private final ScheduledExecutorService delayer = Executors.newScheduledThreadPool(1, new DefaultThreadFactory("delayer")); + private final boolean acceptGzipCompression; - public AsyncHttpConnector(Client client, ClientConfigurationData conf, int autoCertRefreshTimeSeconds) { + public AsyncHttpConnector(Client client, ClientConfigurationData conf, int autoCertRefreshTimeSeconds, + boolean acceptGzipCompression) { this((int) client.getConfiguration().getProperty(ClientProperties.CONNECT_TIMEOUT), (int) client.getConfiguration().getProperty(ClientProperties.READ_TIMEOUT), PulsarAdminImpl.DEFAULT_REQUEST_TIMEOUT_SECONDS * 1000, autoCertRefreshTimeSeconds, - conf); + conf, acceptGzipCompression); } @SneakyThrows public AsyncHttpConnector(int connectTimeoutMs, int readTimeoutMs, int requestTimeoutMs, - int autoCertRefreshTimeSeconds, ClientConfigurationData conf) { + int autoCertRefreshTimeSeconds, ClientConfigurationData conf, + boolean acceptGzipCompression) { + this.acceptGzipCompression = acceptGzipCompression; DefaultAsyncHttpClientConfig.Builder confBuilder = new DefaultAsyncHttpClientConfig.Builder(); confBuilder.setUseProxyProperties(true); confBuilder.setFollowRedirect(true); @@ -339,6 +343,10 @@ private CompletableFuture oneShot(InetSocketAddress host, ClientReques } }); + if (acceptGzipCompression) { + builder.setHeader(HttpHeaders.ACCEPT_ENCODING, "gzip"); + } + return builder.execute().toCompletableFuture(); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorProvider.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorProvider.java index 4467f77d1f993..d20dc84849458 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorProvider.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorProvider.java @@ -32,16 +32,19 @@ public class AsyncHttpConnectorProvider implements ConnectorProvider { private final ClientConfigurationData conf; private Connector connector; private final int autoCertRefreshTimeSeconds; + private final boolean acceptGzipCompression; - public AsyncHttpConnectorProvider(ClientConfigurationData conf, int autoCertRefreshTimeSeconds) { + public AsyncHttpConnectorProvider(ClientConfigurationData conf, int autoCertRefreshTimeSeconds, + boolean acceptGzipCompression) { this.conf = conf; this.autoCertRefreshTimeSeconds = autoCertRefreshTimeSeconds; + this.acceptGzipCompression = acceptGzipCompression; } @Override public Connector getConnector(Client client, Configuration runtimeConfig) { if (connector == null) { - connector = new AsyncHttpConnector(client, conf, autoCertRefreshTimeSeconds); + connector = new AsyncHttpConnector(client, conf, autoCertRefreshTimeSeconds, acceptGzipCompression); } return connector; } @@ -50,6 +53,6 @@ public Connector getConnector(Client client, Configuration runtimeConfig) { public AsyncHttpConnector getConnector(int connectTimeoutMs, int readTimeoutMs, int requestTimeoutMs, int autoCertRefreshTimeSeconds) { return new AsyncHttpConnector(connectTimeoutMs, readTimeoutMs, requestTimeoutMs, autoCertRefreshTimeSeconds, - conf); + conf, acceptGzipCompression); } } diff --git a/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminGzipTest.java b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminGzipTest.java new file mode 100644 index 0000000000000..2bfa382be1096 --- /dev/null +++ b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminGzipTest.java @@ -0,0 +1,122 @@ +/* + * 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.admin.internal; + +import static com.github.tomakehurst.wiremock.client.WireMock.aResponse; +import static com.github.tomakehurst.wiremock.client.WireMock.absent; +import static com.github.tomakehurst.wiremock.client.WireMock.equalTo; +import static com.github.tomakehurst.wiremock.client.WireMock.get; +import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo; +import static org.testng.Assert.assertEquals; +import com.github.tomakehurst.wiremock.WireMockServer; +import com.github.tomakehurst.wiremock.core.WireMockConfiguration; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.zip.GZIPOutputStream; +import lombok.Cleanup; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class PulsarAdminGzipTest { + WireMockServer server; + + @BeforeClass(alwaysRun = true) + void beforeClass() throws IOException { + server = new WireMockServer(WireMockConfiguration.wireMockConfig() + .port(0)); + server.start(); + } + + @AfterClass(alwaysRun = true) + void afterClass() { + if (server != null) { + server.stop(); + } + } + + static byte[] gzipContent(String content) throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + try(GZIPOutputStream gzipOutputStream = new GZIPOutputStream(byteArrayOutputStream)) { + gzipOutputStream.write(content.getBytes(StandardCharsets.UTF_8)); + } + return byteArrayOutputStream.toByteArray(); + } + + @AfterMethod + void resetAllMocks() { + server.resetAll(); + } + + @Test + public void testGzipRequestedGzipResponse() throws Exception { + server.stubFor(get(urlEqualTo("/admin/v2/clusters")) + .withHeader("Accept-Encoding", equalTo("gzip")) + .willReturn(aResponse() + .withHeader("Content-Type", "application/json") + .withHeader("Content-Encoding", "gzip") + .withBody(gzipContent("[\"gzip-test\", \"gzip-test2\"]")))); + + @Cleanup + PulsarAdmin admin = PulsarAdmin.builder() + .serviceHttpUrl("http://localhost:" + server.port()) + .acceptGzipCompression(true) + .build(); + + assertEquals(admin.clusters().getClusters(), Arrays.asList("gzip-test", "gzip-test2")); + } + + @Test + public void testGzipRequestedNoGzipResponse() throws Exception { + server.stubFor(get(urlEqualTo("/admin/v2/clusters")) + .withHeader("Accept-Encoding", equalTo("gzip")) + .willReturn(aResponse() + .withHeader("Content-Type", "application/json") + .withBody("[\"test\", \"test2\"]"))); + + @Cleanup + PulsarAdmin admin = PulsarAdmin.builder() + .serviceHttpUrl("http://localhost:" + server.port()) + .acceptGzipCompression(true) + .build(); + + assertEquals(admin.clusters().getClusters(), Arrays.asList("test", "test2")); + } + + @Test + public void testNoGzipRequestedNoGzipResponse() throws Exception { + server.stubFor(get(urlEqualTo("/admin/v2/clusters")) + .withHeader("Accept-Encoding", absent()) + .willReturn(aResponse() + .withHeader("Content-Type", "application/json") + .withBody("[\"test\", \"test2\"]"))); + + @Cleanup + PulsarAdmin admin = PulsarAdmin.builder() + .serviceHttpUrl("http://localhost:" + server.port()) + .acceptGzipCompression(false) + .build(); + + assertEquals(admin.clusters().getClusters(), Arrays.asList("test", "test2")); + } +} From dbe1a4816c12535da2013ed5da5ee7796d8b4638 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 12 Apr 2024 21:52:34 +0800 Subject: [PATCH 063/580] [improve][broker] Reduce the duplicated null check for LeaderElectionService (#22465) --- .../broker/namespace/NamespaceService.java | 17 +++-------------- 1 file changed, 3 insertions(+), 14 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 6228703f03ab9..4492f9c809435 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -539,23 +539,12 @@ private CompletableFuture> findBrokerServiceUrl( private void searchForCandidateBroker(NamespaceBundle bundle, CompletableFuture> lookupFuture, LookupOptions options) { - if (null == pulsar.getLeaderElectionService()) { - LOG.warn("The leader election has not yet been completed! NamespaceBundle[{}]", bundle); - lookupFuture.completeExceptionally( - new IllegalStateException("The leader election has not yet been completed!")); - return; - } String candidateBroker; - LeaderElectionService les = pulsar.getLeaderElectionService(); if (les == null) { - // The leader election service was not initialized yet. This can happen because the broker service is - // initialized first, and it might start receiving lookup requests before the leader election service is - // fully initialized. - LOG.warn("Leader election service isn't initialized yet. " - + "Returning empty result to lookup. NamespaceBundle[{}]", - bundle); - lookupFuture.complete(Optional.empty()); + LOG.warn("The leader election has not yet been completed! NamespaceBundle[{}]", bundle); + lookupFuture.completeExceptionally( + new IllegalStateException("The leader election has not yet been completed!")); return; } From b85730069ee4c5f96406a075e354d0592fdab434 Mon Sep 17 00:00:00 2001 From: Mukesh Kumar <65598381+mukesh154@users.noreply.github.com> Date: Fri, 12 Apr 2024 22:07:28 +0530 Subject: [PATCH 064/580] [improve][broker] backlog quota exceed limit log replaced with `debug` (#22488) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 3ceecd7f4aa20..3c9ab04d79a0d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -3224,14 +3224,14 @@ public CompletableFuture checkBacklogQuotaExceeded(String producerName, Ba if ((retentionPolicy == BacklogQuota.RetentionPolicy.producer_request_hold || retentionPolicy == BacklogQuota.RetentionPolicy.producer_exception)) { if (backlogQuotaType == BacklogQuotaType.destination_storage && isSizeBacklogExceeded()) { - log.info("[{}] Size backlog quota exceeded. Cannot create producer [{}]", this.getName(), + log.debug("[{}] Size backlog quota exceeded. Cannot create producer [{}]", this.getName(), producerName); return FutureUtil.failedFuture(new TopicBacklogQuotaExceededException(retentionPolicy)); } if (backlogQuotaType == BacklogQuotaType.message_age) { return checkTimeBacklogExceeded().thenCompose(isExceeded -> { if (isExceeded) { - log.info("[{}] Time backlog quota exceeded. Cannot create producer [{}]", this.getName(), + log.debug("[{}] Time backlog quota exceeded. Cannot create producer [{}]", this.getName(), producerName); return FutureUtil.failedFuture(new TopicBacklogQuotaExceededException(retentionPolicy)); } else { From d1748573f1cb294838b69b5d80af672c3ee9e453 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 10:09:54 -0700 Subject: [PATCH 065/580] [fix][test] Fix NPE in BookKeeperClusterTestCase tearDown (#22493) --- .../apache/bookkeeper/test/BookKeeperClusterTestCase.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index 0ddd04ebc4830..a323ecfeb8ea6 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -86,7 +86,7 @@ public abstract class BookKeeperClusterTestCase { protected String testName; - @BeforeMethod + @BeforeMethod(alwaysRun = true) public void handleTestMethodName(Method method) { testName = method.getName(); } @@ -148,7 +148,7 @@ public BookKeeperClusterTestCase(int numBookies, int numOfZKNodes, int testTimeo } } - @BeforeTest + @BeforeTest(alwaysRun = true) public void setUp() throws Exception { setUp(getLedgersRootPath()); } @@ -222,7 +222,9 @@ public void tearDown() throws Exception { tearDownException = e; } - executor.shutdownNow(); + if (executor != null) { + executor.shutdownNow(); + } LOG.info("Tearing down test {} in {} ms.", testName, sw.elapsed(TimeUnit.MILLISECONDS)); if (tearDownException != null) { From 15ed6595af5489a007db82002ed3391589bad54d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 10:35:09 -0700 Subject: [PATCH 066/580] [improve][broker] Improve Gzip compression, allow excluding specific paths or disabling it (#22370) --- .../pulsar/broker/ServiceConfiguration.java | 13 +++++ .../pulsar/broker/web/GzipHandlerUtil.java | 48 +++++++++++++++++++ .../apache/pulsar/broker/web/WebService.java | 10 ++-- .../proxy/server/AdminProxyHandler.java | 1 + 4 files changed, 66 insertions(+), 6 deletions(-) create mode 100644 pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java 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 a7deda752fdde..38a4c552f0b6b 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 @@ -331,6 +331,19 @@ public class ServiceConfiguration implements PulsarConfiguration { + "(0 to disable limiting)") private int maxHttpServerConnections = 2048; + @FieldContext(category = CATEGORY_SERVER, doc = + "Gzip compression is enabled by default. Specific paths can be excluded from compression.\n" + + "There are 2 syntaxes supported, Servlet url-pattern based, and Regex based.\n" + + "If the spec starts with '^' the spec is assumed to be a regex based path spec and will match " + + "with normal Java regex rules.\n" + + "If the spec starts with '/' then spec is assumed to be a Servlet url-pattern rules path spec " + + "for either an exact match or prefix based match.\n" + + "If the spec starts with '*.' then spec is assumed to be a Servlet url-pattern rules path spec " + + "for a suffix based match.\n" + + "All other syntaxes are unsupported.\n" + + "Disable all compression with ^.* or ^.*$") + private List httpServerGzipCompressionExcludedPaths = new ArrayList<>(); + @FieldContext(category = CATEGORY_SERVER, doc = "Whether to enable the delayed delivery for messages.") private boolean delayedDeliveryEnabled = true; diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java new file mode 100644 index 0000000000000..37c9c05e5d53c --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java @@ -0,0 +1,48 @@ +/* + * 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.web; + +import java.util.List; +import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; + +public class GzipHandlerUtil { + public static Handler wrapWithGzipHandler(Handler innerHandler, List gzipCompressionExcludedPaths) { + Handler wrappedHandler; + if (isGzipCompressionCompletelyDisabled(gzipCompressionExcludedPaths)) { + // no need to add GZIP handler if it's disabled by setting the excluded path to "^.*" or "^.*$" + wrappedHandler = innerHandler; + } else { + // add GZIP handler which is active when the request contains "Accept-Encoding: gzip" header + GzipHandler gzipHandler = new GzipHandler(); + gzipHandler.setHandler(innerHandler); + if (gzipCompressionExcludedPaths != null && gzipCompressionExcludedPaths.size() > 0) { + gzipHandler.setExcludedPaths(gzipCompressionExcludedPaths.toArray(new String[0])); + } + wrappedHandler = gzipHandler; + } + return wrappedHandler; + } + + public static boolean isGzipCompressionCompletelyDisabled(List gzipCompressionExcludedPaths) { + return gzipCompressionExcludedPaths != null && gzipCompressionExcludedPaths.size() == 1 + && (gzipCompressionExcludedPaths.get(0).equals("^.*") + || gzipCompressionExcludedPaths.get(0).equals("^.*$")); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java index a7c4244899069..8dc36e2917ed1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java @@ -44,7 +44,6 @@ import org.eclipse.jetty.server.handler.RequestLogHandler; import org.eclipse.jetty.server.handler.ResourceHandler; import org.eclipse.jetty.server.handler.StatisticsHandler; -import org.eclipse.jetty.server.handler.gzip.GzipHandler; import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; @@ -268,9 +267,7 @@ public void addServlet(String path, ServletHolder servletHolder, boolean require } filterInitializer.addFilters(servletContextHandler, requiresAuthentication); - GzipHandler gzipHandler = new GzipHandler(); - gzipHandler.setHandler(servletContextHandler); - handlers.add(gzipHandler); + handlers.add(servletContextHandler); } public void addStaticResources(String basePath, String resourcePath) { @@ -294,8 +291,10 @@ public void start() throws PulsarServerException { ContextHandlerCollection contexts = new ContextHandlerCollection(); contexts.setHandlers(handlers.toArray(new Handler[handlers.size()])); + Handler handlerForContexts = GzipHandlerUtil.wrapWithGzipHandler(contexts, + pulsar.getConfig().getHttpServerGzipCompressionExcludedPaths()); HandlerCollection handlerCollection = new HandlerCollection(); - handlerCollection.setHandlers(new Handler[] { contexts, new DefaultHandler(), requestLogHandler }); + handlerCollection.setHandlers(new Handler[] {handlerForContexts, new DefaultHandler(), requestLogHandler}); // Metrics handler StatisticsHandler stats = new StatisticsHandler(); @@ -306,7 +305,6 @@ public void start() throws PulsarServerException { } catch (IllegalArgumentException e) { // Already registered. Eg: in unit tests } - handlers.add(stats); server.setHandler(stats); server.start(); diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java index c528ceb2cf5b7..caaa99c5d40cc 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java @@ -158,6 +158,7 @@ protected HttpClient createHttpClient() throws ServletException { client.start(); // Content must not be decoded, otherwise the client gets confused. + // Allow encoded content, such as "Content-Encoding: gzip", to pass through without decoding it. client.getContentDecoderFactories().clear(); // Pass traffic to the client, only intercept what's necessary. From 97153dcf6f0f958a2ee2816ea64115a9a9ef4635 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 11:38:42 -0700 Subject: [PATCH 067/580] [fix][test] Move ExtensibleLoadManagerImplTest to flaky tests (#22495) --- .../loadbalance/extensions/ExtensibleLoadManagerImplTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index aee57f9d26093..e87d6c994cd76 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -142,7 +142,7 @@ * Unit test for {@link ExtensibleLoadManagerImpl}. */ @Slf4j -@Test(groups = "broker") +@Test(groups = "flaky") @SuppressWarnings("unchecked") public class ExtensibleLoadManagerImplTest extends ExtensibleLoadManagerImplBaseTest { From 51ecd0235ce5d5ad03c563a3338b29c6a117d216 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Sat, 13 Apr 2024 08:58:53 +0800 Subject: [PATCH 068/580] [cleanup][broker] Remove unused NamespaceBundleFactory parameter when creating OwnershipCache (#22482) --- .../broker/namespace/NamespaceService.java | 2 +- .../broker/namespace/OwnershipCache.java | 4 +--- .../broker/namespace/OwnershipCacheTest.java | 19 ++++++++----------- 3 files changed, 10 insertions(+), 15 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 4492f9c809435..7c62f264c78d4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -192,7 +192,7 @@ public NamespaceService(PulsarService pulsar) { this.config = pulsar.getConfiguration(); this.loadManager = pulsar.getLoadManager(); this.bundleFactory = new NamespaceBundleFactory(pulsar, Hashing.crc32()); - this.ownershipCache = new OwnershipCache(pulsar, bundleFactory, this); + this.ownershipCache = new OwnershipCache(pulsar, this); this.namespaceClients = ConcurrentOpenHashMap.newBuilder().build(); this.bundleOwnershipListeners = new CopyOnWriteArrayList<>(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java index 0033abf36c78c..9a4534f538774 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java @@ -36,7 +36,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.common.naming.NamespaceBundle; -import org.apache.pulsar.common.naming.NamespaceBundleFactory; import org.apache.pulsar.common.naming.NamespaceBundles; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.coordination.LockManager; @@ -115,8 +114,7 @@ public CompletableFuture asyncLoad(NamespaceBundle namespaceBundle, * * the local broker URL that will be set as owner for the ServiceUnit */ - public OwnershipCache(PulsarService pulsar, NamespaceBundleFactory bundleFactory, - NamespaceService namespaceService) { + public OwnershipCache(PulsarService pulsar, NamespaceService namespaceService) { this.namespaceService = namespaceService; this.pulsar = pulsar; this.ownerBrokerUrl = pulsar.getBrokerServiceUrl(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnershipCacheTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnershipCacheTest.java index c92127457aaf2..2c3182659f022 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnershipCacheTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnershipCacheTest.java @@ -55,15 +55,12 @@ import org.apache.pulsar.metadata.coordination.impl.CoordinationServiceImpl; import org.apache.pulsar.zookeeper.ZookeeperServerTest; import org.awaitility.Awaitility; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @Test(groups = "broker") public class OwnershipCacheTest { - private static final Logger log = LoggerFactory.getLogger(OwnershipCacheTest.class); private PulsarService pulsar; private ServiceConfiguration config; @@ -123,14 +120,14 @@ public void teardown() throws Exception { @Test public void testConstructor() { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); assertNotNull(cache); assertNotNull(cache.getOwnedBundles()); } @Test public void testDisableOwnership() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceBundle testBundle = new NamespaceBundle(NamespaceName.get("pulsar/test/ns-1"), Range.closedOpen(0L, (long) Integer.MAX_VALUE), @@ -148,7 +145,7 @@ public void testDisableOwnership() throws Exception { @Test public void testGetOrSetOwner() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceBundle testFullBundle = new NamespaceBundle(NamespaceName.get("pulsar/test/ns-2"), Range.closedOpen(0L, (long) Integer.MAX_VALUE), bundleFactory); @@ -194,7 +191,7 @@ public void testGetOrSetOwner() throws Exception { @Test public void testGetOwner() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceBundle testBundle = new NamespaceBundle(NamespaceName.get("pulsar/test/ns-3"), Range.closedOpen(0L, (long) Integer.MAX_VALUE), bundleFactory); @@ -241,7 +238,7 @@ public void testGetOwner() throws Exception { @Test public void testGetOwnedServiceUnit() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceName testNs = NamespaceName.get("pulsar/test/ns-5"); NamespaceBundle testBundle = new NamespaceBundle(testNs, Range.closedOpen(0L, (long) Integer.MAX_VALUE), @@ -301,7 +298,7 @@ public void testGetOwnedServiceUnit() throws Exception { @Test public void testGetOwnedServiceUnits() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceName testNs = NamespaceName.get("pulsar/test/ns-6"); NamespaceBundle testBundle = new NamespaceBundle(testNs, Range.closedOpen(0L, (long) Integer.MAX_VALUE), @@ -347,7 +344,7 @@ public void testGetOwnedServiceUnits() throws Exception { @Test public void testRemoveOwnership() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceName testNs = NamespaceName.get("pulsar/test/ns-7"); NamespaceBundle bundle = new NamespaceBundle(testNs, Range.closedOpen(0L, (long) Integer.MAX_VALUE), @@ -373,7 +370,7 @@ public void testRemoveOwnership() throws Exception { @Test public void testReestablishOwnership() throws Exception { - OwnershipCache cache = new OwnershipCache(this.pulsar, bundleFactory, nsService); + OwnershipCache cache = new OwnershipCache(this.pulsar, nsService); NamespaceBundle testFullBundle = new NamespaceBundle(NamespaceName.get("pulsar/test/ns-8"), Range.closedOpen(0L, (long) Integer.MAX_VALUE), bundleFactory); From 7009071b6d53bbc3d740ea99cdc0c010692679ab Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 13 Apr 2024 10:00:23 -0700 Subject: [PATCH 069/580] [fix][broker] Optimize /metrics, fix unbounded request queue issue and fix race conditions in metricsBufferResponse mode (#22494) --- conf/proxy.conf | 6 +- .../PrometheusMetricsGeneratorUtils.java | 2 +- .../prometheus/PrometheusMetricsServlet.java | 149 +++++--- .../pulsar/broker/stats/TimeWindow.java | 94 ----- .../pulsar/broker/stats/WindowWrap.java | 56 --- .../stats/prometheus/MetricsExports.java | 68 ++++ .../prometheus/PrometheusMetricStreams.java | 2 +- .../PrometheusMetricsGenerator.java | 328 +++++++++++------- .../PulsarPrometheusMetricsServlet.java | 140 +++++++- .../broker/stats/prometheus/TopicStats.java | 12 +- .../pulsar/PrometheusMetricsTestUtil.java | 84 +++++ .../persistent/BucketDelayedDeliveryTest.java | 6 +- .../persistent/PersistentTopicTest.java | 4 +- .../service/schema/SchemaServiceTest.java | 4 +- .../broker/stats/ConsumerStatsTest.java | 4 +- .../broker/stats/MetadataStoreStatsTest.java | 6 +- .../broker/stats/PrometheusMetricsTest.java | 120 ++++--- .../broker/stats/SubscriptionStatsTest.java | 4 +- .../pulsar/broker/stats/TimeWindowTest.java | 83 ----- .../broker/stats/TransactionMetricsTest.java | 18 +- .../buffer/TransactionBufferClientTest.java | 4 +- .../pendingack/PendingAckPersistentTest.java | 4 +- .../pulsar/broker/web/WebServiceTest.java | 4 +- .../common/util/SimpleTextOutputStream.java | 16 +- .../proxy/server/ProxyConfiguration.java | 6 + .../pulsar/proxy/server/ProxyService.java | 3 +- .../proxy/server/ProxyServiceStarter.java | 40 ++- 27 files changed, 739 insertions(+), 528 deletions(-) delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java delete mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java diff --git a/conf/proxy.conf b/conf/proxy.conf index 8285e1cb75320..5a9d433f39ceb 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -376,5 +376,7 @@ zooKeeperCacheExpirySeconds=-1 enableProxyStatsEndpoints=true # Whether the '/metrics' endpoint requires authentication. Defaults to true authenticateMetricsEndpoint=true -# Enable cache metrics data, default value is false -metricsBufferResponse=false +# Time in milliseconds that metrics endpoint would time out. Default is 30s. +# Set it to 0 to disable timeout. +metricsServletTimeoutMs=30000 + diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java index 828d9871bb3de..077d5280b5102 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java @@ -76,7 +76,7 @@ public static void generateSystemMetrics(SimpleTextOutputStream stream, String c } for (int j = 0; j < sample.labelNames.size(); j++) { String labelValue = sample.labelValues.get(j); - if (labelValue != null) { + if (labelValue != null && labelValue.indexOf('"') > -1) { labelValue = labelValue.replace("\"", "\\\""); } if (j > 0) { diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java index 64d1fcdab6f14..8a41bed29d44f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java @@ -25,9 +25,13 @@ import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import javax.servlet.AsyncContext; +import javax.servlet.AsyncEvent; +import javax.servlet.AsyncListener; import javax.servlet.ServletException; -import javax.servlet.ServletOutputStream; import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; @@ -35,67 +39,132 @@ import org.slf4j.LoggerFactory; public class PrometheusMetricsServlet extends HttpServlet { - private static final long serialVersionUID = 1L; - private static final int HTTP_STATUS_OK_200 = 200; - private static final int HTTP_STATUS_INTERNAL_SERVER_ERROR_500 = 500; - - private final long metricsServletTimeoutMs; - private final String cluster; + static final int HTTP_STATUS_OK_200 = 200; + static final int HTTP_STATUS_INTERNAL_SERVER_ERROR_500 = 500; + protected final long metricsServletTimeoutMs; + protected final String cluster; protected List metricsProviders; - private ExecutorService executor = null; + protected ExecutorService executor = null; + protected final int executorMaxThreads; public PrometheusMetricsServlet(long metricsServletTimeoutMs, String cluster) { + this(metricsServletTimeoutMs, cluster, 1); + } + + public PrometheusMetricsServlet(long metricsServletTimeoutMs, String cluster, int executorMaxThreads) { this.metricsServletTimeoutMs = metricsServletTimeoutMs; this.cluster = cluster; + this.executorMaxThreads = executorMaxThreads; } @Override public void init() throws ServletException { - executor = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("prometheus-stats")); + if (executorMaxThreads > 0) { + executor = + Executors.newScheduledThreadPool(executorMaxThreads, new DefaultThreadFactory("prometheus-stats")); + } } @Override protected void doGet(HttpServletRequest request, HttpServletResponse response) { AsyncContext context = request.startAsync(); - context.setTimeout(metricsServletTimeoutMs); - executor.execute(() -> { - long start = System.currentTimeMillis(); - HttpServletResponse res = (HttpServletResponse) context.getResponse(); - try { - res.setStatus(HTTP_STATUS_OK_200); - res.setContentType("text/plain;charset=utf-8"); - generateMetrics(cluster, res.getOutputStream()); - } catch (Exception e) { - long end = System.currentTimeMillis(); - long time = end - start; - if (e instanceof EOFException) { - // NO STACKTRACE - log.error("Failed to send metrics, " - + "likely the client or this server closed " - + "the connection due to a timeout ({} ms elapsed): {}", time, e + ""); - } else { - log.error("Failed to generate prometheus stats, {} ms elapsed", time, e); + // set hard timeout to 2 * timeout + if (metricsServletTimeoutMs > 0) { + context.setTimeout(metricsServletTimeoutMs * 2); + } + long startNanos = System.nanoTime(); + AtomicBoolean taskStarted = new AtomicBoolean(false); + Future future = executor.submit(() -> { + taskStarted.set(true); + long elapsedNanos = System.nanoTime() - startNanos; + // check if the request has been timed out, implement a soft timeout + // so that response writing can continue to up to 2 * timeout + if (metricsServletTimeoutMs > 0 && elapsedNanos > TimeUnit.MILLISECONDS.toNanos(metricsServletTimeoutMs)) { + log.warn("Prometheus metrics request was too long in queue ({}ms). Skipping sending metrics.", + TimeUnit.NANOSECONDS.toMillis(elapsedNanos)); + if (!response.isCommitted()) { + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); } - res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); - } finally { - long end = System.currentTimeMillis(); - long time = end - start; - try { - context.complete(); - } catch (IllegalStateException e) { - // this happens when metricsServletTimeoutMs expires - // java.lang.IllegalStateException: AsyncContext completed and/or Request lifecycle recycled - log.error("Failed to generate prometheus stats, " - + "this is likely due to metricsServletTimeoutMs: {} ms elapsed: {}", time, e + ""); + context.complete(); + return; + } + handleAsyncMetricsRequest(context); + }); + context.addListener(new AsyncListener() { + @Override + public void onComplete(AsyncEvent asyncEvent) throws IOException { + if (!taskStarted.get()) { + future.cancel(false); } } + + @Override + public void onTimeout(AsyncEvent asyncEvent) throws IOException { + if (!taskStarted.get()) { + future.cancel(false); + } + log.warn("Prometheus metrics request timed out"); + HttpServletResponse res = (HttpServletResponse) context.getResponse(); + if (!res.isCommitted()) { + res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + context.complete(); + } + + @Override + public void onError(AsyncEvent asyncEvent) throws IOException { + if (!taskStarted.get()) { + future.cancel(false); + } + } + + @Override + public void onStartAsync(AsyncEvent asyncEvent) throws IOException { + + } }); + + } + + private void handleAsyncMetricsRequest(AsyncContext context) { + long start = System.currentTimeMillis(); + HttpServletResponse res = (HttpServletResponse) context.getResponse(); + try { + generateMetricsSynchronously(res); + } catch (Exception e) { + long end = System.currentTimeMillis(); + long time = end - start; + if (e instanceof EOFException) { + // NO STACKTRACE + log.error("Failed to send metrics, " + + "likely the client or this server closed " + + "the connection due to a timeout ({} ms elapsed): {}", time, e + ""); + } else { + log.error("Failed to generate prometheus stats, {} ms elapsed", time, e); + } + if (!res.isCommitted()) { + res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + } finally { + long end = System.currentTimeMillis(); + long time = end - start; + try { + context.complete(); + } catch (IllegalStateException e) { + // this happens when metricsServletTimeoutMs expires + // java.lang.IllegalStateException: AsyncContext completed and/or Request lifecycle recycled + log.error("Failed to generate prometheus stats, " + + "this is likely due to metricsServletTimeoutMs: {} ms elapsed: {}", time, e + ""); + } + } } - protected void generateMetrics(String cluster, ServletOutputStream outputStream) throws IOException { - PrometheusMetricsGeneratorUtils.generate(cluster, outputStream, metricsProviders); + private void generateMetricsSynchronously(HttpServletResponse res) throws IOException { + res.setStatus(HTTP_STATUS_OK_200); + res.setContentType("text/plain;charset=utf-8"); + PrometheusMetricsGeneratorUtils.generate(cluster, res.getOutputStream(), metricsProviders); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java deleted file mode 100644 index 08730189322ee..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java +++ /dev/null @@ -1,94 +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.broker.stats; - -import java.util.concurrent.atomic.AtomicReferenceArray; -import java.util.function.Function; - -public final class TimeWindow { - private final int interval; - private final int sampleCount; - private final AtomicReferenceArray> array; - - public TimeWindow(int sampleCount, int interval) { - this.sampleCount = sampleCount; - this.interval = interval; - this.array = new AtomicReferenceArray<>(sampleCount); - } - - /** - * return current time window data. - * - * @param function generate data. - * @return - */ - public synchronized WindowWrap current(Function function) { - long millis = System.currentTimeMillis(); - - if (millis < 0) { - return null; - } - int idx = calculateTimeIdx(millis); - long windowStart = calculateWindowStart(millis); - while (true) { - WindowWrap old = array.get(idx); - if (old == null) { - WindowWrap window = new WindowWrap<>(interval, windowStart, null); - if (array.compareAndSet(idx, null, window)) { - T value = null == function ? null : function.apply(null); - window.value(value); - return window; - } else { - Thread.yield(); - } - } else if (windowStart == old.start()) { - return old; - } else if (windowStart > old.start()) { - T value = null == function ? null : function.apply(old.value()); - old.value(value); - old.resetWindowStart(windowStart); - return old; - } else { - //it should never goes here - throw new IllegalStateException(); - } - } - } - - private int calculateTimeIdx(long timeMillis) { - long timeId = timeMillis / this.interval; - return (int) (timeId % sampleCount); - } - - private long calculateWindowStart(long timeMillis) { - return timeMillis - timeMillis % this.interval; - } - - public int sampleCount() { - return sampleCount; - } - - public int interval() { - return interval; - } - - public long currentWindowStart(long millis) { - return this.calculateWindowStart(millis); - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java deleted file mode 100644 index 12869b82921e5..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java +++ /dev/null @@ -1,56 +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.broker.stats; - -public final class WindowWrap { - private final long interval; - private long start; - private T value; - - public WindowWrap(long interval, long windowStart, T value) { - this.interval = interval; - this.start = windowStart; - this.value = value; - } - - public long interval() { - return this.interval; - } - - public long start() { - return this.start; - } - - public T value() { - return value; - } - - public void value(T value) { - this.value = value; - } - - public WindowWrap resetWindowStart(long startTime) { - this.start = startTime; - return this; - } - - public boolean isTimeInWindow(long timeMillis) { - return start <= timeMillis && timeMillis < start + interval; - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java new file mode 100644 index 0000000000000..b80e5747d8a5a --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java @@ -0,0 +1,68 @@ +/* + * 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.stats.prometheus; + +import static org.apache.pulsar.common.stats.JvmMetrics.getJvmDirectMemoryUsed; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.Gauge; +import io.prometheus.client.hotspot.DefaultExports; +import org.apache.pulsar.PulsarVersion; +import org.apache.pulsar.common.util.DirectMemoryUtils; + +public class MetricsExports { + private static boolean initialized = false; + + private MetricsExports() { + } + + public static synchronized void initialize() { + if (!initialized) { + DefaultExports.initialize(); + register(CollectorRegistry.defaultRegistry); + initialized = true; + } + } + + public static void register(CollectorRegistry registry) { + Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Gauge.Child() { + @Override + public double get() { + return getJvmDirectMemoryUsed(); + } + }).register(registry); + + Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Gauge.Child() { + @Override + public double get() { + return DirectMemoryUtils.jvmMaxDirectMemory(); + } + }).register(registry); + + // metric to export pulsar version info + Gauge.build("pulsar_version_info", "-") + .labelNames("version", "commit").create() + .setChild(new Gauge.Child() { + @Override + public double get() { + return 1.0; + } + }, PulsarVersion.getVersion(), PulsarVersion.getGitSha()) + .register(registry); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java index 93cbad4e19503..5a5a61404b87f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java @@ -42,7 +42,7 @@ void writeSample(String metricName, Number value, String... labelsAndValuesArray stream.write(metricName).write('{'); for (int i = 0; i < labelsAndValuesArray.length; i += 2) { String labelValue = labelsAndValuesArray[i + 1]; - if (labelValue != null) { + if (labelValue != null && labelValue.indexOf('"') > -1) { labelValue = labelValue.replace("\"", "\\\""); } stream.write(labelsAndValuesArray[i]).write("=\"").write(labelValue).write('\"'); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java index 124f0d3e54e4f..bbd09335c0a97 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java @@ -20,40 +20,39 @@ import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGeneratorUtils.generateSystemMetrics; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGeneratorUtils.getTypeStr; -import static org.apache.pulsar.common.stats.JvmMetrics.getJvmDirectMemoryUsed; import io.netty.buffer.ByteBuf; -import io.netty.buffer.UnpooledByteBufAllocator; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.PooledByteBufAllocator; import io.prometheus.client.Collector; -import io.prometheus.client.CollectorRegistry; -import io.prometheus.client.Gauge; -import io.prometheus.client.Gauge.Child; -import io.prometheus.client.hotspot.DefaultExports; +import java.io.BufferedOutputStream; import java.io.IOException; import java.io.OutputStream; -import java.io.StringWriter; +import java.io.OutputStreamWriter; import java.io.Writer; -import java.nio.ByteBuffer; +import java.nio.CharBuffer; +import java.nio.charset.StandardCharsets; +import java.time.Clock; import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; -import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.stats.TimeWindow; -import org.apache.pulsar.broker.stats.WindowWrap; import org.apache.pulsar.broker.stats.metrics.ManagedCursorMetrics; import org.apache.pulsar.broker.stats.metrics.ManagedLedgerCacheMetrics; import org.apache.pulsar.broker.stats.metrics.ManagedLedgerMetrics; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.common.util.DirectMemoryUtils; import org.apache.pulsar.common.util.SimpleTextOutputStream; -import org.eclipse.jetty.server.HttpOutput; /** * Generate metrics aggregated at the namespace level and optionally at a topic level and formats them out @@ -62,123 +61,80 @@ * href="https://prometheus.io/docs/instrumenting/exposition_formats/">Exposition Formats */ @Slf4j -public class PrometheusMetricsGenerator { - private static volatile TimeWindow timeWindow; - private static final int MAX_COMPONENTS = 64; - - static { - DefaultExports.initialize(); - - Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { - @Override - public double get() { - return getJvmDirectMemoryUsed(); - } - }).register(CollectorRegistry.defaultRegistry); - - Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { - @Override - public double get() { - return DirectMemoryUtils.jvmMaxDirectMemory(); - } - }).register(CollectorRegistry.defaultRegistry); - - // metric to export pulsar version info - Gauge.build("pulsar_version_info", "-") - .labelNames("version", "commit").create() - .setChild(new Child() { - @Override - public double get() { - return 1.0; - } - }, PulsarVersion.getVersion(), PulsarVersion.getGitSha()) - .register(CollectorRegistry.defaultRegistry); - } - - public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, OutputStream out) throws IOException { - generate(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, false, out, null); - } - - public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, - OutputStream out) throws IOException { - generate(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, out, null); - } - - public static synchronized void generate(PulsarService pulsar, boolean includeTopicMetrics, - boolean includeConsumerMetrics, boolean includeProducerMetrics, - boolean splitTopicAndPartitionIndexLabel, OutputStream out, - List metricsProviders) throws IOException { - ByteBuf buffer; - boolean exposeBufferMetrics = pulsar.getConfiguration().isMetricsBufferResponse(); +public class PrometheusMetricsGenerator implements AutoCloseable { + private static final int DEFAULT_INITIAL_BUFFER_SIZE = 1024 * 1024; // 1MB + private static final int MINIMUM_FOR_MAX_COMPONENTS = 64; + + private volatile MetricsBuffer metricsBuffer; + private static AtomicReferenceFieldUpdater metricsBufferFieldUpdater = + AtomicReferenceFieldUpdater.newUpdater(PrometheusMetricsGenerator.class, MetricsBuffer.class, + "metricsBuffer"); + private volatile boolean closed; + + public static class MetricsBuffer { + private final CompletableFuture bufferFuture; + private final long createTimeslot; + private final AtomicInteger refCnt = new AtomicInteger(2); + + MetricsBuffer(long timeslot) { + bufferFuture = new CompletableFuture<>(); + createTimeslot = timeslot; + } - if (!exposeBufferMetrics) { - buffer = generate0(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, metricsProviders); - } else { - if (null == timeWindow) { - int period = pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds(); - timeWindow = new TimeWindow<>(1, (int) TimeUnit.SECONDS.toMillis(period)); - } - WindowWrap window = timeWindow.current(oldBuf -> { - // release expired buffer, in case of memory leak - if (oldBuf != null && oldBuf.refCnt() > 0) { - oldBuf.release(); - log.debug("Cached metrics buffer released"); - } + public CompletableFuture getBufferFuture() { + return bufferFuture; + } - try { - ByteBuf buf = generate0(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, metricsProviders); - log.debug("Generated metrics buffer size {}", buf.readableBytes()); - return buf; - } catch (IOException e) { - log.error("Generate metrics failed", e); - //return empty buffer if exception happens - return PulsarByteBufAllocator.DEFAULT.heapBuffer(0); - } - }); + long getCreateTimeslot() { + return createTimeslot; + } - if (null == window || null == window.value()) { - return; - } - buffer = window.value(); - log.debug("Current window start {}, current cached buf size {}", window.start(), buffer.readableBytes()); + /** + * Retain the buffer. This is allowed, only when the buffer is not already released. + * + * @return true if the buffer is retained successfully, false otherwise. + */ + boolean retain() { + return refCnt.updateAndGet(x -> x > 0 ? x + 1 : x) > 0; } - try { - if (out instanceof HttpOutput) { - HttpOutput output = (HttpOutput) out; - //no mem_copy and memory allocations here - ByteBuffer[] buffers = buffer.nioBuffers(); - for (ByteBuffer buffer0 : buffers) { - output.write(buffer0); - } - } else { - //read data from buffer and write it to output stream, with no more heap buffer(byte[]) allocation. - //not modify buffer readIndex/writeIndex here. - int readIndex = buffer.readerIndex(); - int readableBytes = buffer.readableBytes(); - for (int i = 0; i < readableBytes; i++) { - out.write(buffer.getByte(readIndex + i)); - } - } - } finally { - if (!exposeBufferMetrics && buffer.refCnt() > 0) { - buffer.release(); - log.debug("Metrics buffer released."); + /** + * Release the buffer. + */ + public void release() { + int newValue = refCnt.decrementAndGet(); + if (newValue == 0) { + bufferFuture.whenComplete((byteBuf, throwable) -> { + if (byteBuf != null) { + byteBuf.release(); + } + }); } } } - private static ByteBuf generate0(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, - List metricsProviders) throws IOException { - //Use unpooled buffers here to avoid direct buffer usage increasing. - //when write out 200MB data, MAX_COMPONENTS = 64 needn't mem_copy. see: CompositeByteBuf#consolidateIfNeeded() - ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.compositeDirectBuffer(MAX_COMPONENTS); + private final PulsarService pulsar; + private final boolean includeTopicMetrics; + private final boolean includeConsumerMetrics; + private final boolean includeProducerMetrics; + private final boolean splitTopicAndPartitionIndexLabel; + private final Clock clock; + + private volatile int initialBufferSize = DEFAULT_INITIAL_BUFFER_SIZE; + + public PrometheusMetricsGenerator(PulsarService pulsar, boolean includeTopicMetrics, + boolean includeConsumerMetrics, boolean includeProducerMetrics, + boolean splitTopicAndPartitionIndexLabel, Clock clock) { + this.pulsar = pulsar; + this.includeTopicMetrics = includeTopicMetrics; + this.includeConsumerMetrics = includeConsumerMetrics; + this.includeProducerMetrics = includeProducerMetrics; + this.splitTopicAndPartitionIndexLabel = splitTopicAndPartitionIndexLabel; + this.clock = clock; + } + + private ByteBuf generate0(List metricsProviders) { + ByteBuf buf = allocateMultipartCompositeDirectBuffer(); boolean exceptionHappens = false; //Used in namespace/topic and transaction aggregators as share metric names PrometheusMetricStreams metricStreams = new PrometheusMetricStreams(); @@ -220,10 +176,34 @@ private static ByteBuf generate0(PulsarService pulsar, boolean includeTopicMetri //if exception happens, release buffer if (exceptionHappens) { buf.release(); + } else { + // for the next time, the initial buffer size will be suggested by the last buffer size + initialBufferSize = Math.max(DEFAULT_INITIAL_BUFFER_SIZE, buf.readableBytes()); } } } + private ByteBuf allocateMultipartCompositeDirectBuffer() { + // use composite buffer with pre-allocated buffers to ensure that the pooled allocator can be used + // for allocating the buffers + ByteBufAllocator byteBufAllocator = PulsarByteBufAllocator.DEFAULT; + int chunkSize; + if (byteBufAllocator instanceof PooledByteBufAllocator) { + PooledByteBufAllocator pooledByteBufAllocator = (PooledByteBufAllocator) byteBufAllocator; + chunkSize = Math.max(pooledByteBufAllocator.metric().chunkSize(), DEFAULT_INITIAL_BUFFER_SIZE); + } else { + chunkSize = DEFAULT_INITIAL_BUFFER_SIZE; + } + CompositeByteBuf buf = byteBufAllocator.compositeDirectBuffer( + Math.max(MINIMUM_FOR_MAX_COMPONENTS, (initialBufferSize / chunkSize) + 1)); + int totalLen = 0; + while (totalLen < initialBufferSize) { + totalLen += chunkSize; + buf.addComponent(false, byteBufAllocator.directBuffer(chunkSize)); + } + return buf; + } + private static void generateBrokerBasicMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { String clusterName = pulsar.getConfiguration().getClusterName(); // generate managedLedgerCache metrics @@ -269,12 +249,13 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, String name = key.substring(0, nameIndex); value = key.substring(nameIndex + 1); if (!names.contains(name)) { - stream.write("# TYPE ").write(name.replace("brk_", "pulsar_")).write(' ') - .write(getTypeStr(metricType)).write("\n"); + stream.write("# TYPE "); + writeNameReplacingBrkPrefix(stream, name); + stream.write(' ').write(getTypeStr(metricType)).write("\n"); names.add(name); } - stream.write(name.replace("brk_", "pulsar_")) - .write("{cluster=\"").write(cluster).write('"'); + writeNameReplacingBrkPrefix(stream, name); + stream.write("{cluster=\"").write(cluster).write('"'); } catch (Exception e) { continue; } @@ -283,12 +264,13 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, String name = entry.getKey(); if (!names.contains(name)) { - stream.write("# TYPE ").write(entry.getKey().replace("brk_", "pulsar_")).write(' ') - .write(getTypeStr(metricType)).write('\n'); + stream.write("# TYPE "); + writeNameReplacingBrkPrefix(stream, entry.getKey()); + stream.write(' ').write(getTypeStr(metricType)).write('\n'); names.add(name); } - stream.write(name.replace("brk_", "pulsar_")) - .write("{cluster=\"").write(cluster).write('"'); + writeNameReplacingBrkPrefix(stream, name); + stream.write("{cluster=\"").write(cluster).write('"'); } //to avoid quantile label duplicated @@ -308,18 +290,98 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, } } + private static SimpleTextOutputStream writeNameReplacingBrkPrefix(SimpleTextOutputStream stream, String name) { + if (name.startsWith("brk_")) { + return stream.write("pulsar_").write(CharBuffer.wrap(name).position("brk_".length())); + } else { + return stream.write(name); + } + } + private static void generateManagedLedgerBookieClientMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { StatsProvider statsProvider = pulsar.getManagedLedgerClientFactory().getStatsProvider(); if (statsProvider instanceof NullStatsProvider) { return; } - try { - Writer writer = new StringWriter(); + try (Writer writer = new OutputStreamWriter(new BufferedOutputStream(new OutputStream() { + @Override + public void write(int b) throws IOException { + stream.writeByte(b); + } + + @Override + public void write(byte b[], int off, int len) throws IOException { + stream.write(b, off, len); + } + }), StandardCharsets.UTF_8)) { statsProvider.writeAllMetrics(writer); - stream.write(writer.toString()); } catch (IOException e) { - // nop + log.error("Failed to write managed ledger bookie client metrics", e); + } + } + + public MetricsBuffer renderToBuffer(Executor executor, List metricsProviders) { + boolean cacheMetricsResponse = pulsar.getConfiguration().isMetricsBufferResponse(); + while (!closed && !Thread.currentThread().isInterrupted()) { + long currentTimeSlot = cacheMetricsResponse ? calculateCurrentTimeSlot() : 0; + MetricsBuffer currentMetricsBuffer = metricsBuffer; + if (currentMetricsBuffer == null || currentMetricsBuffer.getBufferFuture().isCompletedExceptionally() + || (currentMetricsBuffer.getBufferFuture().isDone() + && (currentMetricsBuffer.getCreateTimeslot() != 0 + && currentTimeSlot > currentMetricsBuffer.getCreateTimeslot()))) { + MetricsBuffer newMetricsBuffer = new MetricsBuffer(currentTimeSlot); + if (metricsBufferFieldUpdater.compareAndSet(this, currentMetricsBuffer, newMetricsBuffer)) { + if (currentMetricsBuffer != null) { + currentMetricsBuffer.release(); + } + CompletableFuture bufferFuture = newMetricsBuffer.getBufferFuture(); + executor.execute(() -> { + try { + bufferFuture.complete(generate0(metricsProviders)); + } catch (Exception e) { + bufferFuture.completeExceptionally(e); + } finally { + if (currentTimeSlot == 0) { + // if the buffer is not cached, release it after the future is completed + metricsBufferFieldUpdater.compareAndSet(this, newMetricsBuffer, null); + newMetricsBuffer.release(); + } + } + }); + // no need to retain before returning since the new buffer starts with refCnt 2 + return newMetricsBuffer; + } else { + currentMetricsBuffer = metricsBuffer; + } + } + // retain the buffer before returning + // if the buffer is already released, retaining won't succeed, retry in that case + if (currentMetricsBuffer != null && currentMetricsBuffer.retain()) { + return currentMetricsBuffer; + } + } + return null; + } + + /** + * Calculate the current time slot based on the current time. + * This is to ensure that cached metrics are refreshed consistently at a fixed interval regardless of the request + * time. + */ + private long calculateCurrentTimeSlot() { + long cacheTimeoutMillis = + TimeUnit.SECONDS.toMillis(Math.max(1, pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds())); + long now = clock.millis(); + return now / cacheTimeoutMillis; + } + + @Override + public void close() { + closed = true; + MetricsBuffer buffer = metricsBufferFieldUpdater.getAndSet(this, null); + if (buffer != null) { + buffer.release(); } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java index 42bd2652883b6..7fcc74e965c24 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java @@ -18,34 +18,142 @@ */ package org.apache.pulsar.broker.stats.prometheus; +import java.io.EOFException; import java.io.IOException; +import java.nio.ByteBuffer; +import java.time.Clock; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.servlet.AsyncContext; +import javax.servlet.AsyncEvent; +import javax.servlet.AsyncListener; import javax.servlet.ServletOutputStream; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; +import org.eclipse.jetty.server.HttpOutput; +@Slf4j public class PulsarPrometheusMetricsServlet extends PrometheusMetricsServlet { - private static final long serialVersionUID = 1L; + private static final int EXECUTOR_MAX_THREADS = 4; - private final PulsarService pulsar; - private final boolean shouldExportTopicMetrics; - private final boolean shouldExportConsumerMetrics; - private final boolean shouldExportProducerMetrics; - private final boolean splitTopicAndPartitionLabel; + private final PrometheusMetricsGenerator prometheusMetricsGenerator; public PulsarPrometheusMetricsServlet(PulsarService pulsar, boolean includeTopicMetrics, - boolean includeConsumerMetrics, boolean shouldExportProducerMetrics, + boolean includeConsumerMetrics, boolean includeProducerMetrics, boolean splitTopicAndPartitionLabel) { - super(pulsar.getConfiguration().getMetricsServletTimeoutMs(), pulsar.getConfiguration().getClusterName()); - this.pulsar = pulsar; - this.shouldExportTopicMetrics = includeTopicMetrics; - this.shouldExportConsumerMetrics = includeConsumerMetrics; - this.shouldExportProducerMetrics = shouldExportProducerMetrics; - this.splitTopicAndPartitionLabel = splitTopicAndPartitionLabel; + super(pulsar.getConfiguration().getMetricsServletTimeoutMs(), pulsar.getConfiguration().getClusterName(), + EXECUTOR_MAX_THREADS); + MetricsExports.initialize(); + prometheusMetricsGenerator = + new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, + includeProducerMetrics, splitTopicAndPartitionLabel, Clock.systemUTC()); } + @Override - protected void generateMetrics(String cluster, ServletOutputStream outputStream) throws IOException { - PrometheusMetricsGenerator.generate(pulsar, shouldExportTopicMetrics, shouldExportConsumerMetrics, - shouldExportProducerMetrics, splitTopicAndPartitionLabel, outputStream, metricsProviders); + public void destroy() { + super.destroy(); + prometheusMetricsGenerator.close(); + } + + protected void doGet(HttpServletRequest request, HttpServletResponse response) { + AsyncContext context = request.startAsync(); + // set hard timeout to 2 * timeout + if (metricsServletTimeoutMs > 0) { + context.setTimeout(metricsServletTimeoutMs * 2); + } + long startNanos = System.nanoTime(); + AtomicBoolean skipWritingResponse = new AtomicBoolean(false); + context.addListener(new AsyncListener() { + @Override + public void onComplete(AsyncEvent event) throws IOException { + } + + @Override + public void onTimeout(AsyncEvent event) throws IOException { + log.warn("Prometheus metrics request timed out"); + skipWritingResponse.set(true); + HttpServletResponse res = (HttpServletResponse) context.getResponse(); + if (!res.isCommitted()) { + res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + context.complete(); + } + + @Override + public void onError(AsyncEvent event) throws IOException { + skipWritingResponse.set(true); + } + + @Override + public void onStartAsync(AsyncEvent event) throws IOException { + } + }); + PrometheusMetricsGenerator.MetricsBuffer metricsBuffer = + prometheusMetricsGenerator.renderToBuffer(executor, metricsProviders); + if (metricsBuffer == null) { + log.info("Service is closing, skip writing metrics."); + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + context.complete(); + return; + } + metricsBuffer.getBufferFuture().whenComplete((buffer, ex) -> executor.execute(() -> { + try { + long elapsedNanos = System.nanoTime() - startNanos; + // check if the request has been timed out, implement a soft timeout + // so that response writing can continue to up to 2 * timeout + if (metricsServletTimeoutMs > 0 && elapsedNanos > TimeUnit.MILLISECONDS.toNanos( + metricsServletTimeoutMs)) { + log.warn("Prometheus metrics request was too long in queue ({}ms). Skipping sending metrics.", + TimeUnit.NANOSECONDS.toMillis(elapsedNanos)); + if (!response.isCommitted() && !skipWritingResponse.get()) { + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + return; + } + if (skipWritingResponse.get()) { + log.warn("Response has timed or failed, skip writing metrics."); + return; + } + if (response.isCommitted()) { + log.warn("Response is already committed, cannot write metrics"); + return; + } + if (ex != null) { + log.error("Failed to generate metrics", ex); + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + return; + } + if (buffer == null) { + log.error("Failed to generate metrics, buffer is null"); + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } else { + response.setStatus(HTTP_STATUS_OK_200); + response.setContentType("text/plain;charset=utf-8"); + ServletOutputStream outputStream = response.getOutputStream(); + if (outputStream instanceof HttpOutput) { + HttpOutput output = (HttpOutput) outputStream; + for (ByteBuffer nioBuffer : buffer.nioBuffers()) { + output.write(nioBuffer); + } + } else { + int length = buffer.readableBytes(); + if (length > 0) { + buffer.duplicate().readBytes(outputStream, length); + } + } + } + } catch (EOFException e) { + log.error("Failed to write metrics to response due to EOFException"); + } catch (IOException e) { + log.error("Failed to write metrics to response", e); + } finally { + metricsBuffer.release(); + context.complete(); + } + })); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java index 4be006423f509..27288291d2969 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java @@ -507,7 +507,9 @@ private static void writeConsumerMetric(PrometheusMetricStreams stream, String m static void writeTopicMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel, String... extraLabelsAndValues) { - String[] labelsAndValues = new String[splitTopicAndPartitionIndexLabel ? 8 : 6]; + int baseLabelCount = splitTopicAndPartitionIndexLabel ? 8 : 6; + String[] labelsAndValues = + new String[baseLabelCount + (extraLabelsAndValues != null ? extraLabelsAndValues.length : 0)]; labelsAndValues[0] = "cluster"; labelsAndValues[1] = cluster; labelsAndValues[2] = "namespace"; @@ -527,7 +529,11 @@ static void writeTopicMetric(PrometheusMetricStreams stream, String metricName, } else { labelsAndValues[5] = topic; } - String[] labels = ArrayUtils.addAll(labelsAndValues, extraLabelsAndValues); - stream.writeSample(metricName, value, labels); + if (extraLabelsAndValues != null) { + for (int i = 0; i < extraLabelsAndValues.length; i++) { + labelsAndValues[baseLabelCount + i] = extraLabelsAndValues[i]; + } + } + stream.writeSample(metricName, value, labelsAndValues); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java new file mode 100644 index 0000000000000..fcc3b6aa88fb4 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java @@ -0,0 +1,84 @@ +/* + * 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; + +import com.google.common.util.concurrent.MoreExecutors; +import io.netty.buffer.ByteBuf; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.time.Clock; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; +import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; +import org.eclipse.jetty.server.HttpOutput; + +public class PrometheusMetricsTestUtil { + public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, + boolean includeProducerMetrics, OutputStream out) throws IOException { + generate(new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, + includeProducerMetrics, false, Clock.systemUTC()), out, null); + } + + public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, + boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, + OutputStream out) throws IOException { + generate(new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, + includeProducerMetrics, splitTopicAndPartitionIndexLabel, Clock.systemUTC()), out, null); + } + + public static void generate(PrometheusMetricsGenerator metricsGenerator, OutputStream out, + List metricsProviders) throws IOException { + PrometheusMetricsGenerator.MetricsBuffer metricsBuffer = + metricsGenerator.renderToBuffer(MoreExecutors.directExecutor(), metricsProviders); + try { + ByteBuf buffer = null; + try { + buffer = metricsBuffer.getBufferFuture().get(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException(e); + } catch (ExecutionException | TimeoutException e) { + throw new IOException(e); + } + if (buffer == null) { + return; + } + if (out instanceof HttpOutput) { + HttpOutput output = (HttpOutput) out; + ByteBuffer[] nioBuffers = buffer.nioBuffers(); + for (ByteBuffer nioBuffer : nioBuffers) { + output.write(nioBuffer); + } + } else { + int length = buffer.readableBytes(); + if (length > 0) { + buffer.duplicate().readBytes(out, length); + } + } + } finally { + metricsBuffer.release(); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index 8be0aa4bc7dbd..ff8e418c024a0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -40,10 +40,10 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.service.Dispatcher; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -218,7 +218,7 @@ public void testBucketDelayedIndexMetrics() throws Exception { Thread.sleep(2000); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, true, true, output); + PrometheusMetricsTestUtil.generate(pulsar, true, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); Multimap metricsMap = parseMetrics(metricsStr); @@ -304,7 +304,7 @@ public void testBucketDelayedIndexMetrics() throws Exception { assertEquals(opLatencyMetricsSum.intValue(), opLatencyTopicMetrics.get().value); ByteArrayOutputStream namespaceOutput = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, true, true, namespaceOutput); + PrometheusMetricsTestUtil.generate(pulsar, false, true, true, namespaceOutput); Multimap namespaceMetricsMap = parseMetrics(namespaceOutput.toString(StandardCharsets.UTF_8)); Optional namespaceMetric = diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index c214634e6ed32..44d24668cc381 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -65,11 +65,11 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -367,7 +367,7 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex latch.await(10, TimeUnit.SECONDS); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output); + PrometheusMetricsTestUtil.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); Multimap metricsMap = parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java index a520b8c241bd1..3a4016eb79c21 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java @@ -43,11 +43,11 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; import org.apache.pulsar.common.naming.TopicName; @@ -121,7 +121,7 @@ public void testSchemaRegistryMetrics() throws Exception { deleteSchema(schemaId, version(1)); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, output); output.flush(); String metricsStr = output.toString(StandardCharsets.UTF_8); Multimap metrics = parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index eb4500c13667a..512a5cfcab661 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -45,6 +45,7 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; @@ -52,7 +53,6 @@ import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterProducerTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -335,7 +335,7 @@ private void testMessageAckRateMetric(String topicName, boolean exposeTopicLevel consumer2.updateRates(); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output); + PrometheusMetricsTestUtil.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricStr = output.toString(StandardCharsets.UTF_8); Multimap metricsMap = parseMetrics(metricStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java index 15f41365da8d1..726bde3f3d0a9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java @@ -30,10 +30,10 @@ import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; import org.apache.pulsar.broker.service.BrokerTestBase; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; @@ -101,7 +101,7 @@ public void testMetadataStoreStats() throws Exception { } ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); Multimap metricsMap = parseMetrics(metricsStr); @@ -191,7 +191,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { } ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); Multimap metricsMap = parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index d3891931496c5..1fe0e99b49874 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -21,7 +21,10 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -35,6 +38,7 @@ import java.math.BigDecimal; import java.math.RoundingMode; import java.nio.charset.StandardCharsets; +import java.time.Clock; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -51,6 +55,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -61,6 +66,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; @@ -85,7 +91,6 @@ import org.apache.pulsar.compaction.PulsarCompactionServiceFactory; import org.apache.zookeeper.CreateMode; import org.awaitility.Awaitility; -import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -155,7 +160,7 @@ public void testPublishRateLimitedTimes() throws Exception { }); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_publish_rate_limit_times")); @@ -185,7 +190,7 @@ public void testPublishRateLimitedTimes() throws Exception { @Cleanup ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut2); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); Multimap metrics2 = parseMetrics(metricsStr2); assertTrue(metrics2.containsKey("pulsar_publish_rate_limit_times")); @@ -217,7 +222,7 @@ public void testMetricsTopicCount() throws Exception { Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Collection metric = metrics.get("pulsar_topics_count"); @@ -254,7 +259,7 @@ public void testMetricsAvgMsgSize2() throws Exception { producerInServer.getStats().msgThroughputIn = 100; @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, true, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, true, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_average_msg_size")); @@ -297,7 +302,7 @@ public void testPerTopicStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -395,7 +400,7 @@ public void testPerBrokerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -504,7 +509,7 @@ public void testPerTopicStatsReconnect() throws Exception { c2.close(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -582,7 +587,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception // includeTopicMetric true ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -614,7 +619,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception // includeTopicMetric false ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut2); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); Multimap metrics2 = parseMetrics(metricsStr2); @@ -698,7 +703,7 @@ public void testPerTopicExpiredStat() throws Exception { Awaitility.await().until(() -> sub2.getExpiredMessageRate() != 0.0); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); // There should be 2 metrics with different tags for each topic @@ -780,15 +785,15 @@ public void testBundlesMetrics() throws Exception { for (var latencyMetric : UnloadManager.LatencyMetric.values()) { var serviceUnit = "serviceUnit"; var brokerLookupAddress = "lookupAddress"; - var serviceUnitStateData = Mockito.mock(ServiceUnitStateData.class); - Mockito.when(serviceUnitStateData.sourceBroker()).thenReturn(brokerLookupAddress); - Mockito.when(serviceUnitStateData.dstBroker()).thenReturn(brokerLookupAddress); + var serviceUnitStateData = mock(ServiceUnitStateData.class); + when(serviceUnitStateData.sourceBroker()).thenReturn(brokerLookupAddress); + when(serviceUnitStateData.dstBroker()).thenReturn(brokerLookupAddress); latencyMetric.beginMeasurement(serviceUnit, brokerLookupAddress, serviceUnitStateData); latencyMetric.endMeasurement(serviceUnit); } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_bundle_msg_rate_in")); @@ -838,7 +843,7 @@ public void testNonPersistentSubMetrics() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_subscription_back_log")); @@ -885,7 +890,7 @@ public void testPerNamespaceStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -958,7 +963,7 @@ public void testPerProducerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, true, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, true, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1026,7 +1031,7 @@ public void testPerConsumerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, true, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, true, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1113,7 +1118,7 @@ public void testDuplicateMetricTypeDefinitions() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Map typeDefs = new HashMap<>(); @@ -1217,7 +1222,7 @@ public void testManagedLedgerCacheStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1253,7 +1258,7 @@ public void testManagedLedgerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1331,7 +1336,7 @@ public void testManagedLedgerBookieClientStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1412,7 +1417,7 @@ public String getCommandData() { }); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_authentication_success_total"); @@ -1473,7 +1478,7 @@ public String getCommandData() { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_expired_token_total"); @@ -1514,7 +1519,7 @@ public String getCommandData() { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Metric countMetric = ((List) metrics.get("pulsar_expiring_token_minutes_count")).get(0); @@ -1588,7 +1593,7 @@ public void testManagedCursorPersistStats() throws Exception { // enable ExposeManagedCursorMetricsInPrometheus pulsar.getConfiguration().setExposeManagedCursorMetricsInPrometheus(true); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1601,7 +1606,7 @@ public void testManagedCursorPersistStats() throws Exception { // disable ExposeManagedCursorMetricsInPrometheus pulsar.getConfiguration().setExposeManagedCursorMetricsInPrometheus(false); ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut2); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); Multimap metrics2 = parseMetrics(metricsStr2); List cm2 = (List) metrics2.get("pulsar_ml_cursor_persistLedgerSucceed"); @@ -1620,7 +1625,7 @@ public void testBrokerConnection() throws Exception { .create(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_connection_created_total_count"); @@ -1637,7 +1642,7 @@ public void testBrokerConnection() throws Exception { pulsarClient.close(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); @@ -1660,7 +1665,7 @@ public void testBrokerConnection() throws Exception { pulsarClient.close(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); @@ -1704,7 +1709,7 @@ public void testCompaction() throws Exception { .messageRoutingMode(MessageRoutingMode.SinglePartition) .create(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_compaction_removed_event_count"); @@ -1739,7 +1744,7 @@ public void testCompaction() throws Exception { Compactor compactor = ((PulsarCompactionServiceFactory)pulsar.getCompactionServiceFactory()).getCompactor(); compactor.compact(topicName).get(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); cm = (List) metrics.get("pulsar_compaction_removed_event_count"); @@ -1772,31 +1777,36 @@ public void testCompaction() throws Exception { @Test public void testMetricsWithCache() throws Throwable { - ServiceConfiguration configuration = Mockito.mock(ServiceConfiguration.class); - Mockito.when(configuration.getManagedLedgerStatsPeriodSeconds()).thenReturn(2); - Mockito.when(configuration.isMetricsBufferResponse()).thenReturn(true); - Mockito.when(configuration.getClusterName()).thenReturn(configClusterName); - Mockito.when(pulsar.getConfiguration()).thenReturn(configuration); + ServiceConfiguration configuration = pulsar.getConfiguration(); + configuration.setManagedLedgerStatsPeriodSeconds(2); + configuration.setMetricsBufferResponse(true); + configuration.setClusterName(configClusterName); - int period = pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds(); - TimeWindow timeWindow = new TimeWindow<>(2, (int) TimeUnit.SECONDS.toMillis(period)); + // create a mock clock to control the time + AtomicLong currentTimeMillis = new AtomicLong(System.currentTimeMillis()); + Clock clock = mock(); + when(clock.millis()).thenAnswer(invocation -> currentTimeMillis.get()); + PrometheusMetricsGenerator prometheusMetricsGenerator = + new PrometheusMetricsGenerator(pulsar, true, false, false, + false, clock); + + String previousMetrics = null; for (int a = 0; a < 4; a++) { - long start = System.currentTimeMillis(); ByteArrayOutputStream statsOut1 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, false, statsOut1, null); + PrometheusMetricsTestUtil.generate(prometheusMetricsGenerator, statsOut1, null); ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, false, statsOut2, null); - long end = System.currentTimeMillis(); - - if (timeWindow.currentWindowStart(start) == timeWindow.currentWindowStart(end)) { - String metricsStr1 = statsOut1.toString(); - String metricsStr2 = statsOut2.toString(); - assertEquals(metricsStr1, metricsStr2); - Multimap metrics = parseMetrics(metricsStr1); - } + PrometheusMetricsTestUtil.generate(prometheusMetricsGenerator, statsOut2, null); + + String metricsStr1 = statsOut1.toString(); + String metricsStr2 = statsOut2.toString(); + assertTrue(metricsStr1.length() > 1000); + assertEquals(metricsStr1, metricsStr2); + assertNotEquals(metricsStr1, previousMetrics); + previousMetrics = metricsStr1; - Thread.sleep(TimeUnit.SECONDS.toMillis(period / 2)); + // move time forward + currentTimeMillis.addAndGet(TimeUnit.SECONDS.toMillis(2)); } } @@ -1824,7 +1834,7 @@ public void testSplitTopicAndPartitionLabel() throws Exception { .subscribe(); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, true, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, true, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Collection metric = metrics.get("pulsar_consumers_count"); @@ -1860,7 +1870,7 @@ public void testMetricsGroupedByTypeDefinitions() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Pattern typePattern = Pattern.compile("^#\\s+TYPE\\s+(\\w+)\\s+(\\w+)"); @@ -1920,7 +1930,7 @@ public void testEscapeLabelValue() throws Exception { .subscribe(); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); final List subCountLines = metricsStr.lines() diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java index e39860274d12f..3e71d8f211101 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java @@ -31,13 +31,13 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.EntryFilterSupport; import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -233,7 +233,7 @@ public void testSubscriptionStats(final String topic, final String subName, bool } ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, enableTopicStats, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, enableTopicStats, false, false, output); String metricsStr = output.toString(); Multimap metrics = parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java deleted file mode 100644 index 89528c1965397..0000000000000 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java +++ /dev/null @@ -1,83 +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.broker.stats; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotNull; -import org.testng.annotations.Test; - -public class TimeWindowTest { - - @Test - public void windowTest() throws Exception { - int intervalInMs = 1000; - int sampleCount = 2; - TimeWindow timeWindow = new TimeWindow<>(sampleCount, intervalInMs); - - WindowWrap expect1 = timeWindow.current(oldValue -> 1); - WindowWrap expect2 = timeWindow.current(oldValue -> null); - assertNotNull(expect1); - assertNotNull(expect2); - - if (expect1.start() == expect2.start()) { - assertEquals((int) expect1.value(), 1); - assertEquals(expect1, expect2); - assertEquals(expect1.value(), expect2.value()); - } - - Thread.sleep(intervalInMs); - - WindowWrap expect3 = timeWindow.current(oldValue -> 2); - WindowWrap expect4 = timeWindow.current(oldValue -> null); - assertNotNull(expect3); - assertNotNull(expect4); - - if (expect3.start() == expect4.start()) { - assertEquals((int) expect3.value(), 2); - assertEquals(expect3, expect4); - assertEquals(expect3.value(), expect4.value()); - } - - Thread.sleep(intervalInMs); - - WindowWrap expect5 = timeWindow.current(oldValue -> 3); - WindowWrap expect6 = timeWindow.current(oldValue -> null); - assertNotNull(expect5); - assertNotNull(expect6); - - if (expect5.start() == expect6.start()) { - assertEquals((int) expect5.value(), 3); - assertEquals(expect5, expect6); - assertEquals(expect5.value(), expect6.value()); - } - - Thread.sleep(intervalInMs); - - WindowWrap expect7 = timeWindow.current(oldValue -> 4); - WindowWrap expect8 = timeWindow.current(oldValue -> null); - assertNotNull(expect7); - assertNotNull(expect8); - - if (expect7.start() == expect8.start()) { - assertEquals((int) expect7.value(), 4); - assertEquals(expect7, expect8); - assertEquals(expect7.value(), expect8.value()); - } - } -} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java index 723a493eca1df..8d5cb9dc39148 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java @@ -38,9 +38,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerTestBase; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -118,7 +118,7 @@ public void testTransactionCoordinatorMetrics() throws Exception { pulsar.getTransactionMetadataStoreService().getStores() .get(transactionCoordinatorIDTwo).newTransaction(timeout, null).get(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Collection metric = metrics.get("pulsar_txn_active_count"); @@ -186,7 +186,7 @@ public void testTransactionCoordinatorRateMetrics() throws Exception { pulsar.getBrokerService().updateRates(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -216,7 +216,7 @@ public void testTransactionCoordinatorRateMetrics() throws Exception { }); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); @@ -272,7 +272,7 @@ public void testManagedLedgerMetrics() throws Exception { producer.send("hello pulsar".getBytes()); consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -290,7 +290,7 @@ public void testManagedLedgerMetrics() throws Exception { checkManagedLedgerMetrics(MLTransactionLogImpl.TRANSACTION_SUBSCRIPTION_NAME, 126, metric); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); metric = metrics.get("pulsar_storage_size"); @@ -334,7 +334,7 @@ public void testManagedLedgerMetricsWhenPendingAckNotInit() throws Exception { producer.send("hello pulsar".getBytes()); consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -359,7 +359,7 @@ public void testManagedLedgerMetricsWhenPendingAckNotInit() throws Exception { consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); metric = metrics.get("pulsar_storage_size"); @@ -393,7 +393,7 @@ public void testDuplicateMetricTypeDefinitions() throws Exception { .send(); } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Map typeDefs = new HashMap<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java index be036a0cf590b..1c3de777e9349 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java @@ -46,9 +46,9 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferHandlerImpl; @@ -229,7 +229,7 @@ public void testTransactionBufferMetrics() throws Exception { @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsarServiceList.get(0), true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metricsMap = parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java index 6c24b6b3f0151..db9daf56104c9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java @@ -46,12 +46,12 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; @@ -255,7 +255,7 @@ public void testPendingAckMetrics() throws Exception { @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsarServiceList.get(0), true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metricsMap = parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index d2b59ed0e4997..17588a7ecac8b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -55,9 +55,9 @@ import javax.net.ssl.TrustManager; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; @@ -107,7 +107,7 @@ public class WebServiceTest { public void testWebExecutorMetrics() throws Exception { setupEnv(true, false, false, false, -1, false); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java index c8c639606aa3e..9bf6302f50f02 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java @@ -20,6 +20,7 @@ import io.netty.buffer.ByteBuf; import io.netty.util.CharsetUtil; +import java.nio.CharBuffer; /** * Format strings and numbers into a ByteBuf without any memory allocation. @@ -28,6 +29,7 @@ public class SimpleTextOutputStream { private final ByteBuf buffer; private static final char[] hexChars = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'}; + private final CharBuffer singleCharBuffer = CharBuffer.allocate(1); public SimpleTextOutputStream(ByteBuf buffer) { this.buffer = buffer; @@ -44,11 +46,17 @@ public SimpleTextOutputStream write(byte[] a, int offset, int len) { } public SimpleTextOutputStream write(char c) { - write(String.valueOf(c)); + // In UTF-8, any character from U+0000 to U+007F is encoded in one byte + if (c <= '\u007F') { + buffer.writeByte((byte) c); + return this; + } + singleCharBuffer.put(0, c); + buffer.writeCharSequence(singleCharBuffer, CharsetUtil.UTF_8); return this; } - public SimpleTextOutputStream write(String s) { + public SimpleTextOutputStream write(CharSequence s) { if (s == null) { return this; } @@ -136,4 +144,8 @@ public void write(ByteBuf byteBuf) { public ByteBuf getBuffer() { return buffer; } + + public void writeByte(int b) { + buffer.writeByte(b); + } } 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 db2969e3c3920..39c8fb5e086fd 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 @@ -392,6 +392,12 @@ public class ProxyConfiguration implements PulsarConfiguration { ) private boolean authenticateMetricsEndpoint = true; + @FieldContext( + category = CATEGORY_HTTP, + doc = "Time in milliseconds that metrics endpoint would time out. Default is 30s.\n" + + " Set it to 0 to disable timeout." + ) + private long metricsServletTimeoutMs = 30000; @FieldContext( category = CATEGORY_SASL_AUTH, 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 61b00871cecdb..ea9e4ebfaa9b8 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 @@ -299,7 +299,8 @@ public void start() throws Exception { } private synchronized void createMetricsServlet() { - this.metricsServlet = new PrometheusMetricsServlet(-1L, proxyConfig.getClusterName()); + this.metricsServlet = + new PrometheusMetricsServlet(proxyConfig.getMetricsServletTimeoutMs(), proxyConfig.getClusterName()); if (pendingMetricsProviders != null) { pendingMetricsProviders.forEach(provider -> metricsServlet.addRawMetricsProvider(provider)); this.pendingMetricsProviders = null; diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index 72d54601995f1..50a8e3ab7d753 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -24,6 +24,7 @@ import static org.apache.commons.lang3.StringUtils.isNotBlank; import static org.apache.pulsar.common.stats.JvmMetrics.getJvmDirectMemoryUsed; import com.google.common.annotations.VisibleForTesting; +import io.prometheus.client.Collector; import io.prometheus.client.CollectorRegistry; import io.prometheus.client.Gauge; import io.prometheus.client.Gauge.Child; @@ -236,21 +237,36 @@ public void start() throws Exception { if (!metricsInitialized) { // Setup metrics DefaultExports.initialize(); + CollectorRegistry registry = CollectorRegistry.defaultRegistry; // Report direct memory from Netty counters - Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { - @Override - public double get() { - return getJvmDirectMemoryUsed(); - } - }).register(CollectorRegistry.defaultRegistry); + Collector jvmMemoryDirectBytesUsed = + Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { + @Override + public double get() { + return getJvmDirectMemoryUsed(); + } + }); + try { + registry.register(jvmMemoryDirectBytesUsed); + } catch (IllegalArgumentException e) { + // workaround issue in tests where the metric is already registered + log.debug("Failed to register jvm_memory_direct_bytes_used metric: {}", e.getMessage()); + } - Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { - @Override - public double get() { - return DirectMemoryUtils.jvmMaxDirectMemory(); - } - }).register(CollectorRegistry.defaultRegistry); + Collector jvmMemoryDirectBytesMax = + Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { + @Override + public double get() { + return DirectMemoryUtils.jvmMaxDirectMemory(); + } + }); + try { + registry.register(jvmMemoryDirectBytesMax); + } catch (IllegalArgumentException e) { + // workaround issue in tests where the metric is already registered + log.debug("Failed to register jvm_memory_direct_bytes_max metric: {}", e.getMessage()); + } metricsInitialized = true; } From 837f8bca7ddbbad4354f9a89e36fcd6aea1be85c Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 15 Apr 2024 00:13:49 +0800 Subject: [PATCH 070/580] [fix] [broker] Prevent long deduplication cursor backlog so that topic loading wouldn't timeout (#22479) --- .../pulsar/broker/service/BrokerService.java | 4 +- .../persistent/MessageDeduplication.java | 18 +- .../service/persistent/PersistentTopic.java | 2 +- .../DeduplicationDisabledBrokerLevelTest.java | 161 ++++++++++++++++++ 4 files changed, 177 insertions(+), 8 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java 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 b4d0f38b4a4dc..2687532693a45 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 @@ -631,8 +631,10 @@ protected void startStatsUpdater(int statsUpdateInitialDelayInSecs, int statsUpd } protected void startDeduplicationSnapshotMonitor() { + // We do not know whether users will enable deduplication on namespace level/topic level or not, so keep this + // scheduled task runs. int interval = pulsar().getConfiguration().getBrokerDeduplicationSnapshotFrequencyInSeconds(); - if (interval > 0 && pulsar().getConfiguration().isBrokerDeduplicationEnabled()) { + if (interval > 0) { this.deduplicationSnapshotMonitor = OrderedScheduler.newSchedulerBuilder() .name("deduplication-snapshot-monitor") .numThreads(1) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java index 802dd91796127..e508661364d74 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java @@ -157,9 +157,14 @@ private CompletableFuture recoverSequenceIdsMap() { // Replay all the entries and apply all the sequence ids updates log.info("[{}] Replaying {} entries for deduplication", topic.getName(), managedCursor.getNumberOfEntries()); - CompletableFuture future = new CompletableFuture<>(); + CompletableFuture future = new CompletableFuture<>(); replayCursor(future); - return future; + return future.thenAccept(lastPosition -> { + if (lastPosition != null && snapshotCounter >= snapshotInterval) { + snapshotCounter = 0; + takeSnapshot(lastPosition); + } + }); } /** @@ -168,11 +173,11 @@ private CompletableFuture recoverSequenceIdsMap() { * * @param future future to trigger when the replay is complete */ - private void replayCursor(CompletableFuture future) { + private void replayCursor(CompletableFuture future) { managedCursor.asyncReadEntries(100, new ReadEntriesCallback() { @Override public void readEntriesComplete(List entries, Object ctx) { - + Position lastPosition = null; for (Entry entry : entries) { ByteBuf messageMetadataAndPayload = entry.getDataBuffer(); MessageMetadata md = Commands.parseMessageMetadata(messageMetadataAndPayload); @@ -182,7 +187,8 @@ public void readEntriesComplete(List entries, Object ctx) { highestSequencedPushed.put(producerName, sequenceId); highestSequencedPersisted.put(producerName, sequenceId); producerRemoved(producerName); - + snapshotCounter++; + lastPosition = entry.getPosition(); entry.release(); } @@ -191,7 +197,7 @@ public void readEntriesComplete(List entries, Object ctx) { pulsar.getExecutor().execute(() -> replayCursor(future)); } else { // Done replaying - future.complete(null); + future.complete(lastPosition); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 3c9ab04d79a0d..e4441969101c1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -208,7 +208,7 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal private volatile List shadowTopics; private final TopicName shadowSourceTopic; - static final String DEDUPLICATION_CURSOR_NAME = "pulsar.dedup"; + public static final String DEDUPLICATION_CURSOR_NAME = "pulsar.dedup"; public static boolean isDedupCursorName(String name) { return DEDUPLICATION_CURSOR_NAME.equals(name); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java new file mode 100644 index 0000000000000..2ce4ea9b00b2e --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java @@ -0,0 +1,161 @@ +/* + * 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.service; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.persistent.MessageDeduplication; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class DeduplicationDisabledBrokerLevelTest extends ProducerConsumerBase { + + private int deduplicationSnapshotFrequency = 5; + private int brokerDeduplicationEntriesInterval = 1000; + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + protected void doInitConf() throws Exception { + this.conf.setBrokerDeduplicationEnabled(false); + this.conf.setBrokerDeduplicationSnapshotFrequencyInSeconds(deduplicationSnapshotFrequency); + this.conf.setBrokerDeduplicationEntriesInterval(brokerDeduplicationEntriesInterval); + } + + @Test + public void testNoBacklogOnDeduplication() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic); + final PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + final ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + // deduplication enabled: + // broker level: "false" + // topic level: "true". + // So it is enabled. + admin.topicPolicies().setDeduplicationStatus(topic, true); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor = + (ManagedCursorImpl) ml.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor); + }); + + // Verify: regarding deduplication cursor, messages will be acknowledged automatically. + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + producer.send("1"); + producer.send("2"); + producer.send("3"); + producer.close(); + ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + Awaitility.await().atMost(Duration.ofSeconds(deduplicationSnapshotFrequency * 3)).untilAsserted(() -> { + PositionImpl LAC = (PositionImpl) ml.getLastConfirmedEntry(); + PositionImpl cursorMD = (PositionImpl) cursor.getMarkDeletedPosition(); + assertTrue(LAC.compareTo(cursorMD) <= 0); + }); + + // cleanup. + admin.topics().delete(topic); + } + + @Test + public void testSnapshotCounterAfterUnload() throws Exception { + final int originalDeduplicationSnapshotFrequency = deduplicationSnapshotFrequency; + deduplicationSnapshotFrequency = 3600; + cleanup(); + setup(); + + // Create a topic and wait deduplication is started. + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic); + final PersistentTopic persistentTopic1 = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + final ManagedLedgerImpl ml1 = (ManagedLedgerImpl) persistentTopic1.getManagedLedger(); + admin.topicPolicies().setDeduplicationStatus(topic, true); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor1 = + (ManagedCursorImpl) ml1.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor1); + }); + final MessageDeduplication deduplication1 = persistentTopic1.getMessageDeduplication(); + + // 1. Send 999 messages, it is less than "brokerDeduplicationEntriesIntervaddl". + // 2. Unload topic. + // 3. Send 1 messages, there are 1099 messages have not been snapshot now. + // 4. Verify the snapshot has been taken. + // step 1. + final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + for (int i = 0; i < brokerDeduplicationEntriesInterval - 1; i++) { + producer.send(i + ""); + } + int snapshotCounter1 = WhiteboxImpl.getInternalState(deduplication1, "snapshotCounter"); + assertEquals(snapshotCounter1, brokerDeduplicationEntriesInterval - 1); + admin.topics().unload(topic); + PersistentTopic persistentTopic2 = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) persistentTopic2.getManagedLedger(); + MessageDeduplication deduplication2 = persistentTopic2.getMessageDeduplication(); + admin.topicPolicies().setDeduplicationStatus(topic, true); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor = + (ManagedCursorImpl) ml2.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor); + }); + // step 3. + producer.send("last message"); + ml2.trimConsumedLedgersInBackground(new CompletableFuture<>()); + // step 4. + Awaitility.await().untilAsserted(() -> { + int snapshotCounter3 = WhiteboxImpl.getInternalState(deduplication2, "snapshotCounter"); + assertTrue(snapshotCounter3 < brokerDeduplicationEntriesInterval); + // Verify: the previous ledger will be removed because all messages have been acked. + assertEquals(ml2.getLedgersInfo().size(), 1); + }); + + // cleanup. + producer.close(); + admin.topics().delete(topic); + deduplicationSnapshotFrequency = originalDeduplicationSnapshotFrequency; + cleanup(); + setup(); + } +} From 5d18ff7b70f9de3b95d83f6a8fd4756b1c34567b Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 15 Apr 2024 08:43:12 +0800 Subject: [PATCH 071/580] [fix][txn]Handle exceptions in the transaction pending ack init (#21274) Co-authored-by: Baodi Shi --- .../pulsar/broker/service/ServerCnx.java | 2 +- .../pendingack/impl/PendingAckHandleImpl.java | 54 ++++++++++-- .../broker/transaction/TransactionTest.java | 2 +- .../pendingack/PendingAckPersistentTest.java | 82 +++++++++++++++++++ 4 files changed, 132 insertions(+), 8 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 4ee6ac43465f4..a60f1d805ceb6 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 @@ -1376,7 +1376,7 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { // Send error back to client, only if not completed already. if (consumerFuture.completeExceptionally(exception)) { commandSender.sendErrorResponse(requestId, - BrokerServiceException.getClientErrorCode(exception), + BrokerServiceException.getClientErrorCode(exception.getCause()), exception.getCause().getMessage()); } consumers.remove(consumerId, consumerFuture); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java index 7dbe0385fd7e9..5ed271c6fd414 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java @@ -22,6 +22,7 @@ import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.compareToWithAckSet; import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.isAckSetOverlap; import com.google.common.annotations.VisibleForTesting; +import io.netty.util.Timer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -35,9 +36,11 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; @@ -45,6 +48,7 @@ import org.apache.commons.lang3.tuple.MutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException; import org.apache.pulsar.broker.service.Consumer; @@ -53,7 +57,9 @@ import org.apache.pulsar.broker.transaction.pendingack.PendingAckHandleStats; import org.apache.pulsar.broker.transaction.pendingack.PendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.transaction.TxnID; +import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.policies.data.TransactionInPendingAckStats; import org.apache.pulsar.common.policies.data.TransactionPendingAckStats; @@ -134,6 +140,12 @@ public class PendingAckHandleImpl extends PendingAckHandleState implements Pendi public final RecoverTimeRecord recoverTime = new RecoverTimeRecord(); + private final long pendingAckInitFailureBackoffInitialTimeInMs = 100; + + public final Backoff backoff = new Backoff(pendingAckInitFailureBackoffInitialTimeInMs, TimeUnit.MILLISECONDS, + 1, TimeUnit.MINUTES, 0, TimeUnit.MILLISECONDS); + + private final Timer transactionOpTimer; public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { super(State.None); @@ -153,7 +165,11 @@ public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { this.pendingAckStoreProvider = this.persistentSubscription.getTopic() .getBrokerService().getPulsar().getTransactionPendingAckStoreProvider(); + transactionOpTimer = persistentSubscription.getTopic().getBrokerService().getPulsar().getTransactionTimer(); + init(); + } + private void init() { pendingAckStoreProvider.checkInitializedBefore(persistentSubscription) .thenAcceptAsync(init -> { if (init) { @@ -164,9 +180,9 @@ public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { }, internalPinnedExecutor) .exceptionallyAsync(e -> { Throwable t = FutureUtil.unwrapCompletionException(e); - changeToErrorState(); + // Handling the exceptions in `exceptionHandleFuture`, + // it will be helpful to make the exception handling clearer. exceptionHandleFuture(t); - this.pendingAckStoreFuture.completeExceptionally(t); return null; }, internalPinnedExecutor); } @@ -180,9 +196,8 @@ private void initPendingAckStore() { recoverTime.setRecoverStartTime(System.currentTimeMillis()); pendingAckStore.replayAsync(this, internalPinnedExecutor); }).exceptionallyAsync(e -> { - handleCacheRequest(); - changeToErrorState(); - log.error("PendingAckHandleImpl init fail! TopicName : {}, SubName: {}", topicName, subName, e); + // Handling the exceptions in `exceptionHandleFuture`, + // it will be helpful to make the exception handling clearer. exceptionHandleFuture(e.getCause()); return null; }, internalPinnedExecutor); @@ -945,12 +960,39 @@ public void completeHandleFuture() { } public void exceptionHandleFuture(Throwable t) { - final boolean completedNow = this.pendingAckHandleCompletableFuture.completeExceptionally(t); + if (isRetryableException(t)) { + this.state = State.None; + long retryTime = backoff.next(); + log.warn("[{}][{}] Failed to init transaction pending ack. It will be retried in {} Ms", + persistentSubscription.getTopic().getName(), subName, retryTime, t); + transactionOpTimer.newTimeout((timeout) -> init(), retryTime, TimeUnit.MILLISECONDS); + return; + } + log.error("[{}] [{}] PendingAckHandleImpl init fail!", topicName, subName, t); + handleCacheRequest(); + changeToErrorState(); + // ToDo: Add a new serverError `TransactionComponentLoadFailedException` + // and before that a `Unknown` will be returned first. + this.pendingAckStoreFuture = FutureUtil.failedFuture(new BrokerServiceException( + String.format("[%s][%s] Failed to init transaction pending ack.", topicName, subName))); + final boolean completedNow = this.pendingAckHandleCompletableFuture.completeExceptionally( + new BrokerServiceException( + String.format("[%s][%s] Failed to init transaction pending ack.", topicName, subName))); if (completedNow) { recoverTime.setRecoverEndTime(System.currentTimeMillis()); } } + private static boolean isRetryableException(Throwable ex) { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + return (realCause instanceof ManagedLedgerException + && !(realCause instanceof ManagedLedgerException.ManagedLedgerFencedException) + && !(realCause instanceof ManagedLedgerException.NonRecoverableLedgerException)) + || realCause instanceof PulsarClientException.BrokerPersistenceException + || realCause instanceof PulsarClientException.LookupException + || realCause instanceof PulsarClientException.ConnectException; + } + @Override public TransactionInPendingAckStats getTransactionInPendingAckStats(TxnID txnID) { TransactionInPendingAckStats transactionInPendingAckStats = new TransactionInPendingAckStats(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index ddfa82f52886f..e45924e8bb4f2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -1517,7 +1517,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { fail("Expect failure by PendingAckHandle closed, but success"); } catch (ExecutionException executionException){ Throwable t = executionException.getCause(); - Assert.assertTrue(t instanceof BrokerServiceException.ServiceUnitNotReadyException); + Assert.assertTrue(t instanceof BrokerServiceException); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java index db9daf56104c9..93a2f274517d5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java @@ -23,7 +23,9 @@ import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; import static org.testng.AssertJUnit.assertFalse; import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertTrue; @@ -44,9 +46,11 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.pulsar.PrometheusMetricsTestUtil; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -59,6 +63,7 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; @@ -100,6 +105,83 @@ protected void cleanup() { super.internalCleanup(); } + /** + * Test consumer can be built successfully with retryable exception + * and get correct error with no-retryable exception. + * @throws Exception + */ + @Test(timeOut = 60000) + public void testBuildConsumerEncounterPendingAckInitFailure() throws Exception { + // 1. Prepare and make sure the consumer can be built successfully. + String topic = NAMESPACE1 + "/testUnloadSubscriptionWhenFailedInitPendingAck"; + @Cleanup + Consumer consumer1 = pulsarClient.newConsumer() + .subscriptionName("subName1") + .topic(topic) + .subscribe(); + // 2. Mock a transactionPendingAckStoreProvider to test building consumer + // failing at transactionPendingAckStoreProvider::checkInitializedBefore. + Field transactionPendingAckStoreProviderField = PulsarService.class + .getDeclaredField("transactionPendingAckStoreProvider"); + transactionPendingAckStoreProviderField.setAccessible(true); + TransactionPendingAckStoreProvider pendingAckStoreProvider = + (TransactionPendingAckStoreProvider) transactionPendingAckStoreProviderField + .get(pulsarServiceList.get(0)); + TransactionPendingAckStoreProvider mockProvider = mock(pendingAckStoreProvider.getClass()); + // 3. Test retryable exception when checkInitializedBefore: + // The consumer will be built successfully after one time retry. + when(mockProvider.checkInitializedBefore(any())) + // First, the method checkInitializedBefore will fail with a retryable exception. + .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException("mock fail initialize"))) + // Then, the method will be executed successfully. + .thenReturn(CompletableFuture.completedFuture(false)); + transactionPendingAckStoreProviderField.set(pulsarServiceList.get(0), mockProvider); + @Cleanup + Consumer consumer2 = pulsarClient.newConsumer() + .subscriptionName("subName2") + .topic(topic) + .subscribe(); + + // 4. Test retryable exception when newPendingAckStore: + // The consumer will be built successfully after one time retry. + when(mockProvider.checkInitializedBefore(any())) + .thenReturn(CompletableFuture.completedFuture(true)); + + when(mockProvider.newPendingAckStore(any())) + // First, the method newPendingAckStore will fail with a retryable exception. + .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException("mock fail new store"))) + // Then, the method will be executed successfully. + .thenCallRealMethod(); + transactionPendingAckStoreProviderField.set(pulsarServiceList.get(0), mockProvider); + @Cleanup + Consumer consumer3 = pulsarClient.newConsumer() + .subscriptionName("subName3") + .topic(topic) + .subscribe(); + + // 5. Test no-retryable exception: + // The consumer building will be failed without retrying. + when(mockProvider.checkInitializedBefore(any())) + // The method checkInitializedBefore will fail with a no-retryable exception without retrying. + .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException + .NonRecoverableLedgerException("mock fail"))) + .thenReturn(CompletableFuture.completedFuture(false)); + @Cleanup PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(pulsarServiceList.get(0).getBrokerServiceUrl()) + .operationTimeout(3, TimeUnit.SECONDS) + .build(); + try { + @Cleanup + Consumer consumer4 = pulsarClient.newConsumer() + .subscriptionName("subName4") + .topic(topic) + .subscribe(); + fail(); + } catch (Exception exception) { + assertTrue(exception.getMessage().contains("Failed to init transaction pending ack.")); + } + } + @Test public void individualPendingAckReplayTest() throws Exception { int messageCount = 1000; From d9a43dd21605930e16bb038095e36fceff3a4a40 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 15 Apr 2024 13:55:34 +0800 Subject: [PATCH 072/580] [fix][test] Flaky-test: testMessageExpiryWithTimestampNonRecoverableException and testIncorrectClientClock (#22489) --- .../service/PersistentMessageFinderTest.java | 42 ++++++++++--------- 1 file changed, 22 insertions(+), 20 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java index 6965ac28068c1..0972c9098b55b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service; -import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.retryStrategically; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.doAnswer; @@ -383,7 +382,7 @@ public static Set getBrokerEntryMetadataIntercep * * @throws Exception */ - @Test(groups = "flaky") + @Test void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { final String ledgerAndCursorName = "testPersistentMessageExpiryWithNonRecoverableLedgers"; @@ -402,11 +401,15 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { for (int i = 0; i < totalEntries; i++) { ledger.addEntry(createMessageWrittenToLedger("msg" + i)); } + Awaitility.await().untilAsserted(() -> + assertEquals(ledger.getState(), ManagedLedgerImpl.State.LedgerOpened)); List ledgers = ledger.getLedgersInfoAsList(); LedgerInfo lastLedgerInfo = ledgers.get(ledgers.size() - 1); - - assertEquals(ledgers.size(), totalEntries / entriesPerLedger); + // The `lastLedgerInfo` should be newly opened, and it does not contain any entries. + // Please refer to: https://github.com/apache/pulsar/pull/22034 + assertEquals(lastLedgerInfo.getEntries(), 0); + assertEquals(ledgers.size(), totalEntries / entriesPerLedger + 1); // this will make sure that all entries should be deleted Thread.sleep(TimeUnit.SECONDS.toMillis(ttlSeconds)); @@ -420,19 +423,13 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null); - Position previousMarkDelete = null; - for (int i = 0; i < totalEntries; i++) { - monitor.expireMessages(1); - Position previousPos = previousMarkDelete; - retryStrategically( - (test) -> c1.getMarkDeletedPosition() != null && !c1.getMarkDeletedPosition().equals(previousPos), - 5, 100); - previousMarkDelete = c1.getMarkDeletedPosition(); - } - - PositionImpl markDeletePosition = (PositionImpl) c1.getMarkDeletedPosition(); - assertEquals(lastLedgerInfo.getLedgerId(), markDeletePosition.getLedgerId()); - assertEquals(lastLedgerInfo.getEntries() - 1, markDeletePosition.getEntryId()); + assertTrue(monitor.expireMessages(ttlSeconds)); + Awaitility.await().untilAsserted(() -> { + PositionImpl markDeletePosition = (PositionImpl) c1.getMarkDeletedPosition(); + // The markDeletePosition points to the last entry of the previous ledger in lastLedgerInfo. + assertEquals(markDeletePosition.getLedgerId(), lastLedgerInfo.getLedgerId() - 1); + assertEquals(markDeletePosition.getEntryId(), entriesPerLedger - 1); + }); c1.close(); ledger.close(); @@ -440,20 +437,25 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { } - @Test(groups = "flaky") + @Test public void testIncorrectClientClock() throws Exception { final String ledgerAndCursorName = "testIncorrectClientClock"; int maxTTLSeconds = 1; + int entriesNum = 10; ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(1); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(ledgerAndCursorName, config); ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(ledgerAndCursorName); // set client clock to 10 days later long incorrectPublishTimestamp = System.currentTimeMillis() + TimeUnit.DAYS.toMillis(10); - for (int i = 0; i < 10; i++) { + for (int i = 0; i < entriesNum; i++) { ledger.addEntry(createMessageWrittenToLedger("msg" + i, incorrectPublishTimestamp)); } - assertEquals(ledger.getLedgersInfoAsList().size(), 10); + Awaitility.await().untilAsserted(() -> + assertEquals(ledger.getState(), ManagedLedgerImpl.State.LedgerOpened)); + // The number of ledgers should be (entriesNum / MaxEntriesPerLedger) + 1 + // Please refer to: https://github.com/apache/pulsar/pull/22034 + assertEquals(ledger.getLedgersInfoAsList().size(), entriesNum + 1); PersistentTopic mock = mock(PersistentTopic.class); when(mock.getName()).thenReturn("topicname"); when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); From 9d72e6bd847df85a7d18f1827274df96a446798f Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Mon, 15 Apr 2024 16:15:59 +0800 Subject: [PATCH 073/580] [improve][test] Add topic operation checker for topic API (#22468) --- .../pulsar/broker/admin/TopicAuthZTest.java | 156 +++++++++++++++--- 1 file changed, 135 insertions(+), 21 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index d09bc0a3ffde1..e6ff0ce2bb43a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -20,9 +20,18 @@ package org.apache.pulsar.broker.admin; import io.jsonwebtoken.Jwts; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import lombok.SneakyThrows; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -37,22 +46,21 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.security.MockedPulsarStandalone; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicInteger; @Test(groups = "broker-admin") public class TopicAuthZTest extends MockedPulsarStandalone { @@ -61,13 +69,17 @@ public class TopicAuthZTest extends MockedPulsarStandalone { private PulsarAdmin tenantManagerAdmin; + private AuthorizationService authorizationService; + + private AuthorizationService orignalAuthorizationService; + private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); private static final String TENANT_ADMIN_TOKEN = Jwts.builder() .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); @SneakyThrows @BeforeClass(alwaysRun = true) - public void before() { + public void setup() { configureTokenAuthentication(); configureDefaultAuthorization(); enableTransaction(); @@ -99,7 +111,7 @@ protected void createTransactionCoordinatorAssign(int numPartitionsOfTC) throws @SneakyThrows @AfterClass(alwaysRun = true) - public void after() { + public void cleanup() { if (superUserAdmin != null) { superUserAdmin.close(); } @@ -109,6 +121,51 @@ public void after() { close(); } + @BeforeMethod + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + authorizationService = Mockito.spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + authorizationService, true); + } + + @AfterMethod + public void after() throws IllegalAccessException { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + orignalAuthorizationService, true); + } + + private AtomicBoolean setAuthorizationTopicOperationChecker(String role, Object operation) { + AtomicBoolean execFlag = new AtomicBoolean(false); + if (operation instanceof TopicOperation) { + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(2); + if (role.equals(role_)) { + TopicOperation operation_ = invocationOnMock.getArgument(1); + Assert.assertEquals(operation_, operation); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowTopicOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any()); + } else if (operation instanceof NamespaceOperation) { + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(2); + if (role.equals(role_)) { + TopicOperation operation_ = invocationOnMock.getArgument(1); + Assert.assertEquals(operation_, operation); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowNamespaceOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any()); + } else { + throw new IllegalArgumentException(""); + } + + return execFlag; + } + @DataProvider(name = "partitioned") public static Object[][] partitioned() { return new Object[][] { @@ -204,6 +261,8 @@ public void testGetManagedLedgerInfo() { Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getInternalInfo(topic)); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.GET_STATS); + for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); if (action == AuthAction.produce || action == AuthAction.consume) { @@ -214,6 +273,9 @@ public void testGetManagedLedgerInfo() { } superUserAdmin.topics().revokePermissions(topic, subject); } + + Assert.assertTrue(execFlag.get()); + superUserAdmin.topics().deletePartitionedTopic(topic, true); } @@ -244,8 +306,10 @@ public void testGetPartitionedStatsAndInternalStats() { Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getPartitionedStats(topic, false)); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.GET_STATS); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getPartitionedInternalStats(topic)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -313,14 +377,20 @@ public void testCreateSubscriptionAndUpdateSubscriptionPropertiesAndAnalyzeSubsc tenantManagerAdmin.topics().analyzeSubscriptionBacklog(TopicName.get(topic).getPartition(0).getLocalName(), "test-sub", Optional.empty()); // test nobody + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().updateSubscriptionProperties(topic, "test-sub", properties)); + Assert.assertTrue(execFlag.get()); + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getSubscriptionProperties(topic, "test-sub")); + Assert.assertTrue(execFlag.get()); + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().analyzeSubscriptionBacklog(TopicName.get(topic).getPartition(0).getLocalName(), "test-sub", Optional.empty())); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -362,10 +432,15 @@ public void testCreateMissingPartition() { superUserAdmin.topics().createMissedPartitions(topic); // test tenant manager + + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, NamespaceOperation.CREATE_TOPIC); tenantManagerAdmin.topics().createMissedPartitions(topic); + Assert.assertTrue(execFlag.get()); + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.LOOKUP); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().createMissedPartitions(topic)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -396,8 +471,10 @@ public void testPartitionedTopicMetadata(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().getPartitionedTopicMetadata(topic); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.LOOKUP); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getPartitionedTopicMetadata(topic)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -434,16 +511,18 @@ public void testGetProperties(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().getProperties(topic); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.GET_METADATA); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getProperties(topic)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); if (AuthAction.produce == action || AuthAction.consume == action) { - subAdmin.topics().getPartitionedTopicMetadata(topic); + subAdmin.topics().getProperties(topic); } else { Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.topics().getPartitionedTopicMetadata(topic)); + () -> subAdmin.topics().getProperties(topic)); } superUserAdmin.topics().revokePermissions(topic, subject); } @@ -472,8 +551,10 @@ public void testUpdateProperties(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().updateProperties(topic, properties); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.UPDATE_METADATA); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().updateProperties(topic, properties)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -504,8 +585,10 @@ public void testRemoveProperties(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().removeProperties(topic, "key1"); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.DELETE_METADATA); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().removeProperties(topic, "key1")); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -539,8 +622,11 @@ public void testDeletePartitionedTopic() { tenantManagerAdmin.topics().deletePartitionedTopic(topic); createTopic(topic, true); + + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, NamespaceOperation.DELETE_TOPIC); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().deletePartitionedTopic(topic)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(ns, subject, Set.of(action)); @@ -548,7 +634,6 @@ public void testDeletePartitionedTopic() { () -> subAdmin.topics().deletePartitionedTopic(topic)); superUserAdmin.namespaces().revokePermissionsOnNamespace(ns, subject); } - deleteTopic(topic, true); } @Test(dataProvider = "partitioned") @@ -571,8 +656,10 @@ public void testGetSubscription(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().getSubscriptions(topic); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.GET_SUBSCRIPTIONS); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getSubscriptions(topic)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -616,6 +703,7 @@ public void testGetInternalStats(boolean partitioned) { } + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.GET_STATS); if (partitioned) { Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getPartitionedInternalStats(topic)); @@ -623,6 +711,7 @@ public void testGetInternalStats(boolean partitioned) { Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getInternalStats(topic)); } + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -671,8 +760,10 @@ public void testDeleteSubscription(boolean partitioned) { tenantManagerAdmin.topics().deleteSubscription(topic, subName); superUserAdmin.topics().createSubscription(topic, subName, MessageId.latest); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.UNSUBSCRIBE); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().deleteSubscription(topic, subName)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -708,9 +799,10 @@ public void testSkipAllMessage(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().skipAllMessages(topic, subName); - + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.SKIP); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().skipAllMessages(topic, subName)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -746,10 +838,10 @@ public void testSkipMessage() { // test tenant manager tenantManagerAdmin.topics().skipMessages(topic, subName, 1); - + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.SKIP); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().skipMessages(topic, subName, 1)); - + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); if (AuthAction.consume == action) { @@ -782,10 +874,10 @@ public void testExpireMessagesForAllSubscriptions(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().expireMessagesForAllSubscriptions(topic, 1); - + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.EXPIRE_MESSAGES); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().expireMessagesForAllSubscriptions(topic, 1)); - + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); if (AuthAction.consume == action) { @@ -820,10 +912,10 @@ public void testResetCursor(boolean partitioned) { // test tenant manager tenantManagerAdmin.topics().resetCursor(topic, subName, System.currentTimeMillis()); - + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.RESET_CURSOR); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().resetCursor(topic, subName, System.currentTimeMillis())); - + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); if (AuthAction.consume == action) { @@ -858,10 +950,10 @@ public void testResetCursorOnPosition() { // test tenant manager tenantManagerAdmin.topics().resetCursor(topic, subName, MessageId.latest); - + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.RESET_CURSOR); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().resetCursor(topic, subName, MessageId.latest)); - + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); if (AuthAction.consume == action) { @@ -903,8 +995,10 @@ public void testGetMessageById() { // test tenant manager tenantManagerAdmin.topics().getMessagesById(topic, messageId.getLedgerId(), messageId.getEntryId()); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PEEK_MESSAGES); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().getMessagesById(topic, messageId.getLedgerId(), messageId.getEntryId())); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -947,9 +1041,10 @@ public void testPeekNthMessage() { // test tenant manager tenantManagerAdmin.topics().peekMessages(topic, subName, 1); - + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PEEK_MESSAGES); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().peekMessages(topic, subName, 1)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -992,8 +1087,10 @@ public void testExamineMessage() { // test tenant manager tenantManagerAdmin.topics().examineMessage(topic, "latest", 1); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PEEK_MESSAGES); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().examineMessage(topic, "latest", 1)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -1039,7 +1136,9 @@ public void testExpireMessage(boolean partitioned) { superUserAdmin.topics().expireMessages(topic, subName, 1); // test tenant manager + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.EXPIRE_MESSAGES); tenantManagerAdmin.topics().expireMessages(topic, subName, 1); + Assert.assertTrue(execFlag.get()); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().expireMessages(topic, subName, 1)); @@ -1090,8 +1189,10 @@ public void testExpireMessageByPosition() { // test tenant manager tenantManagerAdmin.topics().expireMessages(topic, subName, MessageId.earliest, false); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.EXPIRE_MESSAGES); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topics().expireMessages(topic, subName, MessageId.earliest, false)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -1294,6 +1395,15 @@ public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer adminConsumer.accept(subAdmin)); } + AtomicBoolean execFlag = null; + if (topicOpType == OperationAuthType.Lookup) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.LOOKUP); + } else if (topicOpType == OperationAuthType.Produce) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PRODUCE); + } else if (topicOpType == OperationAuthType.Consume) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); + } + for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(testTopic, subject, Set.of(action)); @@ -1305,6 +1415,10 @@ public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer Date: Tue, 16 Apr 2024 00:21:04 +0900 Subject: [PATCH 074/580] [fix][sec] Upgrade Bouncycastle to 1.78 (#22509) Co-authored-by: hoguni --- bouncy-castle/bc/LICENSE | 6 +++--- distribution/server/src/assemble/LICENSE.bin.txt | 8 ++++---- distribution/shell/src/assemble/LICENSE.bin.txt | 8 ++++---- pom.xml | 2 +- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/bouncy-castle/bc/LICENSE b/bouncy-castle/bc/LICENSE index dae8f16df5b82..14f4e76e921d3 100644 --- a/bouncy-castle/bc/LICENSE +++ b/bouncy-castle/bc/LICENSE @@ -205,6 +205,6 @@ This projects includes binary packages with the following licenses: Bouncy Castle License * Bouncy Castle -- licenses/LICENSE-bouncycastle.txt - - org.bouncycastle-bcpkix-jdk18on-1.75.jar - - org.bouncycastle-bcprov-jdk18on-1.75.jar - - org.bouncycastle-bcprov-ext-jdk18on-1.75.jar + - org.bouncycastle-bcpkix-jdk18on-1.78.jar + - org.bouncycastle-bcprov-jdk18on-1.78.jar + - org.bouncycastle-bcprov-ext-jdk18on-1.78.jar diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index a409ad07ed1b4..4dc6e4341672c 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -613,10 +613,10 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- ../licenses/LICENSE-bouncycastle.txt - - org.bouncycastle-bcpkix-jdk18on-1.75.jar - - org.bouncycastle-bcprov-ext-jdk18on-1.75.jar - - org.bouncycastle-bcprov-jdk18on-1.75.jar - - org.bouncycastle-bcutil-jdk18on-1.75.jar + - org.bouncycastle-bcpkix-jdk18on-1.78.jar + - org.bouncycastle-bcprov-ext-jdk18on-1.78.jar + - org.bouncycastle-bcprov-jdk18on-1.78.jar + - org.bouncycastle-bcutil-jdk18on-1.78.jar ------------------------ diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 3ac489fa49a68..069e61b89b55a 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -473,10 +473,10 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- ../licenses/LICENSE-bouncycastle.txt - - bcpkix-jdk18on-1.75.jar - - bcprov-ext-jdk18on-1.75.jar - - bcprov-jdk18on-1.75.jar - - bcutil-jdk18on-1.75.jar + - bcpkix-jdk18on-1.78.jar + - bcprov-ext-jdk18on-1.78.jar + - bcprov-jdk18on-1.78.jar + - bcutil-jdk18on-1.78.jar ------------------------ diff --git a/pom.xml b/pom.xml index 47ac21b62bfed..8a43e536cdb03 100644 --- a/pom.xml +++ b/pom.xml @@ -156,7 +156,7 @@ flexible messaging model and an intuitive client API. 1.7.32 4.4 2.23.1 - 1.75 + 1.78 1.0.6 1.0.2.4 2.14.2 From bbff29d8ecc2f6c7ec91e0a48085fe14c8ffd6b8 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Tue, 16 Apr 2024 08:04:11 +0800 Subject: [PATCH 075/580] [fix][io] Kafka Source connector maybe stuck (#22511) --- .../pulsar/io/kafka/KafkaAbstractSource.java | 28 +++++- .../kafka/source/KafkaAbstractSourceTest.java | 89 +++++++++++++++++++ 2 files changed, 116 insertions(+), 1 deletion(-) diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java index 782f9d5d57dbb..7eba7438b2b1d 100644 --- a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java @@ -27,6 +27,7 @@ import java.util.Optional; import java.util.Properties; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; @@ -63,6 +64,7 @@ public abstract class KafkaAbstractSource extends PushSource { private volatile boolean running = false; private KafkaSourceConfig kafkaSourceConfig; private Thread runnerThread; + private long maxPollIntervalMs; @Override public void open(Map config, SourceContext sourceContext) throws Exception { @@ -126,6 +128,13 @@ public void open(Map config, SourceContext sourceContext) throws props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, kafkaSourceConfig.getAutoOffsetReset()); props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, kafkaSourceConfig.getKeyDeserializationClass()); props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, kafkaSourceConfig.getValueDeserializationClass()); + if (props.containsKey(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG)) { + maxPollIntervalMs = Long.parseLong(props.get(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG).toString()); + } else { + maxPollIntervalMs = Long.parseLong( + ConsumerConfig.configDef().defaultValues().get(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG) + .toString()); + } try { consumer = new KafkaConsumer<>(beforeCreateConsumer(props)); } catch (Exception ex) { @@ -175,7 +184,9 @@ public void start() { index++; } if (!kafkaSourceConfig.isAutoCommitEnabled()) { - CompletableFuture.allOf(futures).get(); + // Wait about 2/3 of the time of maxPollIntervalMs. + // so as to avoid waiting for the timeout to be kicked out of the consumer group. + CompletableFuture.allOf(futures).get(maxPollIntervalMs * 2 / 3, TimeUnit.MILLISECONDS); consumer.commitSync(); } } catch (Exception e) { @@ -253,6 +264,21 @@ public void ack() { completableFuture.complete(null); } + @Override + public void fail() { + completableFuture.completeExceptionally( + new RuntimeException( + String.format( + "Failed to process record with kafka topic: %s partition: %d offset: %d key: %s", + record.topic(), + record.partition(), + record.offset(), + getKey() + ) + ) + ); + } + @Override public Schema getSchema() { return schema; diff --git a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java index 7675de0636e8a..6b4719709a178 100644 --- a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java +++ b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java @@ -21,12 +21,18 @@ import com.google.common.collect.ImmutableMap; import java.time.Duration; import java.util.Collections; +import java.util.Arrays; import java.lang.reflect.Field; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.kafka.KafkaAbstractSource; import org.apache.pulsar.io.kafka.KafkaSourceConfig; @@ -46,6 +52,7 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; import static org.testng.Assert.fail; @@ -218,6 +225,88 @@ public final void throwExceptionByPoll() throws Exception { source.read(); } + @Test + public final void throwExceptionBySendFail() throws Exception { + KafkaAbstractSource source = new DummySource(); + + KafkaSourceConfig kafkaSourceConfig = new KafkaSourceConfig(); + kafkaSourceConfig.setTopic("test-topic"); + kafkaSourceConfig.setAutoCommitEnabled(false); + Field kafkaSourceConfigField = KafkaAbstractSource.class.getDeclaredField("kafkaSourceConfig"); + kafkaSourceConfigField.setAccessible(true); + kafkaSourceConfigField.set(source, kafkaSourceConfig); + + Field defaultMaxPollIntervalMsField = KafkaAbstractSource.class.getDeclaredField("maxPollIntervalMs"); + defaultMaxPollIntervalMsField.setAccessible(true); + defaultMaxPollIntervalMsField.set(source, 300000); + + Consumer consumer = mock(Consumer.class); + ConsumerRecord consumerRecord = new ConsumerRecord<>("topic", 0, 0, + "t-key", "t-value".getBytes(StandardCharsets.UTF_8)); + ConsumerRecords consumerRecords = new ConsumerRecords<>(Collections.singletonMap( + new TopicPartition("topic", 0), + Arrays.asList(consumerRecord))); + Mockito.doReturn(consumerRecords).when(consumer).poll(Mockito.any(Duration.class)); + + Field consumerField = KafkaAbstractSource.class.getDeclaredField("consumer"); + consumerField.setAccessible(true); + consumerField.set(source, consumer); + source.start(); + + // Mock send message fail + Record record = source.read(); + record.fail(); + + // read again will throw RuntimeException. + try { + source.read(); + fail("Should throw exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof RuntimeException); + assertTrue(e.getCause().getMessage().contains("Failed to process record with kafka topic")); + } + } + + @Test + public final void throwExceptionBySendTimeOut() throws Exception { + KafkaAbstractSource source = new DummySource(); + + KafkaSourceConfig kafkaSourceConfig = new KafkaSourceConfig(); + kafkaSourceConfig.setTopic("test-topic"); + kafkaSourceConfig.setAutoCommitEnabled(false); + Field kafkaSourceConfigField = KafkaAbstractSource.class.getDeclaredField("kafkaSourceConfig"); + kafkaSourceConfigField.setAccessible(true); + kafkaSourceConfigField.set(source, kafkaSourceConfig); + + Field defaultMaxPollIntervalMsField = KafkaAbstractSource.class.getDeclaredField("maxPollIntervalMs"); + defaultMaxPollIntervalMsField.setAccessible(true); + defaultMaxPollIntervalMsField.set(source, 1); + + Consumer consumer = mock(Consumer.class); + ConsumerRecord consumerRecord = new ConsumerRecord<>("topic", 0, 0, + "t-key", "t-value".getBytes(StandardCharsets.UTF_8)); + ConsumerRecords consumerRecords = new ConsumerRecords<>(Collections.singletonMap( + new TopicPartition("topic", 0), + Arrays.asList(consumerRecord))); + Mockito.doReturn(consumerRecords).when(consumer).poll(Mockito.any(Duration.class)); + + Field consumerField = KafkaAbstractSource.class.getDeclaredField("consumer"); + consumerField.setAccessible(true); + consumerField.set(source, consumer); + source.start(); + + // Mock send message fail, just read do noting. + source.read(); + + // read again will throw TimeOutException. + try { + source.read(); + fail("Should throw exception"); + } catch (Exception e) { + assertTrue(e instanceof TimeoutException); + } + } + private File getFile(String name) { ClassLoader classLoader = getClass().getClassLoader(); return new File(classLoader.getResource(name).getFile()); From 203f305bf449dd335b39501177f210cfcb73d5fa Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Tue, 16 Apr 2024 00:34:59 -0700 Subject: [PATCH 076/580] [fix][broker] Fix Replicated Topic unload bug when ExtensibleLoadManager is enabled (#22496) --- .../channel/ServiceUnitStateChannelImpl.java | 13 ++++++------- .../broker/namespace/NamespaceService.java | 5 +++++ .../nonpersistent/NonPersistentTopic.java | 3 ++- .../service/persistent/PersistentTopic.java | 3 ++- .../service/ReplicatorGlobalNSTest.java | 16 ++++++++++++++++ .../broker/service/ReplicatorTestBase.java | 19 +++++++++++++++++-- 6 files changed, 48 insertions(+), 11 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index 68b38080e73a1..e355187af4ba2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -88,7 +88,6 @@ import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.common.naming.NamespaceBundle; @@ -1381,8 +1380,8 @@ private synchronized void doCleanup(String broker) { } try { - producer.flush(); - } catch (PulsarClientException e) { + producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + } catch (Exception e) { log.error("Failed to flush the in-flight non-system bundle override messages.", e); } @@ -1405,8 +1404,8 @@ private synchronized void doCleanup(String broker) { } try { - producer.flush(); - } catch (PulsarClientException e) { + producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + } catch (Exception e) { log.error("Failed to flush the in-flight system bundle override messages.", e); } @@ -1584,8 +1583,8 @@ protected void monitorOwnerships(List brokers) { } try { - producer.flush(); - } catch (PulsarClientException e) { + producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + } catch (Exception e) { log.error("Failed to flush the in-flight messages.", e); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 7c62f264c78d4..3e7bb9560e327 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -824,6 +824,11 @@ public CompletableFuture unloadNamespaceBundle(NamespaceBundle bundle, } public CompletableFuture isNamespaceBundleOwned(NamespaceBundle bundle) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { + ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); + return extensibleLoadManager.getOwnershipAsync(Optional.empty(), bundle) + .thenApply(Optional::isPresent); + } return pulsar.getLocalMetadataStore().exists(ServiceUnitUtils.path(bundle)); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 0ac06d6883ff1..9a3a0a7d83d50 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -589,7 +589,8 @@ public CompletableFuture stopReplProducers() { @Override public CompletableFuture checkReplication() { TopicName name = TopicName.get(topic); - if (!name.isGlobal() || NamespaceService.isHeartbeatNamespace(name)) { + if (!name.isGlobal() || NamespaceService.isHeartbeatNamespace(name) + || ExtensibleLoadManagerImpl.isInternalTopic(topic)) { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index e4441969101c1..936091edce557 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -1704,7 +1704,8 @@ CompletableFuture checkPersistencePolicies() { @Override public CompletableFuture checkReplication() { TopicName name = TopicName.get(topic); - if (!name.isGlobal() || NamespaceService.isHeartbeatNamespace(name)) { + if (!name.isGlobal() || NamespaceService.isHeartbeatNamespace(name) + || ExtensibleLoadManagerImpl.isInternalTopic(topic)) { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java index 4296f3f416868..eed849ef1a01e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java @@ -21,6 +21,8 @@ import com.google.common.collect.Sets; import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.ConsumerImpl; @@ -32,6 +34,8 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; import java.lang.reflect.Method; @@ -41,6 +45,18 @@ public class ReplicatorGlobalNSTest extends ReplicatorTestBase { protected String methodName; + @DataProvider(name = "loadManagerClassName") + public static Object[][] loadManagerClassName() { + return new Object[][]{ + {ModularLoadManagerImpl.class.getName()}, + {ExtensibleLoadManagerImpl.class.getName()} + }; + } + + @Factory(dataProvider = "loadManagerClassName") + public ReplicatorGlobalNSTest(String loadManagerClassName) { + this.loadManagerClassName = loadManagerClassName; + } @BeforeMethod public void beforeMethod(Method m) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java index 11d663ff9f4f4..ba9f850ff0cc1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java @@ -119,6 +119,11 @@ public abstract class ReplicatorTestBase extends TestRetrySupport { protected final String cluster2 = "r2"; protected final String cluster3 = "r3"; protected final String cluster4 = "r4"; + protected String loadManagerClassName; + + protected String getLoadManagerClassName() { + return loadManagerClassName; + } // Default frequency public int getBrokerServicePurgeInactiveFrequency() { @@ -271,8 +276,9 @@ protected void setup() throws Exception { .brokerClientTlsTrustStoreType(keyStoreType) .build()); - admin1.tenants().createTenant("pulsar", - new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r1", "r2", "r3"))); + updateTenantInfo("pulsar", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), + Sets.newHashSet("r1", "r2", "r3"))); admin1.namespaces().createNamespace("pulsar/ns", Sets.newHashSet("r1", "r2", "r3")); admin1.namespaces().createNamespace("pulsar/ns1", Sets.newHashSet("r1", "r2")); @@ -344,6 +350,7 @@ private void setConfigDefaults(ServiceConfiguration config, String clusterName, config.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); config.setEnableReplicatedSubscriptions(true); config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); + config.setLoadManagerClassName(getLoadManagerClassName()); } public void resetConfig1() { @@ -409,6 +416,14 @@ protected void cleanup() throws Exception { resetConfig4(); } + protected void updateTenantInfo(String tenant, TenantInfoImpl tenantInfo) throws Exception { + if (!admin1.tenants().getTenants().contains(tenant)) { + admin1.tenants().createTenant(tenant, tenantInfo); + } else { + admin1.tenants().updateTenant(tenant, tenantInfo); + } + } + static class MessageProducer implements AutoCloseable { URL url; String namespace; From 70b401b1de9df685283140cff1f83252abc27045 Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Tue, 16 Apr 2024 19:53:29 +0800 Subject: [PATCH 077/580] [improve][fn] Expose `RuntimeFlags` as CLI option for Pulsar Functions and Connectors (#22514) --- .../java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java | 4 +++- .../java/org/apache/pulsar/admin/cli/CmdFunctions.java | 7 +++++++ .../main/java/org/apache/pulsar/admin/cli/CmdSinks.java | 6 ++++++ .../main/java/org/apache/pulsar/admin/cli/CmdSources.java | 6 ++++++ 4 files changed, 22 insertions(+), 1 deletion(-) diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java index 4d906af9424f5..d3087b7fc873c 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java @@ -165,7 +165,8 @@ public void testCreateFunction() throws Exception { "--className", DummyFunction.class.getName(), "--dead-letter-topic", "test-dead-letter-topic", "--custom-runtime-options", "custom-runtime-options", - "--user-config", "{\"key\": [\"value1\", \"value2\"]}" + "--user-config", "{\"key\": [\"value1\", \"value2\"]}", + "--runtime-flags", "--add-opens java.base/java.lang=ALL-UNNAMED" }); CreateFunction creater = cmd.getCreater(); @@ -175,6 +176,7 @@ public void testCreateFunction() throws Exception { assertEquals(Boolean.FALSE, creater.getAutoAck()); assertEquals("test-dead-letter-topic", creater.getDeadLetterTopic()); assertEquals("custom-runtime-options", creater.getCustomRuntimeOptions()); + assertEquals("--add-opens java.base/java.lang=ALL-UNNAMED", creater.getRuntimeFlags()); verify(functions, times(1)).createFunction(any(FunctionConfig.class), anyString()); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdFunctions.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdFunctions.java index 15b8fca076104..5e80c168d920b 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdFunctions.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdFunctions.java @@ -377,6 +377,9 @@ abstract class FunctionDetailsCommand extends BaseCommand { @Option(names = "--dead-letter-topic", description = "The topic where messages that are not processed successfully are sent to #Java") protected String deadLetterTopic; + @Option(names = "--runtime-flags", description = "Any flags that you want to pass to a runtime" + + " (for process & Kubernetes runtime only).") + protected String runtimeFlags; protected FunctionConfig functionConfig; protected String userCodeFile; @@ -676,6 +679,10 @@ void processArguments() throws Exception { userCodeFile = functionConfig.getGo(); } + if (null != runtimeFlags) { + functionConfig.setRuntimeFlags(runtimeFlags); + } + // check if configs are valid validateFunctionConfigs(functionConfig); } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java index f3172a49b0154..be1cd0af96085 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java @@ -404,6 +404,9 @@ abstract class SinkDetailsCommand extends BaseCommand { protected String transformFunctionConfig; @Option(names = "--log-topic", description = "The topic to which the logs of a Pulsar Sink are produced") protected String logTopic; + @Option(names = "--runtime-flags", description = "Any flags that you want to pass to a runtime" + + " (for process & Kubernetes runtime only).") + protected String runtimeFlags; protected SinkConfig sinkConfig; @@ -602,6 +605,9 @@ void processArguments() throws Exception { if (null != logTopic) { sinkConfig.setLogTopic(logTopic); } + if (null != runtimeFlags) { + sinkConfig.setRuntimeFlags(runtimeFlags); + } // check if configs are valid validateSinkConfigs(sinkConfig); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSources.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSources.java index 03df3903a6c16..e691d7c126778 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSources.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSources.java @@ -359,6 +359,9 @@ abstract class SourceDetailsCommand extends BaseCommand { protected String secretsString; @Option(names = "--log-topic", description = "The topic to which the logs of a Pulsar Sink are produced") protected String logTopic; + @Option(names = "--runtime-flags", description = "Any flags that you want to pass to a runtime" + + " (for process & Kubernetes runtime only).") + protected String runtimeFlags; protected SourceConfig sourceConfig; @@ -497,6 +500,9 @@ void processArguments() throws Exception { if (null != logTopic) { sourceConfig.setLogTopic(logTopic); } + if (null != runtimeFlags) { + sourceConfig.setRuntimeFlags(runtimeFlags); + } // check if source configs are valid validateSourceConfigs(sourceConfig); From ffdfc0c4e0881c682132e79c3cbf9768b1ab4f89 Mon Sep 17 00:00:00 2001 From: sinan liu Date: Tue, 16 Apr 2024 21:19:44 +0800 Subject: [PATCH 078/580] [fix][test] SchemaMap in AutoConsumeSchema has been reused (#22500) --- .../api/SimpleProducerConsumerTest.java | 66 +++++++++++-------- 1 file changed, 38 insertions(+), 28 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index 7552b84a1c553..691f501777eda 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -4329,6 +4329,10 @@ public static Object[] avroSchemaProvider() { public void testAccessAvroSchemaMetadata(Schema schema) throws Exception { log.info("-- Starting {} test --", methodName); + if (pulsarClient == null) { + pulsarClient = newPulsarClient(lookupUrl.toString(), 0); + } + final String topic = "persistent://my-property/my-ns/accessSchema"; Consumer consumer = pulsarClient.newConsumer(Schema.AUTO_CONSUME()) .topic(topic) @@ -4344,37 +4348,43 @@ public void testAccessAvroSchemaMetadata(Schema schema) throws Exception producer.send(payload); producer.close(); - GenericRecord res = consumer.receive(RECEIVE_TIMEOUT_SECONDS, TimeUnit.SECONDS).getValue(); - consumer.close(); - assertEquals(schema.getSchemaInfo().getType(), res.getSchemaType()); - org.apache.avro.generic.GenericRecord nativeAvroRecord = null; - JsonNode nativeJsonRecord = null; - if (schema.getSchemaInfo().getType() == SchemaType.AVRO) { - nativeAvroRecord = (org.apache.avro.generic.GenericRecord) res.getNativeObject(); - assertNotNull(nativeAvroRecord); - } else { - nativeJsonRecord = (JsonNode) res.getNativeObject(); - assertNotNull(nativeJsonRecord); - } - for (org.apache.pulsar.client.api.schema.Field f : res.getFields()) { - log.info("field {} {}", f.getName(), res.getField(f)); - assertEquals("field", f.getName()); - assertEquals("aaaaaaaaaaaaaaaaaaaaaaaaa", res.getField(f)); - - if (nativeAvroRecord != null) { - // test that the native schema is accessible - org.apache.avro.Schema.Field fieldDetails = nativeAvroRecord.getSchema().getField(f.getName()); - // a nullable string is an UNION - assertEquals(org.apache.avro.Schema.Type.UNION, fieldDetails.schema().getType()); - assertTrue(fieldDetails.schema().getTypes().stream().anyMatch(s -> s.getType() == org.apache.avro.Schema.Type.STRING)); - assertTrue(fieldDetails.schema().getTypes().stream().anyMatch(s -> s.getType() == org.apache.avro.Schema.Type.NULL)); + try { + GenericRecord res = consumer.receive(RECEIVE_TIMEOUT_SECONDS, TimeUnit.SECONDS).getValue(); + consumer.close(); + assertEquals(schema.getSchemaInfo().getType(), res.getSchemaType()); + org.apache.avro.generic.GenericRecord nativeAvroRecord = null; + JsonNode nativeJsonRecord = null; + if (schema.getSchemaInfo().getType() == SchemaType.AVRO) { + nativeAvroRecord = (org.apache.avro.generic.GenericRecord) res.getNativeObject(); + assertNotNull(nativeAvroRecord); } else { - assertEquals(JsonNodeType.STRING, nativeJsonRecord.get("field").getNodeType()); + nativeJsonRecord = (JsonNode) res.getNativeObject(); + assertNotNull(nativeJsonRecord); + } + for (org.apache.pulsar.client.api.schema.Field f : res.getFields()) { + log.info("field {} {}", f.getName(), res.getField(f)); + assertEquals("field", f.getName()); + assertEquals("aaaaaaaaaaaaaaaaaaaaaaaaa", res.getField(f)); + + if (nativeAvroRecord != null) { + // test that the native schema is accessible + org.apache.avro.Schema.Field fieldDetails = nativeAvroRecord.getSchema().getField(f.getName()); + // a nullable string is an UNION + assertEquals(org.apache.avro.Schema.Type.UNION, fieldDetails.schema().getType()); + assertTrue(fieldDetails.schema().getTypes().stream().anyMatch(s -> s.getType() == org.apache.avro.Schema.Type.STRING)); + assertTrue(fieldDetails.schema().getTypes().stream().anyMatch(s -> s.getType() == org.apache.avro.Schema.Type.NULL)); + } else { + assertEquals(JsonNodeType.STRING, nativeJsonRecord.get("field").getNodeType()); + } } + assertEquals(1, res.getFields().size()); + } catch (Exception e) { + fail(); + } finally { + pulsarClient.shutdown(); + pulsarClient = null; + admin.schemas().deleteSchema(topic); } - assertEquals(1, res.getFields().size()); - - admin.schemas().deleteSchema(topic); } @Test(timeOut = 100000) From 4ca4e2855267e3b36ee1a27f7144b89ba9194821 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 17 Apr 2024 03:07:30 +0800 Subject: [PATCH 079/580] [improve] Make the config `metricsBufferResponse` description more effective (#22490) --- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 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 38a4c552f0b6b..2b58cbc2d1178 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 @@ -2928,8 +2928,10 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece private boolean exposeTopicLevelMetricsInPrometheus = true; @FieldContext( category = CATEGORY_METRICS, - doc = "If true, export buffered metrics" - ) + doc = "Set to true to enable the broker to cache the metrics response; the default is false. " + + "The caching period is defined by `managedLedgerStatsPeriodSeconds`. " + + "The broker returns the same response for subsequent requests within the same period. " + + "Ensure that the scrape interval of your monitoring system matches the caching period.") private boolean metricsBufferResponse = false; @FieldContext( category = CATEGORY_METRICS, From d5b36da9a2e0d4f17bea8e033180e494e93dc442 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 17 Apr 2024 03:12:34 +0800 Subject: [PATCH 080/580] [improve][broker] Add topic name to emitted error messages. (#22506) --- .../pulsar/broker/service/AbstractTopic.java | 17 +++++++++-------- .../pulsar/broker/admin/AdminApi2Test.java | 6 ++++-- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index 05defa60c050b..e772486fcc6ea 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -867,7 +867,7 @@ protected CompletableFuture> incrementTopicEpochIfNeeded(Producer } } catch (Exception e) { - log.error("Encountered unexpected error during exclusive producer creation", e); + log.error("[{}] Encountered unexpected error during exclusive producer creation", topic, e); return FutureUtil.failedFuture(new BrokerServiceException(e)); } finally { lock.writeLock().unlock(); @@ -941,14 +941,14 @@ protected void checkTopicFenced() throws BrokerServiceException { protected CompletableFuture internalAddProducer(Producer producer) { if (isProducersExceeded(producer)) { log.warn("[{}] Attempting to add producer to topic which reached max producers limit", topic); - return CompletableFuture.failedFuture( - new BrokerServiceException.ProducerBusyException("Topic reached max producers limit")); + return CompletableFuture.failedFuture(new BrokerServiceException.ProducerBusyException( + "Topic '" + topic + "' reached max producers limit")); } if (isSameAddressProducersExceeded(producer)) { log.warn("[{}] Attempting to add producer to topic which reached max same address producers limit", topic); - return CompletableFuture.failedFuture( - new BrokerServiceException.ProducerBusyException("Topic reached max same address producers limit")); + return CompletableFuture.failedFuture(new BrokerServiceException.ProducerBusyException( + "Topic '" + topic + "' reached max same address producers limit")); } if (log.isDebugEnabled()) { @@ -983,7 +983,7 @@ private CompletableFuture tryOverwriteOldProducer(Producer oldProducer, Pr if (previousIsActive.isEmpty() || previousIsActive.get()) { return CompletableFuture.failedFuture(new BrokerServiceException.NamingException( "Producer with name '" + newProducer.getProducerName() - + "' is already connected to topic")); + + "' is already connected to topic '" + topic + "'")); } else { // If the connection of the previous producer is not active, the method // "cnx().checkConnectionLiveness()" will trigger the close for it and kick off the previous @@ -996,7 +996,8 @@ private CompletableFuture tryOverwriteOldProducer(Producer oldProducer, Pr }); } return CompletableFuture.failedFuture(new BrokerServiceException.NamingException( - "Producer with name '" + newProducer.getProducerName() + "' is already connected to topic")); + "Producer with name '" + newProducer.getProducerName() + "' is already connected to topic '" + + topic + "'")); } } @@ -1346,7 +1347,7 @@ public static Optional getMigratedClusterUrl(PulsarService pulsar, S return getMigratedClusterUrlAsync(pulsar, topic) .get(pulsar.getPulsarResources().getClusterResources().getOperationTimeoutSec(), TimeUnit.SECONDS); } catch (Exception e) { - log.warn("Failed to get migration cluster URL", e); + log.warn("[{}] Failed to get migration cluster URL", topic, e); } return Optional.empty(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java index 6bc64f4dd65d0..249dd3c4607be 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java @@ -2888,7 +2888,8 @@ public void testMaxProducersPerTopicUnlimited() throws Exception { Producer producer = pulsarClient.newProducer().topic(topic).create(); fail("should fail"); } catch (PulsarClientException e) { - assertTrue(e.getMessage().contains("Topic reached max producers limit")); + String expectMsg = "Topic '" + topic + "' reached max producers limit"; + assertTrue(e.getMessage().contains(expectMsg)); } //set the limit to 3 admin.namespaces().setMaxProducersPerTopic(myNamespace, 3); @@ -2902,7 +2903,8 @@ public void testMaxProducersPerTopicUnlimited() throws Exception { Producer producer1 = pulsarClient.newProducer().topic(topic).create(); fail("should fail"); } catch (PulsarClientException e) { - assertTrue(e.getMessage().contains("Topic reached max producers limit")); + String expectMsg = "Topic '" + topic + "' reached max producers limit"; + assertTrue(e.getMessage().contains(expectMsg)); } //clean up From 1dd82a0affd6ec3686fa85d444c35bbbb4e9ce12 Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 17 Apr 2024 18:14:38 +0800 Subject: [PATCH 081/580] [improve][broker] Repeat the handleMetadataChanges callback when configurationMetadataStore equals localMetadataStore (#22519) --- .../java/org/apache/pulsar/broker/service/BrokerService.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 2687532693a45..249008bad91ad 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 @@ -357,7 +357,9 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws this.entryFilterProvider = new EntryFilterProvider(pulsar.getConfiguration()); pulsar.getLocalMetadataStore().registerListener(this::handleMetadataChanges); - pulsar.getConfigurationMetadataStore().registerListener(this::handleMetadataChanges); + if (pulsar.getConfigurationMetadataStore() != pulsar.getLocalMetadataStore()) { + pulsar.getConfigurationMetadataStore().registerListener(this::handleMetadataChanges); + } this.inactivityMonitor = OrderedScheduler.newSchedulerBuilder() .name("pulsar-inactivity-monitor") From 94f6c7ccd2bf8bc261d45ab41f6c7f123359fa47 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 17 Apr 2024 03:15:01 -0700 Subject: [PATCH 082/580] [improve][broker] Optimize gzip compression for /metrics endpoint by sharing/caching compressed result (#22521) --- .../prometheus/PrometheusMetricsServlet.java | 1 + .../pulsar/broker/web/GzipHandlerUtil.java | 21 +++ .../broker/web/GzipHandlerUtilTest.java | 36 ++++ .../apache/pulsar/broker/PulsarService.java | 3 +- .../PrometheusMetricsGenerator.java | 176 ++++++++++++++++-- .../PulsarPrometheusMetricsServlet.java | 28 ++- .../pulsar/PrometheusMetricsTestUtil.java | 2 +- 7 files changed, 253 insertions(+), 14 deletions(-) create mode 100644 pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/GzipHandlerUtilTest.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java index 8a41bed29d44f..8685348174cd6 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java @@ -39,6 +39,7 @@ import org.slf4j.LoggerFactory; public class PrometheusMetricsServlet extends HttpServlet { + public static final String DEFAULT_METRICS_PATH = "/metrics"; private static final long serialVersionUID = 1L; static final int HTTP_STATUS_OK_200 = 200; static final int HTTP_STATUS_INTERNAL_SERVER_ERROR_500 = 500; diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java index 37c9c05e5d53c..9e980cecb791f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java @@ -19,8 +19,10 @@ package org.apache.pulsar.broker.web; import java.util.List; +import org.eclipse.jetty.http.pathmap.PathSpecSet; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.handler.gzip.GzipHandler; +import org.eclipse.jetty.util.IncludeExclude; public class GzipHandlerUtil { public static Handler wrapWithGzipHandler(Handler innerHandler, List gzipCompressionExcludedPaths) { @@ -45,4 +47,23 @@ public static boolean isGzipCompressionCompletelyDisabled(List gzipCompr && (gzipCompressionExcludedPaths.get(0).equals("^.*") || gzipCompressionExcludedPaths.get(0).equals("^.*$")); } + + /** + * Check if GZIP compression is enabled for the given endpoint. + * @param gzipCompressionExcludedPaths list of paths that should not be compressed + * @param endpoint the endpoint to check + * @return true if GZIP compression is enabled for the endpoint, false otherwise + */ + public static boolean isGzipCompressionEnabledForEndpoint(List gzipCompressionExcludedPaths, + String endpoint) { + if (gzipCompressionExcludedPaths == null || gzipCompressionExcludedPaths.isEmpty()) { + return true; + } + if (isGzipCompressionCompletelyDisabled(gzipCompressionExcludedPaths)) { + return false; + } + IncludeExclude paths = new IncludeExclude<>(PathSpecSet.class); + paths.exclude(gzipCompressionExcludedPaths.toArray(new String[0])); + return paths.test(endpoint); + } } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/GzipHandlerUtilTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/GzipHandlerUtilTest.java new file mode 100644 index 0000000000000..d6958695dec9f --- /dev/null +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/GzipHandlerUtilTest.java @@ -0,0 +1,36 @@ +/* + * 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.web; + +import static org.testng.Assert.*; +import java.util.Arrays; +import org.testng.annotations.Test; + +public class GzipHandlerUtilTest { + + @Test + public void testIsGzipCompressionEnabledForEndpoint() { + assertTrue(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(null, "/metrics")); + assertFalse(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("^.*"), "/metrics")); + assertFalse(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("^.*$"), "/metrics")); + assertFalse(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("/metrics"), "/metrics")); + assertTrue(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("/metrics"), "/metrics2")); + assertTrue(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("/admin", "/custom"), "/metrics")); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 9f7b40cc38334..7613a13db22de 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -110,6 +110,7 @@ import org.apache.pulsar.broker.service.schema.SchemaStorageFactory; import org.apache.pulsar.broker.stats.MetricsGenerator; import org.apache.pulsar.broker.stats.PulsarBrokerOpenTelemetry; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; import org.apache.pulsar.broker.stats.prometheus.PulsarPrometheusMetricsServlet; import org.apache.pulsar.broker.storage.ManagedLedgerStorage; @@ -1040,7 +1041,7 @@ private void addWebServerHandlers(WebService webService, true, attributeMap, true, Topics.class); // Add metrics servlet - webService.addServlet("/metrics", + webService.addServlet(PrometheusMetricsServlet.DEFAULT_METRICS_PATH, new ServletHolder(metricsServlet), config.isAuthenticateMetricsEndpoint(), attributeMap); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java index bbd09335c0a97..8cd68caf1ee26 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java @@ -30,6 +30,8 @@ import java.io.OutputStream; import java.io.OutputStreamWriter; import java.io.Writer; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.nio.CharBuffer; import java.nio.charset.StandardCharsets; import java.time.Clock; @@ -43,6 +45,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.zip.CRC32; +import java.util.zip.Deflater; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; @@ -72,7 +76,7 @@ public class PrometheusMetricsGenerator implements AutoCloseable { private volatile boolean closed; public static class MetricsBuffer { - private final CompletableFuture bufferFuture; + private final CompletableFuture bufferFuture; private final long createTimeslot; private final AtomicInteger refCnt = new AtomicInteger(2); @@ -81,7 +85,7 @@ public static class MetricsBuffer { createTimeslot = timeslot; } - public CompletableFuture getBufferFuture() { + public CompletableFuture getBufferFuture() { return bufferFuture; } @@ -113,6 +117,151 @@ public void release() { } } + /** + * A wraps the response buffer and asynchronously provides a gzip compressed buffer when requested. + */ + public static class ResponseBuffer { + private final ByteBuf uncompressedBuffer; + private boolean released = false; + private CompletableFuture compressedBuffer; + + private ResponseBuffer(final ByteBuf uncompressedBuffer) { + this.uncompressedBuffer = uncompressedBuffer; + } + + public ByteBuf getUncompressedBuffer() { + return uncompressedBuffer; + } + + public synchronized CompletableFuture getCompressedBuffer(Executor executor) { + if (released) { + throw new IllegalStateException("Already released!"); + } + if (compressedBuffer == null) { + compressedBuffer = new CompletableFuture<>(); + ByteBuf retainedDuplicate = uncompressedBuffer.retainedDuplicate(); + executor.execute(() -> { + try { + compressedBuffer.complete(compress(retainedDuplicate)); + } catch (Exception e) { + compressedBuffer.completeExceptionally(e); + } finally { + retainedDuplicate.release(); + } + }); + } + return compressedBuffer; + } + + private ByteBuf compress(ByteBuf uncompressedBuffer) { + GzipByteBufferWriter gzipByteBufferWriter = new GzipByteBufferWriter(uncompressedBuffer.alloc(), + uncompressedBuffer.readableBytes()); + return gzipByteBufferWriter.compress(uncompressedBuffer); + } + + public synchronized void release() { + released = true; + uncompressedBuffer.release(); + if (compressedBuffer != null) { + compressedBuffer.whenComplete((byteBuf, throwable) -> { + if (byteBuf != null) { + byteBuf.release(); + } + }); + } + } + } + + /** + * Compress input nio buffers into gzip format with output in a Netty composite ByteBuf. + */ + private static class GzipByteBufferWriter { + private static final byte[] GZIP_HEADER = + new byte[] {(byte) 0x1f, (byte) 0x8b, Deflater.DEFLATED, 0, 0, 0, 0, 0, 0, 0}; + private final ByteBufAllocator bufAllocator; + private final Deflater deflater; + private final CRC32 crc; + private final int bufferSize; + private final CompositeByteBuf resultBuffer; + private ByteBuf backingCompressBuffer; + private ByteBuffer compressBuffer; + + GzipByteBufferWriter(ByteBufAllocator bufAllocator, int readableBytes) { + deflater = new Deflater(Deflater.DEFAULT_COMPRESSION, true); + crc = new CRC32(); + this.bufferSize = Math.max(Math.min(resolveChunkSize(bufAllocator), readableBytes), 8192); + this.bufAllocator = bufAllocator; + this.resultBuffer = bufAllocator.compositeDirectBuffer(readableBytes / bufferSize + 1); + allocateBuffer(); + } + + /** + * Compress the input Netty buffer and append it to the result buffer in gzip format. + * @param uncompressedBuffer + */ + public ByteBuf compress(ByteBuf uncompressedBuffer) { + try { + ByteBuffer[] nioBuffers = uncompressedBuffer.nioBuffers(); + for (int i = 0, nioBuffersLength = nioBuffers.length; i < nioBuffersLength; i++) { + ByteBuffer nioBuffer = nioBuffers[i]; + compressAndAppend(nioBuffer, i == 0, i == nioBuffersLength - 1); + } + return resultBuffer; + } finally { + close(); + } + } + + private void compressAndAppend(ByteBuffer nioBuffer, boolean isFirst, boolean isLast) { + if (isFirst) { + // write gzip header + compressBuffer.put(GZIP_HEADER); + } + nioBuffer.mark(); + crc.update(nioBuffer); + nioBuffer.reset(); + deflater.setInput(nioBuffer); + if (isLast) { + deflater.finish(); + } + while (!deflater.needsInput() && !deflater.finished()) { + int written = deflater.deflate(compressBuffer); + if (written == 0 && !compressBuffer.hasRemaining()) { + backingCompressBuffer.setIndex(0, compressBuffer.position()); + resultBuffer.addComponent(true, backingCompressBuffer); + allocateBuffer(); + } + } + if (isLast) { + // write gzip footer, integer values are in little endian byte order + compressBuffer.order(ByteOrder.LITTLE_ENDIAN); + // write CRC32 checksum + compressBuffer.putInt((int) crc.getValue()); + // write uncompressed size + compressBuffer.putInt(deflater.getTotalIn()); + // append the last compressed buffer + backingCompressBuffer.setIndex(0, compressBuffer.position()); + resultBuffer.addComponent(true, backingCompressBuffer); + backingCompressBuffer = null; + compressBuffer = null; + } + } + + private void allocateBuffer() { + backingCompressBuffer = bufAllocator.directBuffer(bufferSize); + compressBuffer = backingCompressBuffer.nioBuffer(0, bufferSize); + } + + private void close() { + if (deflater != null) { + deflater.end(); + } + if (backingCompressBuffer != null) { + backingCompressBuffer.release(); + } + } + } + private final PulsarService pulsar; private final boolean includeTopicMetrics; private final boolean includeConsumerMetrics; @@ -187,13 +336,7 @@ private ByteBuf allocateMultipartCompositeDirectBuffer() { // use composite buffer with pre-allocated buffers to ensure that the pooled allocator can be used // for allocating the buffers ByteBufAllocator byteBufAllocator = PulsarByteBufAllocator.DEFAULT; - int chunkSize; - if (byteBufAllocator instanceof PooledByteBufAllocator) { - PooledByteBufAllocator pooledByteBufAllocator = (PooledByteBufAllocator) byteBufAllocator; - chunkSize = Math.max(pooledByteBufAllocator.metric().chunkSize(), DEFAULT_INITIAL_BUFFER_SIZE); - } else { - chunkSize = DEFAULT_INITIAL_BUFFER_SIZE; - } + int chunkSize = resolveChunkSize(byteBufAllocator); CompositeByteBuf buf = byteBufAllocator.compositeDirectBuffer( Math.max(MINIMUM_FOR_MAX_COMPONENTS, (initialBufferSize / chunkSize) + 1)); int totalLen = 0; @@ -204,6 +347,17 @@ private ByteBuf allocateMultipartCompositeDirectBuffer() { return buf; } + private static int resolveChunkSize(ByteBufAllocator byteBufAllocator) { + int chunkSize; + if (byteBufAllocator instanceof PooledByteBufAllocator) { + PooledByteBufAllocator pooledByteBufAllocator = (PooledByteBufAllocator) byteBufAllocator; + chunkSize = Math.max(pooledByteBufAllocator.metric().chunkSize(), DEFAULT_INITIAL_BUFFER_SIZE); + } else { + chunkSize = DEFAULT_INITIAL_BUFFER_SIZE; + } + return chunkSize; + } + private static void generateBrokerBasicMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { String clusterName = pulsar.getConfiguration().getClusterName(); // generate managedLedgerCache metrics @@ -335,10 +489,10 @@ public MetricsBuffer renderToBuffer(Executor executor, List bufferFuture = newMetricsBuffer.getBufferFuture(); + CompletableFuture bufferFuture = newMetricsBuffer.getBufferFuture(); executor.execute(() -> { try { - bufferFuture.complete(generate0(metricsProviders)); + bufferFuture.complete(new ResponseBuffer(generate0(metricsProviders))); } catch (Exception e) { bufferFuture.completeExceptionally(e); } finally { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java index 7fcc74e965c24..43514d481dcab 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java @@ -18,10 +18,13 @@ */ package org.apache.pulsar.broker.stats.prometheus; +import static org.apache.pulsar.broker.web.GzipHandlerUtil.isGzipCompressionEnabledForEndpoint; import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; import java.time.Clock; +import java.util.Arrays; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import javax.servlet.AsyncContext; @@ -40,6 +43,7 @@ public class PulsarPrometheusMetricsServlet extends PrometheusMetricsServlet { private static final int EXECUTOR_MAX_THREADS = 4; private final PrometheusMetricsGenerator prometheusMetricsGenerator; + private final boolean gzipCompressionEnabledForMetrics; public PulsarPrometheusMetricsServlet(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, boolean includeProducerMetrics, @@ -50,6 +54,8 @@ public PulsarPrometheusMetricsServlet(PulsarService pulsar, boolean includeTopic prometheusMetricsGenerator = new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, splitTopicAndPartitionLabel, Clock.systemUTC()); + gzipCompressionEnabledForMetrics = isGzipCompressionEnabledForEndpoint( + pulsar.getConfiguration().getHttpServerGzipCompressionExcludedPaths(), DEFAULT_METRICS_PATH); } @@ -100,7 +106,14 @@ public void onStartAsync(AsyncEvent event) throws IOException { context.complete(); return; } - metricsBuffer.getBufferFuture().whenComplete((buffer, ex) -> executor.execute(() -> { + boolean compressOutput = gzipCompressionEnabledForMetrics && isGzipAccepted(request); + metricsBuffer.getBufferFuture().thenCompose(responseBuffer -> { + if (compressOutput) { + return responseBuffer.getCompressedBuffer(executor); + } else { + return CompletableFuture.completedFuture(responseBuffer.getUncompressedBuffer()); + } + }).whenComplete((buffer, ex) -> executor.execute(() -> { try { long elapsedNanos = System.nanoTime() - startNanos; // check if the request has been timed out, implement a soft timeout @@ -133,6 +146,9 @@ public void onStartAsync(AsyncEvent event) throws IOException { } else { response.setStatus(HTTP_STATUS_OK_200); response.setContentType("text/plain;charset=utf-8"); + if (compressOutput) { + response.setHeader("Content-Encoding", "gzip"); + } ServletOutputStream outputStream = response.getOutputStream(); if (outputStream instanceof HttpOutput) { HttpOutput output = (HttpOutput) outputStream; @@ -156,4 +172,14 @@ public void onStartAsync(AsyncEvent event) throws IOException { } })); } + + private boolean isGzipAccepted(HttpServletRequest request) { + String acceptEncoding = request.getHeader("Accept-Encoding"); + if (acceptEncoding != null) { + return Arrays.stream(acceptEncoding.split(",")) + .map(String::trim) + .anyMatch(str -> "gzip".equalsIgnoreCase(str)); + } + return false; + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java index fcc3b6aa88fb4..68826372b7bd6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java @@ -55,7 +55,7 @@ public static void generate(PrometheusMetricsGenerator metricsGenerator, OutputS try { ByteBuf buffer = null; try { - buffer = metricsBuffer.getBufferFuture().get(5, TimeUnit.SECONDS); + buffer = metricsBuffer.getBufferFuture().get(5, TimeUnit.SECONDS).getUncompressedBuffer(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException(e); From 8ca01cd42edfd4efd986f752f6f8538ea5bf4f94 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Wed, 17 Apr 2024 18:46:22 +0800 Subject: [PATCH 083/580] [improve][admin] Align the auth and check it at the first place for topic related API (#22507) --- .../admin/impl/PersistentTopicsBase.java | 419 ++++++++---------- .../broker/admin/v2/PersistentTopics.java | 44 +- .../pulsar/broker/admin/TopicAuthZTest.java | 257 +++++++++-- 3 files changed, 447 insertions(+), 273 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index ab74b1e2bcc0e..1f8d06571908e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -128,8 +128,6 @@ import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.Policies; -import org.apache.pulsar.common.policies.data.PolicyName; -import org.apache.pulsar.common.policies.data.PolicyOperation; import org.apache.pulsar.common.policies.data.PublishRate; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; @@ -2727,14 +2725,14 @@ public String toString() { } protected CompletableFuture internalGetMessageIdByTimestampAsync(long timestamp, boolean authoritative) { - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES); return future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> { if (topicName.isPartitioned()) { return CompletableFuture.completedFuture(null); } else { @@ -2748,7 +2746,6 @@ protected CompletableFuture internalGetMessageIdByTimestampAsync(long }); } }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenCompose(topic -> { if (!(topic instanceof PersistentTopic)) { @@ -3158,65 +3155,56 @@ protected CompletableFuture> in protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, MessageIdImpl messageId, boolean authoritative) { - CompletableFuture ret; - // If the topic name is a partition name, no need to get partition topic metadata again - if (!topicName.isPartitioned()) { - ret = getPartitionedTopicMetadataAsync(topicName, authoritative, false) - .thenCompose(topicMetadata -> { - if (topicMetadata.partitions > 0) { - log.warn("[{}] Not supported calculate backlog size operation on partitioned-topic {}", - clientAppId(), topicName); - asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, - "calculate backlog size is not allowed for partitioned-topic")); - } - return CompletableFuture.completedFuture(null); - }); - } else { - ret = CompletableFuture.completedFuture(null); - } - CompletableFuture future; - if (topicName.isGlobal()) { - future = ret.thenCompose(__ -> validateGlobalNamespaceOwnershipAsync(namespaceName)); - } else { - future = ret; - } - future.thenAccept(__ -> validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(unused -> validateTopicOperationAsync(topicName, - TopicOperation.GET_BACKLOG_SIZE)) - .thenCompose(unused -> getTopicReferenceAsync(topicName)) - .thenAccept(t -> { - PersistentTopic topic = (PersistentTopic) t; - PositionImpl pos = new PositionImpl(messageId.getLedgerId(), - messageId.getEntryId()); - if (topic == null) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, - getTopicNotFoundErrorMessage(topicName.toString()))); - return; - } - ManagedLedgerImpl managedLedger = - (ManagedLedgerImpl) topic.getManagedLedger(); - if (messageId.getLedgerId() == -1) { - asyncResponse.resume(managedLedger.getTotalSize()); - } else { - asyncResponse.resume(managedLedger.getEstimatedBacklogSize(pos)); - } - }).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (isNot307And404Exception(ex)) { - log.error("[{}] Failed to get backlog size for topic {}", clientAppId(), - topicName, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - })).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (isNot307And404Exception(ex)) { - log.error("[{}] Failed to validate global namespace ownership " - + "to get backlog size for topic {}", clientAppId(), topicName, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - }); + CompletableFuture ret = validateTopicOperationAsync(topicName, TopicOperation.GET_BACKLOG_SIZE); + ret.thenCompose(__ -> { + // If the topic name is a partition name, no need to get partition topic metadata again + if (!topicName.isPartitioned()) { + return getPartitionedTopicMetadataAsync(topicName, authoritative, false) + .thenCompose(topicMetadata -> { + if (topicMetadata.partitions > 0) { + log.warn("[{}] Not supported calculate backlog size operation on partitioned-topic {}", + clientAppId(), topicName); + asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, + "calculate backlog size is not allowed for partitioned-topic")); + } + return CompletableFuture.completedFuture(null); + }); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(unused -> getTopicReferenceAsync(topicName)) + .thenAccept(t -> { + PersistentTopic topic = (PersistentTopic) t; + PositionImpl pos = new PositionImpl(messageId.getLedgerId(), + messageId.getEntryId()); + if (topic == null) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, + getTopicNotFoundErrorMessage(topicName.toString()))); + return; + } + ManagedLedgerImpl managedLedger = + (ManagedLedgerImpl) topic.getManagedLedger(); + if (messageId.getLedgerId() == -1) { + asyncResponse.resume(managedLedger.getTotalSize()); + } else { + asyncResponse.resume(managedLedger.getEstimatedBacklogSize(pos)); + } + }).exceptionally(ex -> { + // If the exception is not redirect exception we need to log it. + if (!isNot307And404Exception(ex)) { + log.error("[{}] Failed to validate global namespace ownership " + + "to get backlog size for topic {}", clientAppId(), topicName, ex); + } + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } protected CompletableFuture internalSetBacklogQuota(BacklogQuota.BacklogQuotaType backlogQuotaType, @@ -3224,8 +3212,7 @@ protected CompletableFuture internalSetBacklogQuota(BacklogQuota.BacklogQu BacklogQuota.BacklogQuotaType finalBacklogQuotaType = backlogQuotaType == null ? BacklogQuota.BacklogQuotaType.destination_storage : backlogQuotaType; - return validateTopicPolicyOperationAsync(topicName, PolicyName.BACKLOG, PolicyOperation.WRITE) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + return validatePoliciesReadOnlyAccessAsync() .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName, isGlobal)) .thenCompose(op -> { TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); @@ -3266,9 +3253,7 @@ protected CompletableFuture internalSetBacklogQuota(BacklogQuota.BacklogQu } protected CompletableFuture internalSetReplicationClusters(List clusterIds) { - - return validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION, PolicyOperation.WRITE) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + return validatePoliciesReadOnlyAccessAsync() .thenCompose(__ -> { if (CollectionUtils.isEmpty(clusterIds)) { throw new RestException(Status.PRECONDITION_FAILED, "ClusterIds should not be null or empty"); @@ -3306,22 +3291,21 @@ protected CompletableFuture internalSetReplicationClusters(List cl } protected CompletableFuture internalRemoveReplicationClusters() { - return validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION, PolicyOperation.WRITE) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) - .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName).thenCompose(op -> { - TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); - topicPolicies.setReplicationClusters(null); - return pulsar().getTopicPoliciesService().updateTopicPoliciesAsync(topicName, topicPolicies) - .thenRun(() -> { - log.info("[{}] Successfully set replication clusters for namespace={}, " - + "topic={}, clusters={}", - clientAppId(), - namespaceName, - topicName.getLocalName(), - topicPolicies.getReplicationClusters()); - }); - }) - ); + return validatePoliciesReadOnlyAccessAsync() + .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName)) + .thenCompose(op -> { + TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); + topicPolicies.setReplicationClusters(null); + return pulsar().getTopicPoliciesService().updateTopicPoliciesAsync(topicName, topicPolicies) + .thenRun(() -> { + log.info("[{}] Successfully set replication clusters for namespace={}, " + + "topic={}, clusters={}", + clientAppId(), + namespaceName, + topicName.getLocalName(), + topicPolicies.getReplicationClusters()); + }); + }); } protected CompletableFuture internalGetDeduplication(boolean applied, boolean isGlobal) { @@ -3683,29 +3667,29 @@ protected CompletableFuture internalTerminateAsync(boolean authoritat "Termination of a system topic is not allowed")); } - CompletableFuture ret; - if (topicName.isGlobal()) { - ret = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - ret = CompletableFuture.completedFuture(null); - } - return ret.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.TERMINATE)) - .thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) - .thenAccept(partitionMetadata -> { - if (partitionMetadata.partitions > 0) { - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Termination of a partitioned topic is not allowed"); - } - }) - .thenCompose(__ -> getTopicReferenceAsync(topicName)) - .thenCompose(topic -> { - if (!(topic instanceof PersistentTopic)) { - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Termination of a non-persistent topic is not allowed"); - } - return ((PersistentTopic) topic).terminate(); - }); + CompletableFuture ret = validateTopicOperationAsync(topicName, TopicOperation.TERMINATE); + return ret.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) + .thenAccept(partitionMetadata -> { + if (partitionMetadata.partitions > 0) { + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Termination of a partitioned topic is not allowed"); + } + }) + .thenCompose(__ -> getTopicReferenceAsync(topicName)) + .thenCompose(topic -> { + if (!(topic instanceof PersistentTopic)) { + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Termination of a non-persistent topic is not allowed"); + } + return ((PersistentTopic) topic).terminate(); + }); } protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, boolean authoritative) { @@ -3716,73 +3700,63 @@ protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, bo asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, msg)); return; } + CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.TERMINATE); + future.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(unused -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) + .thenAccept(partitionMetadata -> { + if (partitionMetadata.partitions == 0) { + String msg = "Termination of a non-partitioned topic is not allowed using partitioned-terminate" + + ", please use terminate commands"; + log.error("[{}] [{}] {}", clientAppId(), topicName, msg); + asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, msg)); + return; + } + if (partitionMetadata.partitions > 0) { + Map messageIds = new ConcurrentHashMap<>(partitionMetadata.partitions); + final List> futures = + new ArrayList<>(partitionMetadata.partitions); - CompletableFuture future; - if (topicName.isGlobal()) { - future = validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - future = CompletableFuture.completedFuture(null); - } - - future.thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.TERMINATE) - .thenCompose(unused -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) - .thenAccept(partitionMetadata -> { - if (partitionMetadata.partitions == 0) { - String msg = "Termination of a non-partitioned topic is not allowed using partitioned-terminate" - + ", please use terminate commands"; - log.error("[{}] [{}] {}", clientAppId(), topicName, msg); - asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, msg)); - return; + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + int finalI = i; + futures.add(pulsar().getAdminClient().topics() + .terminateTopicAsync(topicNamePartition.toString()) + .whenComplete((messageId, throwable) -> { + if (throwable != null) { + log.error("[{}] Failed to terminate topic {}", clientAppId(), + topicNamePartition, throwable); + asyncResponse.resume(new RestException(throwable)); + } + messageIds.put(finalI, messageId); + })); + } catch (Exception e) { + log.error("[{}] Failed to terminate topic {}", clientAppId(), topicNamePartition, + e); + throw new RestException(e); } - if (partitionMetadata.partitions > 0) { - Map messageIds = new ConcurrentHashMap<>(partitionMetadata.partitions); - final List> futures = - new ArrayList<>(partitionMetadata.partitions); - - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - int finalI = i; - futures.add(pulsar().getAdminClient().topics() - .terminateTopicAsync(topicNamePartition.toString()) - .whenComplete((messageId, throwable) -> { - if (throwable != null) { - log.error("[{}] Failed to terminate topic {}", clientAppId(), - topicNamePartition, throwable); - asyncResponse.resume(new RestException(throwable)); - } - messageIds.put(finalI, messageId); - })); - } catch (Exception e) { - log.error("[{}] Failed to terminate topic {}", clientAppId(), topicNamePartition, - e); - throw new RestException(e); - } + } + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = exception.getCause(); + if (t instanceof NotFoundException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, + getTopicNotFoundErrorMessage(topicName.toString()))); + } else { + log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, t); + asyncResponse.resume(new RestException(t)); } - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = exception.getCause(); - if (t instanceof NotFoundException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, - getTopicNotFoundErrorMessage(topicName.toString()))); - } else { - log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, t); - asyncResponse.resume(new RestException(t)); - } - } - asyncResponse.resume(messageIds); - return null; - }); } - }).exceptionally(ex -> { - // If the exception is not redirect exception we need to log it. - if (isNot307And404Exception(ex)) { - log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); + asyncResponse.resume(messageIds); return null; - }) - ).exceptionally(ex -> { + }); + } + }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. if (isNot307And404Exception(ex)) { log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, ex); @@ -4186,16 +4160,16 @@ protected void internalTriggerCompactionNonPartitionedTopic(AsyncResponse asyncR } protected CompletableFuture internalCompactionStatusAsync(boolean authoritative) { - return validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.COMPACT)) + return validateTopicOperationAsync(topicName, TopicOperation.COMPACT) + .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenApply(topic -> ((PersistentTopic) topic).compactionStatus()); } protected void internalTriggerOffload(AsyncResponse asyncResponse, boolean authoritative, MessageIdImpl messageId) { - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.OFFLOAD)) + validateTopicOperationAsync(topicName, TopicOperation.OFFLOAD) + .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenAccept(topic -> { try { @@ -4221,8 +4195,8 @@ protected void internalTriggerOffload(AsyncResponse asyncResponse, } protected void internalOffloadStatus(AsyncResponse asyncResponse, boolean authoritative) { - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.OFFLOAD)) + validateTopicOperationAsync(topicName, TopicOperation.OFFLOAD) + .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenAccept(topic -> { OffloadProcessStatus offloadProcessStatus = ((PersistentTopic) topic).offloadStatus(); @@ -4482,8 +4456,8 @@ private CompletableFuture validateNonPartitionTopicNameAsync(String topicN } protected void internalGetLastMessageId(AsyncResponse asyncResponse, boolean authoritative) { - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES)) + validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES) + .thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenAccept(topic -> { if (topic == null) { @@ -5207,33 +5181,27 @@ private void internalGetReplicatedSubscriptionStatusForNonPartitionedTopic( } protected CompletableFuture internalGetSchemaCompatibilityStrategy(boolean applied) { - CompletableFuture future = validateTopicPolicyOperationAsync(topicName, - PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.READ); if (applied) { - return future.thenCompose(__ -> getSchemaCompatibilityStrategyAsync()); + return getSchemaCompatibilityStrategyAsync(); } - return future - .thenCompose(n -> getTopicPoliciesAsyncWithRetry(topicName).thenApply(op -> { + return getTopicPoliciesAsyncWithRetry(topicName).thenApply(op -> { if (!op.isPresent()) { return null; } SchemaCompatibilityStrategy strategy = op.get().getSchemaCompatibilityStrategy(); return SchemaCompatibilityStrategy.isUndefined(strategy) ? null : strategy; - })); + }); } protected CompletableFuture internalSetSchemaCompatibilityStrategy(SchemaCompatibilityStrategy strategy) { - return validateTopicPolicyOperationAsync(topicName, - PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, - PolicyOperation.WRITE) - .thenCompose((__) -> getTopicPoliciesAsyncWithRetry(topicName) + return getTopicPoliciesAsyncWithRetry(topicName) .thenCompose(op -> { TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); topicPolicies.setSchemaCompatibilityStrategy( strategy == SchemaCompatibilityStrategy.UNDEFINED ? null : strategy); return pulsar().getTopicPoliciesService() .updateTopicPoliciesAsync(topicName, topicPolicies); - })); + }); } protected CompletableFuture internalGetSchemaValidationEnforced(boolean applied) { @@ -5257,54 +5225,47 @@ protected CompletableFuture internalSetSchemaValidationEnforced(boolean sc } protected CompletableFuture internalGetEntryFilters(boolean applied, boolean isGlobal) { - return validateTopicPolicyOperationAsync(topicName, PolicyName.ENTRY_FILTERS, PolicyOperation.READ) - .thenCompose(__ -> { - if (!applied) { - return getTopicPoliciesAsyncWithRetry(topicName, isGlobal) - .thenApply(op -> op.map(TopicPolicies::getEntryFilters).orElse(null)); - } - if (!pulsar().getConfiguration().isAllowOverrideEntryFilters()) { - return CompletableFuture.completedFuture(new EntryFilters(String.join(",", - pulsar().getConfiguration().getEntryFilterNames()))); + if (!applied) { + return getTopicPoliciesAsyncWithRetry(topicName, isGlobal) + .thenApply(op -> op.map(TopicPolicies::getEntryFilters).orElse(null)); + } + if (!pulsar().getConfiguration().isAllowOverrideEntryFilters()) { + return CompletableFuture.completedFuture(new EntryFilters(String.join(",", + pulsar().getConfiguration().getEntryFilterNames()))); + } + return getTopicPoliciesAsyncWithRetry(topicName, isGlobal) + .thenApply(op -> op.map(TopicPolicies::getEntryFilters)) + .thenCompose(policyEntryFilters -> { + if (policyEntryFilters.isPresent()) { + return CompletableFuture.completedFuture(policyEntryFilters.get()); } - return getTopicPoliciesAsyncWithRetry(topicName, isGlobal) - .thenApply(op -> op.map(TopicPolicies::getEntryFilters)) - .thenCompose(policyEntryFilters -> { - if (policyEntryFilters.isPresent()) { - return CompletableFuture.completedFuture(policyEntryFilters.get()); + return getNamespacePoliciesAsync(namespaceName) + .thenApply(policies -> policies.entryFilters) + .thenCompose(nsEntryFilters -> { + if (nsEntryFilters != null) { + return CompletableFuture.completedFuture(nsEntryFilters); } - return getNamespacePoliciesAsync(namespaceName) - .thenApply(policies -> policies.entryFilters) - .thenCompose(nsEntryFilters -> { - if (nsEntryFilters != null) { - return CompletableFuture.completedFuture(nsEntryFilters); - } - return CompletableFuture.completedFuture(new EntryFilters(String.join(",", - pulsar().getConfiguration().getEntryFilterNames()))); - }); + return CompletableFuture.completedFuture(new EntryFilters(String.join(",", + pulsar().getConfiguration().getEntryFilterNames()))); }); }); } protected CompletableFuture internalSetEntryFilters(EntryFilters entryFilters, boolean isGlobal) { - - return validateTopicPolicyOperationAsync(topicName, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE) - .thenAccept(__ -> validateEntryFilters(entryFilters)) - .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName, isGlobal) + validateEntryFilters(entryFilters); + return getTopicPoliciesAsyncWithRetry(topicName, isGlobal) .thenCompose(op -> { TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); topicPolicies.setEntryFilters(entryFilters); topicPolicies.setIsGlobal(isGlobal); return pulsar().getTopicPoliciesService() .updateTopicPoliciesAsync(topicName, topicPolicies); - })); + }); } protected CompletableFuture internalRemoveEntryFilters(boolean isGlobal) { - return validateTopicPolicyOperationAsync(topicName, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE) - .thenCompose(__ -> - getTopicPoliciesAsyncWithRetry(topicName, isGlobal) + return getTopicPoliciesAsyncWithRetry(topicName, isGlobal) .thenCompose(op -> { if (!op.isPresent()) { return CompletableFuture.completedFuture(null); @@ -5312,7 +5273,7 @@ protected CompletableFuture internalRemoveEntryFilters(boolean isGlobal) { op.get().setEntryFilters(null); op.get().setIsGlobal(isGlobal); return pulsar().getTopicPoliciesService().updateTopicPoliciesAsync(topicName, op.get()); - })); + }); } protected CompletableFuture validateShadowTopics(List shadowTopics) { @@ -5348,8 +5309,7 @@ protected CompletableFuture internalSetShadowTopic(List shadowTopi return FutureUtil.failedFuture(new RestException(Status.PRECONDITION_FAILED, "Cannot specify empty shadow topics, please use remove command instead.")); } - return validateTopicPolicyOperationAsync(topicName, PolicyName.SHADOW_TOPIC, PolicyOperation.WRITE) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + return validatePoliciesReadOnlyAccessAsync() .thenCompose(__ -> validateShadowTopics(shadowTopics)) .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName)) .thenCompose(op -> { @@ -5361,8 +5321,7 @@ protected CompletableFuture internalSetShadowTopic(List shadowTopi } protected CompletableFuture internalDeleteShadowTopics() { - return validateTopicPolicyOperationAsync(topicName, PolicyName.SHADOW_TOPIC, PolicyOperation.WRITE) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + return validatePoliciesReadOnlyAccessAsync() .thenCompose(shadowTopicName -> getTopicPoliciesAsyncWithRetry(topicName)) .thenCompose(op -> { TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 90f0208c81cd6..7e138442ae228 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -2149,7 +2149,8 @@ public void setBacklogQuota( @QueryParam("backlogQuotaType") BacklogQuotaType backlogQuotaType, @ApiParam(value = "backlog quota policies for the specified topic") BacklogQuotaImpl backlogQuota) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.BACKLOG, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetBacklogQuota(backlogQuotaType, backlogQuota, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -2174,7 +2175,8 @@ public void removeBacklogQuota(@Suspended final AsyncResponse asyncResponse, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.BACKLOG, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetBacklogQuota(backlogQuotaType, null, isGlobal)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -2237,7 +2239,8 @@ public void setReplicationClusters( @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "List of replication clusters", required = true) List clusterIds) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetReplicationClusters(clusterIds)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -2260,7 +2263,8 @@ public void removeReplicationClusters(@Suspended final AsyncResponse asyncRespon @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveReplicationClusters()) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -4405,8 +4409,8 @@ public void getSchemaCompatibilityStrategy( @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__-> internalGetSchemaCompatibilityStrategy(applied)) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { @@ -4436,8 +4440,8 @@ public void setSchemaCompatibilityStrategy( @ApiParam(value = "Strategy used to check the compatibility of new schema") SchemaCompatibilityStrategy strategy) { validateTopicName(tenant, namespace, encodedTopic); - - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetSchemaCompatibilityStrategy(strategy)) .thenRun(() -> { log.info( @@ -4476,8 +4480,8 @@ public void removeSchemaCompatibilityStrategy( @ApiParam(value = "Strategy used to check the compatibility of new schema") SchemaCompatibilityStrategy strategy) { validateTopicName(tenant, namespace, encodedTopic); - - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetSchemaCompatibilityStrategy(null)) .thenRun(() -> { log.info( @@ -4568,7 +4572,8 @@ public void getEntryFilters(@Suspended AsyncResponse asyncResponse, + "broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.ENTRY_FILTERS, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetEntryFilters(applied, isGlobal)) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { @@ -4596,7 +4601,8 @@ public void setEntryFilters(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Entry filters for the specified topic") EntryFilters entryFilters) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetEntryFilters(entryFilters, isGlobal)) .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -4622,7 +4628,8 @@ public void removeEntryFilters(@Suspended final AsyncResponse asyncResponse, + "call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveEntryFilters(isGlobal)) .thenRun(() -> { log.info( @@ -4655,9 +4662,8 @@ public void getShadowTopics( @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) - .thenCompose(__ -> validateTopicPolicyOperationAsync(topicName, PolicyName.SHADOW_TOPIC, - PolicyOperation.READ)) + validateTopicPolicyOperationAsync(topicName, PolicyName.SHADOW_TOPIC, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName)) .thenAccept(op -> asyncResponse.resume(op.map(TopicPolicies::getShadowTopics).orElse(null))) .exceptionally(ex -> { @@ -4684,7 +4690,8 @@ public void setShadowTopics( @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "List of shadow topics", required = true) List shadowTopics) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SHADOW_TOPIC, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetShadowTopic(shadowTopics)) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { @@ -4710,7 +4717,8 @@ public void deleteShadowTopics( @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.SHADOW_TOPIC, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalDeleteShadowTopics()) .thenRun(() -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index e6ff0ce2bb43a..3c0596d531f41 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -19,48 +19,54 @@ package org.apache.pulsar.broker.admin; +import com.google.common.collect.Lists; import io.jsonwebtoken.Jwts; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import lombok.SneakyThrows; import org.apache.commons.lang3.reflect.FieldUtils; -import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.broker.service.plugin.EntryFilterDefinition; +import org.apache.pulsar.broker.service.plugin.EntryFilterProvider; +import org.apache.pulsar.broker.service.plugin.EntryFilterTest; +import org.apache.pulsar.broker.testcontext.MockEntryFilterProvider; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.auth.AuthenticationToken; -import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; -import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.EntryFilters; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.security.MockedPulsarStandalone; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.metadata.api.MetadataStoreException; -import org.apache.pulsar.security.MockedPulsarStandalone; import org.mockito.Mockito; -import org.testng.Assert; -import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Test; @Test(groups = "broker-admin") public class TopicAuthZTest extends MockedPulsarStandalone { @@ -1105,15 +1111,15 @@ public void testExamineMessage() { deleteTopic(topic, false); } - @Test(dataProvider = "partitioned", groups = "flaky") + @Test @SneakyThrows - public void testExpireMessage(boolean partitioned) { + public void testExpireMessage() { final String random = UUID.randomUUID().toString(); final String topic = "persistent://public/default/" + random; final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); - createTopic(topic, partitioned); + createTopic(topic, false); @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) @@ -1153,7 +1159,7 @@ public void testExpireMessage(boolean partitioned) { } superUserAdmin.topics().revokePermissions(topic, subject); } - deleteTopic(topic, partitioned); + deleteTopic(topic, false); } @Test @@ -1373,6 +1379,37 @@ public Object[][] authFunction () throws Exception { }; } + @Test + @SneakyThrows + public void testSchemaCompatibility() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + superUserAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, true); + + // test tenant manager + tenantManagerAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, true); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, false)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getSchemaCompatibilityStrategy(topic, false)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + } + @Test(dataProvider = "authFunction") public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer adminConsumer, OperationAuthType topicOpType) throws Exception { @@ -1380,6 +1417,7 @@ public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer subAdmin.topicPolicies().getEntryFiltersPerTopic(topic, false)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getEntryFiltersPerTopic(topic, false)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testSetEntryFilter() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + final EntryFilterProvider oldEntryFilterProvider = getPulsarService().getBrokerService().getEntryFilterProvider(); + @Cleanup + final MockEntryFilterProvider testEntryFilterProvider = + new MockEntryFilterProvider(getServiceConfiguration()); + + testEntryFilterProvider + .setMockEntryFilters(new EntryFilterDefinition( + "test", + null, + EntryFilterTest.class.getName() + )); + FieldUtils.writeField(getPulsarService().getBrokerService(), + "entryFilterProvider", testEntryFilterProvider, true); + final EntryFilters entryFilter = new EntryFilters("test"); + superUserAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter); + + // test tenant manager + tenantManagerAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + FieldUtils.writeField(getPulsarService().getBrokerService(), + "entryFilterProvider", oldEntryFilterProvider, true); + } + + @Test + @SneakyThrows + public void testRemoveEntryFilter() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + final EntryFilterProvider oldEntryFilterProvider = getPulsarService().getBrokerService().getEntryFilterProvider(); + @Cleanup + final MockEntryFilterProvider testEntryFilterProvider = + new MockEntryFilterProvider(getServiceConfiguration()); + + testEntryFilterProvider + .setMockEntryFilters(new EntryFilterDefinition( + "test", + null, + EntryFilterTest.class.getName() + )); + FieldUtils.writeField(getPulsarService().getBrokerService(), + "entryFilterProvider", testEntryFilterProvider, true); + final EntryFilters entryFilter = new EntryFilters("test"); + superUserAdmin.topicPolicies().removeEntryFiltersPerTopic(topic); + // test tenant manager + tenantManagerAdmin.topicPolicies().removeEntryFiltersPerTopic(topic); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeEntryFiltersPerTopic(topic)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeEntryFiltersPerTopic(topic)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + FieldUtils.writeField(getPulsarService().getBrokerService(), + "entryFilterProvider", oldEntryFilterProvider, true); + } + + @Test + @SneakyThrows + public void testShadowTopic() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + + String shadowTopic = topic + "-shadow-topic"; + superUserAdmin.topics().createShadowTopic(shadowTopic, topic); + superUserAdmin.topics().setShadowTopics(topic, Lists.newArrayList(shadowTopic)); + superUserAdmin.topics().getShadowTopics(topic); + superUserAdmin.topics().removeShadowTopics(topic); + + + // test tenant manager + tenantManagerAdmin.topics().setShadowTopics(topic, Lists.newArrayList(shadowTopic)); + tenantManagerAdmin.topics().getShadowTopics(topic); + tenantManagerAdmin.topics().removeShadowTopics(topic); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().setShadowTopics(topic, Lists.newArrayList(shadowTopic))); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getShadowTopics(topic)); + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().setShadowTopics(topic, Lists.newArrayList(shadowTopic))); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getShadowTopics(topic)); + superUserAdmin.topics().revokePermissions(topic, subject); + } + deleteTopic(topic, false); + } + private void createTopic(String topic, boolean partitioned) throws Exception { if (partitioned) { superUserAdmin.topics().createPartitionedTopic(topic, 2); From 56970b714f5adb606b02d12a99db1ceec3fa7832 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 17 Apr 2024 12:46:43 -0700 Subject: [PATCH 084/580] [improve][test] Move ShadowManagedLedgerImplTest to flaky tests (#22526) --- .../bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java index cc4b3f2481152..2aa04197ab91e 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java @@ -51,7 +51,7 @@ private ShadowManagedLedgerImpl openShadowManagedLedger(String name, String sour return (ShadowManagedLedgerImpl) shadowML; } - @Test + @Test(groups = "flaky") public void testShadowWrites() throws Exception { ManagedLedgerImpl sourceML = (ManagedLedgerImpl) factory.open("source_ML", new ManagedLedgerConfig() .setMaxEntriesPerLedger(2) From d0b9d471d53d2db600b55a04d6255688d1fd2d27 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Thu, 18 Apr 2024 09:48:14 +0800 Subject: [PATCH 085/580] [fix][broker] Check the broker is available for the SLA monitor bundle when the ExtensibleLoadManager is enabled (#22485) --- .../extensions/ExtensibleLoadManagerImpl.java | 39 ++++++--------- .../broker/namespace/NamespaceService.java | 47 ++++++++++++++----- .../ExtensibleLoadManagerImplTest.java | 43 +++++++++++++++++ 3 files changed, 94 insertions(+), 35 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 0c9448ab69c38..c8cf1c05756a6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -492,30 +492,20 @@ public CompletableFuture> assign(Optional { + if (candidateBrokerId != null) { + return CompletableFuture.completedFuture(Optional.of(candidateBrokerId)); + } + return getOrSelectOwnerAsync(serviceUnit, bundle).thenApply(Optional::ofNullable); + }); } return getBrokerLookupData(owner, bundle); }); } - private String getHeartbeatOrSLAMonitorBrokerId(ServiceUnitId serviceUnit) { - // Check if this is Heartbeat or SLAMonitor namespace - String candidateBroker = NamespaceService.checkHeartbeatNamespace(serviceUnit); - if (candidateBroker == null) { - candidateBroker = NamespaceService.checkHeartbeatNamespaceV2(serviceUnit); - } - if (candidateBroker == null) { - candidateBroker = NamespaceService.getSLAMonitorBrokerName(serviceUnit); - } - if (candidateBroker != null) { - return candidateBroker.substring(candidateBroker.lastIndexOf('/') + 1); - } - return candidateBroker; + private CompletableFuture getHeartbeatOrSLAMonitorBrokerId(ServiceUnitId serviceUnit) { + return pulsar.getNamespaceService().getHeartbeatOrSLAMonitorBrokerId(serviceUnit, + cb -> brokerRegistry.lookupAsync(cb).thenApply(Optional::isPresent)); } private CompletableFuture getOrSelectOwnerAsync(ServiceUnitId serviceUnit, @@ -662,11 +652,12 @@ public CompletableFuture> getOwnershipAsync(Optional { + if (candidateBroker != null) { + return CompletableFuture.completedFuture(Optional.of(candidateBroker)); + } + return serviceUnitStateChannel.getOwnerAsync(bundle); + }); } public CompletableFuture> getOwnershipWithLookupDataAsync(ServiceUnitId bundleUnit) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 3e7bb9560e327..65081f2ea42b6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -46,6 +46,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -536,6 +537,38 @@ private CompletableFuture> findBrokerServiceUrl( }); } + /** + * Check if this is Heartbeat or SLAMonitor namespace and return the broker id. + * + * @param serviceUnit the service unit + * @param isBrokerActive the function to check if the broker is active + * @return the broker id + */ + public CompletableFuture getHeartbeatOrSLAMonitorBrokerId( + ServiceUnitId serviceUnit, Function> isBrokerActive) { + String candidateBroker = NamespaceService.checkHeartbeatNamespace(serviceUnit); + if (candidateBroker != null) { + return CompletableFuture.completedFuture(candidateBroker); + } + candidateBroker = NamespaceService.checkHeartbeatNamespaceV2(serviceUnit); + if (candidateBroker != null) { + return CompletableFuture.completedFuture(candidateBroker); + } + candidateBroker = NamespaceService.getSLAMonitorBrokerName(serviceUnit); + if (candidateBroker != null) { + // Check if the broker is available + final String finalCandidateBroker = candidateBroker; + return isBrokerActive.apply(candidateBroker).thenApply(isActive -> { + if (isActive) { + return finalCandidateBroker; + } else { + return null; + } + }); + } + return CompletableFuture.completedFuture(null); + } + private void searchForCandidateBroker(NamespaceBundle bundle, CompletableFuture> lookupFuture, LookupOptions options) { @@ -552,17 +585,9 @@ private void searchForCandidateBroker(NamespaceBundle bundle, try { // check if this is Heartbeat or SLAMonitor namespace - candidateBroker = checkHeartbeatNamespace(bundle); - if (candidateBroker == null) { - candidateBroker = checkHeartbeatNamespaceV2(bundle); - } - if (candidateBroker == null) { - String broker = getSLAMonitorBrokerName(bundle); - // checking if the broker is up and running - if (broker != null && isBrokerActive(broker)) { - candidateBroker = broker; - } - } + candidateBroker = getHeartbeatOrSLAMonitorBrokerId(bundle, cb -> + CompletableFuture.completedFuture(isBrokerActive(cb))) + .get(config.getMetadataStoreOperationTimeoutSeconds(), SECONDS); if (candidateBroker == null) { Optional currentLeader = pulsar.getLeaderElectionService().getCurrentLeader(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index e87d6c994cd76..b72ab77e81447 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -50,6 +50,7 @@ import static org.mockito.Mockito.verify; 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.assertNull; import static org.testng.Assert.assertTrue; @@ -113,6 +114,7 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; @@ -998,6 +1000,47 @@ public void testDeployAndRollbackLoadManager() throws Exception { pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); } } + // Check if the broker is available + var wrapper = (ExtensibleLoadManagerWrapper) pulsar4.getLoadManager().get(); + var loadManager4 = spy((ExtensibleLoadManagerImpl) + FieldUtils.readField(wrapper, "loadManager", true)); + loadManager4.getBrokerRegistry().unregister(); + + NamespaceName slaMonitorNamespace = + getSLAMonitorNamespace(pulsar4.getBrokerId(), pulsar.getConfiguration()); + String slaMonitorTopic = slaMonitorNamespace.getPersistentTopicName("test"); + String result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertNotEquals(result, pulsar4.getBrokerServiceUrl()); + + Producer producer = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer.send("t1"); + + // Test re-register broker and check the lookup result + loadManager4.getBrokerRegistry().register(); + + result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertEquals(result, pulsar4.getBrokerServiceUrl()); + + producer.send("t2"); + Producer producer1 = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer1.send("t3"); + + producer.close(); + producer1.close(); + @Cleanup + Consumer consumer = pulsar.getClient().newConsumer(Schema.STRING) + .topic(slaMonitorTopic) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName("test") + .subscribe(); + // receive message t1 t2 t3 + assertEquals(consumer.receive().getValue(), "t1"); + assertEquals(consumer.receive().getValue(), "t2"); + assertEquals(consumer.receive().getValue(), "t3"); } } } From 72474d7a2dabdf7acf0b158bd07f1bc8b69b790e Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Wed, 17 Apr 2024 23:59:36 -0700 Subject: [PATCH 086/580] [fix][broker] Fix a deadlock in SystemTopicBasedTopicPoliciesService during NamespaceEventsSystemTopicFactory init (#22528) --- .../SystemTopicBasedTopicPoliciesService.java | 49 ++++++++++++------- 1 file changed, 32 insertions(+), 17 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index 4e9e875bcf4c3..0449e5c885cd3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -34,6 +34,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nonnull; +import org.apache.commons.lang3.concurrent.ConcurrentInitializer; +import org.apache.commons.lang3.concurrent.LazyInitializer; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -70,7 +72,19 @@ public class SystemTopicBasedTopicPoliciesService implements TopicPoliciesServic private final PulsarService pulsarService; private final HashSet localCluster; private final String clusterName; - private volatile NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; + + private final ConcurrentInitializer + namespaceEventsSystemTopicFactoryLazyInitializer = new LazyInitializer<>() { + @Override + protected NamespaceEventsSystemTopicFactory initialize() { + try { + return new NamespaceEventsSystemTopicFactory(pulsarService.getClient()); + } catch (PulsarServerException e) { + log.error("Create namespace event system topic factory error.", e); + throw new RuntimeException(e); + } + } + }; @VisibleForTesting final Map policiesCache = new ConcurrentHashMap<>(); @@ -102,7 +116,7 @@ public SystemTopicBasedTopicPoliciesService(PulsarService pulsarService) { }); }) .buildAsync((namespaceName, executor) -> { - SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(namespaceName); return systemTopicClient.newWriterAsync(); }); @@ -301,7 +315,7 @@ public CompletableFuture getTopicPoliciesBypassCacheAsync(TopicNa result.complete(null); return result; } - SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(topicName.getNamespaceObject()); systemTopicClient.newReaderAsync().thenAccept(r -> fetchTopicPoliciesAsyncAndCloseReader(r, topicName, null, result)); @@ -373,7 +387,7 @@ protected CompletableFuture> createSystemT } catch (PulsarServerException ex) { return FutureUtil.failedFuture(ex); } - final SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + final SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(namespace); return systemTopicClient.newReaderAsync(); } @@ -561,7 +575,7 @@ private void refreshTopicPoliciesCache(Message msg) { log.error("Failed to create system topic factory"); break; } - SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(topicName.getNamespaceObject()); systemTopicClient.newWriterAsync().thenAccept(writer -> writer.deleteAsync(getEventKey(topicName), @@ -595,18 +609,19 @@ private boolean hasReplicateTo(Message message) { } private void createSystemTopicFactoryIfNeeded() throws PulsarServerException { - if (namespaceEventsSystemTopicFactory == null) { - synchronized (this) { - if (namespaceEventsSystemTopicFactory == null) { - try { - namespaceEventsSystemTopicFactory = - new NamespaceEventsSystemTopicFactory(pulsarService.getClient()); - } catch (PulsarServerException e) { - log.error("Create namespace event system topic factory error.", e); - throw e; - } - } - } + try { + getNamespaceEventsSystemTopicFactory(); + } catch (Exception e) { + throw new PulsarServerException(e); + } + } + + private NamespaceEventsSystemTopicFactory getNamespaceEventsSystemTopicFactory() { + try { + return namespaceEventsSystemTopicFactoryLazyInitializer.get(); + } catch (Exception e) { + log.error("Create namespace event system topic factory error.", e); + throw new RuntimeException(e); } } From 990b8d0334c75255e25899df869887711059cb33 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 18 Apr 2024 07:48:55 -0700 Subject: [PATCH 087/580] [improve][build] Upgrade OWASP Dependency check version to 9.1.0 (#22530) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8a43e536cdb03..c7fba94abd8ea 100644 --- a/pom.xml +++ b/pom.xml @@ -307,7 +307,7 @@ flexible messaging model and an intuitive client API. 0.1.21 1.3 0.4 - 9.0.7 + 9.1.0 0.9.44 1.6.1 6.4.0 From 7acbc4c9f4ce74979a5ae9b6f0721956edaf9295 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 18 Apr 2024 12:35:30 -0700 Subject: [PATCH 088/580] [improve] Run Trivy image scanner with the current image (#22386) --- .../workflows/ci-trivy-container-scan.yaml | 66 ------------------- .github/workflows/pulsar-ci.yaml | 17 +++++ 2 files changed, 17 insertions(+), 66 deletions(-) delete mode 100644 .github/workflows/ci-trivy-container-scan.yaml diff --git a/.github/workflows/ci-trivy-container-scan.yaml b/.github/workflows/ci-trivy-container-scan.yaml deleted file mode 100644 index 47ebe654369d5..0000000000000 --- a/.github/workflows/ci-trivy-container-scan.yaml +++ /dev/null @@ -1,66 +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. -# - -name: CI - Trivy Container Scan -on: - schedule: - - cron: '0 8 * * *' # Every day at 8am UTC - workflow_dispatch: - inputs: - severity: - description: "Severities to include (comma-separated or 'ALL' to include all)" - required: false - default: 'CRITICAL,HIGH' - -jobs: - container_scan: - if: ${{ github.repository == 'apache/pulsar' }} - name: Trivy Docker image vulnerability scan - runs-on: ubuntu-latest - strategy: - fail-fast: false - matrix: - docker-image: - - 'apachepulsar/pulsar' - docker-tag: - - 'latest' - env: - IMAGE_REF: '${{ matrix.docker-image }}:${{ matrix.docker-tag }}' - steps: - - id: prepare-vars - shell: bash - run: | - IMAGE_REF_CLEAN="$(echo $IMAGE_REF | sed 's/-/_/g; s/\./_/g; s/:/_/g; s/\//_/g')" - echo "image_ref_clean=$IMAGE_REF_CLEAN" >> "$GITHUB_OUTPUT" - echo "report_filename=trivy-scan-$IMAGE_REF_CLEAN.${{ inputs.report-format }}" >> "$GITHUB_OUTPUT" - - name: Run Trivy container scan - uses: aquasecurity/trivy-action@master - with: - image-ref: ${{ env.IMAGE_REF }} - scanners: vuln - severity: ${{ inputs.severity != 'ALL' && inputs.severity || 'UNKNOWN,LOW,MEDIUM,HIGH,CRITICAL' }} - limit-severities-for-sarif: true - format: 'sarif' - output: ${{ steps.prepare-vars.outputs.report_filename }} - exit-code: 1 - - name: Upload Trivy scan results to GitHub Security tab - uses: github/codeql-action/upload-sarif@v2 - if: ${{ failure() }} - with: - sarif_file: '${{ github.workspace }}/${{ steps.prepare-vars.outputs.report_filename }}' diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 22d061ac58094..aa33d7ae197d1 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -876,6 +876,23 @@ jobs: - name: Check binary licenses run: src/check-binary-license.sh ./distribution/server/target/apache-pulsar-*-bin.tar.gz && src/check-binary-license.sh ./distribution/shell/target/apache-pulsar-shell-*-bin.tar.gz + - name: Run Trivy container scan + uses: aquasecurity/trivy-action@master + if: ${{ github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} + with: + image-ref: "apachepulsar/pulsar:latest" + scanners: vuln + severity: CRITICAL,HIGH,MEDIUM,LOW + limit-severities-for-sarif: true + format: 'sarif' + output: 'trivy-results.sarif' + + - name: Upload Trivy scan results to GitHub Security tab + uses: github/codeql-action/upload-sarif@v2 + if: ${{ github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} + with: + sarif_file: 'trivy-results.sarif' + - name: Clean up disk space run: | # release disk space since saving docker image consumes local disk space From 7aedb6b20c120ec0a7cc096e33e6305caca26786 Mon Sep 17 00:00:00 2001 From: hanmz Date: Fri, 19 Apr 2024 06:49:18 +0800 Subject: [PATCH 089/580] [fix][broker] Fix typos in Consumer class (#22532) --- .../java/org/apache/pulsar/broker/service/Consumer.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index 4cd54420200be..6b2028095e205 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -147,7 +147,7 @@ public class Consumer { @Setter private volatile long consumerEpoch; - private long negtiveUnackedMsgsTimestamp; + private long negativeUnackedMsgsTimestamp; @Getter private final SchemaType schemaType; @@ -1102,8 +1102,8 @@ private int addAndGetUnAckedMsgs(Consumer consumer, int ackedMessages) { subscription.addUnAckedMessages(ackedMessages); unackedMsgs = UNACKED_MESSAGES_UPDATER.addAndGet(consumer, ackedMessages); } - if (unackedMsgs < 0 && System.currentTimeMillis() - negtiveUnackedMsgsTimestamp >= 10_000) { - negtiveUnackedMsgsTimestamp = System.currentTimeMillis(); + if (unackedMsgs < 0 && System.currentTimeMillis() - negativeUnackedMsgsTimestamp >= 10_000) { + negativeUnackedMsgsTimestamp = System.currentTimeMillis(); log.warn("unackedMsgs is : {}, ackedMessages : {}, consumer : {}", unackedMsgs, ackedMessages, consumer); } return unackedMsgs; From 2badcf6bd0be1aad2a5ec6da552185b4ef5b745b Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 19 Apr 2024 09:13:19 +0800 Subject: [PATCH 090/580] [fix][broker] Fix NPE causing dispatching to stop when using Key_Shared mode and allowOutOfOrderDelivery=true (#22533) --- ...ntStickyKeyDispatcherMultipleConsumers.java | 10 ++++++++++ .../client/api/KeySharedSubscriptionTest.java | 18 ++++++++++++++++-- 2 files changed, 26 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index ee2ebd7ca867e..2df9f38531f5d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -457,6 +457,11 @@ private int getAvailablePermits(Consumer c) { @Override protected synchronized NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", + // So skip this filter out. + if (isAllowOutOfOrderDelivery()) { + return src; + } if (src.isEmpty()) { return src; } @@ -501,6 +506,11 @@ protected synchronized NavigableSet filterOutEntriesWillBeDiscarde */ @Override protected boolean hasConsumersNeededNormalRead() { + // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", + // So the method "filterOutEntriesWillBeDiscarded" will filter out nothing, just return "true" here. + if (isAllowOutOfOrderDelivery()) { + return true; + } for (Consumer consumer : consumerList) { if (consumer == null || consumer.isBlocked()) { continue; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index 7219555050839..27aa98597ec12 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -1741,6 +1741,14 @@ public void testNoRepeatedReadAndDiscard() throws Exception { admin.topics().delete(topic, false); } + @DataProvider(name = "allowKeySharedOutOfOrder") + public Object[][] allowKeySharedOutOfOrder() { + return new Object[][]{ + {true}, + {false} + }; + } + /** * This test is in order to guarantee the feature added by https://github.com/apache/pulsar/pull/7105. * 1. Start 3 consumers: @@ -1755,8 +1763,8 @@ public void testNoRepeatedReadAndDiscard() throws Exception { * - no repeated Read-and-discard. * - at last, all messages will be received. */ - @Test(timeOut = 180 * 1000) // the test will be finished in 60s. - public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { + @Test(timeOut = 180 * 1000, dataProvider = "allowKeySharedOutOfOrder") // the test will be finished in 60s. + public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedOutOfOrder) throws Exception { final int messagesSentPerTime = 100; final Set totalReceivedMessages = new TreeSet<>(); final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); @@ -1775,6 +1783,8 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { log.info("Published message :{}", messageId); } + KeySharedPolicy keySharedPolicy = KeySharedPolicy.autoSplitHashRange() + .setAllowOutOfOrderDelivery(allowKeySharedOutOfOrder); // 1. Start 3 consumers and make ack holes. // - one consumer will be closed and trigger a messages redeliver. // - one consumer will not ack any messages to make the new consumer joined late will be stuck due to the @@ -1785,18 +1795,21 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(keySharedPolicy) .subscribe(); Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(keySharedPolicy) .subscribe(); Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(keySharedPolicy) .subscribe(); List msgList1 = new ArrayList<>(); List msgList2 = new ArrayList<>(); @@ -1845,6 +1858,7 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { .subscriptionName(subName) .receiverQueueSize(1000) .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(keySharedPolicy) .subscribe(); consumerWillBeClose.close(); From fbf4cb71a3f3ed08786205dc5e60b810f3d62605 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Fri, 19 Apr 2024 21:51:10 +0800 Subject: [PATCH 091/580] [improve][offload] Apply autoSkipNonRecoverableData configuration to tiered storage (#22531) --- .../impl/BlobStoreBackedInputStreamImpl.java | 8 +++ .../impl/BlobStoreBackedReadHandleImpl.java | 13 ++++- .../impl/BlobStoreBackedReadHandleImplV2.java | 13 ++++- .../BlobStoreBackedInputStreamTest.java | 5 +- ...reManagedLedgerOffloaderStreamingTest.java | 54 +++++++++++++++++++ .../BlobStoreManagedLedgerOffloaderTest.java | 23 ++++++++ 6 files changed, 110 insertions(+), 6 deletions(-) diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java index 0dea46726f50a..6cb60e14984f9 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java @@ -28,6 +28,7 @@ import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.naming.TopicName; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.jclouds.blobstore.options.GetOptions; import org.slf4j.Logger; @@ -95,6 +96,9 @@ private boolean refillBufferIfNeeded() throws IOException { try { long startReadTime = System.nanoTime(); Blob blob = blobStore.getBlob(bucket, key, new GetOptions().range(startRange, endRange)); + if (blob == null) { + throw new KeyNotFoundException(bucket, key, ""); + } versionCheck.check(key, blob); try (InputStream stream = blob.getPayload().openStream()) { @@ -121,6 +125,10 @@ private boolean refillBufferIfNeeded() throws IOException { if (null != this.offloaderStats) { this.offloaderStats.recordReadOffloadError(this.topicName); } + // If the blob is not found, the original exception is thrown and handled by the caller. + if (e instanceof KeyNotFoundException) { + throw e; + } throw new IOException("Error reading from BlobStore", e); } } diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java index 5346be6a044c8..4f68f90370e6f 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java @@ -49,6 +49,7 @@ import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.naming.TopicName; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -202,7 +203,11 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr } catch (Throwable t) { log.error("Failed to read entries {} - {} from the offloader in ledger {}", firstEntry, lastEntry, ledgerId, t); - promise.completeExceptionally(t); + if (t instanceof KeyNotFoundException) { + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + } else { + promise.completeExceptionally(t); + } entries.forEach(LedgerEntry::close); } }); @@ -265,7 +270,7 @@ public static ReadHandle open(ScheduledExecutorService executor, VersionCheck versionCheck, long ledgerId, int readBufferSize, LedgerOffloaderStats offloaderStats, String managedLedgerName) - throws IOException { + throws IOException, BKException.BKNoSuchLedgerExistsException { int retryCount = 3; OffloadIndexBlock index = null; IOException lastException = null; @@ -278,6 +283,10 @@ public static ReadHandle open(ScheduledExecutorService executor, while (retryCount-- > 0) { long readIndexStartTime = System.nanoTime(); Blob blob = blobStore.getBlob(bucket, indexKey); + if (blob == null) { + log.error("{} not found in container {}", indexKey, bucket); + throw new BKException.BKNoSuchLedgerExistsException(); + } offloaderStats.recordReadOffloadIndexLatency(topicName, System.nanoTime() - readIndexStartTime, TimeUnit.NANOSECONDS); versionCheck.check(indexKey, blob); diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java index 53d96e08abf5e..502f475174cee 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java @@ -49,6 +49,7 @@ import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.naming.TopicName; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -224,7 +225,11 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr } } } catch (Throwable t) { - promise.completeExceptionally(t); + if (t instanceof KeyNotFoundException) { + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + } else { + promise.completeExceptionally(t); + } entries.forEach(LedgerEntry::close); } @@ -303,7 +308,7 @@ public static ReadHandle open(ScheduledExecutorService executor, VersionCheck versionCheck, long ledgerId, int readBufferSize, LedgerOffloaderStats offloaderStats, String managedLedgerName) - throws IOException { + throws IOException, BKException.BKNoSuchLedgerExistsException { List inputStreams = new LinkedList<>(); List indice = new LinkedList<>(); String topicName = TopicName.fromPersistenceNamingEncoding(managedLedgerName); @@ -313,6 +318,10 @@ public static ReadHandle open(ScheduledExecutorService executor, log.debug("open bucket: {} index key: {}", bucket, indexKey); long startTime = System.nanoTime(); Blob blob = blobStore.getBlob(bucket, indexKey); + if (blob == null) { + log.error("{} not found in container {}", indexKey, bucket); + throw new BKException.BKNoSuchLedgerExistsException(); + } offloaderStats.recordReadOffloadIndexLatency(topicName, System.nanoTime() - startTime, TimeUnit.NANOSECONDS); log.debug("indexKey blob: {} {}", indexKey, blob); diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java index 775310925a1a3..3e5c4b609dfec 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java @@ -32,6 +32,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.offload.jcloud.impl.BlobStoreBackedInputStreamImpl; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.jclouds.io.Payload; import org.jclouds.io.Payloads; @@ -142,8 +143,8 @@ public void testReadingFullObjectByBytes() throws Exception { assertStreamsMatchByBytes(toTest, toCompare); } - @Test(expectedExceptions = IOException.class) - public void testErrorOnRead() throws Exception { + @Test(expectedExceptions = KeyNotFoundException.class) + public void testNotFoundOnRead() throws Exception { BackedInputStream toTest = new BlobStoreBackedInputStreamImpl(blobStore, BUCKET, "doesn't exist", (key, md) -> {}, 1234, 1000); diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java index 9056281a308f2..ad1529072f813 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java @@ -18,16 +18,19 @@ */ package org.apache.bookkeeper.mledger.offload.jcloud.impl; +import static org.apache.bookkeeper.client.api.BKException.Code.NoSuchLedgerExistsException; import static org.mockito.AdditionalAnswers.delegatesTo; import static org.mockito.Mockito.mock; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.fail; import java.io.IOException; import java.util.HashMap; import java.util.LinkedList; import java.util.Map; import java.util.Random; import java.util.UUID; +import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.api.LedgerEntries; import org.apache.bookkeeper.client.api.LedgerEntry; import org.apache.bookkeeper.client.api.ReadHandle; @@ -445,4 +448,55 @@ public void testInvalidEntryIds() throws Exception { } catch (Exception e) { } } + + @Test + public void testReadNotExistLedger() throws Exception { + LedgerOffloader offloader = getOffloader(new HashMap() {{ + put(TieredStorageConfiguration.MAX_OFFLOAD_SEGMENT_SIZE_IN_BYTES, "1000"); + put(config.getKeys(TieredStorageConfiguration.METADATA_FIELD_MAX_BLOCK_SIZE).get(0), "5242880"); + put(TieredStorageConfiguration.MAX_OFFLOAD_SEGMENT_ROLLOVER_TIME_SEC, "600"); + }}); + ManagedLedger ml = createMockManagedLedger(); + UUID uuid = UUID.randomUUID(); + long beginLedger = 0; + long beginEntry = 0; + + Map driverMeta = new HashMap() {{ + put(TieredStorageConfiguration.METADATA_FIELD_BUCKET, BUCKET); + }}; + OffloadHandle offloadHandle = offloader + .streamingOffload(ml, uuid, beginLedger, beginEntry, driverMeta).get(); + + // Segment should closed because size in bytes full + final LinkedList entries = new LinkedList<>(); + for (int i = 0; i < 10; i++) { + final byte[] data = new byte[100]; + random.nextBytes(data); + final EntryImpl entry = EntryImpl.create(0, i, data); + offloadHandle.offerEntry(entry); + entries.add(entry); + } + + final LedgerOffloader.OffloadResult offloadResult = offloadHandle.getOffloadResultAsync().get(); + assertEquals(offloadResult.endLedger, 0); + assertEquals(offloadResult.endEntry, 9); + final OffloadContext.Builder contextBuilder = OffloadContext.newBuilder(); + contextBuilder.addOffloadSegment( + MLDataFormats.OffloadSegment.newBuilder() + .setUidLsb(uuid.getLeastSignificantBits()) + .setUidMsb(uuid.getMostSignificantBits()) + .setComplete(true).setEndEntryId(9).build()); + + final ReadHandle readHandle = offloader.readOffloaded(0, contextBuilder.build(), driverMeta).get(); + + // delete blob(ledger) + blobStore.removeBlob(BUCKET, uuid.toString()); + + try { + readHandle.read(0, 9); + fail("Should be read fail"); + } catch (BKException e) { + assertEquals(e.getCode(), NoSuchLedgerExistsException); + } + } } diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java index bb4cb286680f5..4419210c251f1 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.mledger.offload.jcloud.impl; +import static org.apache.bookkeeper.client.api.BKException.Code.NoSuchLedgerExistsException; import static org.mockito.AdditionalAnswers.delegatesTo; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; @@ -26,6 +27,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -601,4 +603,25 @@ public void testReadWithAClosedLedgerHandler() throws Exception { throw e; } } + + @Test + public void testReadNotExistLedger() throws Exception { + ReadHandle toWrite = buildReadHandle(DEFAULT_BLOCK_SIZE, 3); + LedgerOffloader offloader = getOffloader(); + + UUID uuid = UUID.randomUUID(); + offloader.offload(toWrite, uuid, new HashMap<>()).get(); + ReadHandle offloadRead = offloader.readOffloaded(toWrite.getId(), uuid, Collections.emptyMap()).get(); + assertEquals(offloadRead.getLastAddConfirmed(), toWrite.getLastAddConfirmed()); + + // delete blob(ledger) + blobStore.removeBlob(BUCKET, DataBlockUtils.dataBlockOffloadKey(toWrite.getId(), uuid)); + + try { + offloadRead.read(0, offloadRead.getLastAddConfirmed()); + fail("Should be read fail"); + } catch (BKException e) { + assertEquals(e.getCode(), NoSuchLedgerExistsException); + } + } } From 59daac64c210f539e733f883edad09d08333aa62 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 19 Apr 2024 10:30:55 -0700 Subject: [PATCH 092/580] [fix][broker] Fix broken topic policy implementation compatibility with old pulsar version (#22535) --- .../pulsar/broker/service/AbstractTopic.java | 52 +++++++++++-------- ...ternalClientConfigurationOverrideTest.java | 42 ++++++++++++++- 2 files changed, 72 insertions(+), 22 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index e772486fcc6ea..44a4ca42cea46 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -220,13 +220,16 @@ protected void updateTopicPolicy(TopicPolicies data) { topicPolicies.getRetentionPolicies().updateTopicValue(data.getRetentionPolicies()); topicPolicies.getDispatcherPauseOnAckStatePersistentEnabled() .updateTopicValue(data.getDispatcherPauseOnAckStatePersistentEnabled()); - topicPolicies.getMaxSubscriptionsPerTopic().updateTopicValue(data.getMaxSubscriptionsPerTopic()); - topicPolicies.getMaxUnackedMessagesOnConsumer().updateTopicValue(data.getMaxUnackedMessagesOnConsumer()); + topicPolicies.getMaxSubscriptionsPerTopic() + .updateTopicValue(normalizeValue(data.getMaxSubscriptionsPerTopic())); + topicPolicies.getMaxUnackedMessagesOnConsumer() + .updateTopicValue(normalizeValue(data.getMaxUnackedMessagesOnConsumer())); topicPolicies.getMaxUnackedMessagesOnSubscription() - .updateTopicValue(data.getMaxUnackedMessagesOnSubscription()); - topicPolicies.getMaxProducersPerTopic().updateTopicValue(data.getMaxProducerPerTopic()); - topicPolicies.getMaxConsumerPerTopic().updateTopicValue(data.getMaxConsumerPerTopic()); - topicPolicies.getMaxConsumersPerSubscription().updateTopicValue(data.getMaxConsumersPerSubscription()); + .updateTopicValue(normalizeValue(data.getMaxUnackedMessagesOnSubscription())); + topicPolicies.getMaxProducersPerTopic().updateTopicValue(normalizeValue(data.getMaxProducerPerTopic())); + topicPolicies.getMaxConsumerPerTopic().updateTopicValue(normalizeValue(data.getMaxConsumerPerTopic())); + topicPolicies.getMaxConsumersPerSubscription() + .updateTopicValue(normalizeValue(data.getMaxConsumersPerSubscription())); topicPolicies.getInactiveTopicPolicies().updateTopicValue(data.getInactiveTopicPolicies()); topicPolicies.getDeduplicationEnabled().updateTopicValue(data.getDeduplicationEnabled()); topicPolicies.getDeduplicationSnapshotIntervalSeconds().updateTopicValue( @@ -237,8 +240,8 @@ protected void updateTopicPolicy(TopicPolicies data) { Arrays.stream(BacklogQuota.BacklogQuotaType.values()).forEach(type -> this.topicPolicies.getBackLogQuotaMap().get(type).updateTopicValue( data.getBackLogQuotaMap() == null ? null : data.getBackLogQuotaMap().get(type.toString()))); - topicPolicies.getTopicMaxMessageSize().updateTopicValue(data.getMaxMessageSize()); - topicPolicies.getMessageTTLInSeconds().updateTopicValue(data.getMessageTTLInSeconds()); + topicPolicies.getTopicMaxMessageSize().updateTopicValue(normalizeValue(data.getMaxMessageSize())); + topicPolicies.getMessageTTLInSeconds().updateTopicValue(normalizeValue(data.getMessageTTLInSeconds())); topicPolicies.getPublishRate().updateTopicValue(PublishRate.normalize(data.getPublishRate())); topicPolicies.getDelayedDeliveryEnabled().updateTopicValue(data.getDelayedDeliveryEnabled()); topicPolicies.getReplicatorDispatchRate().updateTopicValue( @@ -268,15 +271,19 @@ protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) { topicPolicies.getReplicationClusters().updateNamespaceValue( new ArrayList<>(CollectionUtils.emptyIfNull(namespacePolicies.replication_clusters))); topicPolicies.getMaxUnackedMessagesOnConsumer() - .updateNamespaceValue(namespacePolicies.max_unacked_messages_per_consumer); + .updateNamespaceValue(normalizeValue(namespacePolicies.max_unacked_messages_per_consumer)); topicPolicies.getMaxUnackedMessagesOnSubscription() - .updateNamespaceValue(namespacePolicies.max_unacked_messages_per_subscription); - topicPolicies.getMessageTTLInSeconds().updateNamespaceValue(namespacePolicies.message_ttl_in_seconds); - topicPolicies.getMaxSubscriptionsPerTopic().updateNamespaceValue(namespacePolicies.max_subscriptions_per_topic); - topicPolicies.getMaxProducersPerTopic().updateNamespaceValue(namespacePolicies.max_producers_per_topic); - topicPolicies.getMaxConsumerPerTopic().updateNamespaceValue(namespacePolicies.max_consumers_per_topic); + .updateNamespaceValue(normalizeValue(namespacePolicies.max_unacked_messages_per_subscription)); + topicPolicies.getMessageTTLInSeconds() + .updateNamespaceValue(normalizeValue(namespacePolicies.message_ttl_in_seconds)); + topicPolicies.getMaxSubscriptionsPerTopic() + .updateNamespaceValue(normalizeValue(namespacePolicies.max_subscriptions_per_topic)); + topicPolicies.getMaxProducersPerTopic() + .updateNamespaceValue(normalizeValue(namespacePolicies.max_producers_per_topic)); + topicPolicies.getMaxConsumerPerTopic() + .updateNamespaceValue(normalizeValue(namespacePolicies.max_consumers_per_topic)); topicPolicies.getMaxConsumersPerSubscription() - .updateNamespaceValue(namespacePolicies.max_consumers_per_subscription); + .updateNamespaceValue(normalizeValue(namespacePolicies.max_consumers_per_subscription)); topicPolicies.getInactiveTopicPolicies().updateNamespaceValue(namespacePolicies.inactive_topic_policies); topicPolicies.getDeduplicationEnabled().updateNamespaceValue(namespacePolicies.deduplicationEnabled); topicPolicies.getDeduplicationSnapshotIntervalSeconds().updateNamespaceValue( @@ -312,6 +319,10 @@ protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) { updateEntryFilters(); } + private Integer normalizeValue(Integer policyValue) { + return policyValue != null && policyValue < 0 ? null : policyValue; + } + private void updateNamespaceDispatchRate(Policies namespacePolicies, String cluster) { DispatchRateImpl dispatchRate = namespacePolicies.topicDispatchRate.get(cluster); if (dispatchRate == null) { @@ -370,12 +381,11 @@ private void updateTopicPolicyByBrokerConfig() { topicPolicies.getMaxConsumerPerTopic().updateBrokerValue(config.getMaxConsumersPerTopic()); topicPolicies.getMaxConsumersPerSubscription().updateBrokerValue(config.getMaxConsumersPerSubscription()); topicPolicies.getDeduplicationEnabled().updateBrokerValue(config.isBrokerDeduplicationEnabled()); - topicPolicies.getRetentionPolicies().updateBrokerValue(new RetentionPolicies( - config.getDefaultRetentionTimeInMinutes(), config.getDefaultRetentionSizeInMB())); - topicPolicies.getDeduplicationSnapshotIntervalSeconds().updateBrokerValue( - config.getBrokerDeduplicationSnapshotIntervalSeconds()); - topicPolicies.getMaxUnackedMessagesOnConsumer() - .updateBrokerValue(config.getMaxUnackedMessagesPerConsumer()); + topicPolicies.getRetentionPolicies().updateBrokerValue( + new RetentionPolicies(config.getDefaultRetentionTimeInMinutes(), config.getDefaultRetentionSizeInMB())); + topicPolicies.getDeduplicationSnapshotIntervalSeconds() + .updateBrokerValue(config.getBrokerDeduplicationSnapshotIntervalSeconds()); + topicPolicies.getMaxUnackedMessagesOnConsumer().updateBrokerValue(config.getMaxUnackedMessagesPerConsumer()); topicPolicies.getMaxUnackedMessagesOnSubscription() .updateBrokerValue(config.getMaxUnackedMessagesPerSubscription()); //init backlogQuota diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java index 1b1b383e930e3..f33202c3c4033 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java @@ -18,17 +18,21 @@ */ package org.apache.pulsar.broker.service; +import static org.testng.Assert.assertEquals; import org.apache.pulsar.broker.PulsarServerException; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; +import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.Policies; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; - +import lombok.Cleanup; import java.util.Optional; import java.util.Properties; @@ -112,4 +116,40 @@ public void testNamespaceServicePulsarClientConfiguration() { Assert.assertEquals(clientConf.getMemoryLimitBytes(), 100000); } + @Test + public void testOldNamespacePolicy() throws Exception { + + String ns = "prop/oldNsWithDefaultNonNullValues"; + String topic = "persistent://" + ns + "/t1"; + Policies policies = new Policies(); + policies.max_consumers_per_subscription = -1; + policies.max_consumers_per_topic = -1; + policies.max_producers_per_topic = -1; + policies.max_subscriptions_per_topic = -1; + policies.max_topics_per_namespace = -1; + policies.max_unacked_messages_per_consumer = -1; + policies.max_unacked_messages_per_subscription = -1; + admin.namespaces().createNamespace(ns, policies); + + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topic).create(); + PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get(); + assertEquals(topicRef.topicPolicies.getMaxUnackedMessagesOnSubscription().get(), + conf.getMaxUnackedMessagesPerSubscription()); + assertEquals(topicRef.topicPolicies.getMaxConsumersPerSubscription().get(), + conf.getMaxConsumersPerSubscription()); + assertEquals(topicRef.topicPolicies.getMaxConsumerPerTopic().get(), + conf.getMaxConsumersPerTopic()); + assertEquals(topicRef.topicPolicies.getMaxProducersPerTopic().get(), + conf.getMaxProducersPerTopic()); + assertEquals(topicRef.topicPolicies.getMaxSubscriptionsPerTopic().get(), + conf.getMaxSubscriptionsPerTopic()); + assertEquals(topicRef.topicPolicies.getTopicMaxMessageSize().get(), + conf.getMaxMessageSize()); + assertEquals(topicRef.topicPolicies.getMaxUnackedMessagesOnConsumer().get(), + conf.getMaxUnackedMessagesPerConsumer()); + + + } } From 21647a1fc69ff46e65b6eaa37dd6d435e9f8eaef Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 19 Apr 2024 19:12:34 -0700 Subject: [PATCH 093/580] [fix] Bump golang.org/x/net from 0.17.0 to 0.23.0 in /pulsar-function-go (#22540) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Matteo Merli --- pulsar-function-go/examples/go.mod | 8 ++++---- pulsar-function-go/examples/go.sum | 16 ++++++++-------- pulsar-function-go/go.mod | 8 ++++---- pulsar-function-go/go.sum | 16 ++++++++-------- 4 files changed, 24 insertions(+), 24 deletions(-) diff --git a/pulsar-function-go/examples/go.mod b/pulsar-function-go/examples/go.mod index 31e1cc7769b92..59e695f5a33eb 100644 --- a/pulsar-function-go/examples/go.mod +++ b/pulsar-function-go/examples/go.mod @@ -42,11 +42,11 @@ require ( github.com/spaolacci/murmur3 v1.1.0 // indirect github.com/stretchr/testify v1.8.4 // indirect go.uber.org/atomic v1.7.0 // indirect - golang.org/x/crypto v0.17.0 // indirect - golang.org/x/net v0.17.0 // indirect + golang.org/x/crypto v0.21.0 // indirect + golang.org/x/net v0.23.0 // indirect golang.org/x/oauth2 v0.13.0 // indirect - golang.org/x/sys v0.15.0 // indirect - golang.org/x/term v0.15.0 // indirect + golang.org/x/sys v0.18.0 // indirect + golang.org/x/term v0.18.0 // indirect golang.org/x/text v0.14.0 // indirect google.golang.org/appengine v1.6.8 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 // indirect diff --git a/pulsar-function-go/examples/go.sum b/pulsar-function-go/examples/go.sum index 5d2429673f028..85390cf32e59a 100644 --- a/pulsar-function-go/examples/go.sum +++ b/pulsar-function-go/examples/go.sum @@ -393,8 +393,8 @@ golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= -golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= +golang.org/x/crypto v0.21.0 h1:X31++rzVUdKhX5sWmSOFZxx8UW/ldWx55cbf08iNAMA= +golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -473,8 +473,8 @@ golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLd golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= -golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= +golang.org/x/net v0.23.0 h1:7EYJ93RZ9vYSZAIb2x3lnuvqO5zneoD6IvWjuhfxjTs= +golang.org/x/net v0.23.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -559,12 +559,12 @@ golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= -golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= +golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.15.0 h1:y/Oo/a/q3IXu26lQgl04j/gjuBDOBlx7X6Om1j2CPW4= -golang.org/x/term v0.15.0/go.mod h1:BDl952bC7+uMoWR75FIrCDx79TPU9oHkTZ9yRbYOrX0= +golang.org/x/term v0.18.0 h1:FcHjZXDMxI8mM3nwhX9HlKop4C0YQvCVCdwYl2wOtE8= +golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= diff --git a/pulsar-function-go/go.mod b/pulsar-function-go/go.mod index 1a0f2990f006f..bb5c18a4499e2 100644 --- a/pulsar-function-go/go.mod +++ b/pulsar-function-go/go.mod @@ -45,11 +45,11 @@ require ( github.com/rogpeppe/go-internal v1.12.0 // indirect github.com/spaolacci/murmur3 v1.1.0 // indirect go.uber.org/atomic v1.7.0 // indirect - golang.org/x/crypto v0.17.0 // indirect - golang.org/x/net v0.17.0 // indirect + golang.org/x/crypto v0.21.0 // indirect + golang.org/x/net v0.23.0 // indirect golang.org/x/oauth2 v0.13.0 // indirect - golang.org/x/sys v0.15.0 // indirect - golang.org/x/term v0.15.0 // indirect + golang.org/x/sys v0.18.0 // indirect + golang.org/x/term v0.18.0 // indirect golang.org/x/text v0.14.0 // indirect google.golang.org/appengine v1.6.8 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 // indirect diff --git a/pulsar-function-go/go.sum b/pulsar-function-go/go.sum index 2cadeb1331f30..d840906772c56 100644 --- a/pulsar-function-go/go.sum +++ b/pulsar-function-go/go.sum @@ -393,8 +393,8 @@ golang.org/x/crypto v0.0.0-20190820162420-60c769a6c586/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= -golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= +golang.org/x/crypto v0.21.0 h1:X31++rzVUdKhX5sWmSOFZxx8UW/ldWx55cbf08iNAMA= +golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -473,8 +473,8 @@ golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLd golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= -golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= +golang.org/x/net v0.23.0 h1:7EYJ93RZ9vYSZAIb2x3lnuvqO5zneoD6IvWjuhfxjTs= +golang.org/x/net v0.23.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -559,12 +559,12 @@ golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= -golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= +golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.15.0 h1:y/Oo/a/q3IXu26lQgl04j/gjuBDOBlx7X6Om1j2CPW4= -golang.org/x/term v0.15.0/go.mod h1:BDl952bC7+uMoWR75FIrCDx79TPU9oHkTZ9yRbYOrX0= +golang.org/x/term v0.18.0 h1:FcHjZXDMxI8mM3nwhX9HlKop4C0YQvCVCdwYl2wOtE8= +golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From 4a887217d835629cafb393ddf331441b484d4e2c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 22 Apr 2024 07:49:34 +0300 Subject: [PATCH 094/580] [improve][broker] Support X-Forwarded-For and HA Proxy Protocol for resolving original client IP of http/https requests (#22524) --- conf/broker.conf | 10 + conf/functions_worker.yml | 10 + conf/proxy.conf | 10 + conf/standalone.conf | 10 + conf/websocket.conf | 10 + pom.xml | 1 + .../pulsar/broker/ServiceConfiguration.java | 16 ++ .../broker/web/JettyRequestLogFactory.java | 195 +++++++++++++++++- pulsar-broker/pom.xml | 7 + .../apache/pulsar/broker/web/WebService.java | 37 +++- .../web/WebServiceOriginalClientIPTest.java | 155 ++++++++++++++ pulsar-broker/src/test/resources/log4j2.xml | 3 +- .../pulsar/functions/worker/WorkerConfig.java | 16 ++ .../functions/worker/rest/WorkerServer.java | 38 +++- pulsar-proxy/pom.xml | 6 + .../proxy/server/ProxyConfiguration.java | 16 ++ .../proxy/server/ProxyServiceStarter.java | 31 ++- .../apache/pulsar/proxy/server/WebServer.java | 34 ++- .../server/ProxyOriginalClientIPTest.java | 157 ++++++++++++++ ...roxyServiceStarterDisableZeroCopyTest.java | 2 +- .../proxy/server/ProxyServiceStarterTest.java | 2 +- .../server/ProxyServiceTlsStarterTest.java | 2 +- pulsar-proxy/src/test/resources/log4j2.xml | 36 ++++ .../pulsar/websocket/service/ProxyServer.java | 39 +++- .../service/WebSocketProxyConfiguration.java | 14 ++ 25 files changed, 835 insertions(+), 22 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java create mode 100644 pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java create mode 100644 pulsar-proxy/src/test/resources/log4j2.xml diff --git a/conf/broker.conf b/conf/broker.conf index fd6bba0f45d2c..d482f77da7cb5 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -88,6 +88,16 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Number of threads to config Netty Acceptor. Default is 1 numAcceptorThreads= diff --git a/conf/functions_worker.yml b/conf/functions_worker.yml index 3871c74a88778..6f995576ebd64 100644 --- a/conf/functions_worker.yml +++ b/conf/functions_worker.yml @@ -27,6 +27,16 @@ workerHostname: localhost workerPort: 6750 workerPortTls: 6751 +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled: false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor: false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses: null + # The Configuration metadata store url # Examples: # * zk:my-zk-1:2181,my-zk-2:2181,my-zk-3:2181 diff --git a/conf/proxy.conf b/conf/proxy.conf index 5a9d433f39ceb..6e6c960e8009e 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -63,6 +63,16 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Enables zero-copy transport of data across network interfaces using the splice system call. # Zero copy mode cannot be used when TLS is enabled or when proxyLogLevel is > 0. proxyZeroCopyModeEnabled=true diff --git a/conf/standalone.conf b/conf/standalone.conf index 5c94d63817a12..b04e5ccefa640 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -51,6 +51,16 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Number of threads to use for Netty IO. Default is set to 2 * Runtime.getRuntime().availableProcessors() numIOThreads= diff --git a/conf/websocket.conf b/conf/websocket.conf index 490cff2722ee5..9051f3b590c8e 100644 --- a/conf/websocket.conf +++ b/conf/websocket.conf @@ -46,6 +46,16 @@ statusFilePath= # Hostname or IP address the service binds on, default is 0.0.0.0. bindAddress=0.0.0.0 +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Name of the pulsar cluster to connect to clusterName= diff --git a/pom.xml b/pom.xml index c7fba94abd8ea..d4b14efc356ba 100644 --- a/pom.xml +++ b/pom.xml @@ -278,6 +278,7 @@ flexible messaging model and an intuitive client API. 1.5.4 5.4.0 2.33.2 + 1.0.3 0.6.1 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 2b58cbc2d1178..156c83bd6960c 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 @@ -250,6 +250,22 @@ public class ServiceConfiguration implements PulsarConfiguration { + " when getting topic statistics data.") private boolean haProxyProtocolEnabled; + @FieldContext(category = CATEGORY_SERVER, + doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext( category = CATEGORY_SERVER, doc = "Number of threads to use for Netty Acceptor." diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java index e5daa5852b51f..fc88647eb49ea 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java @@ -18,9 +18,23 @@ */ package org.apache.pulsar.broker.web; +import java.net.InetSocketAddress; import java.util.TimeZone; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.extern.slf4j.Slf4j; +import org.eclipse.jetty.io.Connection; +import org.eclipse.jetty.io.EndPoint; +import org.eclipse.jetty.server.Connector; import org.eclipse.jetty.server.CustomRequestLog; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.Request; +import org.eclipse.jetty.server.RequestLog; +import org.eclipse.jetty.server.Response; +import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.Slf4jRequestLogWriter; +import org.eclipse.jetty.util.HostPort; +import org.eclipse.jetty.util.component.ContainerLifeCycle; /** * Class to standardize initialization of a Jetty request logger for all pulsar components. @@ -58,7 +72,184 @@ public class JettyRequestLogFactory { * Build a new Jetty request logger using the format defined in this class. * @return a request logger */ - public static CustomRequestLog createRequestLogger() { - return new CustomRequestLog(new Slf4jRequestLogWriter(), LOG_FORMAT); + public static RequestLog createRequestLogger() { + return createRequestLogger(false, null); + } + + /** + * Build a new Jetty request logger using the format defined in this class. + * @param showDetailedAddresses whether to show detailed addresses and ports in logs + * @return a request logger + */ + public static RequestLog createRequestLogger(boolean showDetailedAddresses, Server server) { + if (!showDetailedAddresses) { + return new CustomRequestLog(new Slf4jRequestLogWriter(), LOG_FORMAT); + } else { + return new OriginalClientIPRequestLog(server); + } + } + + /** + * Logs the original and real remote (client) and local (server) IP addresses + * when detailed addresses are enabled. + * Tracks the real addresses of remote and local using a registered Connection.Listener + * when detailed addresses are enabled. + * This is necessary when Proxy Protocol is used to pass the original client IP. + */ + @Slf4j + private static class OriginalClientIPRequestLog extends ContainerLifeCycle implements RequestLog { + private final ThreadLocal requestLogStringBuilder = ThreadLocal.withInitial(StringBuilder::new); + private final CustomRequestLog delegate; + private final Slf4jRequestLogWriter delegateLogWriter; + + OriginalClientIPRequestLog(Server server) { + delegate = new CustomRequestLog(this::write, LOG_FORMAT); + addBean(delegate); + delegateLogWriter = new Slf4jRequestLogWriter(); + addBean(delegateLogWriter); + if (server != null) { + for (Connector connector : server.getConnectors()) { + // adding the listener is only necessary for connectors that use ProxyConnectionFactory + if (connector.getDefaultConnectionFactory() instanceof ProxyConnectionFactory) { + connector.addBean(proxyProtocolOriginalEndpointListener); + } + } + } + } + + void write(String requestEntry) { + StringBuilder sb = requestLogStringBuilder.get(); + sb.setLength(0); + sb.append(requestEntry); + } + + @Override + public void log(Request request, Response response) { + delegate.log(request, response); + StringBuilder sb = requestLogStringBuilder.get(); + sb.append(" [R:"); + sb.append(request.getRemoteHost()); + sb.append(':'); + sb.append(request.getRemotePort()); + InetSocketAddress realRemoteAddress = lookupRealAddress(request.getHttpChannel().getRemoteAddress()); + if (realRemoteAddress != null) { + String realRemoteHost = HostPort.normalizeHost(realRemoteAddress.getHostString()); + int realRemotePort = realRemoteAddress.getPort(); + if (!realRemoteHost.equals(request.getRemoteHost()) || realRemotePort != request.getRemotePort()) { + sb.append(" via "); + sb.append(realRemoteHost); + sb.append(':'); + sb.append(realRemotePort); + } + } + sb.append("]->[L:"); + InetSocketAddress realLocalAddress = lookupRealAddress(request.getHttpChannel().getLocalAddress()); + if (realLocalAddress != null) { + String realLocalHost = HostPort.normalizeHost(realLocalAddress.getHostString()); + int realLocalPort = realLocalAddress.getPort(); + sb.append(realLocalHost); + sb.append(':'); + sb.append(realLocalPort); + if (!realLocalHost.equals(request.getLocalAddr()) || realLocalPort != request.getLocalPort()) { + sb.append(" dst "); + sb.append(request.getLocalAddr()); + sb.append(':'); + sb.append(request.getLocalPort()); + } + } else { + sb.append(request.getLocalAddr()); + sb.append(':'); + sb.append(request.getLocalPort()); + } + sb.append(']'); + try { + delegateLogWriter.write(sb.toString()); + } catch (Exception e) { + log.warn("Failed to write request log", e); + } + } + + private InetSocketAddress lookupRealAddress(InetSocketAddress socketAddress) { + if (socketAddress == null) { + return null; + } + if (proxyProtocolRealAddressMapping.isEmpty()) { + return socketAddress; + } + AddressEntry entry = proxyProtocolRealAddressMapping.get(new AddressKey(socketAddress.getHostString(), + socketAddress.getPort())); + if (entry != null) { + return entry.realAddress; + } else { + return socketAddress; + } + } + + private final Connection.Listener proxyProtocolOriginalEndpointListener = + new ProxyProtocolOriginalEndpointListener(); + + private final ConcurrentHashMap proxyProtocolRealAddressMapping = + new ConcurrentHashMap<>(); + + // Use a record as key since InetSocketAddress hash code changes if the address gets resolved + record AddressKey(String hostString, int port) { + + } + + record AddressEntry(InetSocketAddress realAddress, AtomicInteger referenceCount) { + + } + + // Tracks the real addresses of remote and local when detailed addresses are enabled. + // This is necessary when Proxy Protocol is used to pass the original client IP. + // The Proxy Protocol implementation in Jetty wraps the original endpoint with a ProxyEndPoint + // and the real endpoint information isn't available in the request object. + // This listener is added to all connectors to track the real addresses of the client and server. + class ProxyProtocolOriginalEndpointListener implements Connection.Listener { + @Override + public void onOpened(Connection connection) { + handleConnection(connection, true); + } + + @Override + public void onClosed(Connection connection) { + handleConnection(connection, false); + } + + private void handleConnection(Connection connection, boolean increment) { + if (connection.getEndPoint() instanceof ProxyConnectionFactory.ProxyEndPoint) { + ProxyConnectionFactory.ProxyEndPoint proxyEndPoint = + (ProxyConnectionFactory.ProxyEndPoint) connection.getEndPoint(); + EndPoint originalEndpoint = proxyEndPoint.unwrap(); + mapAddress(proxyEndPoint.getLocalAddress(), originalEndpoint.getLocalAddress(), increment); + mapAddress(proxyEndPoint.getRemoteAddress(), originalEndpoint.getRemoteAddress(), increment); + } + } + + private void mapAddress(InetSocketAddress current, InetSocketAddress real, boolean increment) { + // don't add the mapping if the current address is the same as the real address + if (real != null && current != null && current.equals(real)) { + return; + } + AddressKey key = new AddressKey(current.getHostString(), current.getPort()); + proxyProtocolRealAddressMapping.compute(key, (__, entry) -> { + if (entry == null) { + if (increment) { + entry = new AddressEntry(real, new AtomicInteger(1)); + } + } else { + if (increment) { + entry.referenceCount.incrementAndGet(); + } else { + if (entry.referenceCount.decrementAndGet() == 0) { + // remove the entry if the reference count drops to 0 + entry = null; + } + } + } + return entry; + }); + } + } } } diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index e15e024ea8158..3548877912199 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -176,6 +176,13 @@ test + + io.github.hakky54 + consolecaptor + ${consolecaptor.version} + test + + io.streamnative.oxia oxia-testcontainers diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java index 8dc36e2917ed1..9a439268a8b4f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java @@ -31,12 +31,18 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.RequestLog; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandler; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; @@ -103,9 +109,18 @@ public WebService(PulsarService pulsar) throws PulsarServerException { Optional port = config.getWebServicePort(); HttpConfiguration httpConfig = new HttpConfiguration(); + if (config.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } httpConfig.setRequestHeaderSize(pulsar.getConfig().getHttpMaxRequestHeaderSize()); + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); if (port.isPresent()) { - httpConnector = new ServerConnector(server, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + httpConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpConnector.setPort(port.get()); httpConnector.setHost(pulsar.getBindAddress()); connectors.add(httpConnector); @@ -144,7 +159,18 @@ public WebService(PulsarService pulsar) throws PulsarServerException { config.getWebServiceTlsProtocols(), config.getTlsCertRefreshCheckDurationSec()); } - httpsConnector = new ServerConnector(server, sslCtxFactory, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + httpsConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpsConnector.setPort(tlsPort.get()); httpsConnector.setHost(pulsar.getBindAddress()); connectors.add(httpsConnector); @@ -284,7 +310,12 @@ public void addStaticResources(String basePath, String resourcePath) { public void start() throws PulsarServerException { try { RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = pulsar.getConfiguration().getWebServiceLogDetailedAddresses() != null + ? pulsar.getConfiguration().getWebServiceLogDetailedAddresses() : + (pulsar.getConfiguration().isWebServiceHaProxyProtocolEnabled() + || pulsar.getConfiguration().isWebServiceTrustXForwardedFor()); + RequestLog requestLogger = JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server); + requestLogHandler.setRequestLog(requestLogger); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java new file mode 100644 index 0000000000000..7f7fa85bd3bb4 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java @@ -0,0 +1,155 @@ +/* + * 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.web; + +import static org.testng.Assert.assertTrue; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import nl.altindag.console.ConsoleCaptor; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.assertj.core.api.ThrowingConsumer; +import org.awaitility.Awaitility; +import org.eclipse.jetty.client.HttpClient; +import org.eclipse.jetty.client.ProxyProtocolClientConnectionFactory.V2; +import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.util.ssl.SslContextFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class WebServiceOriginalClientIPTest extends MockedPulsarServiceBaseTest { + HttpClient httpClient; + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + httpClient = new HttpClient(new SslContextFactory(true)); + httpClient.start(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + if (httpClient != null) { + httpClient.stop(); + } + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setWebServiceTrustXForwardedFor(true); + conf.setWebServiceHaProxyProtocolEnabled(true); + conf.setWebServicePortTls(Optional.of(0)); + conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); + conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + } + + @DataProvider(name = "tlsEnabled") + public Object[][] tlsEnabled() { + return new Object[][] { { true }, { false } }; + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromXForwardedForHeaderAndLogged(boolean tlsEnabled) throws Exception { + String metricsUrl = + (tlsEnabled ? pulsar.getWebServiceAddressTls() : pulsar.getWebServiceAddress()) + "/metrics/"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(metricsUrl) + .header("X-Forwarded-For", "11.22.33.44:12345") + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("process_cpu_seconds_total")); + + // Validate that the client IP passed in X-Forwarded-For is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("RequestLog") && line.contains("[R:11.22.33.44:12345 via "))); + }); + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromForwardedHeaderAndLogged(boolean tlsEnabled) throws Exception { + String metricsUrl = + (tlsEnabled ? pulsar.getWebServiceAddressTls() : pulsar.getWebServiceAddress()) + "/metrics/"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(metricsUrl) + .header("Forwarded", "for=11.22.33.44:12345") + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("process_cpu_seconds_total")); + + // Validate that the client IP passed in Forwarded is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("RequestLog") && line.contains("[R:11.22.33.44:12345 via "))); + }); + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromHAProxyProtocolAndLogged(boolean tlsEnabled) throws Exception { + String metricsUrl = (tlsEnabled ? pulsar.getWebServiceAddressTls() : pulsar.getWebServiceAddress()) + "/metrics/"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(metricsUrl) + // Jetty client will add HA Proxy protocol header with the given IP to the request + .tag(new V2.Tag(V2.Tag.Command.PROXY, null, V2.Tag.Protocol.STREAM, + // source IP and port + "99.22.33.44", 1234, + // destination IP and port + "5.4.3.1", 4321, + null)) + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("process_cpu_seconds_total")); + + // Validate that the client IP and destination IP passed in HA Proxy protocol is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("RequestLog") && line.contains("[R:99.22.33.44:1234 via ") + && line.contains(" dst 5.4.3.1:4321]"))); + }); + } + + void performLoggingTest(ThrowingConsumer testFunction) { + ConsoleCaptor consoleCaptor = new ConsoleCaptor(); + try { + Awaitility.await().atMost(Duration.of(2, ChronoUnit.SECONDS)).untilAsserted(() -> { + consoleCaptor.clearOutput(); + testFunction.accept(consoleCaptor); + }); + } finally { + consoleCaptor.close(); + System.out.println("--- Captured console output:"); + consoleCaptor.getStandardOutput().forEach(System.out::println); + consoleCaptor.getErrorOutput().forEach(System.err::println); + System.out.println("--- End of captured console output"); + } + } +} diff --git a/pulsar-broker/src/test/resources/log4j2.xml b/pulsar-broker/src/test/resources/log4j2.xml index 38a57df80d57b..09a89702ee2ac 100644 --- a/pulsar-broker/src/test/resources/log4j2.xml +++ b/pulsar-broker/src/test/resources/log4j2.xml @@ -23,7 +23,8 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://logging.apache.org/log4j/2.0/config https://logging.apache.org/log4j/2.0/log4j-core.xsd"> - + + diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java index ec0e620d0ae8b..036311ea13230 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java @@ -163,6 +163,22 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { + "(0 to disable limiting)") private int maxHttpServerConnections = 2048; + @FieldContext(category = CATEGORY_WORKER, + doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(category = CATEGORY_WORKER, doc = + "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(category = CATEGORY_WORKER, doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext( category = CATEGORY_WORKER, required = false, diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java index 2b3ea30121015..583d8ce558b08 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java @@ -35,10 +35,17 @@ import org.apache.pulsar.functions.worker.rest.api.v2.WorkerApiV2Resource; import org.apache.pulsar.functions.worker.rest.api.v2.WorkerStatsApiV2Resource; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.HttpConfiguration; +import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerCollection; @@ -88,10 +95,21 @@ private void init() { server.addBean(new ConnectionLimit(workerConfig.getMaxHttpServerConnections(), server)); } + HttpConfiguration httpConfig = new HttpConfiguration(); + if (workerConfig.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); + List connectors = new ArrayList<>(); if (this.workerConfig.getWorkerPort() != null) { log.info("Configuring http server on port={}", this.workerConfig.getWorkerPort()); - httpConnector = new ServerConnector(server); + List connectionFactories = new ArrayList<>(); + if (workerConfig.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + httpConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpConnector.setPort(this.workerConfig.getWorkerPort()); connectors.add(httpConnector); } @@ -109,7 +127,10 @@ private void init() { workerConfig.isAuthenticateMetricsEndpoint(), filterInitializer)); RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = workerConfig.getWebServiceLogDetailedAddresses() != null + ? workerConfig.getWebServiceLogDetailedAddresses() : + (workerConfig.isWebServiceHaProxyProtocolEnabled() || workerConfig.isWebServiceTrustXForwardedFor()); + requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); @@ -161,7 +182,18 @@ private void init() { workerConfig.getTlsCertRefreshCheckDurationSec() ); } - httpsConnector = new ServerConnector(server, sslCtxFactory); + List connectionFactories = new ArrayList<>(); + if (workerConfig.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + httpsConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpsConnector.setPort(this.workerConfig.getWorkerPortTls()); connectors.add(httpsConnector); } catch (Exception e) { diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 64ca301facf4d..a30e23b8d4781 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -209,6 +209,12 @@ ${wiremock.version} test + + io.github.hakky54 + consolecaptor + ${consolecaptor.version} + test + 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 39c8fb5e086fd..d65408748f432 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 @@ -268,6 +268,22 @@ public class ProxyConfiguration implements PulsarConfiguration { doc = "Enable or disable the proxy protocol.") private boolean haProxyProtocolEnabled; + @FieldContext(category = CATEGORY_SERVER, + doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext(category = CATEGORY_SERVER, doc = "Enables zero-copy transport of data across network interfaces using the spice. " + "Zero copy mode cannot be used when TLS is enabled or when proxyLogLevel is > 0.") diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index 50a8e3ab7d753..10121e7f5d61d 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -35,6 +35,7 @@ import java.util.Collections; import java.util.Date; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import lombok.Getter; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.core.util.datetime.FixedDateFormat; @@ -109,8 +110,15 @@ public class ProxyServiceStarter { private WebServer server; private WebSocketService webSocketService; private static boolean metricsInitialized; + private boolean embeddedMode; public ProxyServiceStarter(String[] args) throws Exception { + this(args, null, false); + } + + public ProxyServiceStarter(String[] args, Consumer proxyConfigurationCustomizer, + boolean embeddedMode) throws Exception { + this.embeddedMode = embeddedMode; try { DateFormat dateFormat = new SimpleDateFormat( FixedDateFormat.FixedFormat.ISO8601_OFFSET_DATE_TIME_HHMM.getPattern()); @@ -132,15 +140,26 @@ public ProxyServiceStarter(String[] args) throws Exception { CmdGenerateDocs cmd = new CmdGenerateDocs("pulsar"); cmd.addCommand("proxy", commander); cmd.run(null); - System.exit(0); + if (embeddedMode) { + return; + } else { + System.exit(0); + } } } catch (Exception e) { commander.getErr().println(e); - System.exit(1); + if (embeddedMode) { + return; + } else { + System.exit(1); + } } // load config file config = PulsarConfigurationLoader.create(configFile, ProxyConfiguration.class); + if (proxyConfigurationCustomizer != null) { + proxyConfigurationCustomizer.accept(config); + } if (!isBlank(zookeeperServers)) { // Use zookeeperServers from command line @@ -230,7 +249,9 @@ public void start() throws Exception { // create a web-service server = new WebServer(config, authenticationService); - Runtime.getRuntime().addShutdownHook(new Thread(this::close)); + if (!embeddedMode) { + Runtime.getRuntime().addShutdownHook(new Thread(this::close)); + } proxyService.start(); @@ -293,7 +314,9 @@ public void close() { } catch (Exception e) { log.warn("server couldn't stop gracefully {}", e.getMessage(), e); } finally { - LogManager.shutdown(); + if (!embeddedMode) { + LogManager.shutdown(); + } } } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java index b95bbcab08b11..478b911eb23cf 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java @@ -37,13 +37,18 @@ import org.apache.pulsar.broker.web.RateLimitingFilter; import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; import org.eclipse.jetty.server.Connector; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerCollection; @@ -93,12 +98,21 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication List connectors = new ArrayList<>(); HttpConfiguration httpConfig = new HttpConfiguration(); + if (config.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } httpConfig.setOutputBufferSize(config.getHttpOutputBufferSize()); httpConfig.setRequestHeaderSize(config.getHttpMaxRequestHeaderSize()); + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); if (config.getWebServicePort().isPresent()) { this.externalServicePort = config.getWebServicePort().get(); - connector = new ServerConnector(server, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + connector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connector.setHost(config.getBindAddress()); connector.setPort(externalServicePort); connectors.add(connector); @@ -133,7 +147,18 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication config.getWebServiceTlsProtocols(), config.getTlsCertRefreshCheckDurationSec()); } - connectorTls = new ServerConnector(server, sslCtxFactory, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + connectorTls = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connectorTls.setPort(config.getWebServicePortTls().get()); connectorTls.setHost(config.getBindAddress()); connectors.add(connectorTls); @@ -281,7 +306,10 @@ public int getExternalServicePort() { public void start() throws Exception { RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = config.getWebServiceLogDetailedAddresses() != null + ? config.getWebServiceLogDetailedAddresses() : + (config.isWebServiceHaProxyProtocolEnabled() || config.isWebServiceTrustXForwardedFor()); + requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java new file mode 100644 index 0000000000000..b267439d47113 --- /dev/null +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java @@ -0,0 +1,157 @@ +/* + * 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.testng.Assert.assertTrue; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import nl.altindag.console.ConsoleCaptor; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.assertj.core.api.ThrowingConsumer; +import org.awaitility.Awaitility; +import org.eclipse.jetty.client.HttpClient; +import org.eclipse.jetty.client.ProxyProtocolClientConnectionFactory.V2; +import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.util.ssl.SslContextFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class ProxyOriginalClientIPTest extends MockedPulsarServiceBaseTest { + static final String[] ARGS = new String[]{"-c", "./src/test/resources/proxy.conf"}; + HttpClient httpClient; + ProxyServiceStarter serviceStarter; + String webServiceUrl; + String webServiceUrlTls; + + @Override + @BeforeClass + protected void setup() throws Exception { + internalSetup(); + serviceStarter = new ProxyServiceStarter(ARGS, proxyConfig -> { + proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); + proxyConfig.setBrokerWebServiceURL(pulsar.getWebServiceAddress()); + proxyConfig.setWebServicePort(Optional.of(0)); + proxyConfig.setWebServicePortTls(Optional.of(0)); + proxyConfig.setTlsEnabledWithBroker(false); + proxyConfig.setTlsCertificateFilePath(PROXY_CERT_FILE_PATH); + proxyConfig.setTlsKeyFilePath(PROXY_KEY_FILE_PATH); + proxyConfig.setServicePort(Optional.of(0)); + proxyConfig.setWebSocketServiceEnabled(true); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setClusterName(configClusterName); + proxyConfig.setWebServiceTrustXForwardedFor(true); + proxyConfig.setWebServiceHaProxyProtocolEnabled(true); + }, true); + serviceStarter.start(); + webServiceUrl = "http://localhost:" + serviceStarter.getServer().getListenPortHTTP().get(); + webServiceUrlTls = "https://localhost:" + serviceStarter.getServer().getListenPortHTTPS().get(); + httpClient = new HttpClient(new SslContextFactory(true)); + httpClient.start(); + } + + @Override + @AfterClass(alwaysRun = true) + protected void cleanup() throws Exception { + internalCleanup(); + if (serviceStarter != null) { + serviceStarter.close(); + } + if (httpClient != null) { + httpClient.stop(); + } + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setWebServiceTrustXForwardedFor(true); + } + + @DataProvider(name = "tlsEnabled") + public Object[][] tlsEnabled() { + return new Object[][] { { true }, { false } }; + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromXForwardedForHeaderAndLogged(boolean tlsEnabled) throws Exception { + String url = (tlsEnabled ? webServiceUrlTls : webServiceUrl) + "/admin/v2/brokers/leaderBroker"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(url) + .header("X-Forwarded-For", "11.22.33.44") + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("\"brokerId\":\"" + pulsar.getBrokerId() + "\"")); + + // Validate that the client IP passed in X-Forwarded-For is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-external-web-") && line.contains("RequestLog") + && line.contains("R:11.22.33.44")), "Expected to find client IP in proxy logs"); + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-web-") && line.contains("RequestLog") + && line.contains("R:11.22.33.44")), "Expected to find client IP in broker logs"); + }); + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromHAProxyProtocolAndLogged(boolean tlsEnabled) throws Exception { + String url = (tlsEnabled ? webServiceUrlTls : webServiceUrl) + "/admin/v2/brokers/leaderBroker"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(url) + // Jetty client will add HA Proxy protocol header with the given IP to the request + .tag(new V2.Tag("99.22.33.44", 1234)) + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("\"brokerId\":\"" + pulsar.getBrokerId() + "\"")); + + // Validate that the client IP passed in HA proxy protocol is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-external-web-") && line.contains("RequestLog") + && line.contains("R:99.22.33.44")), "Expected to find client IP in proxy logs"); + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-web-") && line.contains("RequestLog") + && line.contains("R:99.22.33.44")), "Expected to find client IP in broker logs"); + }); + } + + void performLoggingTest(ThrowingConsumer testFunction) { + ConsoleCaptor consoleCaptor = new ConsoleCaptor(); + try { + Awaitility.await().atMost(Duration.of(2, ChronoUnit.SECONDS)).untilAsserted(() -> { + consoleCaptor.clearOutput(); + testFunction.accept(consoleCaptor); + }); + } finally { + consoleCaptor.close(); + System.out.println("--- Captured console output:"); + consoleCaptor.getStandardOutput().forEach(System.out::println); + consoleCaptor.getErrorOutput().forEach(System.err::println); + System.out.println("--- End of captured console output"); + } + } +} diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java index 3e598a57277a2..937526629acf0 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterDisableZeroCopyTest.java @@ -27,7 +27,7 @@ public class ProxyServiceStarterDisableZeroCopyTest extends ProxyServiceStarterT @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS); + serviceStarter = new ProxyServiceStarter(ARGS, null, true); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); serviceStarter.getConfig().setWebServicePort(Optional.of(0)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java index f263286125353..0b9b6f17d1254 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java @@ -54,7 +54,7 @@ public class ProxyServiceStarterTest extends MockedPulsarServiceBaseTest { @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS); + serviceStarter = new ProxyServiceStarter(ARGS, null, true); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); serviceStarter.getConfig().setWebServicePort(Optional.of(0)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java index 61718bbac3ab0..770424d93747c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java @@ -55,7 +55,7 @@ public class ProxyServiceTlsStarterTest extends MockedPulsarServiceBaseTest { @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS); + serviceStarter = new ProxyServiceStarter(ARGS, null, true); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerServiceURLTLS(pulsar.getBrokerServiceUrlTls()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); diff --git a/pulsar-proxy/src/test/resources/log4j2.xml b/pulsar-proxy/src/test/resources/log4j2.xml new file mode 100644 index 0000000000000..261bd2edf6980 --- /dev/null +++ b/pulsar-proxy/src/test/resources/log4j2.xml @@ -0,0 +1,36 @@ + + + + + + + + + + + + + + + diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java index 7aed43d056c67..bbb34a3e3f73d 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java @@ -35,10 +35,17 @@ import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.HttpConfiguration; +import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerCollection; @@ -73,10 +80,22 @@ public ProxyServer(WebSocketProxyConfiguration config) if (config.getMaxHttpServerConnections() > 0) { server.addBean(new ConnectionLimit(config.getMaxHttpServerConnections(), server)); } + + HttpConfiguration httpConfig = new HttpConfiguration(); + if (config.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); + List connectors = new ArrayList<>(); if (config.getWebServicePort().isPresent()) { - connector = new ServerConnector(server); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + connector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connector.setPort(config.getWebServicePort().get()); connectors.add(connector); } @@ -111,7 +130,18 @@ public ProxyServer(WebSocketProxyConfiguration config) config.getWebServiceTlsProtocols(), config.getTlsCertRefreshCheckDurationSec()); } - connectorTls = new ServerConnector(server, sslCtxFactory); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + connectorTls = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connectorTls.setPort(config.getWebServicePortTls().get()); connectors.add(connectorTls); } catch (Exception e) { @@ -169,7 +199,10 @@ public void start() throws PulsarServerException { .map(ServerConnector.class::cast).map(ServerConnector::getPort).map(Object::toString) .collect(Collectors.joining(","))); RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = conf.getWebServiceLogDetailedAddresses() != null + ? conf.getWebServiceLogDetailedAddresses() : + (conf.isWebServiceHaProxyProtocolEnabled() || conf.isWebServiceTrustXForwardedFor()); + requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java index 7acfd4a64ad35..3fcbcf4b21567 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java @@ -96,6 +96,20 @@ public class WebSocketProxyConfiguration implements PulsarConfiguration { @FieldContext(doc = "Hostname or IP address the service binds on, default is 0.0.0.0.") private String bindAddress = "0.0.0.0"; + @FieldContext(doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(doc = "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext(doc = "Maximum size of a text message during parsing in WebSocket proxy") private int webSocketMaxTextFrameSize = 1024 * 1024; // --- Authentication --- From 3a0f908e80d0863920a1258362fd782e95fe8f17 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Mon, 22 Apr 2024 19:47:03 +0800 Subject: [PATCH 095/580] [improve][test] Add topic policy test for topic API (#22546) --- .../apache/pulsar/broker/admin/AuthZTest.java | 113 ++ .../pulsar/broker/admin/TopicAuthZTest.java | 1121 ++++++++++++----- .../admin/TransactionAndSchemaAuthZTest.java | 359 ++++++ 3 files changed, 1270 insertions(+), 323 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java new file mode 100644 index 0000000000000..a710a03970d06 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java @@ -0,0 +1,113 @@ +/* + * 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.admin; + +import io.jsonwebtoken.Jwts; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.TopicOperation; +import org.apache.pulsar.security.MockedPulsarStandalone; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import static org.mockito.Mockito.doReturn; + +public class AuthZTest extends MockedPulsarStandalone { + + protected PulsarAdmin superUserAdmin; + + protected PulsarAdmin tenantManagerAdmin; + + protected AuthorizationService authorizationService; + + protected AuthorizationService orignalAuthorizationService; + + protected static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); + protected static final String TENANT_ADMIN_TOKEN = Jwts.builder() + .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); + + @BeforeMethod(alwaysRun = true) + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + authorizationService = Mockito.spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + authorizationService, true); + } + + @AfterMethod(alwaysRun = true) + public void after() throws IllegalAccessException { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + orignalAuthorizationService, true); + } + + protected AtomicBoolean setAuthorizationTopicOperationChecker(String role, Object operation) { + AtomicBoolean execFlag = new AtomicBoolean(false); + if (operation instanceof TopicOperation) { + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(2); + if (role.equals(role_)) { + TopicOperation operation_ = invocationOnMock.getArgument(1); + Assert.assertEquals(operation_, operation); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowTopicOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any()); + } else if (operation instanceof NamespaceOperation) { + doReturn(true) + .when(authorizationService).isValidOriginalPrincipal(Mockito.any(), Mockito.any(), Mockito.any()); + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(2); + if (role.equals(role_)) { + TopicOperation operation_ = invocationOnMock.getArgument(1); + Assert.assertEquals(operation_, operation); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowNamespaceOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any()); + } else { + throw new IllegalArgumentException(""); + } + + + return execFlag; + } + + protected void createTopic(String topic, boolean partitioned) throws Exception { + if (partitioned) { + superUserAdmin.topics().createPartitionedTopic(topic, 2); + } else { + superUserAdmin.topics().createNonPartitionedTopic(topic); + } + } + + protected void deleteTopic(String topic, boolean partitioned) throws Exception { + if (partitioned) { + superUserAdmin.topics().deletePartitionedTopic(topic, true); + } else { + superUserAdmin.topics().delete(topic, true); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index 3c0596d531f41..ad47ac74a8980 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -20,6 +20,7 @@ package org.apache.pulsar.broker.admin; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import io.jsonwebtoken.Jwts; import lombok.Cleanup; import lombok.SneakyThrows; @@ -38,59 +39,48 @@ import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.AutoSubscriptionCreationOverride; +import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; +import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.EntryFilters; +import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; +import org.apache.pulsar.common.policies.data.OffloadPolicies; +import org.apache.pulsar.common.policies.data.PersistencePolicies; +import org.apache.pulsar.common.policies.data.PolicyName; +import org.apache.pulsar.common.policies.data.PolicyOperation; +import org.apache.pulsar.common.policies.data.PublishRate; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.TenantInfo; -import org.apache.pulsar.security.MockedPulsarStandalone; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.UUID; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.pulsar.broker.authorization.AuthorizationService; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.transaction.Transaction; -import org.apache.pulsar.common.naming.SystemTopicNames; -import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.TopicOperation; -import org.apache.pulsar.common.schema.SchemaInfo; -import org.apache.pulsar.common.schema.SchemaType; -import org.apache.pulsar.metadata.api.MetadataStoreException; import org.mockito.Mockito; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; +import static org.mockito.Mockito.doReturn; @Test(groups = "broker-admin") -public class TopicAuthZTest extends MockedPulsarStandalone { - - private PulsarAdmin superUserAdmin; - - private PulsarAdmin tenantManagerAdmin; - - private AuthorizationService authorizationService; - - private AuthorizationService orignalAuthorizationService; - - private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); - private static final String TENANT_ADMIN_TOKEN = Jwts.builder() - .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); +public class TopicAuthZTest extends AuthZTest { @SneakyThrows @BeforeClass(alwaysRun = true) public void setup() { configureTokenAuthentication(); configureDefaultAuthorization(); - enableTransaction(); start(); - createTransactionCoordinatorAssign(16); this.superUserAdmin =PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) @@ -103,16 +93,6 @@ public void setup() { .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) .build(); - superUserAdmin.tenants().createTenant("pulsar", tenantInfo); - superUserAdmin.namespaces().createNamespace("pulsar/system"); - } - - protected void createTransactionCoordinatorAssign(int numPartitionsOfTC) throws MetadataStoreException { - getPulsarService().getPulsarResources() - .getNamespaceResources() - .getPartitionedTopicResources() - .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, - new PartitionedTopicMetadata(numPartitionsOfTC)); } @SneakyThrows @@ -127,48 +107,28 @@ public void cleanup() { close(); } - @BeforeMethod - public void before() throws IllegalAccessException { - orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); - authorizationService = Mockito.spy(orignalAuthorizationService); - FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", - authorizationService, true); - } + private AtomicBoolean setAuthorizationPolicyOperationChecker(String role, Object policyName, Object operation) { + AtomicBoolean execFlag = new AtomicBoolean(false); + if (operation instanceof PolicyOperation ) { - @AfterMethod - public void after() throws IllegalAccessException { - FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", - orignalAuthorizationService, true); - } + doReturn(true) + .when(authorizationService).isValidOriginalPrincipal(Mockito.any(), Mockito.any(), Mockito.any()); - private AtomicBoolean setAuthorizationTopicOperationChecker(String role, Object operation) { - AtomicBoolean execFlag = new AtomicBoolean(false); - if (operation instanceof TopicOperation) { - Mockito.doAnswer(invocationOnMock -> { - String role_ = invocationOnMock.getArgument(2); - if (role.equals(role_)) { - TopicOperation operation_ = invocationOnMock.getArgument(1); - Assert.assertEquals(operation_, operation); - } - execFlag.set(true); - return invocationOnMock.callRealMethod(); - }).when(authorizationService).allowTopicOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), - Mockito.any(), Mockito.any()); - } else if (operation instanceof NamespaceOperation) { Mockito.doAnswer(invocationOnMock -> { - String role_ = invocationOnMock.getArgument(2); - if (role.equals(role_)) { - TopicOperation operation_ = invocationOnMock.getArgument(1); - Assert.assertEquals(operation_, operation); - } - execFlag.set(true); - return invocationOnMock.callRealMethod(); - }).when(authorizationService).allowNamespaceOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), - Mockito.any(), Mockito.any()); + String role_ = invocationOnMock.getArgument(4); + if (role.equals(role_)) { + PolicyName policyName_ = invocationOnMock.getArgument(1); + PolicyOperation operation_ = invocationOnMock.getArgument(2); + Assert.assertEquals(operation_, operation); + Assert.assertEquals(policyName_, policyName); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowTopicPolicyOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any(), Mockito.any()); } else { throw new IllegalArgumentException(""); } - return execFlag; } @@ -1213,171 +1173,8 @@ public void testExpireMessageByPosition() { deleteTopic(topic, false); } - public enum OperationAuthType { - Lookup, - Produce, - Consume, - AdminOrSuperUser, - NOAuth - } - private final String testTopic = "persistent://public/default/" + UUID.randomUUID().toString(); - @FunctionalInterface - public interface ThrowingBiConsumer { - void accept(T t) throws PulsarAdminException; - } - @DataProvider(name = "authFunction") - public Object[][] authFunction () throws Exception { - String sub = "my-sub"; - createTopic(testTopic, false); - @Cleanup final PulsarClient pulsarClient = PulsarClient.builder() - .serviceUrl(getPulsarService().getBrokerServiceUrl()) - .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) - .enableTransaction(true) - .build(); - @Cleanup final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(testTopic).create(); - - @Cleanup final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic(testTopic) - .subscriptionName(sub) - .subscribe(); - - Transaction transaction = pulsarClient.newTransaction().withTransactionTimeout(5, TimeUnit.MINUTES) - .build().get(); - MessageIdImpl messageId = (MessageIdImpl) producer.newMessage().value("test message").send(); - - consumer.acknowledgeAsync(messageId, transaction).get(); - - return new Object[][]{ - // SCHEMA - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo(testTopic), - OperationAuthType.Lookup - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo( - testTopic, 0), - OperationAuthType.Lookup - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().getAllSchemas( - testTopic), - OperationAuthType.Lookup - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().createSchema(testTopic, - SchemaInfo.builder().type(SchemaType.STRING).build()), - OperationAuthType.Produce - }, - // TODO: improve the authorization check for testCompatibility and deleteSchema - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().testCompatibility( - testTopic, SchemaInfo.builder().type(SchemaType.STRING).build()), - OperationAuthType.AdminOrSuperUser - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.schemas().deleteSchema( - testTopic), - OperationAuthType.AdminOrSuperUser - }, - - // TRANSACTION - - // Modify transaction coordinator - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .abortTransaction(transaction.getTxnID()), - OperationAuthType.AdminOrSuperUser - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .scaleTransactionCoordinators(17), - OperationAuthType.AdminOrSuperUser - }, - // TODO: fix authorization check of check transaction coordinator stats. - // Check transaction coordinator stats - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getCoordinatorInternalStats(1, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getCoordinatorStats(), - OperationAuthType.AdminOrSuperUser - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getSlowTransactionsByCoordinatorId(1, 5, TimeUnit.SECONDS), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionMetadata(transaction.getTxnID()), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .listTransactionCoordinators(), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getSlowTransactions(5, TimeUnit.SECONDS), - OperationAuthType.AdminOrSuperUser - }, - - // TODO: Check the authorization of the topic when get stats of TB or TP - // Check stats related to transaction buffer and transaction pending ack - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getPendingAckInternalStats(testTopic, sub, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getPendingAckStats(testTopic, sub, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getPositionStatsInPendingAck(testTopic, sub, messageId.getLedgerId(), - messageId.getEntryId(), null), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionBufferInternalStats(testTopic, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionBufferStats(testTopic, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionBufferStats(testTopic, false), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionInBufferStats(transaction.getTxnID(), testTopic), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionInBufferStats(transaction.getTxnID(), testTopic), - OperationAuthType.NOAuth - }, - new Object[] { - (ThrowingBiConsumer) (admin) -> admin.transactions() - .getTransactionInPendingAckStats(transaction.getTxnID(), testTopic, sub), - OperationAuthType.NOAuth - }, - }; - } @Test @SneakyThrows @@ -1410,82 +1207,7 @@ public void testSchemaCompatibility() { deleteTopic(topic, false); } - @Test(dataProvider = "authFunction") - public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer adminConsumer, OperationAuthType topicOpType) - throws Exception { - final String subject = UUID.randomUUID().toString(); - final String token = Jwts.builder() - .claim("sub", subject).signWith(SECRET_KEY).compact(); - - - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() - .serviceHttpUrl(getPulsarService().getWebServiceAddress()) - .authentication(new AuthenticationToken(token)) - .build(); - // test tenant manager - if (topicOpType != OperationAuthType.AdminOrSuperUser) { - adminConsumer.accept(tenantManagerAdmin); - } - - if (topicOpType != OperationAuthType.NOAuth) { - Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> adminConsumer.accept(subAdmin)); - } - - AtomicBoolean execFlag = null; - if (topicOpType == OperationAuthType.Lookup) { - execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.LOOKUP); - } else if (topicOpType == OperationAuthType.Produce) { - execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PRODUCE); - } else if (topicOpType == OperationAuthType.Consume) { - execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); - } - - for (AuthAction action : AuthAction.values()) { - superUserAdmin.topics().grantPermission(testTopic, subject, Set.of(action)); - - if (authActionMatchOperation(topicOpType, action)) { - adminConsumer.accept(subAdmin); - } else { - Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> adminConsumer.accept(subAdmin)); - } - superUserAdmin.topics().revokePermissions(testTopic, subject); - } - - if (execFlag != null) { - Assert.assertTrue(execFlag.get()); - } - - } - private boolean authActionMatchOperation(OperationAuthType operationAuthType, AuthAction action) { - switch (operationAuthType) { - case Lookup -> { - if (AuthAction.consume == action || AuthAction.produce == action) { - return true; - } - } - case Consume -> { - if (AuthAction.consume == action) { - return true; - } - } - case Produce -> { - if (AuthAction.produce == action) { - return true; - } - } - case AdminOrSuperUser -> { - return false; - } - case NOAuth -> { - return true; - } - } - return false; - } @Test @SneakyThrows @@ -1507,8 +1229,10 @@ public void testGetEntryFilter() { // test tenant manager tenantManagerAdmin.topicPolicies().getEntryFiltersPerTopic(topic, true); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ENTRY_FILTERS, PolicyOperation.READ); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topicPolicies().getEntryFiltersPerTopic(topic, false)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -1553,8 +1277,10 @@ public void testSetEntryFilter() { // test tenant manager tenantManagerAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.topicPolicies().setEntryFiltersPerTopic(topic, entryFilter)); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.topics().grantPermission(topic, subject, Set.of(action)); @@ -1656,19 +1382,768 @@ public void testShadowTopic() { deleteTopic(topic, false); } - private void createTopic(String topic, boolean partitioned) throws Exception { - if (partitioned) { - superUserAdmin.topics().createPartitionedTopic(topic, 2); - } else { - superUserAdmin.topics().createNonPartitionedTopic(topic); - } + @Test + @SneakyThrows + public void testList() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationTopicOperationChecker(subject, NamespaceOperation.GET_TOPICS); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getList("public/default")); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationTopicOperationChecker(subject, NamespaceOperation.GET_TOPICS); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPartitionedTopicList("public/default")); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); } - private void deleteTopic(String topic, boolean partitioned) throws Exception { - if (partitioned) { - superUserAdmin.topics().deletePartitionedTopic(topic, true); - } else { - superUserAdmin.topics().delete(topic, true); - } + @Test + @SneakyThrows + public void testPermissionsOnTopic() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // + superUserAdmin.topics().getPermissions(topic); + superUserAdmin.topics().grantPermission(topic, subject, Sets.newHashSet(AuthAction.functions)); + superUserAdmin.topics().revokePermissions(topic, subject); + + // test tenant manager + tenantManagerAdmin.topics().getPermissions(topic); + tenantManagerAdmin.topics().grantPermission(topic, subject, Sets.newHashSet(AuthAction.functions)); + tenantManagerAdmin.topics().revokePermissions(topic, subject); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getPermissions(topic)); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().grantPermission(topic, subject, Sets.newHashSet(AuthAction.functions))); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().revokePermissions(topic, subject)); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testOffloadPolicies() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getOffloadPolicies(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setOffloadPolicies(topic, OffloadPolicies.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeOffloadPolicies(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxUnackedMessagesOnConsumer() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxUnackedMessagesOnConsumer(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxUnackedMessagesOnConsumer(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxUnackedMessagesOnConsumer(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testDeduplicationSnapshotInterval() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getDeduplicationSnapshotInterval(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setDeduplicationSnapshotInterval(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeDeduplicationSnapshotInterval(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testInactiveTopicPolicies() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.INACTIVE_TOPIC, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getInactiveTopicPolicies(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.INACTIVE_TOPIC, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setInactiveTopicPolicies(topic, new InactiveTopicPolicies())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.INACTIVE_TOPIC, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeInactiveTopicPolicies(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxUnackedMessagesOnSubscription() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxUnackedMessagesOnSubscription(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxUnackedMessagesOnSubscription(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxUnackedMessagesOnSubscription(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testDelayedDeliveryPolicies() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DELAYED_DELIVERY, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getDelayedDeliveryPolicy(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DELAYED_DELIVERY, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setDelayedDeliveryPolicy(topic, DelayedDeliveryPolicies.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DELAYED_DELIVERY, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeDelayedDeliveryPolicy(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testAutoSubscriptionCreation() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_SUBSCRIPTION_CREATION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getAutoSubscriptionCreation(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_SUBSCRIPTION_CREATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setAutoSubscriptionCreation(topic, AutoSubscriptionCreationOverride.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_SUBSCRIPTION_CREATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeAutoSubscriptionCreation(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testSubscribeRate() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getSubscribeRate(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setSubscribeRate(topic, new SubscribeRate())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeSubscribeRate(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testSubscriptionTypesEnabled() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getSubscriptionTypesEnabled(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setSubscriptionTypesEnabled(topic, new HashSet<>())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeSubscriptionTypesEnabled(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testPublishRate() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getPublishRate(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setPublishRate(topic, new PublishRate())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removePublishRate(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxConsumersPerSubscription() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxConsumersPerSubscription(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxConsumersPerSubscription(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxConsumersPerSubscription(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testCompactionThreshold() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.COMPACTION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getCompactionThreshold(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.COMPACTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setCompactionThreshold(topic, 20000)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.COMPACTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeCompactionThreshold(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testDispatchRate() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getDispatchRate(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setDispatchRate(topic, DispatchRate.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeDispatchRate(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxConsumers() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxConsumers(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxConsumers(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxConsumers(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMaxProducers() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_PRODUCERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMaxProducers(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_PRODUCERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMaxProducers(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_PRODUCERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMaxProducers(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testReplicatorDispatchRate() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION_RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getReplicatorDispatchRate(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION_RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setReplicatorDispatchRate(topic, DispatchRate.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION_RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeReplicatorDispatchRate(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testPersistence() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.PERSISTENCE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getPersistence(topic)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.PERSISTENCE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setPersistence(topic, new PersistencePolicies())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.PERSISTENCE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removePersistence(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testRetention() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RETENTION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getRetention(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RETENTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setRetention(topic, new RetentionPolicies())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RETENTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeRetention(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testDeduplication() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getDeduplicationStatus(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setDeduplicationStatus(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeDeduplicationStatus(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testMessageTTL() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.TTL, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getMessageTTL(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.TTL, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setMessageTTL(topic, 2)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.TTL, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeMessageTTL(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testBacklogQuota() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.BACKLOG, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().getBacklogQuotaMap(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.BACKLOG, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().setBacklogQuota(topic, BacklogQuota.builder().build())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.BACKLOG, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topicPolicies().removeBacklogQuota(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); + } + + @Test + @SneakyThrows + public void testReplicationClusters() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + createTopic(topic, false); + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().getReplicationClusters(topic, false)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().setReplicationClusters(topic, new ArrayList<>())); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.topics().removeReplicationClusters(topic)); + Assert.assertTrue(execFlag.get()); + + deleteTopic(topic, false); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java new file mode 100644 index 0000000000000..1bca6f6e30835 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java @@ -0,0 +1,359 @@ +/* + * 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.admin; + +import io.jsonwebtoken.Jwts; +import lombok.Cleanup; +import lombok.SneakyThrows; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TopicOperation; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +@Test(groups = "broker-admin") +public class TransactionAndSchemaAuthZTest extends AuthZTest { + + @SneakyThrows + @BeforeClass(alwaysRun = true) + public void setup() { + configureTokenAuthentication(); + configureDefaultAuthorization(); + enableTransaction(); + start(); + createTransactionCoordinatorAssign(16); + this.superUserAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + final TenantInfo tenantInfo = superUserAdmin.tenants().getTenantInfo("public"); + tenantInfo.getAdminRoles().add(TENANT_ADMIN_SUBJECT); + superUserAdmin.tenants().updateTenant("public", tenantInfo); + this.tenantManagerAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) + .build(); + + superUserAdmin.tenants().createTenant("pulsar", tenantInfo); + superUserAdmin.namespaces().createNamespace("pulsar/system"); + } + + @SneakyThrows + @AfterClass(alwaysRun = true) + public void cleanup() { + if (superUserAdmin != null) { + superUserAdmin.close(); + } + if (tenantManagerAdmin != null) { + tenantManagerAdmin.close(); + } + close(); + } + + @BeforeMethod + public void before() throws IllegalAccessException { + orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); + authorizationService = Mockito.spy(orignalAuthorizationService); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + authorizationService, true); + } + + @AfterMethod + public void after() throws IllegalAccessException { + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + orignalAuthorizationService, true); + } + + protected void createTransactionCoordinatorAssign(int numPartitionsOfTC) throws MetadataStoreException { + getPulsarService().getPulsarResources() + .getNamespaceResources() + .getPartitionedTopicResources() + .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, + new PartitionedTopicMetadata(numPartitionsOfTC)); + } + + public enum OperationAuthType { + Lookup, + Produce, + Consume, + AdminOrSuperUser, + NOAuth + } + + private final String testTopic = "persistent://public/default/" + UUID.randomUUID().toString(); + @FunctionalInterface + public interface ThrowingBiConsumer { + void accept(T t) throws PulsarAdminException; + } + + @DataProvider(name = "authFunction") + public Object[][] authFunction () throws Exception { + String sub = "my-sub"; + createTopic(testTopic, false); + @Cleanup final PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(getPulsarService().getBrokerServiceUrl()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .enableTransaction(true) + .build(); + @Cleanup final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(testTopic).create(); + + @Cleanup final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(testTopic) + .subscriptionName(sub) + .subscribe(); + + Transaction transaction = pulsarClient.newTransaction().withTransactionTimeout(5, TimeUnit.MINUTES) + .build().get(); + MessageIdImpl messageId = (MessageIdImpl) producer.newMessage().value("test message").send(); + + consumer.acknowledgeAsync(messageId, transaction).get(); + + return new Object[][]{ + // SCHEMA + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo(testTopic), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getSchemaInfo( + testTopic, 0), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().getAllSchemas( + testTopic), + OperationAuthType.Lookup + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().createSchema(testTopic, + SchemaInfo.builder().type(SchemaType.STRING).build()), + OperationAuthType.Produce + }, + // TODO: improve the authorization check for testCompatibility and deleteSchema + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().testCompatibility( + testTopic, SchemaInfo.builder().type(SchemaType.STRING).build()), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.schemas().deleteSchema( + testTopic), + OperationAuthType.AdminOrSuperUser + }, + + // TRANSACTION + + // Modify transaction coordinator + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .abortTransaction(transaction.getTxnID()), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .scaleTransactionCoordinators(17), + OperationAuthType.AdminOrSuperUser + }, + // TODO: fix authorization check of check transaction coordinator stats. + // Check transaction coordinator stats + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getCoordinatorInternalStats(1, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getCoordinatorStats(), + OperationAuthType.AdminOrSuperUser + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getSlowTransactionsByCoordinatorId(1, 5, TimeUnit.SECONDS), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionMetadata(transaction.getTxnID()), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .listTransactionCoordinators(), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getSlowTransactions(5, TimeUnit.SECONDS), + OperationAuthType.AdminOrSuperUser + }, + + // TODO: Check the authorization of the topic when get stats of TB or TP + // Check stats related to transaction buffer and transaction pending ack + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPendingAckInternalStats(testTopic, sub, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPendingAckStats(testTopic, sub, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getPositionStatsInPendingAck(testTopic, sub, messageId.getLedgerId(), + messageId.getEntryId(), null), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferInternalStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionBufferStats(testTopic, false), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInBufferStats(transaction.getTxnID(), testTopic), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInBufferStats(transaction.getTxnID(), testTopic), + OperationAuthType.NOAuth + }, + new Object[] { + (ThrowingBiConsumer) (admin) -> admin.transactions() + .getTransactionInPendingAckStats(transaction.getTxnID(), testTopic, sub), + OperationAuthType.NOAuth + }, + }; + } + + @Test(dataProvider = "authFunction") + public void testSchemaAndTransactionAuthorization(ThrowingBiConsumer adminConsumer, OperationAuthType topicOpType) + throws Exception { + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + // test tenant manager + if (topicOpType != OperationAuthType.AdminOrSuperUser) { + adminConsumer.accept(tenantManagerAdmin); + } + + if (topicOpType != OperationAuthType.NOAuth) { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> adminConsumer.accept(subAdmin)); + } + + AtomicBoolean execFlag = null; + if (topicOpType == OperationAuthType.Lookup) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.LOOKUP); + } else if (topicOpType == OperationAuthType.Produce) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.PRODUCE); + } else if (topicOpType == OperationAuthType.Consume) { + execFlag = setAuthorizationTopicOperationChecker(subject, TopicOperation.CONSUME); + } + + for (AuthAction action : AuthAction.values()) { + superUserAdmin.topics().grantPermission(testTopic, subject, Set.of(action)); + + if (authActionMatchOperation(topicOpType, action)) { + adminConsumer.accept(subAdmin); + } else { + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> adminConsumer.accept(subAdmin)); + } + superUserAdmin.topics().revokePermissions(testTopic, subject); + } + + if (execFlag != null) { + Assert.assertTrue(execFlag.get()); + } + + } + + private boolean authActionMatchOperation(OperationAuthType operationAuthType, AuthAction action) { + switch (operationAuthType) { + case Lookup -> { + if (AuthAction.consume == action || AuthAction.produce == action) { + return true; + } + } + case Consume -> { + if (AuthAction.consume == action) { + return true; + } + } + case Produce -> { + if (AuthAction.produce == action) { + return true; + } + } + case AdminOrSuperUser -> { + return false; + } + case NOAuth -> { + return true; + } + } + return false; + } + +} From e81f37000ec212676c5daffa17faad8fc604ff77 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 22 Apr 2024 18:13:45 +0300 Subject: [PATCH 096/580] [fix][offload] Increase file upload limit from 2048MiB to 4096MiB for GCP/GCS offloading (#22554) --- conf/broker.conf | 11 ++++++----- .../common/policies/data/OffloadPoliciesImpl.java | 7 ++++--- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index d482f77da7cb5..d97e3a5ef89ad 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1670,10 +1670,10 @@ s3ManagedLedgerOffloadBucket= # For Amazon S3 ledger offload, Alternative endpoint to connect to (useful for testing) s3ManagedLedgerOffloadServiceEndpoint= -# For Amazon S3 ledger offload, Max block size in bytes. (64MB by default, 5MB minimum) +# For Amazon S3 ledger offload, Max block size in bytes. (64MiB by default, 5MiB minimum) s3ManagedLedgerOffloadMaxBlockSizeInBytes=67108864 -# For Amazon S3 ledger offload, Read buffer size in bytes (1MB by default) +# For Amazon S3 ledger offload, Read buffer size in bytes (1MiB by default) s3ManagedLedgerOffloadReadBufferSizeInBytes=1048576 # For Google Cloud Storage ledger offload, region where offload bucket is located. @@ -1683,10 +1683,11 @@ gcsManagedLedgerOffloadRegion= # For Google Cloud Storage ledger offload, Bucket to place offloaded ledger into gcsManagedLedgerOffloadBucket= -# For Google Cloud Storage ledger offload, Max block size in bytes. (64MB by default, 5MB minimum) -gcsManagedLedgerOffloadMaxBlockSizeInBytes=67108864 +# For Google Cloud Storage ledger offload, Max block size in bytes. (128MiB by default, 5MiB minimum) +# Since JClouds limits the maximum number of blocks to 32, the maximum size of a ledger is 32 times the block size. +gcsManagedLedgerOffloadMaxBlockSizeInBytes=134217728 -# For Google Cloud Storage ledger offload, Read buffer size in bytes (1MB by default) +# For Google Cloud Storage ledger offload, Read buffer size in bytes (1MiB by default) gcsManagedLedgerOffloadReadBufferSizeInBytes=1048576 # For Google Cloud Storage, path to json file containing service account credentials. diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java index 51e181811c228..6c40aa3f2edd0 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java @@ -79,8 +79,9 @@ public class OffloadPoliciesImpl implements Serializable, OffloadPolicies { } } - public static final int DEFAULT_MAX_BLOCK_SIZE_IN_BYTES = 64 * 1024 * 1024; // 64MB - public static final int DEFAULT_READ_BUFFER_SIZE_IN_BYTES = 1024 * 1024; // 1MB + public static final int DEFAULT_MAX_BLOCK_SIZE_IN_BYTES = 64 * 1024 * 1024; // 64MiB + public static final int DEFAULT_GCS_MAX_BLOCK_SIZE_IN_BYTES = 128 * 1024 * 1024; // 128MiB + public static final int DEFAULT_READ_BUFFER_SIZE_IN_BYTES = 1024 * 1024; // 1MiB public static final int DEFAULT_OFFLOAD_MAX_THREADS = 2; public static final int DEFAULT_OFFLOAD_MAX_PREFETCH_ROUNDS = 1; public static final String DEFAULT_OFFLOADER_DIRECTORY = "./offloaders"; @@ -163,7 +164,7 @@ public class OffloadPoliciesImpl implements Serializable, OffloadPolicies { private String gcsManagedLedgerOffloadBucket = null; @Configuration @JsonProperty(access = JsonProperty.Access.READ_WRITE) - private Integer gcsManagedLedgerOffloadMaxBlockSizeInBytes = DEFAULT_MAX_BLOCK_SIZE_IN_BYTES; + private Integer gcsManagedLedgerOffloadMaxBlockSizeInBytes = DEFAULT_GCS_MAX_BLOCK_SIZE_IN_BYTES; @Configuration @JsonProperty(access = JsonProperty.Access.READ_WRITE) private Integer gcsManagedLedgerOffloadReadBufferSizeInBytes = DEFAULT_READ_BUFFER_SIZE_IN_BYTES; From 35599b7325347838203a92ca63b78d134b7864c2 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Tue, 23 Apr 2024 00:05:41 +0800 Subject: [PATCH 097/580] [fix][ml] Fix NPE of getValidPositionAfterSkippedEntries when recovering a terminated managed ledger (#22552) --- .../mledger/impl/ManagedLedgerImpl.java | 2 +- .../mledger/impl/ManagedCursorTest.java | 61 +++++++++++++++++++ 2 files changed, 62 insertions(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 698563ed7a1f2..70d3c1f74cab3 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -3701,7 +3701,7 @@ public PositionImpl getValidPositionAfterSkippedEntries(final PositionImpl posit Long nextLedgerId = ledgers.ceilingKey(skippedPosition.getLedgerId() + 1); // This means it has jumped to the last position if (nextLedgerId == null) { - if (currentLedgerEntries == 0) { + if (currentLedgerEntries == 0 && currentLedger != null) { return PositionImpl.get(currentLedger.getId(), 0); } return lastConfirmedEntry.getNext(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index c9bd64171c15a..4e3f8b7908438 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -4695,5 +4695,66 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { && cursorReadPosition.getEntryId() == expectReadPosition.getEntryId()); } + @Test + public void testRecoverCursorWithTerminateManagedLedger() throws Exception { + String mlName = "my_test_ledger"; + String cursorName = "c1"; + + ManagedLedgerConfig config = new ManagedLedgerConfig(); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(mlName, config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(cursorName); + + // Write some data. + Position p0 = ledger.addEntry("entry-0".getBytes()); + Position p1 = ledger.addEntry("entry-1".getBytes()); + + // Read message. + List entries = c1.readEntries(2); + assertEquals(entries.size(), 2); + assertEquals(entries.get(0).getPosition(), p0); + assertEquals(entries.get(1).getPosition(), p1); + entries.forEach(Entry::release); + + // Mark delete the last message. + c1.markDelete(p1); + Position markDeletedPosition = c1.getMarkDeletedPosition(); + Assert.assertEquals(markDeletedPosition, p1); + + // Terminate the managed ledger. + Position lastPosition = ledger.terminate(); + assertEquals(lastPosition, p1); + + // Close the ledger. + ledger.close(); + + // Reopen the ledger. + ledger = (ManagedLedgerImpl) factory.open(mlName, config); + BookKeeper mockBookKeeper = mock(BookKeeper.class); + final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, new ManagedLedgerConfig(), ledger, + cursorName); + + CompletableFuture recoverFuture = new CompletableFuture<>(); + // Recover the cursor. + cursor.recover(new VoidCallback() { + @Override + public void operationComplete() { + recoverFuture.complete(null); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + recoverFuture.completeExceptionally(exception); + } + }); + + recoverFuture.join(); + assertTrue(recoverFuture.isDone()); + assertFalse(recoverFuture.isCompletedExceptionally()); + + // Verify the cursor state. + assertEquals(cursor.getMarkDeletedPosition(), markDeletedPosition); + assertEquals(cursor.getReadPosition(), markDeletedPosition.getNext()); + } + private static final Logger log = LoggerFactory.getLogger(ManagedCursorTest.class); } From a037fa33eeeea6b0bc052c4aa960a55ca8bd0ca2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 22 Apr 2024 19:38:11 +0300 Subject: [PATCH 098/580] [improve][misc] Upgrade to Bookkeeper 4.17.0 (#22551) --- .../server/src/assemble/LICENSE.bin.txt | 100 +++++++++--------- .../shell/src/assemble/LICENSE.bin.txt | 8 +- pom.xml | 6 +- 3 files changed, 57 insertions(+), 57 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 4dc6e4341672c..93fd46d44b53f 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -262,7 +262,7 @@ The Apache Software License, Version 2.0 - com.fasterxml.jackson.module-jackson-module-parameter-names-2.14.2.jar * Caffeine -- com.github.ben-manes.caffeine-caffeine-2.9.1.jar * Conscrypt -- org.conscrypt-conscrypt-openjdk-uber-2.5.2.jar - * Proto Google Common Protos -- com.google.api.grpc-proto-google-common-protos-2.9.0.jar + * Proto Google Common Protos -- com.google.api.grpc-proto-google-common-protos-2.17.0.jar * Bitbucket -- org.bitbucket.b_c-jose4j-0.9.4.jar * Gson - com.google.code.gson-gson-2.8.9.jar @@ -356,34 +356,34 @@ The Apache Software License, Version 2.0 - net.java.dev.jna-jna-jpms-5.12.1.jar - net.java.dev.jna-jna-platform-jpms-5.12.1.jar * BookKeeper - - org.apache.bookkeeper-bookkeeper-common-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-proto-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-server-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.5.jar - - org.apache.bookkeeper-circe-checksum-4.16.5.jar - - org.apache.bookkeeper-cpu-affinity-4.16.5.jar - - org.apache.bookkeeper-statelib-4.16.5.jar - - org.apache.bookkeeper-stream-storage-api-4.16.5.jar - - org.apache.bookkeeper-stream-storage-common-4.16.5.jar - - org.apache.bookkeeper-stream-storage-java-client-4.16.5.jar - - org.apache.bookkeeper-stream-storage-java-client-base-4.16.5.jar - - org.apache.bookkeeper-stream-storage-proto-4.16.5.jar - - org.apache.bookkeeper-stream-storage-server-4.16.5.jar - - org.apache.bookkeeper-stream-storage-service-api-4.16.5.jar - - org.apache.bookkeeper-stream-storage-service-impl-4.16.5.jar - - org.apache.bookkeeper.http-http-server-4.16.5.jar - - org.apache.bookkeeper.http-vertx-http-server-4.16.5.jar - - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.5.jar - - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.5.jar - - org.apache.distributedlog-distributedlog-common-4.16.5.jar - - org.apache.distributedlog-distributedlog-core-4.16.5-tests.jar - - org.apache.distributedlog-distributedlog-core-4.16.5.jar - - org.apache.distributedlog-distributedlog-protocol-4.16.5.jar - - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.5.jar - - org.apache.bookkeeper-native-io-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-common-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-common-allocator-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-proto-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-server-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-tools-framework-4.17.0.jar + - org.apache.bookkeeper-circe-checksum-4.17.0.jar + - org.apache.bookkeeper-cpu-affinity-4.17.0.jar + - org.apache.bookkeeper-statelib-4.17.0.jar + - org.apache.bookkeeper-stream-storage-api-4.17.0.jar + - org.apache.bookkeeper-stream-storage-common-4.17.0.jar + - org.apache.bookkeeper-stream-storage-java-client-4.17.0.jar + - org.apache.bookkeeper-stream-storage-java-client-base-4.17.0.jar + - org.apache.bookkeeper-stream-storage-proto-4.17.0.jar + - org.apache.bookkeeper-stream-storage-server-4.17.0.jar + - org.apache.bookkeeper-stream-storage-service-api-4.17.0.jar + - org.apache.bookkeeper-stream-storage-service-impl-4.17.0.jar + - org.apache.bookkeeper.http-http-server-4.17.0.jar + - org.apache.bookkeeper.http-vertx-http-server-4.17.0.jar + - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.17.0.jar + - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.17.0.jar + - org.apache.distributedlog-distributedlog-common-4.17.0.jar + - org.apache.distributedlog-distributedlog-core-4.17.0-tests.jar + - org.apache.distributedlog-distributedlog-core-4.17.0.jar + - org.apache.distributedlog-distributedlog-protocol-4.17.0.jar + - org.apache.bookkeeper.stats-codahale-metrics-provider-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-slogger-api-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.17.0.jar + - org.apache.bookkeeper-native-io-4.17.0.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar @@ -430,23 +430,23 @@ The Apache Software License, Version 2.0 - org.jetbrains.kotlin-kotlin-stdlib-jdk8-1.8.20.jar - org.jetbrains-annotations-13.0.jar * gRPC - - io.grpc-grpc-all-1.55.3.jar - - io.grpc-grpc-auth-1.55.3.jar - - io.grpc-grpc-context-1.55.3.jar - - io.grpc-grpc-core-1.55.3.jar - - io.grpc-grpc-netty-1.55.3.jar - - io.grpc-grpc-protobuf-1.55.3.jar - - io.grpc-grpc-protobuf-lite-1.55.3.jar - - io.grpc-grpc-stub-1.55.3.jar - - io.grpc-grpc-alts-1.55.3.jar - - io.grpc-grpc-api-1.55.3.jar - - io.grpc-grpc-grpclb-1.55.3.jar - - io.grpc-grpc-netty-shaded-1.55.3.jar - - io.grpc-grpc-services-1.55.3.jar - - io.grpc-grpc-xds-1.55.3.jar - - io.grpc-grpc-rls-1.55.3.jar - - io.grpc-grpc-servlet-1.55.3.jar - - io.grpc-grpc-servlet-jakarta-1.55.3.jar + - io.grpc-grpc-all-1.56.0.jar + - io.grpc-grpc-auth-1.56.0.jar + - io.grpc-grpc-context-1.56.0.jar + - io.grpc-grpc-core-1.56.0.jar + - io.grpc-grpc-netty-1.56.0.jar + - io.grpc-grpc-protobuf-1.56.0.jar + - io.grpc-grpc-protobuf-lite-1.56.0.jar + - io.grpc-grpc-stub-1.56.0.jar + - io.grpc-grpc-alts-1.56.0.jar + - io.grpc-grpc-api-1.56.0.jar + - io.grpc-grpc-grpclb-1.56.0.jar + - io.grpc-grpc-netty-shaded-1.56.0.jar + - io.grpc-grpc-services-1.56.0.jar + - io.grpc-grpc-xds-1.56.0.jar + - io.grpc-grpc-rls-1.56.0.jar + - io.grpc-grpc-servlet-1.56.0.jar + - io.grpc-grpc-servlet-jakarta-1.56.0.jar * Perfmark - io.perfmark-perfmark-api-0.26.0.jar * OpenCensus @@ -504,8 +504,8 @@ The Apache Software License, Version 2.0 * Google HTTP Client - com.google.http-client-google-http-client-gson-1.41.0.jar - com.google.http-client-google-http-client-1.41.0.jar - - com.google.auto.value-auto-value-annotations-1.9.jar - - com.google.re2j-re2j-1.6.jar + - com.google.auto.value-auto-value-annotations-1.10.1.jar + - com.google.re2j-re2j-1.7.jar * Jetcd - io.etcd-jetcd-api-0.7.5.jar - io.etcd-jetcd-common-0.7.5.jar @@ -566,8 +566,8 @@ MIT License - com.auth0-jwks-rsa-0.22.0.jar Protocol Buffers License * Protocol Buffers - - com.google.protobuf-protobuf-java-3.19.6.jar -- ../licenses/LICENSE-protobuf.txt - - com.google.protobuf-protobuf-java-util-3.19.6.jar -- ../licenses/LICENSE-protobuf.txt + - com.google.protobuf-protobuf-java-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt + - com.google.protobuf-protobuf-java-util-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 069e61b89b55a..91d4643d9d4bc 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -393,9 +393,9 @@ The Apache Software License, Version 2.0 - opentelemetry-extension-incubator-1.34.1-alpha.jar * BookKeeper - - bookkeeper-common-allocator-4.16.5.jar - - cpu-affinity-4.16.5.jar - - circe-checksum-4.16.5.jar + - bookkeeper-common-allocator-4.17.0.jar + - cpu-affinity-4.17.0.jar + - circe-checksum-4.17.0.jar * AirCompressor - aircompressor-0.20.jar * AsyncHttpClient @@ -429,7 +429,7 @@ MIT License Protocol Buffers License * Protocol Buffers - - protobuf-java-3.19.6.jar -- ../licenses/LICENSE-protobuf.txt + - protobuf-java-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API diff --git a/pom.xml b/pom.xml index d4b14efc356ba..168eddaf2fe1c 100644 --- a/pom.xml +++ b/pom.xml @@ -137,7 +137,7 @@ flexible messaging model and an intuitive client API. 1.26.0 - 4.16.5 + 4.17.0 3.9.2 1.5.0 1.10.0 @@ -168,9 +168,9 @@ flexible messaging model and an intuitive client API. 0.5.0 1.14.12 1.17 - 3.19.6 + 3.22.3 ${protobuf3.version} - 1.55.3 + 1.56.0 1.41.0 0.26.0 ${grpc.version} From c72c135541e14043370836421cfef372b1d0a0ea Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 22 Apr 2024 14:15:36 -0700 Subject: [PATCH 099/580] [improve] Update Oxia client to 0.1.6 (#22525) --- .../licenses/LICENSE-Reactive-gRPC.txt | 29 +++++++++++++++++++ .../server/src/assemble/LICENSE.bin.txt | 10 ++++++- pom.xml | 3 +- pulsar-metadata/pom.xml | 1 - 4 files changed, 39 insertions(+), 4 deletions(-) create mode 100644 distribution/licenses/LICENSE-Reactive-gRPC.txt diff --git a/distribution/licenses/LICENSE-Reactive-gRPC.txt b/distribution/licenses/LICENSE-Reactive-gRPC.txt new file mode 100644 index 0000000000000..bc589401e7bdf --- /dev/null +++ b/distribution/licenses/LICENSE-Reactive-gRPC.txt @@ -0,0 +1,29 @@ +BSD 3-Clause License + +Copyright (c) 2019, Salesforce.com, Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +* Neither the name of the copyright holder nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 93fd46d44b53f..c5642503b25b0 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -481,7 +481,12 @@ The Apache Software License, Version 2.0 * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia - - io.streamnative.oxia-oxia-client-0.1.0-shaded.jar + - io.streamnative.oxia-oxia-client-0.1.6.jar + - io.streamnative.oxia-oxia-client-metrics-api-0.1.6.jar + * OpenHFT + - net.openhft-zero-allocation-hashing-0.16.jar + * Project reactor + - io.projectreactor-reactor-core-3.5.2.jar * Java JSON WebTokens - io.jsonwebtoken-jjwt-api-0.11.1.jar - io.jsonwebtoken-jjwt-impl-0.11.1.jar @@ -548,6 +553,9 @@ BSD 3-clause "New" or "Revised" License * JSR305 -- com.google.code.findbugs-jsr305-3.0.2.jar -- ../licenses/LICENSE-JSR305.txt * JLine -- jline-jline-2.14.6.jar -- ../licenses/LICENSE-JLine.txt * JLine3 -- org.jline-jline-3.21.0.jar -- ../licenses/LICENSE-JLine.txt + * Reactive gRPC + - com.salesforce.servicelibs-reactive-grpc-common-1.2.4.jar -- ../licenses/LICENSE-Reactive-gRPC.txt + - com.salesforce.servicelibs-reactor-grpc-stub-1.2.4.jar -- ../licenses/LICENSE-Reactive-gRPC.txt BSD 2-Clause License * HdrHistogram -- org.hdrhistogram-HdrHistogram-2.1.9.jar -- ../licenses/LICENSE-HdrHistogram.txt diff --git a/pom.xml b/pom.xml index 168eddaf2fe1c..90b6c8cb8edf4 100644 --- a/pom.xml +++ b/pom.xml @@ -248,7 +248,7 @@ flexible messaging model and an intuitive client API. 4.5.13 4.4.15 0.7.5 - 0.1.0 + 0.1.6 2.0 1.10.12 5.3.3 @@ -1193,7 +1193,6 @@ flexible messaging model and an intuitive client API. io.streamnative.oxia oxia-client ${oxia.version} - shaded io.streamnative.oxia diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index 8600d0ea1919b..163a3058dc4bc 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -65,7 +65,6 @@ io.streamnative.oxia oxia-client - shaded From 49240522f543eea0e9307811c92b487eabe431d9 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 23 Apr 2024 09:23:08 +0800 Subject: [PATCH 100/580] [fix] [broker] Part-1: Replicator can not created successfully due to an orphan replicator in the previous topic owner (#21946) --- .../broker/service/AbstractReplicator.java | 332 +++++++++++++----- .../pulsar/broker/service/BrokerService.java | 2 +- .../pulsar/broker/service/Replicator.java | 4 +- .../NonPersistentReplicator.java | 5 +- .../nonpersistent/NonPersistentTopic.java | 10 +- .../persistent/PersistentReplicator.java | 87 +++-- .../service/persistent/PersistentTopic.java | 31 +- .../service/AbstractReplicatorTest.java | 22 +- .../broker/service/OneWayReplicatorTest.java | 276 ++++++++++++++- .../service/OneWayReplicatorTestBase.java | 40 ++- .../broker/service/PersistentTopicTest.java | 6 +- .../pulsar/broker/service/ReplicatorTest.java | 11 +- 12 files changed, 656 insertions(+), 170 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index 1b5b2824257b0..f34144deb0ab0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -18,16 +18,22 @@ */ package org.apache.pulsar.broker.service; +import com.google.common.annotations.VisibleForTesting; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import lombok.Getter; import org.apache.bookkeeper.mledger.Position; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.BrokerServiceException.NamingException; import org.apache.pulsar.broker.service.BrokerServiceException.TopicBusyException; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.Backoff; @@ -39,7 +45,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class AbstractReplicator { +public abstract class AbstractReplicator implements Replicator { protected final BrokerService brokerService; protected final String localTopicName; @@ -64,10 +70,31 @@ public abstract class AbstractReplicator { protected static final AtomicReferenceFieldUpdater STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(AbstractReplicator.class, State.class, "state"); - private volatile State state = State.Stopped; - - protected enum State { - Stopped, Starting, Started, Stopping + @VisibleForTesting + @Getter + protected volatile State state = State.Disconnected; + + public enum State { + /** + * This enum has two mean meanings: + * Init: replicator is just created, has not been started now. + * Disconnected: the producer was closed after {@link PersistentTopic#checkGC} called {@link #disconnect}. + */ + // The internal producer is disconnected. + Disconnected, + // Trying to create a new internal producer. + Starting, + // The internal producer has started, and tries copy data. + Started, + /** + * The producer is closing after {@link PersistentTopic#checkGC} called {@link #disconnect}. + */ + // The internal producer is trying to disconnect. + Disconnecting, + // The replicator is in terminating. + Terminating, + // The replicator is never used again. Pulsar will create a new Replicator when enable replication again. + Terminated; } public AbstractReplicator(String localCluster, Topic localTopic, String remoteCluster, String remoteTopicName, @@ -96,16 +123,16 @@ public AbstractReplicator(String localCluster, Topic localTopic, String remoteCl .sendTimeout(0, TimeUnit.SECONDS) // .maxPendingMessages(producerQueueSize) // .producerName(getProducerName()); - STATE_UPDATER.set(this, State.Stopped); + STATE_UPDATER.set(this, State.Disconnected); } protected abstract String getProducerName(); - protected abstract void readEntries(org.apache.pulsar.client.api.Producer producer); + protected abstract void setProducerAndTriggerReadEntries(org.apache.pulsar.client.api.Producer producer); protected abstract Position getReplicatorReadPosition(); - protected abstract long getNumberOfEntriesInBacklog(); + public abstract long getNumberOfEntriesInBacklog(); protected abstract void disableReplicatorRead(); @@ -113,66 +140,121 @@ public String getRemoteCluster() { return remoteCluster; } - // This method needs to be synchronized with disconnects else if there is a disconnect followed by startProducer - // the end result can be disconnect. - public synchronized void startProducer() { - if (STATE_UPDATER.get(this) == State.Stopping) { - long waitTimeMs = backOff.next(); - if (log.isDebugEnabled()) { - log.debug( - "[{}] waiting for producer to close before attempting to reconnect, retrying in {} s", - replicatorId, waitTimeMs / 1000.0); - } - // BackOff before retrying - brokerService.executor().schedule(this::checkTopicActiveAndRetryStartProducer, waitTimeMs, - TimeUnit.MILLISECONDS); - return; - } - State state = STATE_UPDATER.get(this); - if (!STATE_UPDATER.compareAndSet(this, State.Stopped, State.Starting)) { - if (state == State.Started) { - // Already running + public void startProducer() { + // Guarantee only one task call "producerBuilder.createAsync()". + Pair setStartingRes = compareSetAndGetState(State.Disconnected, State.Starting); + if (!setStartingRes.getLeft()) { + if (setStartingRes.getRight() == State.Starting) { + log.info("[{}] Skip the producer creation since other thread is doing starting, state : {}", + replicatorId, state); + } else if (setStartingRes.getRight() == State.Started) { + // Since the method "startProducer" will be called even if it is started, only print debug-level log. + if (log.isDebugEnabled()) { + log.debug("[{}] Replicator was already running. state: {}", replicatorId, state); + } + } else if (setStartingRes.getRight() == State.Disconnecting) { if (log.isDebugEnabled()) { - log.debug("[{}] Replicator was already running", replicatorId); + log.debug("[{}] Rep.producer is closing, delay to retry(wait the producer close success)." + + " state: {}", replicatorId, state); } + delayStartProducerAfterDisconnected(); } else { - log.info("[{}] Replicator already being started. Replicator state: {}", replicatorId, state); + /** {@link State.Terminating}, {@link State.Terminated}. **/ + log.info("[{}] Skip the producer creation since the replicator state is : {}", replicatorId, state); } - return; } log.info("[{}] Starting replicator", replicatorId); producerBuilder.createAsync().thenAccept(producer -> { - readEntries(producer); + setProducerAndTriggerReadEntries(producer); }).exceptionally(ex -> { - if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Stopped)) { + Pair setDisconnectedRes = compareSetAndGetState(State.Starting, State.Disconnected); + if (setDisconnectedRes.getLeft()) { long waitTimeMs = backOff.next(); log.warn("[{}] Failed to create remote producer ({}), retrying in {} s", replicatorId, ex.getMessage(), waitTimeMs / 1000.0); - // BackOff before retrying - brokerService.executor().schedule(this::checkTopicActiveAndRetryStartProducer, waitTimeMs, - TimeUnit.MILLISECONDS); + scheduleCheckTopicActiveAndStartProducer(waitTimeMs); } else { - log.warn("[{}] Failed to create remote producer. Replicator state: {}", replicatorId, - STATE_UPDATER.get(this), ex); + if (setDisconnectedRes.getRight() == State.Terminating + || setDisconnectedRes.getRight() == State.Terminated) { + log.info("[{}] Skip to create producer, because it has been terminated, state is : {}", + replicatorId, state); + } else { + /** {@link State.Disconnected}, {@link State.Starting}, {@link State.Started} **/ + // Since only one task can call "producerBuilder.createAsync()", this scenario is not expected. + // So print a warn log. + log.warn("[{}] Other thread will try to create the producer again. so skipped current one task." + + " State is : {}", + replicatorId, state); + } } return null; }); + } + /*** + * The producer is disconnecting, delay to start the producer. + * If we start a producer immediately, we will get a conflict producer(same name producer) registered error. + */ + protected void delayStartProducerAfterDisconnected() { + long waitTimeMs = backOff.next(); + if (log.isDebugEnabled()) { + log.debug( + "[{}] waiting for producer to close before attempting to reconnect, retrying in {} s", + replicatorId, waitTimeMs / 1000.0); + } + scheduleCheckTopicActiveAndStartProducer(waitTimeMs); } - protected void checkTopicActiveAndRetryStartProducer() { - isLocalTopicActive().thenAccept(isTopicActive -> { - if (isTopicActive) { - startProducer(); + protected void scheduleCheckTopicActiveAndStartProducer(final long waitTimeMs) { + brokerService.executor().schedule(() -> { + if (state == State.Terminating || state == State.Terminated) { + log.info("[{}] Skip scheduled to start the producer since the replicator state is : {}", + replicatorId, state); + return; } - }).exceptionally(ex -> { - log.warn("[{}] Stop retry to create producer due to topic load fail. Replicator state: {}", replicatorId, - STATE_UPDATER.get(this), ex); - return null; - }); + CompletableFuture> topicFuture = brokerService.getTopics().get(localTopicName); + if (topicFuture == null) { + // Topic closed. + log.info("[{}] Skip scheduled to start the producer since the topic was closed successfully." + + " And trigger a terminate.", replicatorId); + terminate(); + return; + } + topicFuture.thenAccept(optional -> { + if (optional.isEmpty()) { + // Topic closed. + log.info("[{}] Skip scheduled to start the producer since the topic was closed. And trigger a" + + " terminate.", replicatorId); + terminate(); + return; + } + if (optional.get() != localTopic) { + // Topic closed and created a new one, current replicator is outdated. + log.info("[{}] Skip scheduled to start the producer since the topic was closed. And trigger a" + + " terminate.", replicatorId); + terminate(); + return; + } + Replicator replicator = localTopic.getReplicators().get(remoteCluster); + if (replicator != AbstractReplicator.this) { + // Current replicator has been closed, and created a new one. + log.info("[{}] Skip scheduled to start the producer since a new replicator has instead current" + + " one. And trigger a terminate.", replicatorId); + terminate(); + return; + } + startProducer(); + }).exceptionally(ex -> { + log.warn("[{}] [{}] Stop retry to create producer due to unknown error(topic create failed), and" + + " trigger a terminate. Replicator state: {}", + localTopicName, replicatorId, STATE_UPDATER.get(this), ex); + terminate(); + return null; + }); + }, waitTimeMs, TimeUnit.MILLISECONDS); } protected CompletableFuture isLocalTopicActive() { @@ -188,58 +270,130 @@ protected CompletableFuture isLocalTopicActive() { }, brokerService.executor()); } - protected synchronized CompletableFuture closeProducerAsync() { - if (producer == null) { - STATE_UPDATER.set(this, State.Stopped); + /** + * This method only be used by {@link PersistentTopic#checkGC} now. + */ + public CompletableFuture disconnect(boolean failIfHasBacklog, boolean closeTheStartingProducer) { + long backlog = getNumberOfEntriesInBacklog(); + if (failIfHasBacklog && backlog > 0) { + CompletableFuture disconnectFuture = new CompletableFuture<>(); + disconnectFuture.completeExceptionally(new TopicBusyException("Cannot close a replicator with backlog")); + if (log.isDebugEnabled()) { + log.debug("[{}] Replicator disconnect failed since topic has backlog", replicatorId); + } + return disconnectFuture; + } + log.info("[{}] Disconnect replicator at position {} with backlog {}", replicatorId, + getReplicatorReadPosition(), backlog); + return closeProducerAsync(closeTheStartingProducer); + } + + /** + * This method only be used by {@link PersistentTopic#checkGC} now. + */ + protected CompletableFuture closeProducerAsync(boolean closeTheStartingProducer) { + Pair setDisconnectingRes = compareSetAndGetState(State.Started, State.Disconnecting); + if (!setDisconnectingRes.getLeft()) { + if (setDisconnectingRes.getRight() == State.Starting) { + if (closeTheStartingProducer) { + /** + * Delay retry(wait for the start producer task is finish). + * Note: If the producer always start fail, the start producer task will always retry until the + * state changed to {@link State.Terminated}. + * Nit: The better solution is creating a {@link CompletableFuture} to trace the in-progress + * creation and call "inProgressCreationFuture.thenApply(closeProducer())". + */ + long waitTimeMs = backOff.next(); + brokerService.executor().schedule(() -> closeProducerAsync(true), + waitTimeMs, TimeUnit.MILLISECONDS); + } else { + log.info("[{}] Skip current producer closing since the previous producer has been closed," + + " and trying start a new one, state : {}", + replicatorId, setDisconnectingRes.getRight()); + } + } else if (setDisconnectingRes.getRight() == State.Disconnected + || setDisconnectingRes.getRight() == State.Disconnecting) { + log.info("[{}] Skip current producer closing since other thread did closing, state : {}", + replicatorId, setDisconnectingRes.getRight()); + } else if (setDisconnectingRes.getRight() == State.Terminating + || setDisconnectingRes.getRight() == State.Terminated) { + log.info("[{}] Skip current producer closing since other thread is doing termination, state : {}", + replicatorId, state); + } + log.info("[{}] Skip current termination since other thread is doing close producer or termination," + + " state : {}", replicatorId, state); return CompletableFuture.completedFuture(null); } - CompletableFuture future = producer.closeAsync(); + + // Close producer and update state. + return doCloseProducerAsync(producer, () -> { + Pair setDisconnectedRes = compareSetAndGetState(State.Disconnecting, State.Disconnected); + if (setDisconnectedRes.getLeft()) { + this.producer = null; + // deactivate further read + disableReplicatorRead(); + return; + } + if (setDisconnectedRes.getRight() == State.Terminating + || setDisconnectingRes.getRight() == State.Terminated) { + log.info("[{}] Skip setting state to terminated because it was terminated, state : {}", + replicatorId, state); + } else { + // Since only one task can call "doCloseProducerAsync(producer, action)", this scenario is not expected. + // So print a warn log. + log.warn("[{}] Other task has change the state to terminated. so skipped current one task." + + " State is : {}", + replicatorId, state); + } + }); + } + + protected CompletableFuture doCloseProducerAsync(Producer producer, Runnable actionAfterClosed) { + CompletableFuture future = + producer == null ? CompletableFuture.completedFuture(null) : producer.closeAsync(); return future.thenRun(() -> { - STATE_UPDATER.set(this, State.Stopped); - this.producer = null; - // deactivate further read - disableReplicatorRead(); + actionAfterClosed.run(); }).exceptionally(ex -> { long waitTimeMs = backOff.next(); log.warn( - "[{}] Exception: '{}' occurred while trying to close the producer." - + " retrying again in {} s", - replicatorId, ex.getMessage(), waitTimeMs / 1000.0); + "[{}] Exception: '{}' occurred while trying to close the producer. Replicator state: {}." + + " Retrying again in {} s.", + replicatorId, ex.getMessage(), state, waitTimeMs / 1000.0); // BackOff before retrying - brokerService.executor().schedule(this::closeProducerAsync, waitTimeMs, TimeUnit.MILLISECONDS); + brokerService.executor().schedule(() -> doCloseProducerAsync(producer, actionAfterClosed), + waitTimeMs, TimeUnit.MILLISECONDS); return null; }); } - - public CompletableFuture disconnect() { - return disconnect(false); + public CompletableFuture terminate() { + if (!tryChangeStatusToTerminating()) { + log.info("[{}] Skip current termination since other thread is doing termination, state : {}", replicatorId, + state); + return CompletableFuture.completedFuture(null); + } + return doCloseProducerAsync(producer, () -> { + STATE_UPDATER.set(this, State.Terminated); + this.producer = null; + // set the cursor as inactive. + disableReplicatorRead(); + }); } - public synchronized CompletableFuture disconnect(boolean failIfHasBacklog) { - if (failIfHasBacklog && getNumberOfEntriesInBacklog() > 0) { - CompletableFuture disconnectFuture = new CompletableFuture<>(); - disconnectFuture.completeExceptionally(new TopicBusyException("Cannot close a replicator with backlog")); - if (log.isDebugEnabled()) { - log.debug("[{}] Replicator disconnect failed since topic has backlog", replicatorId); - } - return disconnectFuture; + protected boolean tryChangeStatusToTerminating() { + if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Terminating)){ + return true; } - - if (STATE_UPDATER.get(this) == State.Stopping) { - // Do nothing since the all "STATE_UPDATER.set(this, Stopping)" instructions are followed by - // closeProducerAsync() - // which will at some point change the state to stopped - return CompletableFuture.completedFuture(null); + if (STATE_UPDATER.compareAndSet(this, State.Started, State.Terminating)){ + return true; } - - if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Stopping) - || STATE_UPDATER.compareAndSet(this, State.Started, State.Stopping)) { - log.info("[{}] Disconnect replicator at position {} with backlog {}", replicatorId, - getReplicatorReadPosition(), getNumberOfEntriesInBacklog()); + if (STATE_UPDATER.compareAndSet(this, State.Disconnecting, State.Terminating)){ + return true; } - - return closeProducerAsync(); + if (STATE_UPDATER.compareAndSet(this, State.Disconnected, State.Terminating)) { + return true; + } + return false; } public CompletableFuture remove() { @@ -300,4 +454,18 @@ public static CompletableFuture validatePartitionedTopicAsync(String topic public State getState() { return state; } + + protected ImmutablePair compareSetAndGetState(State expect, State update) { + State original1 = state; + if (STATE_UPDATER.compareAndSet(this, expect, update)) { + return ImmutablePair.of(true, expect); + } + State original2 = state; + // Maybe the value changed more than once even if "original1 == original2", but the probability is very small, + // so let's ignore this case for prevent using a lock. + if (original1 == original2) { + return ImmutablePair.of(false, original1); + } + return compareSetAndGetState(expect, update); + } } 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 249008bad91ad..295a9a2954126 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 @@ -744,7 +744,7 @@ public CompletableFuture closeAndRemoveReplicationClient(String clusterNam if (ot.isPresent()) { Replicator r = ot.get().getReplicators().get(clusterName); if (r != null && r.isConnected()) { - r.disconnect(false).whenComplete((v, e) -> f.complete(null)); + r.terminate().whenComplete((v, e) -> f.complete(null)); return; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java index 482fa2cbd2300..8130b855b4e4a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java @@ -29,9 +29,9 @@ public interface Replicator { ReplicatorStatsImpl getStats(); - CompletableFuture disconnect(); + CompletableFuture terminate(); - CompletableFuture disconnect(boolean b); + CompletableFuture disconnect(boolean failIfHasBacklog, boolean closeTheStartingProducer); void updateRates(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java index 087c5f932008f..51509f3818a28 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java @@ -67,7 +67,7 @@ protected String getProducerName() { } @Override - protected void readEntries(Producer producer) { + protected void setProducerAndTriggerReadEntries(Producer producer) { this.producer = (ProducerImpl) producer; if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Started)) { @@ -78,8 +78,7 @@ protected void readEntries(Producer producer) { "[{}] Replicator was stopped while creating the producer." + " Closing it. Replicator state: {}", replicatorId, STATE_UPDATER.get(this)); - STATE_UPDATER.set(this, State.Stopping); - closeProducerAsync(); + doCloseProducerAsync(producer, () -> {}); return; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 9a3a0a7d83d50..586fcd76151e4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -420,7 +420,7 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, boolean c CompletableFuture closeClientFuture = new CompletableFuture<>(); if (closeIfClientsConnected) { List> futures = new ArrayList<>(); - replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); + replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); producers.values().forEach(producer -> futures.add(producer.disconnect())); subscriptions.forEach((s, sub) -> futures.add(sub.close(true, Optional.empty()))); FutureUtil.waitForAll(futures).thenRun(() -> { @@ -523,7 +523,7 @@ public CompletableFuture close( List> futures = new ArrayList<>(); - replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); + replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); if (disconnectClients) { futures.add(ExtensibleLoadManagerImpl.getAssignedBrokerLookupData( brokerService.getPulsar(), topic).thenAccept(lookupData -> { @@ -582,7 +582,7 @@ public CompletableFuture close( public CompletableFuture stopReplProducers() { List> closeFutures = new ArrayList<>(); - replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect())); + replicators.forEach((region, replicator) -> closeFutures.add(replicator.terminate())); return FutureUtil.waitForAll(closeFutures); } @@ -663,7 +663,7 @@ CompletableFuture removeReplicator(String remoteCluster) { String name = NonPersistentReplicator.getReplicatorName(replicatorPrefix, remoteCluster); - replicators.get(remoteCluster).disconnect().thenRun(() -> { + replicators.get(remoteCluster).terminate().thenRun(() -> { log.info("[{}] Successfully removed replicator {}", name, remoteCluster); replicators.remove(remoteCluster); @@ -1032,7 +1032,7 @@ private CompletableFuture disconnectReplicators() { List> futures = new ArrayList<>(); ConcurrentOpenHashMap replicators = getReplicators(); replicators.forEach((r, replicator) -> { - futures.add(replicator.disconnect()); + futures.add(replicator.terminate()); }); return FutureUtil.waitForAll(futures); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index 754d25b8b0ab4..5e1cc4a936a75 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -18,6 +18,10 @@ */ package org.apache.pulsar.broker.service.persistent; +import static org.apache.pulsar.broker.service.AbstractReplicator.State.Started; +import static org.apache.pulsar.broker.service.AbstractReplicator.State.Starting; +import static org.apache.pulsar.broker.service.AbstractReplicator.State.Terminated; +import static org.apache.pulsar.broker.service.AbstractReplicator.State.Terminating; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; @@ -26,7 +30,6 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; @@ -43,10 +46,10 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.AbstractReplicator; import org.apache.pulsar.broker.service.BrokerService; -import org.apache.pulsar.broker.service.BrokerServiceException.TopicBusyException; import org.apache.pulsar.broker.service.MessageExpirer; import org.apache.pulsar.broker.service.Replicator; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; @@ -134,30 +137,44 @@ public PersistentReplicator(String localCluster, PersistentTopic localTopic, Man } @Override - protected void readEntries(Producer producer) { - // Rewind the cursor to be sure to read again all non-acked messages sent while restarting + protected void setProducerAndTriggerReadEntries(Producer producer) { + // Rewind the cursor to be sure to read again all non-acked messages sent while restarting. cursor.rewind(); - cursor.cancelPendingReadRequest(); - HAVE_PENDING_READ_UPDATER.set(this, FALSE); - this.producer = (ProducerImpl) producer; - if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Started)) { - log.info("[{}] Created replicator producer", replicatorId); + /** + * 1. Try change state to {@link Started}. + * 2. Atoms modify multiple properties if change state success, to avoid another thread get a null value + * producer when the state is {@link Started}. + */ + Pair changeStateRes; + changeStateRes = compareSetAndGetState(Starting, Started); + if (changeStateRes.getLeft()) { + this.producer = (ProducerImpl) producer; + HAVE_PENDING_READ_UPDATER.set(this, FALSE); + // Trigger a new read. + log.info("[{}] Created replicator producer, Replicator state: {}", replicatorId, state); backOff.reset(); - // activate cursor: so, entries can be cached + // activate cursor: so, entries can be cached. this.cursor.setActive(); // read entries readMoreEntries(); } else { - log.info( - "[{}] Replicator was stopped while creating the producer." - + " Closing it. Replicator state: {}", - replicatorId, STATE_UPDATER.get(this)); - STATE_UPDATER.set(this, State.Stopping); - closeProducerAsync(); + if (changeStateRes.getRight() == Started) { + // Since only one task can call "producerBuilder.createAsync()", this scenario is not expected. + // So print a warn log. + log.warn("[{}] Replicator was already started by another thread while creating the producer." + + " Closing the producer newly created. Replicator state: {}", replicatorId, state); + } else if (changeStateRes.getRight() == Terminating || changeStateRes.getRight() == Terminated) { + log.info("[{}] Replicator was terminated, so close the producer. Replicator state: {}", + replicatorId, state); + } else { + log.error("[{}] Replicator state is not expected, so close the producer. Replicator state: {}", + replicatorId, changeStateRes.getRight()); + } + // Close the producer if change the state fail. + doCloseProducerAsync(producer, () -> {}); } - } @Override @@ -420,8 +437,8 @@ public CompletableFuture getFuture() { @Override public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - if (STATE_UPDATER.get(this) != State.Started) { - log.info("[{}] Replicator was stopped while reading entries." + if (state != Started) { + log.info("[{}] Replicator was disconnected while reading entries." + " Stop reading. Replicator state: {}", replicatorId, STATE_UPDATER.get(this)); return; @@ -436,8 +453,8 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { log.error("[{}] Error reading entries because replicator is" + " already deleted and cursor is already closed {}, ({})", replicatorId, ctx, exception.getMessage(), exception); - // replicator is already deleted and cursor is already closed so, producer should also be stopped - closeProducerAsync(); + // replicator is already deleted and cursor is already closed so, producer should also be disconnected. + terminate(); return; } else if (!(exception instanceof TooManyRequestsException)) { log.error("[{}] Error reading entries at {}. Retrying to read in {}s. ({})", @@ -555,8 +572,8 @@ public void deleteFailed(ManagedLedgerException exception, Object ctx) { if (exception instanceof CursorAlreadyClosedException) { log.error("[{}] Asynchronous ack failure because replicator is already deleted and cursor is already" + " closed {}, ({})", replicatorId, ctx, exception.getMessage(), exception); - // replicator is already deleted and cursor is already closed so, producer should also be stopped - closeProducerAsync(); + // replicator is already deleted and cursor is already closed so, producer should also be disconnected. + terminate(); return; } if (ctx instanceof PositionImpl) { @@ -675,30 +692,6 @@ protected void checkReplicatedSubscriptionMarker(Position position, MessageImpl< } } - @Override - public CompletableFuture disconnect() { - return disconnect(false); - } - - @Override - public synchronized CompletableFuture disconnect(boolean failIfHasBacklog) { - final CompletableFuture future = new CompletableFuture<>(); - - super.disconnect(failIfHasBacklog).thenRun(() -> { - dispatchRateLimiter.ifPresent(DispatchRateLimiter::close); - future.complete(null); - }).exceptionally(ex -> { - Throwable t = (ex instanceof CompletionException ? ex.getCause() : ex); - if (!(t instanceof TopicBusyException)) { - log.error("[{}] Failed to close dispatch rate limiter: {}", replicatorId, ex.getMessage()); - } - future.completeExceptionally(t); - return null; - }); - - return future; - } - @Override public boolean isConnected() { ProducerImpl producer = this.producer; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 936091edce557..9d6855962ced6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -833,15 +833,15 @@ public CompletableFuture startReplProducers() { public CompletableFuture stopReplProducers() { List> closeFutures = new ArrayList<>(); - replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect())); - shadowReplicators.forEach((__, replicator) -> closeFutures.add(replicator.disconnect())); + replicators.forEach((region, replicator) -> closeFutures.add(replicator.terminate())); + shadowReplicators.forEach((__, replicator) -> closeFutures.add(replicator.terminate())); return FutureUtil.waitForAll(closeFutures); } private synchronized CompletableFuture closeReplProducersIfNoBacklog() { List> closeFutures = new ArrayList<>(); - replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect(true))); - shadowReplicators.forEach((__, replicator) -> closeFutures.add(replicator.disconnect(true))); + replicators.forEach((region, replicator) -> closeFutures.add(replicator.disconnect(true, true))); + shadowReplicators.forEach((__, replicator) -> closeFutures.add(replicator.disconnect(true, true))); return FutureUtil.waitForAll(closeFutures); } @@ -1423,8 +1423,8 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, List> futures = new ArrayList<>(); subscriptions.forEach((s, sub) -> futures.add(sub.close(true, Optional.empty()))); if (closeIfClientsConnected) { - replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); - shadowReplicators.forEach((__, replicator) -> futures.add(replicator.disconnect())); + replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); + shadowReplicators.forEach((__, replicator) -> futures.add(replicator.terminate())); producers.values().forEach(producer -> futures.add(producer.disconnect())); } FutureUtil.waitForAll(futures).thenRunAsync(() -> { @@ -1565,8 +1565,8 @@ public CompletableFuture close( List> futures = new ArrayList<>(); futures.add(transactionBuffer.closeAsync()); - replicators.forEach((cluster, replicator) -> futures.add(replicator.disconnect())); - shadowReplicators.forEach((__, replicator) -> futures.add(replicator.disconnect())); + replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); + shadowReplicators.forEach((__, replicator) -> futures.add(replicator.terminate())); if (disconnectClients) { futures.add(ExtensibleLoadManagerImpl.getAssignedBrokerLookupData( brokerService.getPulsar(), topic).thenAccept(lookupData -> { @@ -1942,7 +1942,7 @@ CompletableFuture removeReplicator(String remoteCluster) { String name = PersistentReplicator.getReplicatorName(replicatorPrefix, remoteCluster); - Optional.ofNullable(replicators.get(remoteCluster)).map(Replicator::disconnect) + Optional.ofNullable(replicators.get(remoteCluster)).map(Replicator::terminate) .orElse(CompletableFuture.completedFuture(null)).thenRun(() -> { ledger.asyncDeleteCursor(name, new DeleteCursorCallback() { @Override @@ -2014,7 +2014,7 @@ CompletableFuture removeShadowReplicator(String shadowTopic) { log.info("[{}] Removing shadow topic replicator to {}", topic, shadowTopic); final CompletableFuture future = new CompletableFuture<>(); String name = ShadowReplicator.getShadowReplicatorName(replicatorPrefix, shadowTopic); - shadowReplicators.get(shadowTopic).disconnect().thenRun(() -> { + shadowReplicators.get(shadowTopic).terminate().thenRun(() -> { ledger.asyncDeleteCursor(name, new DeleteCursorCallback() { @Override @@ -2898,7 +2898,7 @@ private CompletableFuture checkAndDisconnectReplicators() { ConcurrentOpenHashMap replicators = getReplicators(); replicators.forEach((r, replicator) -> { if (replicator.getNumberOfEntriesInBacklog() <= 0) { - futures.add(replicator.disconnect()); + futures.add(replicator.terminate()); } }); return FutureUtil.waitForAll(futures); @@ -2949,6 +2949,15 @@ public void checkGC() { log.debug("[{}] Global topic inactive for {} seconds, closing repl producers.", topic, maxInactiveDurationInSec); } + /** + * There is a race condition that may cause a NPE: + * - task 1: a callback of "replicator.cursor.asyncRead" will trigger a replication. + * - task 2: "closeReplProducersIfNoBacklog" called by current thread will make the variable + * "replicator.producer" to a null value. + * Race condition: task 1 will get a NPE when it tries to send messages using the variable + * "replicator.producer", because task 2 will set this variable to "null". + * TODO Create a seperated PR to fix it. + */ closeReplProducersIfNoBacklog().thenRun(() -> { if (hasRemoteProducers()) { if (log.isDebugEnabled()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java index 8699c73246830..7aebf20896c2c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java @@ -43,6 +43,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; @@ -94,7 +95,7 @@ public void testRetryStartProducerStoppedByTopicRemove() throws Exception { final ReplicatorInTest replicator = new ReplicatorInTest(localCluster, localTopic, remoteCluster, topicName, replicatorPrefix, broker, remoteClient); replicator.startProducer(); - replicator.disconnect(); + replicator.terminate(); // Verify task will done. Awaitility.await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> { @@ -129,7 +130,7 @@ protected String getProducerName() { } @Override - protected void readEntries(Producer producer) { + protected void setProducerAndTriggerReadEntries(Producer producer) { } @@ -139,7 +140,22 @@ protected Position getReplicatorReadPosition() { } @Override - protected long getNumberOfEntriesInBacklog() { + public ReplicatorStatsImpl getStats() { + return null; + } + + @Override + public void updateRates() { + + } + + @Override + public boolean isConnected() { + return false; + } + + @Override + public long getNumberOfEntriesInBacklog() { return 0; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index 1accd04f4918c..f9184f2288f52 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -18,28 +18,56 @@ */ package org.apache.pulsar.broker.service; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertTrue; +import io.netty.util.concurrent.FastThreadLocalThread; import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.time.Duration; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.ProducerBuilderImpl; import org.apache.pulsar.client.impl.ProducerImpl; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +@Slf4j @Test(groups = "broker") public class OneWayReplicatorTest extends OneWayReplicatorTestBase { @@ -78,7 +106,7 @@ private ProducerImpl overrideProducerForReplicator(AbstractReplicator replicator return originalValue; } - @Test + @Test(timeOut = 45 * 1000) public void testReplicatorProducerStatInTopic() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); final String subscribeName = "subscribe_1"; @@ -104,7 +132,7 @@ public void testReplicatorProducerStatInTopic() throws Exception { }); } - @Test + @Test(timeOut = 45 * 1000) public void testCreateRemoteConsumerFirst() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); Producer producer1 = client1.newProducer(Schema.STRING).topic(topicName).create(); @@ -124,29 +152,257 @@ public void testCreateRemoteConsumerFirst() throws Exception { }); } - @Test + @Test(timeOut = 45 * 1000) public void testTopicCloseWhenInternalProducerCloseErrorOnce() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); admin1.topics().createNonPartitionedTopic(topicName); // Wait for replicator started. waitReplicatorStarted(topicName); - PersistentTopic persistentTopic = + PersistentTopic topic1 = (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); - PersistentReplicator replicator = - (PersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + PersistentReplicator replicator1 = + (PersistentReplicator) topic1.getReplicators().values().iterator().next(); // Mock an error when calling "replicator.disconnect()" - ProducerImpl mockProducer = Mockito.mock(ProducerImpl.class); - Mockito.when(mockProducer.closeAsync()).thenReturn(CompletableFuture.failedFuture(new Exception("mocked ex"))); - ProducerImpl originalProducer = overrideProducerForReplicator(replicator, mockProducer); + AtomicBoolean closeFailed = new AtomicBoolean(true); + final ProducerImpl mockProducer = Mockito.mock(ProducerImpl.class); + final AtomicReference originalProducer1 = new AtomicReference(); + doAnswer(invocation -> { + if (closeFailed.get()) { + return CompletableFuture.failedFuture(new Exception("mocked ex")); + } else { + return originalProducer1.get().closeAsync(); + } + }).when(mockProducer).closeAsync(); + originalProducer1.set(overrideProducerForReplicator(replicator1, mockProducer)); // Verify: since the "replicator.producer.closeAsync()" will retry after it failed, the topic unload should be // successful. admin1.topics().unload(topicName); // Verify: After "replicator.producer.closeAsync()" retry again, the "replicator.producer" will be closed // successful. - overrideProducerForReplicator(replicator, originalProducer); + closeFailed.set(false); + AtomicReference topic2 = new AtomicReference(); + AtomicReference replicator2 = new AtomicReference(); Awaitility.await().untilAsserted(() -> { + topic2.set((PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get()); + replicator2.set((PersistentReplicator) topic2.get().getReplicators().values().iterator().next()); + // It is a new Topic after reloading. + assertNotEquals(topic2.get(), topic1); + assertNotEquals(replicator2.get(), replicator1); + }); + Awaitility.await().untilAsserted(() -> { + // Old replicator should be closed. + Assert.assertFalse(replicator1.isConnected()); + Assert.assertFalse(originalProducer1.get().isConnected()); + // New replicator should be connected. + Assert.assertTrue(replicator2.get().isConnected()); + }); + // cleanup. + cleanupTopics(() -> { + admin1.topics().delete(topicName); + admin2.topics().delete(topicName); + }); + } + + private void injectMockReplicatorProducerBuilder( + BiFunction producerDecorator) + throws Exception { + String cluster2 = pulsar2.getConfig().getClusterName(); + BrokerService brokerService = pulsar1.getBrokerService(); + // Wait for the internal client created. + final String topicNameTriggerInternalClientCreate = + BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + admin1.topics().createNonPartitionedTopic(topicNameTriggerInternalClientCreate); + waitReplicatorStarted(topicNameTriggerInternalClientCreate); + cleanupTopics(() -> { + admin1.topics().delete(topicNameTriggerInternalClientCreate); + admin2.topics().delete(topicNameTriggerInternalClientCreate); + }); + + // Inject spy client. + ConcurrentOpenHashMap + replicationClients = WhiteboxImpl.getInternalState(brokerService, "replicationClients"); + PulsarClientImpl internalClient = (PulsarClientImpl) replicationClients.get(cluster2); + PulsarClient spyClient = spy(internalClient); + replicationClients.put(cluster2, spyClient); + + // Inject producer decorator. + doAnswer(invocation -> { + Schema schema = (Schema) invocation.getArguments()[0]; + ProducerBuilderImpl producerBuilder = (ProducerBuilderImpl) internalClient.newProducer(schema); + ProducerBuilder spyProducerBuilder = spy(producerBuilder); + doAnswer(ignore -> { + CompletableFuture producerFuture = new CompletableFuture<>(); + producerBuilder.createAsync().whenComplete((p, t) -> { + if (t != null) { + producerFuture.completeExceptionally(t); + return; + } + ProducerImpl pImpl = (ProducerImpl) p; + new FastThreadLocalThread(() -> { + try { + ProducerImpl newProducer = producerDecorator.apply(producerBuilder.getConf(), pImpl); + producerFuture.complete(newProducer); + } catch (Exception ex) { + producerFuture.completeExceptionally(ex); + } + }).start(); + }); + + return producerFuture; + }).when(spyProducerBuilder).createAsync(); + return spyProducerBuilder; + }).when(spyClient).newProducer(any(Schema.class)); + } + + private SpyCursor spyCursor(PersistentTopic persistentTopic, String cursorName) throws Exception { + ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().get(cursorName); + ManagedCursorImpl spyCursor = spy(cursor); + // remove cursor. + ml.getCursors().removeCursor(cursorName); + ml.deactivateCursor(cursor); + // Add the spy one. addCursor(ManagedCursorImpl cursor) + Method m = ManagedLedgerImpl.class.getDeclaredMethod("addCursor", new Class[]{ManagedCursorImpl.class}); + m.setAccessible(true); + m.invoke(ml, new Object[]{spyCursor}); + return new SpyCursor(cursor, spyCursor); + } + + @Data + @AllArgsConstructor + static class SpyCursor { + ManagedCursorImpl original; + ManagedCursorImpl spy; + } + + private CursorCloseSignal makeCursorClosingDelay(SpyCursor spyCursor) throws Exception { + CountDownLatch startCloseSignal = new CountDownLatch(1); + CountDownLatch startCallbackSignal = new CountDownLatch(1); + doAnswer(invocation -> { + AsyncCallbacks.CloseCallback originalCallback = (AsyncCallbacks.CloseCallback) invocation.getArguments()[0]; + Object ctx = invocation.getArguments()[1]; + AsyncCallbacks.CloseCallback newCallback = new AsyncCallbacks.CloseCallback() { + @Override + public void closeComplete(Object ctx) { + new FastThreadLocalThread(new Runnable() { + @Override + @SneakyThrows + public void run() { + startCallbackSignal.await(); + originalCallback.closeComplete(ctx); + } + }).start(); + } + + @Override + public void closeFailed(ManagedLedgerException exception, Object ctx) { + new FastThreadLocalThread(new Runnable() { + @Override + @SneakyThrows + public void run() { + startCallbackSignal.await(); + originalCallback.closeFailed(exception, ctx); + } + }).start(); + } + }; + startCloseSignal.await(); + spyCursor.original.asyncClose(newCallback, ctx); + return null; + }).when(spyCursor.spy).asyncClose(any(AsyncCallbacks.CloseCallback.class), any()); + return new CursorCloseSignal(startCloseSignal, startCallbackSignal); + } + + @AllArgsConstructor + static class CursorCloseSignal { + CountDownLatch startCloseSignal; + CountDownLatch startCallbackSignal; + + void startClose() { + startCloseSignal.countDown(); + } + + void startCallback() { + startCallbackSignal.countDown(); + } + } + + /** + * See the description and execution flow: https://github.com/apache/pulsar/pull/21946. + * Steps: + * - Create topic, but the internal producer of Replicator created failed. + * - Unload bundle, the Replicator will be closed, but the internal producer creation retry has not executed yet. + * - The internal producer creation retry execute successfully, the "repl.cursor" has not been closed yet. + * - The topic is wholly closed. + * - Verify: the delayed created internal producer will be closed. + */ + @Test(timeOut = 120 * 1000) + public void testConcurrencyOfUnloadBundleAndRecreateProducer() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + // Inject an error for "replicator.producer" creation. + // The delay time of next retry to create producer is below: + // 0.1s, 0.2, 0.4, 0.8, 1.6s, 3.2s, 6.4s... + // If the retry counter is larger than 6, the next creation will be slow enough to close Replicator. + final AtomicInteger createProducerCounter = new AtomicInteger(); + final int failTimes = 6; + injectMockReplicatorProducerBuilder((producerCnf, originalProducer) -> { + if (topicName.equals(producerCnf.getTopicName())) { + // There is a switch to determine create producer successfully or not. + if (createProducerCounter.incrementAndGet() > failTimes) { + return originalProducer; + } + log.info("Retry create replicator.producer count: {}", createProducerCounter); + // Release producer and fail callback. + originalProducer.closeAsync(); + throw new RuntimeException("mock error"); + } + return originalProducer; + }); + + // Create topic. + admin1.topics().createNonPartitionedTopic(topicName); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + PersistentReplicator replicator = + (PersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + // Since we inject a producer creation error, the replicator can not start successfully. + assertFalse(replicator.isConnected()); + + // Stuck the closing of the cursor("pulsar.repl"), until the internal producer of the replicator started. + SpyCursor spyCursor = + spyCursor(persistentTopic, "pulsar.repl." + pulsar2.getConfig().getClusterName()); + CursorCloseSignal cursorCloseSignal = makeCursorClosingDelay(spyCursor); + + // Unload bundle: call "topic.close(false)". + // Stuck start new producer, until the state of replicator change to Stopped. + // The next once of "createProducerSuccessAfterFailTimes" to create producer will be successfully. + Awaitility.await().pollInterval(Duration.ofMillis(100)).atMost(Duration.ofSeconds(60)).untilAsserted(() -> { + assertTrue(createProducerCounter.get() >= failTimes, + "count of retry to create producer is " + createProducerCounter.get()); + }); + CompletableFuture topicCloseFuture = persistentTopic.close(true); + Awaitility.await().atMost(Duration.ofSeconds(30)).untilAsserted(() -> { + String state = String.valueOf(replicator.getState()); + assertTrue(state.equals("Stopped") || state.equals("Terminated")); + }); + + // Delay close cursor, until "replicator.producer" create successfully. + // The next once retry time of create "replicator.producer" will be 3.2s. + Thread.sleep(4 * 1000); + log.info("Replicator.state: {}", replicator.getState()); + cursorCloseSignal.startClose(); + cursorCloseSignal.startCallback(); + + // Wait for topic close successfully. + // Verify there is no orphan producer on the remote cluster. + topicCloseFuture.join(); + Awaitility.await().pollInterval(Duration.ofSeconds(1)).untilAsserted(() -> { + PersistentTopic persistentTopic2 = + (PersistentTopic) pulsar2.getBrokerService().getTopic(topicName, false).join().get(); + assertEquals(persistentTopic2.getProducers().size(), 0); Assert.assertFalse(replicator.isConnected()); }); + // cleanup. cleanupTopics(() -> { admin1.topics().delete(topicName); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index 33620716288af..8e8b444f952c7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -18,21 +18,28 @@ */ package org.apache.pulsar.broker.service; +import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; import com.google.common.collect.Sets; import java.net.URL; +import java.time.Duration; import java.util.Collections; import java.util.Optional; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.tests.TestRetrySupport; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.apache.pulsar.zookeeper.ZookeeperServerTest; +import org.awaitility.Awaitility; +import org.testng.Assert; @Slf4j public abstract class OneWayReplicatorTestBase extends TestRetrySupport { @@ -140,10 +147,32 @@ protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { } protected void cleanupTopics(CleanupTopicAction cleanupTopicAction) throws Exception { + waitChangeEventsInit(defaultNamespace); admin1.namespaces().setNamespaceReplicationClusters(defaultNamespace, Collections.singleton(cluster1)); admin1.namespaces().unload(defaultNamespace); cleanupTopicAction.run(); admin1.namespaces().setNamespaceReplicationClusters(defaultNamespace, Sets.newHashSet(cluster1, cluster2)); + waitChangeEventsInit(defaultNamespace); + } + + protected void waitChangeEventsInit(String namespace) { + PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService() + .getTopic(namespace + "/" + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME, false) + .join().get(); + Awaitility.await().atMost(Duration.ofSeconds(180)).untilAsserted(() -> { + TopicStatsImpl topicStats = topic.getStats(true, false, false); + topicStats.getSubscriptions().entrySet().forEach(entry -> { + // No wait for compaction. + if (COMPACTION_SUBSCRIPTION.equals(entry.getKey())) { + return; + } + // No wait for durable cursor. + if (entry.getValue().isDurable()) { + return; + } + Assert.assertTrue(entry.getValue().getMsgBacklog() == 0, entry.getKey()); + }); + }); } protected interface CleanupTopicAction { @@ -166,7 +195,7 @@ protected void setup() throws Exception { log.info("--- OneWayReplicatorTestBase::setup completed ---"); } - private void setConfigDefaults(ServiceConfiguration config, String clusterName, + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, LocalBookkeeperEnsemble bookkeeperEnsemble, ZookeeperServerTest brokerConfigZk) { config.setClusterName(clusterName); config.setAdvertisedAddress("localhost"); @@ -185,10 +214,19 @@ private void setConfigDefaults(ServiceConfiguration config, String clusterName, config.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); config.setEnableReplicatedSubscriptions(true); config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); + config.setLoadBalancerSheddingEnabled(false); } @Override protected void cleanup() throws Exception { + // delete namespaces. + waitChangeEventsInit(defaultNamespace); + admin1.namespaces().setNamespaceReplicationClusters(defaultNamespace, Sets.newHashSet(cluster1)); + admin1.namespaces().deleteNamespace(defaultNamespace); + admin2.namespaces().setNamespaceReplicationClusters(defaultNamespace, Sets.newHashSet(cluster2)); + admin2.namespaces().deleteNamespace(defaultNamespace); + + // shutdown. markCurrentSetupNumberCleaned(); log.info("--- Shutting down ---"); 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 d5044276a5a63..de9d0272fc002 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 @@ -1799,12 +1799,12 @@ public void testClosingReplicationProducerTwice() throws Exception { any(), eq(null) ); - replicator.disconnect(false); - replicator.disconnect(false); + replicator.terminate(); + replicator.terminate(); replicator.startProducer(); - verify(clientImpl, Mockito.times(2)).createProducerAsync(any(), any(), any()); + verify(clientImpl, Mockito.times(1)).createProducerAsync(any(), any(), any()); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index 88a668e8745d5..a05c3468ea16e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -895,7 +895,7 @@ public void testReplicatorProducerClosing() throws Exception { pulsar2 = null; pulsar3.close(); pulsar3 = null; - replicator.disconnect(false); + replicator.terminate(); Thread.sleep(100); Field field = AbstractReplicator.class.getDeclaredField("producer"); field.setAccessible(true); @@ -1834,7 +1834,7 @@ public void testReplicatorWithTTL() throws Exception { persistentTopic.getReplicators().forEach((cluster, replicator) -> { PersistentReplicator persistentReplicator = (PersistentReplicator) replicator; // Pause replicator - persistentReplicator.disconnect(); + pauseReplicator(persistentReplicator); }); persistentProducer1.send("V2".getBytes()); @@ -1874,4 +1874,11 @@ public void testReplicatorWithTTL() throws Exception { assertEquals(result, Lists.newArrayList("V1", "V2", "V3", "V4")); } + + private void pauseReplicator(PersistentReplicator replicator) { + Awaitility.await().untilAsserted(() -> { + assertTrue(replicator.isConnected()); + }); + replicator.closeProducerAsync(true); + } } From 882ce415c94db215c6f06aa5212b6d321231e35e Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 23 Apr 2024 13:42:22 +0800 Subject: [PATCH 101/580] [improve][broker] Apply loadBalancerDebugModeEnabled in LeastResourceUsageWithWeight (#22549) --- .../extensions/strategy/LeastResourceUsageWithWeight.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeight.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeight.java index 98986d84b9858..9bf16ac179532 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeight.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeight.java @@ -96,8 +96,7 @@ public Optional select( // select one of them at the end. double totalUsage = 0.0d; - // TODO: use loadBalancerDebugModeEnabled too. - boolean debugMode = log.isDebugEnabled(); + boolean debugMode = log.isDebugEnabled() || conf.isLoadBalancerDebugModeEnabled(); for (String broker : candidates) { var brokerLoadDataOptional = context.brokerLoadDataStore().get(broker); if (brokerLoadDataOptional.isEmpty()) { From 7fe92ac43cfd2f2de5576a023498aac8b46c7ac8 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Tue, 23 Apr 2024 15:22:44 +0800 Subject: [PATCH 102/580] [fix][broker] Support lookup options for extensible load manager (#22487) --- .../broker/loadbalance/LoadManager.java | 3 +- .../extensions/ExtensibleLoadManager.java | 5 +- .../extensions/ExtensibleLoadManagerImpl.java | 53 ++++++++------- .../ExtensibleLoadManagerWrapper.java | 15 +++-- .../channel/ServiceUnitStateChannelImpl.java | 4 +- .../extensions/data/BrokerLookupData.java | 17 ++++- .../broker/namespace/NamespaceService.java | 4 +- ...tiAffinityNamespaceGroupExtensionTest.java | 4 +- .../ExtensibleLoadManagerImplBaseTest.java | 4 ++ .../ExtensibleLoadManagerImplTest.java | 65 +++++++++++++++---- .../channel/ServiceUnitStateChannelTest.java | 14 ++-- .../extensions/data/BrokerLookupDataTest.java | 32 ++++++++- .../ExtensibleLoadManagerTest.java | 3 +- 13 files changed, 162 insertions(+), 61 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java index 2cce68b60cb49..0dd5d948480ab 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java @@ -31,6 +31,7 @@ import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; import org.apache.pulsar.broker.lookup.LookupResult; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.common.naming.ServiceUnitId; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.Reflections; @@ -63,7 +64,7 @@ public interface LoadManager { Optional getLeastLoaded(ServiceUnitId su) throws Exception; default CompletableFuture> findBrokerServiceUrl( - Optional topic, ServiceUnitId bundle) { + Optional topic, ServiceUnitId bundle, LookupOptions options) { throw new UnsupportedOperationException(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManager.java index b7da70d1cf1de..eabf6005b439b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManager.java @@ -60,9 +60,12 @@ public interface ExtensibleLoadManager extends Closeable { * (e.g. {@link NamespaceService#internalGetWebServiceUrl(NamespaceBundle, LookupOptions)}), * So the topic is optional. * @param serviceUnit service unit (e.g. bundle). + * @param options The lookup options. * @return The broker lookup data. */ - CompletableFuture> assign(Optional topic, ServiceUnitId serviceUnit); + CompletableFuture> assign(Optional topic, + ServiceUnitId serviceUnit, + LookupOptions options); /** * Check the incoming service unit is owned by the current broker. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index c8cf1c05756a6..a20694356b178 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -88,6 +88,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.strategy.LeastResourceUsageWithWeight; import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceAllocationPolicies; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -482,7 +483,8 @@ public void initialize(PulsarService pulsar) { @Override public CompletableFuture> assign(Optional topic, - ServiceUnitId serviceUnit) { + ServiceUnitId serviceUnit, + LookupOptions options) { final String bundle = serviceUnit.toString(); @@ -496,7 +498,7 @@ public CompletableFuture> assign(Optional getHeartbeatOrSLAMonitorBrokerId(ServiceUnitId } private CompletableFuture getOrSelectOwnerAsync(ServiceUnitId serviceUnit, - String bundle) { + String bundle, + LookupOptions options) { return serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> { // If the bundle not assign yet, select and publish assign event to channel. if (broker.isEmpty()) { - return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> { + return this.selectAsync(serviceUnit, Collections.emptySet(), options).thenCompose(brokerOpt -> { if (brokerOpt.isPresent()) { assignCounter.incrementSuccess(); log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle); return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get()); } - throw new IllegalStateException( - "Failed to select the new owner broker for bundle: " + bundle); + return CompletableFuture.completedFuture(null); }); } assignCounter.incrementSkip(); @@ -534,22 +536,19 @@ private CompletableFuture> getBrokerLookupData( String bundle) { return owner.thenCompose(broker -> { if (broker.isEmpty()) { - String errorMsg = String.format( - "Failed to get or assign the owner for bundle:%s", bundle); - log.error(errorMsg); - throw new IllegalStateException(errorMsg); - } - return CompletableFuture.completedFuture(broker.get()); - }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> { - if (brokerLookupData.isEmpty()) { - String errorMsg = String.format( - "Failed to lookup broker:%s for bundle:%s, the broker has not been registered.", - broker, bundle); - log.error(errorMsg); - throw new IllegalStateException(errorMsg); + return CompletableFuture.completedFuture(Optional.empty()); } - return CompletableFuture.completedFuture(brokerLookupData); - })); + return this.getBrokerRegistry().lookupAsync(broker.get()).thenCompose(brokerLookupData -> { + if (brokerLookupData.isEmpty()) { + String errorMsg = String.format( + "Failed to lookup broker:%s for bundle:%s, the broker has not been registered.", + broker, bundle); + log.error(errorMsg); + throw new IllegalStateException(errorMsg); + } + return CompletableFuture.completedFuture(brokerLookupData); + }); + }); } /** @@ -562,7 +561,7 @@ private CompletableFuture> getBrokerLookupData( public CompletableFuture tryAcquiringOwnership(NamespaceBundle namespaceBundle) { log.info("Try acquiring ownership for bundle: {} - {}.", namespaceBundle, brokerRegistry.getBrokerId()); final String bundle = namespaceBundle.toString(); - return assign(Optional.empty(), namespaceBundle) + return assign(Optional.empty(), namespaceBundle, LookupOptions.builder().readOnly(false).build()) .thenApply(brokerLookupData -> { if (brokerLookupData.isEmpty()) { String errorMsg = String.format( @@ -595,12 +594,12 @@ private CompletableFuture> dedupeLookupRequest( } } - public CompletableFuture> selectAsync(ServiceUnitId bundle) { - return selectAsync(bundle, Collections.emptySet()); - } - public CompletableFuture> selectAsync(ServiceUnitId bundle, - Set excludeBrokerSet) { + Set excludeBrokerSet, + LookupOptions options) { + if (options.isReadOnly()) { + return CompletableFuture.completedFuture(Optional.empty()); + } BrokerRegistry brokerRegistry = getBrokerRegistry(); return brokerRegistry.getAvailableBrokerLookupDataAsync() .thenComposeAsync(availableBrokers -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java index cd1561cb70e2d..25eb27bc58d27 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java @@ -28,10 +28,11 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.loadbalance.LoadManager; import org.apache.pulsar.broker.loadbalance.ResourceUnit; -import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.lookup.LookupResult; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.common.naming.ServiceUnitId; import org.apache.pulsar.common.stats.Metrics; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; public class ExtensibleLoadManagerWrapper implements LoadManager { @@ -62,9 +63,15 @@ public boolean isCentralized() { @Override public CompletableFuture> findBrokerServiceUrl( - Optional topic, ServiceUnitId bundle) { - return loadManager.assign(topic, bundle) - .thenApply(lookupData -> lookupData.map(BrokerLookupData::toLookupResult)); + Optional topic, ServiceUnitId bundle, LookupOptions options) { + return loadManager.assign(topic, bundle, options) + .thenApply(lookupData -> lookupData.map(data -> { + try { + return data.toLookupResult(options); + } catch (PulsarServerException ex) { + throw FutureUtil.wrapToCompletionException(ex); + } + })); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index e355187af4ba2..bf6266482f8f0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -83,6 +83,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.models.Split; import org.apache.pulsar.broker.loadbalance.extensions.models.Unload; import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.client.api.CompressionType; @@ -1430,7 +1431,8 @@ private synchronized void doCleanup(String broker) { private Optional selectBroker(String serviceUnit, String inactiveBroker) { try { return loadManager.selectAsync( - LoadManagerShared.getNamespaceBundle(pulsar, serviceUnit), Set.of(inactiveBroker)) + LoadManagerShared.getNamespaceBundle(pulsar, serviceUnit), + Set.of(inactiveBroker), LookupOptions.builder().build()) .get(inFlightStateWaitingTimeInMillis, MILLISECONDS); } catch (Throwable e) { log.error("Failed to select a broker for serviceUnit:{}", serviceUnit); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java index 41f5b18e321e8..50a2b70404039 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java @@ -18,9 +18,12 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.data; +import java.net.URI; import java.util.Map; import java.util.Optional; +import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.lookup.LookupResult; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.policies.data.loadbalancer.AdvertisedListener; import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData; @@ -79,7 +82,19 @@ public long getStartTimestamp() { return this.startTimestamp; } - public LookupResult toLookupResult() { + public LookupResult toLookupResult(LookupOptions options) throws PulsarServerException { + if (options.hasAdvertisedListenerName()) { + AdvertisedListener listener = advertisedListeners.get(options.getAdvertisedListenerName()); + if (listener == null) { + throw new PulsarServerException("the broker do not have " + + options.getAdvertisedListenerName() + " listener"); + } + URI url = listener.getBrokerServiceUrl(); + URI urlTls = listener.getBrokerServiceUrlTls(); + return new LookupResult(webServiceUrl, webServiceUrlTls, + url == null ? null : url.toString(), + urlTls == null ? null : urlTls.toString(), LookupResult.Type.BrokerUrl, false); + } return new LookupResult(webServiceUrl, webServiceUrlTls, pulsarServiceUrl, pulsarServiceUrlTls, LookupResult.Type.BrokerUrl, false); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 65081f2ea42b6..44cdd6368fe79 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -227,7 +227,7 @@ public CompletableFuture> getBrokerServiceUrlAsync(TopicN return CompletableFuture.completedFuture(optResult); } if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { - return loadManager.get().findBrokerServiceUrl(Optional.of(topic), bundle); + return loadManager.get().findBrokerServiceUrl(Optional.of(topic), bundle, options); } else { // TODO: Add unit tests cover it. return findBrokerServiceUrl(bundle, options); @@ -353,7 +353,7 @@ private CompletableFuture> internalGetWebServiceUrl(@Nullable Serv } CompletableFuture> future = ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar) - ? loadManager.get().findBrokerServiceUrl(Optional.ofNullable(topic), bundle) : + ? loadManager.get().findBrokerServiceUrl(Optional.ofNullable(topic), bundle, options) : findBrokerServiceUrl(bundle, options); return future.thenApply(lookupResult -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/AntiAffinityNamespaceGroupExtensionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/AntiAffinityNamespaceGroupExtensionTest.java index d77490e1b8210..cd653a964be36 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/AntiAffinityNamespaceGroupExtensionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/AntiAffinityNamespaceGroupExtensionTest.java @@ -38,6 +38,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; import org.apache.pulsar.broker.loadbalance.extensions.filter.AntiAffinityGroupPolicyFilter; import org.apache.pulsar.broker.loadbalance.extensions.policies.AntiAffinityGroupPolicyHelper; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; @@ -61,7 +62,8 @@ protected String getLoadManagerClassName() { protected String selectBroker(ServiceUnitId serviceUnit, Object loadManager) { try { - return ((ExtensibleLoadManagerImpl) loadManager).assign(Optional.empty(), serviceUnit).get() + return ((ExtensibleLoadManagerImpl) loadManager) + .assign(Optional.empty(), serviceUnit, LookupOptions.builder().build()).get() .get().getPulsarServiceUrl(); } catch (Throwable e) { throw new RuntimeException(e); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java index 651a544a04e82..4f2c1ae6607bb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java @@ -21,6 +21,8 @@ import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import com.google.common.collect.Sets; + +import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.tuple.Pair; @@ -69,6 +71,8 @@ protected ServiceConfiguration initConfig(ServiceConfiguration conf) { conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); conf.setLoadBalancerSheddingEnabled(false); conf.setLoadBalancerDebugModeEnabled(true); + conf.setWebServicePortTls(Optional.of(0)); + conf.setBrokerServicePortTls(Optional.of(0)); return conf; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index b72ab77e81447..a385b0d3c5cca 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -156,10 +156,12 @@ public ExtensibleLoadManagerImplTest() { public void testAssignInternalTopic() throws Exception { Optional brokerLookupData1 = primaryLoadManager.assign( Optional.of(TopicName.get(ServiceUnitStateChannelImpl.TOPIC)), - getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get()).get(); + getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get(), + LookupOptions.builder().build()).get(); Optional brokerLookupData2 = secondaryLoadManager.assign( Optional.of(TopicName.get(ServiceUnitStateChannelImpl.TOPIC)), - getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get()).get(); + getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get(), + LookupOptions.builder().build()).get(); assertEquals(brokerLookupData1, brokerLookupData2); assertTrue(brokerLookupData1.isPresent()); @@ -167,7 +169,7 @@ public void testAssignInternalTopic() throws Exception { FieldUtils.readField(channel1, "leaderElectionService", true); Optional currentLeader = leaderElectionService.getCurrentLeader(); assertTrue(currentLeader.isPresent()); - assertEquals(brokerLookupData1.get().getWebServiceUrl(), currentLeader.get().getServiceUrl()); + assertEquals(brokerLookupData1.get().getWebServiceUrlTls(), currentLeader.get().getServiceUrl()); } @Test @@ -175,15 +177,17 @@ public void testAssign() throws Exception { Pair topicAndBundle = getBundleIsNotOwnByChangeEventTopic("test-assign"); TopicName topicName = topicAndBundle.getLeft(); NamespaceBundle bundle = topicAndBundle.getRight(); - Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle, + LookupOptions.builder().build()).get(); assertTrue(brokerLookupData.isPresent()); log.info("Assign the bundle {} to {}", bundle, brokerLookupData); // Should get owner info from channel. - Optional brokerLookupData1 = secondaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData1 = secondaryLoadManager.assign(Optional.empty(), bundle, + LookupOptions.builder().build()).get(); assertEquals(brokerLookupData, brokerLookupData1); Optional lookupResult = pulsar2.getNamespaceService() - .getBrokerServiceUrlAsync(topicName, null).get(); + .getBrokerServiceUrlAsync(topicName, LookupOptions.builder().build()).get(); assertTrue(lookupResult.isPresent()); assertEquals(lookupResult.get().getLookupData().getHttpUrl(), brokerLookupData.get().getWebServiceUrl()); @@ -193,6 +197,43 @@ public void testAssign() throws Exception { assertEquals(webServiceUrl.get().toString(), brokerLookupData.get().getWebServiceUrl()); } + @Test + public void testLookupOptions() throws Exception { + Pair topicAndBundle = + getBundleIsNotOwnByChangeEventTopic("test-lookup-options"); + TopicName topicName = topicAndBundle.getLeft(); + NamespaceBundle bundle = topicAndBundle.getRight(); + + admin.topics().createPartitionedTopic(topicName.toString(), 1); + + // Test LookupOptions.readOnly = true when the bundle is not owned by any broker. + Optional webServiceUrlReadOnlyTrue = pulsar1.getNamespaceService() + .getWebServiceUrl(bundle, LookupOptions.builder().readOnly(true).requestHttps(false).build()); + assertTrue(webServiceUrlReadOnlyTrue.isEmpty()); + + // Test LookupOptions.readOnly = false and the bundle assign to some broker. + Optional webServiceUrlReadOnlyFalse = pulsar1.getNamespaceService() + .getWebServiceUrl(bundle, LookupOptions.builder().readOnly(false).requestHttps(false).build()); + assertTrue(webServiceUrlReadOnlyFalse.isPresent()); + + // Test LookupOptions.requestHttps = true + Optional webServiceUrlHttps = pulsar2.getNamespaceService() + .getWebServiceUrl(bundle, LookupOptions.builder().requestHttps(true).build()); + assertTrue(webServiceUrlHttps.isPresent()); + assertTrue(webServiceUrlHttps.get().toString().startsWith("https")); + + // TODO: Support LookupOptions.loadTopicsInBundle = true + + // Test LookupOptions.advertisedListenerName = internal but the broker do not have internal listener. + try { + pulsar2.getNamespaceService() + .getWebServiceUrl(bundle, LookupOptions.builder().advertisedListenerName("internal").build()); + fail(); + } catch (Exception e) { + assertTrue(e.getMessage().contains("the broker do not have internal listener")); + } + } + @Test public void testCheckOwnershipAsync() throws Exception { Pair topicAndBundle = getBundleIsNotOwnByChangeEventTopic("test-check-ownership"); @@ -210,7 +251,7 @@ public void testCheckOwnershipAsync() throws Exception { assertFalse(secondaryLoadManager.checkOwnershipAsync(Optional.empty(), bundle).get()); // 2. Assign the bundle to a broker. - Optional lookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional lookupData = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); assertTrue(lookupData.isPresent()); if (lookupData.get().getPulsarServiceUrl().equals(pulsar1.getBrokerServiceUrl())) { assertTrue(primaryLoadManager.checkOwnershipAsync(Optional.empty(), bundle).get()); @@ -243,7 +284,7 @@ public CompletableFuture> filterAsync(Map brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); assertTrue(brokerLookupData.isPresent()); assertEquals(brokerLookupData.get().getWebServiceUrl(), pulsar2.getWebServiceAddress()); } @@ -263,7 +304,7 @@ public CompletableFuture> filterAsync(Map brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); assertTrue(brokerLookupData.isPresent()); } @@ -272,7 +313,7 @@ public void testUnloadUponTopicLookupFailure() throws Exception { TopicName topicName = TopicName.get("public/test/testUnloadUponTopicLookupFailure"); NamespaceBundle bundle = pulsar1.getNamespaceService().getBundle(topicName); - primaryLoadManager.assign(Optional.empty(), bundle).get(); + primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); CompletableFuture future1 = new CompletableFuture(); CompletableFuture future2 = new CompletableFuture(); @@ -869,7 +910,7 @@ public CompletableFuture> filterAsync(Map brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); Awaitility.waitAtMost(5, TimeUnit.SECONDS).untilAsserted(() -> { assertTrue(brokerLookupData.isPresent()); assertEquals(brokerLookupData.get().getWebServiceUrl(), pulsar2.getWebServiceAddress()); @@ -1564,7 +1605,7 @@ public void testGetOwnedServiceUnitsAndGetOwnedNamespaceStatus() throws Exceptio String topic = "persistent://" + defaultTestNamespace + "/test-get-owned-service-units"; admin.topics().createPartitionedTopic(topic, 1); NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).join(); - CompletableFuture> owner = primaryLoadManager.assign(Optional.empty(), bundle); + CompletableFuture> owner = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()); assertFalse(owner.join().isEmpty()); BrokerLookupData brokerLookupData = owner.join().get(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java index fe8387710eeae..1076f92037f10 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java @@ -502,7 +502,7 @@ public void transferTestWhenDestBrokerFails() // recovered, check the monitor update state : Assigned -> Owned doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); FieldUtils.writeDeclaredField(channel2, "producer", producer, true); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1 , true); @@ -724,7 +724,7 @@ public void handleBrokerDeletionEventTest() var owner1 = channel1.getOwnerAsync(bundle1); var owner2 = channel2.getOwnerAsync(bundle2); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); assertTrue(owner1.get().isEmpty()); assertTrue(owner2.get().isEmpty()); @@ -1126,7 +1126,7 @@ public void assignTestWhenDestBrokerProducerFails() FieldUtils.writeDeclaredField(channel2, "inFlightStateWaitingTimeInMillis", 3 * 1000, true); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); channel1.publishAssignEventAsync(bundle, brokerId2); // channel1 is broken. the assign won't be complete. waitUntilState(channel1, bundle); @@ -1525,7 +1525,7 @@ public void testOverrideInactiveBrokerStateData() // test stable metadata state doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); leaderChannel.handleMetadataSessionEvent(SessionReestablished); followerChannel.handleMetadataSessionEvent(SessionReestablished); FieldUtils.writeDeclaredField(leaderChannel, "lastMetadataSessionEventTimestamp", @@ -1590,7 +1590,7 @@ public void testOverrideOrphanStateData() // test stable metadata state doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); FieldUtils.writeDeclaredField(leaderChannel, "inFlightStateWaitingTimeInMillis", -1, true); FieldUtils.writeDeclaredField(followerChannel, "inFlightStateWaitingTimeInMillis", @@ -1645,7 +1645,7 @@ public void testActiveGetOwner() throws Exception { // simulate ownership cleanup(no selected owner) by the leader channel doReturn(CompletableFuture.completedFuture(Optional.empty())) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); var leaderChannel = channel1; String leader1 = channel1.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); String leader2 = channel2.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); @@ -1669,7 +1669,7 @@ public void testActiveGetOwner() throws Exception { overrideTableViews(bundle, new ServiceUnitStateData(Owned, broker, null, 1)); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); leaderChannel.handleMetadataSessionEvent(SessionReestablished); FieldUtils.writeDeclaredField(leaderChannel, "lastMetadataSessionEventTimestamp", System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java index 0d874e0f77117..66e8c917d1fc5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java @@ -18,13 +18,19 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.data; +import static org.testng.Assert.fail; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertTrue; + +import java.net.URI; +import java.net.URISyntaxException; import java.util.HashMap; import java.util.Map; import java.util.Optional; +import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.broker.lookup.LookupResult; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.policies.data.loadbalancer.AdvertisedListener; import org.testng.annotations.Test; @@ -32,12 +38,20 @@ public class BrokerLookupDataTest { @Test - public void testConstructors() { + public void testConstructors() throws PulsarServerException, URISyntaxException { String webServiceUrl = "http://localhost:8080"; String webServiceUrlTls = "https://localhoss:8081"; String pulsarServiceUrl = "pulsar://localhost:6650"; String pulsarServiceUrlTls = "pulsar+ssl://localhost:6651"; - Map advertisedListeners = new HashMap<>(); + final String listenerUrl = "pulsar://gateway:7000"; + final String listenerUrlTls = "pulsar://gateway:8000"; + final String listener = "internal"; + Map advertisedListeners = new HashMap<>(){{ + put(listener, AdvertisedListener.builder() + .brokerServiceUrl(new URI(listenerUrl)) + .brokerServiceUrlTls(new URI(listenerUrlTls)) + .build()); + }}; Map protocols = new HashMap<>(){{ put("kafka", "9092"); }}; @@ -56,10 +70,22 @@ public void testConstructors() { assertEquals("3.0", lookupData.brokerVersion()); - LookupResult lookupResult = lookupData.toLookupResult(); + LookupResult lookupResult = lookupData.toLookupResult(LookupOptions.builder().build()); assertEquals(webServiceUrl, lookupResult.getLookupData().getHttpUrl()); assertEquals(webServiceUrlTls, lookupResult.getLookupData().getHttpUrlTls()); assertEquals(pulsarServiceUrl, lookupResult.getLookupData().getBrokerUrl()); assertEquals(pulsarServiceUrlTls, lookupResult.getLookupData().getBrokerUrlTls()); + + try { + lookupData.toLookupResult(LookupOptions.builder().advertisedListenerName("others").build()); + fail(); + } catch (PulsarServerException ex) { + assertTrue(ex.getMessage().contains("the broker do not have others listener")); + } + lookupResult = lookupData.toLookupResult(LookupOptions.builder().advertisedListenerName(listener).build()); + assertEquals(listenerUrl, lookupResult.getLookupData().getBrokerUrl()); + assertEquals(listenerUrlTls, lookupResult.getLookupData().getBrokerUrlTls()); + assertEquals(webServiceUrl, lookupResult.getLookupData().getHttpUrl()); + assertEquals(webServiceUrlTls, lookupResult.getLookupData().getHttpUrlTls()); } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java index 1f29e19f01873..ee7497010adfc 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java @@ -403,9 +403,10 @@ public void testIsolationPolicy() throws Exception { () -> { try { admin.lookups().lookupTopicAsync(topic).get(5, TimeUnit.SECONDS); + fail(); } catch (Exception ex) { log.error("Failed to lookup topic: ", ex); - assertThat(ex.getMessage()).contains("Failed to select the new owner broker for bundle"); + assertThat(ex.getMessage()).contains("Service Unavailable"); } } ); From 358c7cc6bbbe1165879377b37b23356deb045e1c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 23 Apr 2024 11:10:26 +0300 Subject: [PATCH 103/580] [fix][ci] Don't allow merging PR without successful result (#22563) --- .github/workflows/pulsar-ci.yaml | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index aa33d7ae197d1..1642b54337fc0 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -1498,12 +1498,16 @@ jobs: with: action: wait - # This job is required for pulls to be merged. + # This job is required for pulls to be merged. This job is referenced by name in .asf.yaml file in the + # protected_branches section for master branch required_status_checks. # It depends on all other jobs in this workflow. - # It cleans up the binaries in the same job in order to not spin up another runner for basically doing nothing. + # This job also cleans up the binaries at the end of the workflow. pulsar-ci-checks-completed: name: "Pulsar CI checks completed" - if: ${{ always() && needs.preconditions.result == 'success' }} + # run always, but skip for other repositories than apache/pulsar when a scheduled workflow is cancelled + # this is to allow the workflow scheduled jobs to show as cancelled instead of failed since scheduled + # jobs are not enabled for other than apache/pulsar repository. + if: ${{ always() && !(cancelled() && github.repository != 'apache/pulsar' && github.event_name == 'schedule') }} runs-on: ubuntu-22.04 timeout-minutes: 10 needs: [ @@ -1521,10 +1525,11 @@ jobs: ] steps: - name: Check that all required jobs were completed successfully - if: ${{ needs.preconditions.outputs.docs_only != 'true' }} + if: ${{ needs.preconditions.result != 'success' || needs.preconditions.outputs.docs_only != 'true' }} run: | if [[ ! ( \ - "${{ needs.unit-tests.result }}" == "success" \ + "${{ needs.preconditions.result }}" == "success" \ + && "${{ needs.unit-tests.result }}" == "success" \ && "${{ needs.integration-tests.result }}" == "success" \ && "${{ needs.system-tests.result }}" == "success" \ && "${{ needs.macos-build.result }}" == "success" \ From 89b201ed8a49877e0a7148b060af945b29074b02 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 23 Apr 2024 18:52:49 +0800 Subject: [PATCH 104/580] [improve][broker] Make BrokerSelectionStrategy pluggable (#22553) --- .../extensions/ExtensibleLoadManagerImpl.java | 11 ++- .../strategy/BrokerSelectionStrategy.java | 2 + .../BrokerSelectionStrategyFactory.java | 27 ++++++ .../CustomBrokerSelectionStrategyTest.java | 86 +++++++++++++++++++ 4 files changed, 123 insertions(+), 3 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategyFactory.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/CustomBrokerSelectionStrategyTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index a20694356b178..41832fb60075d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -85,6 +85,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreException; import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreFactory; import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy; +import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategyFactory; import org.apache.pulsar.broker.loadbalance.extensions.strategy.LeastResourceUsageWithWeight; import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceAllocationPolicies; @@ -104,7 +105,7 @@ import org.slf4j.Logger; @Slf4j -public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { +public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerSelectionStrategyFactory { public static final String BROKER_LOAD_DATA_STORE_TOPIC = TopicName.get( TopicDomain.non_persistent.value(), @@ -252,6 +253,11 @@ public Set getOwnedServiceUnits() { return ownedServiceUnits; } + @Override + public BrokerSelectionStrategy createBrokerSelectionStrategy() { + return new LeastResourceUsageWithWeight(); + } + public enum Role { Leader, Follower @@ -267,8 +273,7 @@ public ExtensibleLoadManagerImpl() { this.brokerFilterPipeline.add(new BrokerLoadManagerClassFilter()); this.brokerFilterPipeline.add(new BrokerMaxTopicCountFilter()); this.brokerFilterPipeline.add(new BrokerVersionFilter()); - // TODO: Make brokerSelectionStrategy configurable. - this.brokerSelectionStrategy = new LeastResourceUsageWithWeight(); + this.brokerSelectionStrategy = createBrokerSelectionStrategy(); } public static boolean isLoadManagerExtensionEnabled(PulsarService pulsar) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategy.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategy.java index e0a9122383c22..b240cb5b5f6a6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategy.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategy.java @@ -21,11 +21,13 @@ import java.util.Optional; import java.util.Set; import org.apache.pulsar.broker.loadbalance.extensions.LoadManagerContext; +import org.apache.pulsar.common.classification.InterfaceStability; import org.apache.pulsar.common.naming.ServiceUnitId; /** * The broker selection strategy is designed to select the broker according to different implementations. */ +@InterfaceStability.Evolving public interface BrokerSelectionStrategy { /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategyFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategyFactory.java new file mode 100644 index 0000000000000..61b9fbcfcb9e5 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/BrokerSelectionStrategyFactory.java @@ -0,0 +1,27 @@ +/* + * 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.loadbalance.extensions.strategy; + +import org.apache.pulsar.common.classification.InterfaceStability; + +@InterfaceStability.Stable +public interface BrokerSelectionStrategyFactory { + + BrokerSelectionStrategy createBrokerSelectionStrategy(); +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/CustomBrokerSelectionStrategyTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/CustomBrokerSelectionStrategyTest.java new file mode 100644 index 0000000000000..3ac6df2595109 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/CustomBrokerSelectionStrategyTest.java @@ -0,0 +1,86 @@ +/* + * 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.loadbalance.extensions.strategy; + +import java.util.Comparator; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import lombok.Cleanup; +import org.apache.pulsar.broker.MultiBrokerBaseTest; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder; +import org.apache.pulsar.client.impl.PartitionedProducerImpl; +import org.apache.pulsar.client.impl.ProducerImpl; +import org.testng.Assert; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class CustomBrokerSelectionStrategyTest extends MultiBrokerBaseTest { + + @Override + protected void startBroker() throws Exception { + addCustomConfigs(conf); + super.startBroker(); + } + + @Override + protected ServiceConfiguration createConfForAdditionalBroker(int additionalBrokerIndex) { + return addCustomConfigs(getDefaultConf()); + } + + private static ServiceConfiguration addCustomConfigs(ServiceConfiguration conf) { + conf.setLoadManagerClassName(CustomExtensibleLoadManager.class.getName()); + conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + conf.setLoadBalancerAutoBundleSplitEnabled(false); + conf.setDefaultNumberOfNamespaceBundles(8); + // Don't consider broker's load so the broker will be selected randomly with the default strategy + conf.setLoadBalancerAverageResourceUsageDifferenceThresholdPercentage(100); + return conf; + } + + @Test + public void testSingleBrokerSelected() throws Exception { + final var topic = "test-single-broker-selected"; + getAllAdmins().get(0).topics().createPartitionedTopic(topic, 16); + @Cleanup final var producer = (PartitionedProducerImpl) getAllClients().get(0).newProducer() + .topic(topic).create(); + Assert.assertNotNull(producer); + final var connections = producer.getProducers().stream().map(ProducerImpl::getClientCnx) + .collect(Collectors.toSet()); + Assert.assertEquals(connections.size(), 1); + final var port = Integer.parseInt(connections.stream().findFirst().orElseThrow().ctx().channel() + .remoteAddress().toString().replaceAll(".*:", "")); + final var expectedPort = Stream.concat(Stream.of(pulsar), additionalBrokers.stream()) + .min(Comparator.comparingInt(o -> o.getListenPortHTTP().orElseThrow())) + .map(PulsarService::getBrokerListenPort) + .orElseThrow().orElseThrow(); + Assert.assertEquals(port, expectedPort); + } + + public static class CustomExtensibleLoadManager extends ExtensibleLoadManagerImpl { + + @Override + public BrokerSelectionStrategy createBrokerSelectionStrategy() { + // The smallest HTTP port will always be selected because the host parts are all "localhost" + return (brokers, __, ___) -> brokers.stream().sorted().findFirst(); + } + } +} From d5c72312ff4d03291e1ea2eb37464250c85bf401 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Tue, 23 Apr 2024 22:04:13 +0800 Subject: [PATCH 105/580] [fix][admin] Fix can't delete tenant for v1 (#22550) --- .../broker/resources/TopicResources.java | 2 +- .../pulsar/broker/auth/AuthorizationTest.java | 29 +++++++++++++++++++ 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java index 0963f25c3d31f..413184764f52b 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java @@ -120,7 +120,7 @@ public CompletableFuture clearTenantPersistence(String tenant) { return store.exists(path) .thenCompose(exists -> { if (exists) { - return store.delete(path, Optional.empty()); + return store.deleteRecursive(path); } else { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java index 01bfd03ceb81a..f59f9d480b8c2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java @@ -33,6 +33,7 @@ import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; +import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; @@ -56,12 +57,17 @@ public AuthorizationTest() { @Override public void setup() throws Exception { conf.setClusterName("c1"); + conf.setSystemTopicEnabled(false); conf.setAuthenticationEnabled(true); + conf.setForceDeleteNamespaceAllowed(true); + conf.setForceDeleteTenantAllowed(true); conf.setAuthenticationProviders( Sets.newHashSet("org.apache.pulsar.broker.auth.MockAuthenticationProvider")); conf.setAuthorizationEnabled(true); conf.setAuthorizationAllowWildcardsMatching(true); conf.setSuperUserRoles(Sets.newHashSet("pulsar.super_user", "pass.pass")); + conf.setBrokerClientAuthenticationPlugin(MockAuthentication.class.getName()); + conf.setBrokerClientAuthenticationParameters("user:pass.pass"); internalSetup(); } @@ -70,6 +76,11 @@ protected void customizeNewPulsarAdminBuilder(PulsarAdminBuilder pulsarAdminBuil pulsarAdminBuilder.authentication(new MockAuthentication("pass.pass")); } + @Override + protected void customizeNewPulsarClientBuilder(ClientBuilder clientBuilder) { + clientBuilder.authentication(new MockAuthentication("pass.pass")); + } + @AfterClass(alwaysRun = true) @Override public void cleanup() throws Exception { @@ -233,6 +244,24 @@ public void simple() throws Exception { admin.namespaces().deleteNamespace("p1/c1/ns1"); admin.tenants().deleteTenant("p1"); + + admin.clusters().deleteCluster("c1"); + } + + @Test + public void testDeleteV1Tenant() throws Exception { + admin.clusters().createCluster("c1", ClusterData.builder().build()); + admin.tenants().createTenant("p1", new TenantInfoImpl(Sets.newHashSet("role1"), Sets.newHashSet("c1"))); + waitForChange(); + admin.namespaces().createNamespace("p1/c1/ns1"); + waitForChange(); + + + String topic = "persistent://p1/c1/ns1/ds2"; + admin.topics().createNonPartitionedTopic(topic); + + admin.namespaces().deleteNamespace("p1/c1/ns1", true); + admin.tenants().deleteTenant("p1", true); admin.clusters().deleteCluster("c1"); } From 77a99ed287cd2c85590f9734190faceb8a50532c Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 23 Apr 2024 22:32:06 -0700 Subject: [PATCH 106/580] [fix] Include swagger annotations in shaded client lib (#22570) --- distribution/shell/src/assemble/LICENSE.bin.txt | 1 + pulsar-client/pom.xml | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 91d4643d9d4bc..f76631dbbf260 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -333,6 +333,7 @@ The Apache Software License, Version 2.0 - listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar * J2ObjC Annotations -- j2objc-annotations-1.3.jar * Netty Reactive Streams -- netty-reactive-streams-2.0.6.jar + * Swagger -- swagger-annotations-1.6.2.jar * DataSketches - memory-0.8.3.jar - sketches-core-0.8.3.jar diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index 3917e2996e180..f79af79d57452 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -86,7 +86,6 @@ io.swagger swagger-annotations - provided From a3cd1f8dd2c9f3fbc128f4ba6fb00f865b3a2316 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Wed, 24 Apr 2024 15:01:19 +0800 Subject: [PATCH 107/580] [fix][io] CompressionEnabled didn't work on elasticsearch sink (#22565) --- .../elastic/ElasticSearchJavaRestClient.java | 1 + .../OpenSearchHighLevelRestClient.java | 1 + .../ElasticSearchClientTests.java | 34 +++++++++++++++++++ 3 files changed, 36 insertions(+) diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticSearchJavaRestClient.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticSearchJavaRestClient.java index 4749ea2e2d383..afda5ba0e7449 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticSearchJavaRestClient.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticSearchJavaRestClient.java @@ -84,6 +84,7 @@ public ElasticSearchJavaRestClient(ElasticSearchConfig elasticSearchConfig, .setConnectionRequestTimeout(config.getConnectionRequestTimeoutInMs()) .setConnectTimeout(config.getConnectTimeoutInMs()) .setSocketTimeout(config.getSocketTimeoutInMs())) + .setCompressionEnabled(config.isCompressionEnabled()) .setHttpClientConfigCallback(this.configCallback) .setFailureListener(new org.elasticsearch.client.RestClient.FailureListener() { public void onFailure(Node node) { diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java index 7b7041967026e..bb92047f17a31 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java @@ -112,6 +112,7 @@ public OpenSearchHighLevelRestClient(ElasticSearchConfig elasticSearchConfig, .setConnectionRequestTimeout(config.getConnectionRequestTimeoutInMs()) .setConnectTimeout(config.getConnectTimeoutInMs()) .setSocketTimeout(config.getSocketTimeoutInMs())) + .setCompressionEnabled(config.isCompressionEnabled()) .setHttpClientConfigCallback(this.configCallback) .setFailureListener(new org.opensearch.client.RestClient.FailureListener() { @Override diff --git a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java index c1e0eafe03a55..468d78d989cf1 100644 --- a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java +++ b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java @@ -30,8 +30,10 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; +import co.elastic.clients.transport.rest_client.RestClientTransport; import eu.rekawek.toxiproxy.model.ToxicDirection; import java.io.IOException; +import java.lang.reflect.Field; import java.util.Map; import java.util.Optional; import java.util.UUID; @@ -46,6 +48,8 @@ import org.apache.pulsar.io.elasticsearch.client.opensearch.OpenSearchHighLevelRestClient; import org.apache.pulsar.io.elasticsearch.testcontainers.ElasticToxiproxiContainer; import org.awaitility.Awaitility; +import org.opensearch.client.RestClient; +import org.opensearch.client.RestHighLevelClient; import org.testcontainers.containers.Network; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testng.annotations.AfterClass; @@ -110,11 +114,41 @@ public void fail() { public void testClientInstance() throws Exception { try (ElasticSearchClient client = new ElasticSearchClient(new ElasticSearchConfig() .setElasticSearchUrl("http://" + container.getHttpHostAddress()) + .setCompressionEnabled(true) .setIndexName(INDEX), mock(SinkContext.class));) { if (elasticImageName.equals(OPENSEARCH) || elasticImageName.equals(ELASTICSEARCH_7)) { assertTrue(client.getRestClient() instanceof OpenSearchHighLevelRestClient); + OpenSearchHighLevelRestClient osRestHighLevelClient = (OpenSearchHighLevelRestClient) client.getRestClient(); + RestHighLevelClient restHighLevelClient = osRestHighLevelClient.getClient(); + assertNotNull(restHighLevelClient); + + Field field = RestHighLevelClient.class.getDeclaredField("client"); + field.setAccessible(true); + RestClient restClient = (RestClient) field.get(restHighLevelClient); + assertNotNull(restClient); + + Field compressionEnabledFiled = RestClient.class.getDeclaredField("compressionEnabled"); + compressionEnabledFiled.setAccessible(true); + boolean compressionEnabled = (boolean) compressionEnabledFiled.get(restClient); + assertTrue(compressionEnabled); } else { assertTrue(client.getRestClient() instanceof ElasticSearchJavaRestClient); + ElasticSearchJavaRestClient javaRestClient = (ElasticSearchJavaRestClient) client.getRestClient(); + + Field field = ElasticSearchJavaRestClient.class.getDeclaredField("transport"); + field.setAccessible(true); + RestClientTransport transport = (RestClientTransport) field.get(javaRestClient); + assertNotNull(transport); + + Field restClientFiled = RestClientTransport.class.getDeclaredField("restClient"); + restClientFiled.setAccessible(true); + org.elasticsearch.client.RestClient restClient = (org.elasticsearch.client.RestClient) restClientFiled.get(transport); + assertNotNull(restClient); + + Field compressionEnabledFiled = org.elasticsearch.client.RestClient.class.getDeclaredField("compressionEnabled"); + compressionEnabledFiled.setAccessible(true); + boolean compressionEnabled = (boolean) compressionEnabledFiled.get(restClient); + assertTrue(compressionEnabled); } } } From d4756557bf4328019dd938a56c3135aecc3147e4 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 24 Apr 2024 15:06:00 +0800 Subject: [PATCH 108/580] [improve] [broker] Create partitioned topics automatically when enable topic level replication (#22537) --- .../pulsar/broker/admin/AdminResource.java | 104 +++++++++++++----- .../admin/impl/PersistentTopicsBase.java | 23 +++- .../broker/service/OneWayReplicatorTest.java | 87 ++++++++++++++- .../service/OneWayReplicatorTestBase.java | 31 +++--- 4 files changed, 196 insertions(+), 49 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index a1bfeb2142ffc..45455f16d4dc1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -23,6 +23,7 @@ import com.google.errorprone.annotations.CanIgnoreReturnValue; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -43,9 +44,11 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.broker.resources.ClusterResources; import org.apache.pulsar.broker.service.plugin.InvalidEntryFilterException; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.internal.TopicsImpl; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.naming.Constants; @@ -621,35 +624,82 @@ protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int n private void internalCreatePartitionedTopicToReplicatedClustersInBackground(int numPartitions) { getNamespaceReplicatedClustersAsync(namespaceName) - .thenAccept(clusters -> { - for (String cluster : clusters) { - if (!cluster.equals(pulsar().getConfiguration().getClusterName())) { - // this call happens in the background without async composition. completion is logged. - pulsar().getPulsarResources().getClusterResources() - .getClusterAsync(cluster) - .thenCompose(clusterDataOp -> - ((TopicsImpl) pulsar().getBrokerService() - .getClusterPulsarAdmin(cluster, - clusterDataOp).topics()) - .createPartitionedTopicAsync( - topicName.getPartitionedTopicName(), - numPartitions, - true, null)) - .whenComplete((__, ex) -> { - if (ex != null) { - log.error( - "[{}] Failed to create partitioned topic {} in cluster {}.", - clientAppId(), topicName, cluster, ex); - } else { - log.info( - "[{}] Successfully created partitioned topic {} in " - + "cluster {}", - clientAppId(), topicName, cluster); - } - }); - } + .thenAccept(clusters -> { + // this call happens in the background without async composition. completion is logged. + internalCreatePartitionedTopicToReplicatedClustersInBackground(clusters, numPartitions); + }); + } + + protected Map> internalCreatePartitionedTopicToReplicatedClustersInBackground( + Set clusters, int numPartitions) { + final String shortTopicName = topicName.getPartitionedTopicName(); + Map> tasksForAllClusters = new HashMap<>(); + for (String cluster : clusters) { + if (cluster.equals(pulsar().getConfiguration().getClusterName())) { + continue; + } + ClusterResources clusterResources = pulsar().getPulsarResources().getClusterResources(); + CompletableFuture createRemoteTopicFuture = new CompletableFuture<>(); + tasksForAllClusters.put(cluster, createRemoteTopicFuture); + clusterResources.getClusterAsync(cluster).whenComplete((clusterData, ex1) -> { + if (ex1 != null) { + // Unexpected error, such as NPE. Catch all error to avoid the "createRemoteTopicFuture" stuck. + log.error("[{}] An un-expected error occurs when trying to create partitioned topic {} in cluster" + + " {}.", clientAppId(), topicName, cluster, ex1); + createRemoteTopicFuture.completeExceptionally(new RestException(ex1)); + return; + } + // Get cluster data success. + TopicsImpl topics = + (TopicsImpl) pulsar().getBrokerService().getClusterPulsarAdmin(cluster, clusterData).topics(); + topics.createPartitionedTopicAsync(shortTopicName, numPartitions, true, null) + .whenComplete((ignore, ex2) -> { + if (ex2 == null) { + // Create success. + log.info("[{}] Successfully created partitioned topic {} in cluster {}", + clientAppId(), topicName, cluster); + createRemoteTopicFuture.complete(null); + return; + } + // Create topic on the remote cluster error. + Throwable unwrapEx2 = FutureUtil.unwrapCompletionException(ex2); + // The topic has been created before, check the partitions count is expected. + if (unwrapEx2 instanceof PulsarAdminException.ConflictException) { + topics.getPartitionedTopicMetadataAsync(shortTopicName).whenComplete((topicMeta, ex3) -> { + if (ex3 != null) { + // Unexpected error, such as NPE. Catch all error to avoid the + // "createRemoteTopicFuture" stuck. + log.error("[{}] Failed to check remote-cluster's topic metadata when creating" + + " partitioned topic {} in cluster {}.", + clientAppId(), topicName, cluster, ex3); + createRemoteTopicFuture.completeExceptionally(new RestException(ex3)); + } + // Call get partitioned metadata of remote cluster success. + if (topicMeta.partitions == numPartitions) { + log.info("[{}] Skip created partitioned topic {} in cluster {}, because that {}", + clientAppId(), topicName, cluster, unwrapEx2.getMessage()); + createRemoteTopicFuture.complete(null); + } else { + String errorMsg = String.format("[%s] There is an exists topic %s with different" + + " partitions %s on the remote cluster %s, you want to create it" + + " with partitions %s", + clientAppId(), shortTopicName, topicMeta.partitions, cluster, + numPartitions); + log.error(errorMsg); + createRemoteTopicFuture.completeExceptionally( + new RestException(Status.PRECONDITION_FAILED, errorMsg)); + } + }); + } else { + // An HTTP error was responded from the remote cluster. + log.error("[{}] Failed to create partitioned topic {} in cluster {}.", + clientAppId(), topicName, cluster, ex2); + createRemoteTopicFuture.completeExceptionally(new RestException(unwrapEx2)); } }); + }); + } + return tasksForAllClusters; } /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 1f8d06571908e..63ea987bb07fe 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -3253,12 +3253,13 @@ protected CompletableFuture internalSetBacklogQuota(BacklogQuota.BacklogQu } protected CompletableFuture internalSetReplicationClusters(List clusterIds) { + if (CollectionUtils.isEmpty(clusterIds)) { + return CompletableFuture.failedFuture(new RestException(Status.PRECONDITION_FAILED, + "ClusterIds should not be null or empty")); + } + Set replicationClusters = Sets.newHashSet(clusterIds); return validatePoliciesReadOnlyAccessAsync() .thenCompose(__ -> { - if (CollectionUtils.isEmpty(clusterIds)) { - throw new RestException(Status.PRECONDITION_FAILED, "ClusterIds should not be null or empty"); - } - Set replicationClusters = Sets.newHashSet(clusterIds); if (replicationClusters.contains("global")) { throw new RestException(Status.PRECONDITION_FAILED, "Cannot specify global in the list of replication clusters"); @@ -3273,6 +3274,20 @@ protected CompletableFuture internalSetReplicationClusters(List cl futures.add(validateClusterForTenantAsync(namespaceName.getTenant(), clusterId)); } return FutureUtil.waitForAll(futures); + }).thenCompose(__ -> { + // Sync to create partitioned topic on the remote cluster if needed. + TopicName topicNameWithoutPartition = TopicName.get(topicName.getPartitionedTopicName()); + return pulsar().getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .getPartitionedTopicMetadataAsync(topicNameWithoutPartition).thenCompose(topicMetaOp -> { + // Skip to create topic if the topic is non-partitioned, because the replicator will create + // it automatically. + if (topicMetaOp.isEmpty()) { + return CompletableFuture.completedFuture(null); + } + return FutureUtil.waitForAll( + internalCreatePartitionedTopicToReplicatedClustersInBackground(replicationClusters, + topicMetaOp.get().partitions).values()); + }); }).thenCompose(__ -> getTopicPoliciesAsyncWithRetry(topicName).thenCompose(op -> { TopicPolicies topicPolicies = op.orElseGet(TopicPolicies::new); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index f9184f2288f52..35073575f34ed 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -25,10 +25,12 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import io.netty.util.concurrent.FastThreadLocalThread; import java.lang.reflect.Field; import java.lang.reflect.Method; import java.time.Duration; +import java.util.Arrays; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; @@ -59,6 +61,9 @@ import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.util.FutureUtil; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; @@ -92,6 +97,20 @@ private void waitReplicatorStarted(String topicName) { }); } + private void waitReplicatorStopped(String topicName) { + Awaitility.await().untilAsserted(() -> { + Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); + assertTrue(topicOptional2.isPresent()); + PersistentTopic persistentTopic2 = (PersistentTopic) topicOptional2.get(); + assertTrue(persistentTopic2.getProducers().isEmpty()); + Optional topicOptional1 = pulsar2.getBrokerService().getTopic(topicName, false).get(); + assertTrue(topicOptional1.isPresent()); + PersistentTopic persistentTopic1 = (PersistentTopic) topicOptional2.get(); + assertTrue(persistentTopic1.getReplicators().isEmpty() + || !persistentTopic1.getReplicators().get(cluster2).isConnected()); + }); + } + /** * Override "AbstractReplicator.producer" by {@param producer} and return the original value. */ @@ -108,7 +127,7 @@ private ProducerImpl overrideProducerForReplicator(AbstractReplicator replicator @Test(timeOut = 45 * 1000) public void testReplicatorProducerStatInTopic() throws Exception { - final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); final String subscribeName = "subscribe_1"; final byte[] msgValue = "test".getBytes(); @@ -134,7 +153,7 @@ public void testReplicatorProducerStatInTopic() throws Exception { @Test(timeOut = 45 * 1000) public void testCreateRemoteConsumerFirst() throws Exception { - final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); Producer producer1 = client1.newProducer(Schema.STRING).topic(topicName).create(); // The topic in cluster2 has a replicator created producer(schema Auto_Produce), but does not have any schema。 @@ -154,7 +173,7 @@ public void testCreateRemoteConsumerFirst() throws Exception { @Test(timeOut = 45 * 1000) public void testTopicCloseWhenInternalProducerCloseErrorOnce() throws Exception { - final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); admin1.topics().createNonPartitionedTopic(topicName); // Wait for replicator started. waitReplicatorStarted(topicName); @@ -210,7 +229,7 @@ private void injectMockReplicatorProducerBuilder( BrokerService brokerService = pulsar1.getBrokerService(); // Wait for the internal client created. final String topicNameTriggerInternalClientCreate = - BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); admin1.topics().createNonPartitionedTopic(topicNameTriggerInternalClientCreate); waitReplicatorStarted(topicNameTriggerInternalClientCreate); cleanupTopics(() -> { @@ -338,7 +357,7 @@ void startCallback() { */ @Test(timeOut = 120 * 1000) public void testConcurrencyOfUnloadBundleAndRecreateProducer() throws Exception { - final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); // Inject an error for "replicator.producer" creation. // The delay time of next retry to create producer is below: // 0.1s, 0.2, 0.4, 0.8, 1.6s, 3.2s, 6.4s... @@ -409,4 +428,62 @@ public void testConcurrencyOfUnloadBundleAndRecreateProducer() throws Exception admin2.topics().delete(topicName); }); } + + @Test + public void testPartitionedTopicLevelReplication() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); + final String partition0 = TopicName.get(topicName).getPartition(0).toString(); + final String partition1 = TopicName.get(topicName).getPartition(1).toString(); + admin1.topics().createPartitionedTopic(topicName, 2); + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); + // Check the partitioned topic has been created at the remote cluster. + PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); + assertEquals(topicMetadata2.partitions, 2); + // cleanup. + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); + waitReplicatorStopped(partition0); + waitReplicatorStopped(partition1); + admin1.topics().deletePartitionedTopic(topicName); + admin2.topics().deletePartitionedTopic(topicName); + } + + @Test + public void testPartitionedTopicLevelReplicationRemoteTopicExist() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); + final String partition0 = TopicName.get(topicName).getPartition(0).toString(); + final String partition1 = TopicName.get(topicName).getPartition(1).toString(); + admin1.topics().createPartitionedTopic(topicName, 2); + admin2.topics().createPartitionedTopic(topicName, 2); + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); + // Check the partitioned topic has been created at the remote cluster. + PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); + assertEquals(topicMetadata2.partitions, 2); + // cleanup. + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); + waitReplicatorStopped(partition0); + waitReplicatorStopped(partition1); + admin1.topics().deletePartitionedTopic(topicName); + admin2.topics().deletePartitionedTopic(topicName); + } + + @Test + public void testPartitionedTopicLevelReplicationRemoteConflictTopicExist() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); + admin2.topics().createPartitionedTopic(topicName, 3); + admin1.topics().createPartitionedTopic(topicName, 2); + try { + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); + fail("Expected error due to a conflict partitioned topic already exists."); + } catch (Exception ex) { + Throwable unWrapEx = FutureUtil.unwrapCompletionException(ex); + assertTrue(unWrapEx.getMessage().contains("with different partitions")); + } + // Check nothing changed. + PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); + assertEquals(topicMetadata2.partitions, 3); + assertEquals(admin1.topics().getReplicationClusters(topicName, true).size(), 1); + // cleanup. + admin1.topics().deletePartitionedTopic(topicName); + admin2.topics().deletePartitionedTopic(topicName); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index 8e8b444f952c7..181721e34aa73 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -45,7 +45,8 @@ public abstract class OneWayReplicatorTestBase extends TestRetrySupport { protected final String defaultTenant = "public"; - protected final String defaultNamespace = defaultTenant + "/default"; + protected final String replicatedNamespace = defaultTenant + "/default"; + protected final String nonReplicatedNamespace = defaultTenant + "/ns1"; protected final String cluster1 = "r1"; protected URL url1; @@ -142,17 +143,19 @@ protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { admin2.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), Sets.newHashSet(cluster1, cluster2))); - admin1.namespaces().createNamespace(defaultNamespace, Sets.newHashSet(cluster1, cluster2)); - admin2.namespaces().createNamespace(defaultNamespace); + admin1.namespaces().createNamespace(replicatedNamespace, Sets.newHashSet(cluster1, cluster2)); + admin2.namespaces().createNamespace(replicatedNamespace); + admin1.namespaces().createNamespace(nonReplicatedNamespace); + admin2.namespaces().createNamespace(nonReplicatedNamespace); } protected void cleanupTopics(CleanupTopicAction cleanupTopicAction) throws Exception { - waitChangeEventsInit(defaultNamespace); - admin1.namespaces().setNamespaceReplicationClusters(defaultNamespace, Collections.singleton(cluster1)); - admin1.namespaces().unload(defaultNamespace); + waitChangeEventsInit(replicatedNamespace); + admin1.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Collections.singleton(cluster1)); + admin1.namespaces().unload(replicatedNamespace); cleanupTopicAction.run(); - admin1.namespaces().setNamespaceReplicationClusters(defaultNamespace, Sets.newHashSet(cluster1, cluster2)); - waitChangeEventsInit(defaultNamespace); + admin1.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster1, cluster2)); + waitChangeEventsInit(replicatedNamespace); } protected void waitChangeEventsInit(String namespace) { @@ -220,11 +223,13 @@ protected void setConfigDefaults(ServiceConfiguration config, String clusterName @Override protected void cleanup() throws Exception { // delete namespaces. - waitChangeEventsInit(defaultNamespace); - admin1.namespaces().setNamespaceReplicationClusters(defaultNamespace, Sets.newHashSet(cluster1)); - admin1.namespaces().deleteNamespace(defaultNamespace); - admin2.namespaces().setNamespaceReplicationClusters(defaultNamespace, Sets.newHashSet(cluster2)); - admin2.namespaces().deleteNamespace(defaultNamespace); + waitChangeEventsInit(replicatedNamespace); + admin1.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster1)); + admin1.namespaces().deleteNamespace(replicatedNamespace); + admin2.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster2)); + admin2.namespaces().deleteNamespace(replicatedNamespace); + admin1.namespaces().deleteNamespace(nonReplicatedNamespace); + admin2.namespaces().deleteNamespace(nonReplicatedNamespace); // shutdown. markCurrentSetupNumberCleaned(); From b774666331db33ea6407174e0fe6e27a73160522 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 25 Apr 2024 01:45:41 +0800 Subject: [PATCH 109/580] [fix] [broker] Part-2: Replicator can not created successfully due to an orphan replicator in the previous topic owner (#21948) --- .../broker/service/AbstractReplicator.java | 10 +- .../pulsar/broker/service/Replicator.java | 2 + .../persistent/PersistentReplicator.java | 9 +- .../service/persistent/PersistentTopic.java | 58 ++++-- .../broker/service/OneWayReplicatorTest.java | 166 ++++++++++++++++++ .../service/OneWayReplicatorTestBase.java | 14 +- .../pulsar/broker/service/ReplicatorTest.java | 2 +- 7 files changed, 239 insertions(+), 22 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index f34144deb0ab0..394fad21ae6dc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -248,7 +248,7 @@ protected void scheduleCheckTopicActiveAndStartProducer(final long waitTimeMs) { } startProducer(); }).exceptionally(ex -> { - log.warn("[{}] [{}] Stop retry to create producer due to unknown error(topic create failed), and" + log.error("[{}] [{}] Stop retry to create producer due to unknown error(topic create failed), and" + " trigger a terminate. Replicator state: {}", localTopicName, replicatorId, STATE_UPDATER.get(this), ex); terminate(); @@ -377,9 +377,13 @@ public CompletableFuture terminate() { this.producer = null; // set the cursor as inactive. disableReplicatorRead(); + // release resources. + doReleaseResources(); }); } + protected void doReleaseResources() {} + protected boolean tryChangeStatusToTerminating() { if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Terminating)){ return true; @@ -468,4 +472,8 @@ protected ImmutablePair compareSetAndGetState(State expect, Stat } return compareSetAndGetState(expect, update); } + + public boolean isTerminated() { + return state == State.Terminating || state == State.Terminated; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java index 8130b855b4e4a..5c314397da80e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java @@ -51,4 +51,6 @@ default Optional getRateLimiter() { boolean isConnected(); long getNumberOfEntriesInBacklog(); + + boolean isTerminated(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index 5e1cc4a936a75..367d19652072d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -450,7 +450,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { long waitTimeMillis = readFailureBackoff.next(); if (exception instanceof CursorAlreadyClosedException) { - log.error("[{}] Error reading entries because replicator is" + log.warn("[{}] Error reading entries because replicator is" + " already deleted and cursor is already closed {}, ({})", replicatorId, ctx, exception.getMessage(), exception); // replicator is already deleted and cursor is already closed so, producer should also be disconnected. @@ -570,7 +570,7 @@ public void deleteFailed(ManagedLedgerException exception, Object ctx) { log.error("[{}] Failed to delete message at {}: {}", replicatorId, ctx, exception.getMessage(), exception); if (exception instanceof CursorAlreadyClosedException) { - log.error("[{}] Asynchronous ack failure because replicator is already deleted and cursor is already" + log.warn("[{}] Asynchronous ack failure because replicator is already deleted and cursor is already" + " closed {}, ({})", replicatorId, ctx, exception.getMessage(), exception); // replicator is already deleted and cursor is already closed so, producer should also be disconnected. terminate(); @@ -698,6 +698,11 @@ public boolean isConnected() { return producer != null && producer.isConnected(); } + @Override + protected void doReleaseResources() { + dispatchRateLimiter.ifPresent(DispatchRateLimiter::close); + } + private static final Logger log = LoggerFactory.getLogger(PersistentReplicator.class); @VisibleForTesting diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 9d6855962ced6..c1a75d67e3c4e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -1731,6 +1731,7 @@ public CompletableFuture checkReplication() { return deleteForcefully(); } + removeTerminatedReplicators(replicators); List> futures = new ArrayList<>(); // Check for missing replicators @@ -1769,6 +1770,8 @@ private CompletableFuture checkShadowReplication() { if (log.isDebugEnabled()) { log.debug("[{}] Checking shadow replication status, shadowTopics={}", topic, configuredShadowTopics); } + + removeTerminatedReplicators(shadowReplicators); List> futures = new ArrayList<>(); // Check for missing replicators @@ -1919,19 +1922,30 @@ protected CompletableFuture addReplicationCluster(String remoteCluster, Ma if (replicationClient == null) { return; } - Replicator replicator = replicators.computeIfAbsent(remoteCluster, r -> { - try { - return new GeoPersistentReplicator(PersistentTopic.this, cursor, localCluster, - remoteCluster, brokerService, (PulsarClientImpl) replicationClient); - } catch (PulsarServerException e) { - log.error("[{}] Replicator startup failed {}", topic, remoteCluster, e); + lock.readLock().lock(); + try { + if (isClosingOrDeleting) { + // Whether is "transferring" or not, do not create new replicator. + log.info("[{}] Skip to create replicator because this topic is closing." + + " remote cluster: {}. State of transferring : {}", + topic, remoteCluster, transferring); + return; } - return null; - }); - - // clean up replicator if startup is failed - if (replicator == null) { - replicators.removeNullValue(remoteCluster); + Replicator replicator = replicators.computeIfAbsent(remoteCluster, r -> { + try { + return new GeoPersistentReplicator(PersistentTopic.this, cursor, localCluster, + remoteCluster, brokerService, (PulsarClientImpl) replicationClient); + } catch (PulsarServerException e) { + log.error("[{}] Replicator startup failed {}", topic, remoteCluster, e); + } + return null; + }); + // clean up replicator if startup is failed + if (replicator == null) { + replicators.removeNullValue(remoteCluster); + } + } finally { + lock.readLock().unlock(); } }); } @@ -3881,9 +3895,27 @@ private void fenceTopicToCloseOrDelete() { } private void unfenceTopicToResume() { - subscriptions.values().forEach(sub -> sub.resumeAfterFence()); isFenced = false; isClosingOrDeleting = false; + subscriptions.values().forEach(sub -> sub.resumeAfterFence()); + unfenceReplicatorsToResume(); + } + + private void unfenceReplicatorsToResume() { + checkReplication(); + checkShadowReplication(); + } + + private void removeTerminatedReplicators(ConcurrentOpenHashMap replicators) { + Map terminatedReplicators = new HashMap<>(); + replicators.forEach((cluster, replicator) -> { + if (replicator.isTerminated()) { + terminatedReplicators.put(cluster, replicator); + } + }); + terminatedReplicators.entrySet().forEach(entry -> { + replicators.remove(entry.getKey(), entry.getValue()); + }); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index 35073575f34ed..9b8b567af081b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -20,18 +20,21 @@ 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.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.google.common.collect.Sets; import io.netty.util.concurrent.FastThreadLocalThread; import java.lang.reflect.Field; import java.lang.reflect.Method; import java.time.Duration; import java.util.Arrays; import java.util.Optional; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -48,6 +51,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.Consumer; @@ -486,4 +490,166 @@ public void testPartitionedTopicLevelReplicationRemoteConflictTopicExist() throw admin1.topics().deletePartitionedTopic(topicName); admin2.topics().deletePartitionedTopic(topicName); } + + /** + * See the description and execution flow: https://github.com/apache/pulsar/pull/21948. + * Steps: + * 1.Create topic, does not enable replication now. + * - The topic will be loaded in the memory. + * 2.Enable namespace level replication. + * - Broker creates a replicator, and the internal producer of replicator is starting. + * - We inject an error to make the internal producer fail to connect,after few seconds, it will retry to start. + * 3.Unload bundle. + * - Starting to close the topic. + * - The replicator will be closed, but it will not close the internal producer, because the producer has not + * been created successfully. + * - We inject a sleeping into the progress of closing the "repl.cursor" to make it stuck. So the topic is still + * in the process of being closed now. + * 4.Internal producer retry to connect. + * - At the next retry, it connected successful. Since the state of "repl.cursor" is not "Closed", this producer + * will not be closed now. + * 5.Topic closed. + * - Cancel the stuck of closing the "repl.cursor". + * - The topic is wholly closed. + * 6.Verify: the delayed created internal producer will be closed. In other words, there is no producer is connected + * to the remote cluster. + */ + @Test + public void testConcurrencyOfUnloadBundleAndRecreateProducer2() throws Exception { + final String namespaceName = defaultTenant + "/" + UUID.randomUUID().toString().replaceAll("-", ""); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + namespaceName + "/tp_"); + // 1.Create topic, does not enable replication now. + admin1.namespaces().createNamespace(namespaceName); + admin2.namespaces().createNamespace(namespaceName); + admin1.topics().createNonPartitionedTopic(topicName); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + + // We inject an error to make the internal producer fail to connect. + // The delay time of next retry to create producer is below: + // 0.1s, 0.2, 0.4, 0.8, 1.6s, 3.2s, 6.4s... + // If the retry counter is larger than 6, the next creation will be slow enough to close Replicator. + final AtomicInteger createProducerCounter = new AtomicInteger(); + final int failTimes = 6; + injectMockReplicatorProducerBuilder((producerCnf, originalProducer) -> { + if (topicName.equals(producerCnf.getTopicName())) { + // There is a switch to determine create producer successfully or not. + if (createProducerCounter.incrementAndGet() > failTimes) { + return originalProducer; + } + log.info("Retry create replicator.producer count: {}", createProducerCounter); + // Release producer and fail callback. + originalProducer.closeAsync(); + throw new RuntimeException("mock error"); + } + return originalProducer; + }); + + // 2.Enable namespace level replication. + admin1.namespaces().setNamespaceReplicationClusters(namespaceName, Sets.newHashSet(cluster1, cluster2)); + AtomicReference replicator = new AtomicReference(); + Awaitility.await().untilAsserted(() -> { + assertFalse(persistentTopic.getReplicators().isEmpty()); + replicator.set( + (PersistentReplicator) persistentTopic.getReplicators().values().iterator().next()); + // Since we inject a producer creation error, the replicator can not start successfully. + assertFalse(replicator.get().isConnected()); + }); + + // We inject a sleeping into the progress of closing the "repl.cursor" to make it stuck, until the internal + // producer of the replicator started. + SpyCursor spyCursor = + spyCursor(persistentTopic, "pulsar.repl." + pulsar2.getConfig().getClusterName()); + CursorCloseSignal cursorCloseSignal = makeCursorClosingDelay(spyCursor); + + // 3.Unload bundle: call "topic.close(false)". + // Stuck start new producer, until the state of replicator change to Stopped. + // The next once of "createProducerSuccessAfterFailTimes" to create producer will be successfully. + Awaitility.await().pollInterval(Duration.ofMillis(100)).atMost(Duration.ofSeconds(60)).untilAsserted(() -> { + assertTrue(createProducerCounter.get() >= failTimes); + }); + CompletableFuture topicCloseFuture = persistentTopic.close(true); + Awaitility.await().atMost(Duration.ofSeconds(30)).untilAsserted(() -> { + String state = String.valueOf(replicator.get().getState()); + log.error("replicator state: {}", state); + assertTrue(state.equals("Disconnected") || state.equals("Terminated")); + }); + + // 5.Delay close cursor, until "replicator.producer" create successfully. + // The next once retry time of create "replicator.producer" will be 3.2s. + Thread.sleep(4 * 1000); + log.info("Replicator.state: {}", replicator.get().getState()); + cursorCloseSignal.startClose(); + cursorCloseSignal.startCallback(); + // Wait for topic close successfully. + topicCloseFuture.join(); + + // 6. Verify there is no orphan producer on the remote cluster. + Awaitility.await().pollInterval(Duration.ofSeconds(1)).untilAsserted(() -> { + PersistentTopic persistentTopic2 = + (PersistentTopic) pulsar2.getBrokerService().getTopic(topicName, false).join().get(); + assertEquals(persistentTopic2.getProducers().size(), 0); + Assert.assertFalse(replicator.get().isConnected()); + }); + + // cleanup. + cleanupTopics(namespaceName, () -> { + admin1.topics().delete(topicName); + admin2.topics().delete(topicName); + }); + admin1.namespaces().setNamespaceReplicationClusters(namespaceName, Sets.newHashSet(cluster1)); + admin1.namespaces().deleteNamespace(namespaceName); + admin2.namespaces().deleteNamespace(namespaceName); + } + + @Test + public void testUnFenceTopicToReuse() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp"); + // Wait for replicator started. + Producer producer1 = client1.newProducer(Schema.STRING).topic(topicName).create(); + waitReplicatorStarted(topicName); + + // Inject an error to make topic close fails. + final String mockProducerName = UUID.randomUUID().toString(); + final org.apache.pulsar.broker.service.Producer mockProducer = + mock(org.apache.pulsar.broker.service.Producer.class); + doAnswer(invocation -> CompletableFuture.failedFuture(new RuntimeException("mocked error"))) + .when(mockProducer).disconnect(any()); + doAnswer(invocation -> CompletableFuture.failedFuture(new RuntimeException("mocked error"))) + .when(mockProducer).disconnect(); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + persistentTopic.getProducers().put(mockProducerName, mockProducer); + + // Do close. + GeoPersistentReplicator replicator1 = + (GeoPersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + try { + persistentTopic.close(true, false).join(); + fail("Expected close fails due to a producer close fails"); + } catch (Exception ex) { + log.info("Expected error: {}", ex.getMessage()); + } + + // Broker will call `topic.unfenceTopicToResume` if close clients fails. + // Verify: the replicator will be re-created. + Awaitility.await().untilAsserted(() -> { + assertTrue(producer1.isConnected()); + GeoPersistentReplicator replicator2 = + (GeoPersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + assertNotEquals(replicator1, replicator2); + assertFalse(replicator1.isConnected()); + assertFalse(replicator1.producer != null && replicator1.producer.isConnected()); + assertTrue(replicator2.isConnected()); + assertTrue(replicator2.producer != null && replicator2.producer.isConnected()); + }); + + // cleanup. + persistentTopic.getProducers().remove(mockProducerName, mockProducer); + producer1.close(); + cleanupTopics(() -> { + admin1.topics().delete(topicName); + admin2.topics().delete(topicName); + }); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index 181721e34aa73..95f976f965a0d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -150,12 +150,16 @@ protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { } protected void cleanupTopics(CleanupTopicAction cleanupTopicAction) throws Exception { - waitChangeEventsInit(replicatedNamespace); - admin1.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Collections.singleton(cluster1)); - admin1.namespaces().unload(replicatedNamespace); + cleanupTopics(replicatedNamespace, cleanupTopicAction); + } + + protected void cleanupTopics(String namespace, CleanupTopicAction cleanupTopicAction) throws Exception { + waitChangeEventsInit(namespace); + admin1.namespaces().setNamespaceReplicationClusters(namespace, Collections.singleton(cluster1)); + admin1.namespaces().unload(namespace); cleanupTopicAction.run(); - admin1.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster1, cluster2)); - waitChangeEventsInit(replicatedNamespace); + admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet(cluster1, cluster2)); + waitChangeEventsInit(namespace); } protected void waitChangeEventsInit(String namespace) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index a05c3468ea16e..0bfcdf563d632 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -152,7 +152,7 @@ public Object[][] partitionedTopicProvider() { return new Object[][] { { Boolean.TRUE }, { Boolean.FALSE } }; } - @Test + @Test(priority = Integer.MAX_VALUE) public void testConfigChange() throws Exception { log.info("--- Starting ReplicatorTest::testConfigChange ---"); // This test is to verify that the config change on global namespace is successfully applied in broker during From 0c097ef2c6c85efbb91d388ffe839ec542e82278 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Thu, 25 Apr 2024 16:55:45 +0800 Subject: [PATCH 110/580] [improve][misc] Upgrade slf4j to 2.0.13 (#22391) Signed-off-by: Zixuan Liu --- buildtools/pom.xml | 10 ++++-- distribution/server/pom.xml | 2 +- .../server/src/assemble/LICENSE.bin.txt | 6 ++-- distribution/shell/pom.xml | 2 +- .../shell/src/assemble/LICENSE.bin.txt | 4 +-- pom.xml | 22 ++++++++++++- pulsar-client-all/pom.xml | 2 +- pulsar-functions/instance/pom.xml | 2 +- pulsar-functions/localrun/pom.xml | 2 +- pulsar-functions/runtime-all/pom.xml | 2 +- pulsar-io/alluxio/pom.xml | 10 ++++++ pulsar-io/hdfs2/pom.xml | 2 +- pulsar-io/hdfs3/pom.xml | 2 +- pulsar-io/rabbitmq/pom.xml | 4 +-- .../io/rabbitmq/RabbitMQBrokerManager.java | 33 ++++++++++--------- pulsar-io/solr/pom.xml | 4 +++ structured-event-log/pom.xml | 2 +- tiered-storage/file-system/pom.xml | 6 +++- 18 files changed, 82 insertions(+), 35 deletions(-) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index cd4d02af3d7b4..58f99e9ea86b5 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -41,7 +41,7 @@ 1.8 3.1.0 2.23.1 - 1.7.32 + 2.0.13 7.7.1 3.11 4.1 @@ -100,6 +100,12 @@ org.testng testng ${testng.version} + + + org.slf4j + * + + org.apache.logging.log4j @@ -111,7 +117,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl org.slf4j diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 599a9755f9155..1c9ea68685308 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -180,7 +180,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index c5642503b25b0..c5c243796b6f3 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -350,7 +350,7 @@ The Apache Software License, Version 2.0 * Log4J - org.apache.logging.log4j-log4j-api-2.23.1.jar - org.apache.logging.log4j-log4j-core-2.23.1.jar - - org.apache.logging.log4j-log4j-slf4j-impl-2.23.1.jar + - org.apache.logging.log4j-log4j-slf4j2-impl-2.23.1.jar - org.apache.logging.log4j-log4j-web-2.23.1.jar * Java Native Access JNA - net.java.dev.jna-jna-jpms-5.12.1.jar @@ -563,8 +563,8 @@ BSD 2-Clause License MIT License * Java SemVer -- com.github.zafarkhaja-java-semver-0.9.0.jar -- ../licenses/LICENSE-SemVer.txt * SLF4J -- ../licenses/LICENSE-SLF4J.txt - - org.slf4j-slf4j-api-1.7.32.jar - - org.slf4j-jcl-over-slf4j-1.7.32.jar + - org.slf4j-slf4j-api-2.0.13.jar + - org.slf4j-jcl-over-slf4j-2.0.13.jar * The Checker Framework - org.checkerframework-checker-qual-3.33.0.jar * oshi diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 5f4fc549ccc62..144f7b1ff6d83 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -51,7 +51,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index f76631dbbf260..41b38f17dce77 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -386,7 +386,7 @@ The Apache Software License, Version 2.0 * Log4J - log4j-api-2.23.1.jar - log4j-core-2.23.1.jar - - log4j-slf4j-impl-2.23.1.jar + - log4j-slf4j2-impl-2.23.1.jar - log4j-web-2.23.1.jar * OpenTelemetry - opentelemetry-api-1.34.1.jar @@ -424,7 +424,7 @@ BSD 3-clause "New" or "Revised" License MIT License * SLF4J -- ../licenses/LICENSE-SLF4J.txt - - slf4j-api-1.7.32.jar + - slf4j-api-2.0.13.jar * The Checker Framework - checker-qual-3.33.0.jar diff --git a/pom.xml b/pom.xml index 90b6c8cb8edf4..585347fb1f855 100644 --- a/pom.xml +++ b/pom.xml @@ -153,7 +153,7 @@ flexible messaging model and an intuitive client API. 0.16.0 4.3.8 7.9.2 - 1.7.32 + 2.0.13 4.4 2.23.1 1.78 @@ -352,6 +352,10 @@ flexible messaging model and an intuitive client API. org.yaml * + + org.slf4j + * + @@ -460,6 +464,10 @@ flexible messaging model and an intuitive client API. org.bouncycastle * + + log4j-slf4j-impl + org.apache.logging.log4j + slf4j-log4j12 org.slf4j @@ -1582,6 +1590,10 @@ flexible messaging model and an intuitive client API. org.apache.zookeeper * + + log4j-slf4j-impl + org.apache.logging.log4j + @@ -2701,5 +2713,13 @@ flexible messaging model and an intuitive client API. false + + + oracle.releases + https://download.oracle.com/maven + + false + + diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 781c03721fb69..eca03ee1502b0 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -66,7 +66,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl test diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index b8d197c0683d3..84d8a032d3bde 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -36,7 +36,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl org.apache.logging.log4j diff --git a/pulsar-functions/localrun/pom.xml b/pulsar-functions/localrun/pom.xml index 5118a30b92f94..001780bdb5178 100644 --- a/pulsar-functions/localrun/pom.xml +++ b/pulsar-functions/localrun/pom.xml @@ -43,7 +43,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl org.apache.logging.log4j diff --git a/pulsar-functions/runtime-all/pom.xml b/pulsar-functions/runtime-all/pom.xml index 5e300c78aac46..d367038ff4945 100644 --- a/pulsar-functions/runtime-all/pom.xml +++ b/pulsar-functions/runtime-all/pom.xml @@ -105,7 +105,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl diff --git a/pulsar-io/alluxio/pom.xml b/pulsar-io/alluxio/pom.xml index 53f2ed095c683..86d76ec9578ee 100644 --- a/pulsar-io/alluxio/pom.xml +++ b/pulsar-io/alluxio/pom.xml @@ -57,6 +57,12 @@ org.alluxio alluxio-core-client-fs ${alluxio.version} + + + log4j-slf4j-impl + org.apache.logging.log4j + + @@ -69,6 +75,10 @@ org.glassfish javax.el + + log4j-slf4j-impl + org.apache.logging.log4j + diff --git a/pulsar-io/hdfs2/pom.xml b/pulsar-io/hdfs2/pom.xml index ef198579544a4..38f1460a33c57 100644 --- a/pulsar-io/hdfs2/pom.xml +++ b/pulsar-io/hdfs2/pom.xml @@ -61,7 +61,7 @@ org.slf4j - slf4j-log4j12 + * org.apache.avro diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index dcb0226a1f8d0..1a7a975098bec 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -65,7 +65,7 @@ org.slf4j - slf4j-log4j12 + * org.apache.avro diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index 9074faed3b7cf..f8a5c0601d146 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -49,7 +49,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl @@ -83,7 +83,7 @@ org.apache.qpid qpid-broker - 6.1.6 + 9.2.0 test diff --git a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java index 507313c86fd7f..83331bf7de810 100644 --- a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java +++ b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java @@ -18,28 +18,29 @@ */ package org.apache.pulsar.io.rabbitmq; -import org.apache.qpid.server.Broker; -import org.apache.qpid.server.BrokerOptions; - import java.io.File; import java.io.FileOutputStream; import java.nio.file.Files; import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; +import org.apache.qpid.server.SystemLauncher; +import org.apache.qpid.server.model.SystemConfig; public class RabbitMQBrokerManager { - private final Broker broker = new Broker(); + private final SystemLauncher systemLauncher = new SystemLauncher(); public void startBroker(String port) throws Exception { - BrokerOptions brokerOptions = getBrokerOptions(port); - broker.startup(brokerOptions); + Map brokerOptions = getBrokerOptions(port); + systemLauncher.startup(brokerOptions); } public void stopBroker() { - broker.shutdown(); + systemLauncher.shutdown(); } - BrokerOptions getBrokerOptions(String port) throws Exception { + Map getBrokerOptions(String port) throws Exception { Path tmpFolder = Files.createTempDirectory("qpidWork"); Path homeFolder = Files.createTempDirectory("qpidHome"); File etc = new File(homeFolder.toFile(), "etc"); @@ -48,15 +49,17 @@ BrokerOptions getBrokerOptions(String port) throws Exception { fos.write("guest:guest\n".getBytes()); fos.close(); - BrokerOptions brokerOptions = new BrokerOptions(); - - brokerOptions.setConfigProperty("qpid.work_dir", tmpFolder.toAbsolutePath().toString()); - brokerOptions.setConfigProperty("qpid.amqp_port", port); - brokerOptions.setConfigProperty("qpid.home_dir", homeFolder.toAbsolutePath().toString()); + Map config = new HashMap<>(); + config.put("qpid.work_dir", tmpFolder.toAbsolutePath().toString()); + config.put("qpid.amqp_port", port); + config.put("qpid.home_dir", homeFolder.toAbsolutePath().toString()); String configPath = getFile("qpid.json").getAbsolutePath(); - brokerOptions.setInitialConfigurationLocation(configPath); - return brokerOptions; + Map context = new HashMap<>(); + context.put(SystemConfig.INITIAL_CONFIGURATION_LOCATION, configPath); + context.put(SystemConfig.TYPE, "Memory"); + context.put(SystemConfig.CONTEXT, config); + return context; } private File getFile(String name) { diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index 5be2639c718fb..2b7893fc945a1 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -70,6 +70,10 @@ jose4j org.bitbucket.b_c + + log4j-slf4j-impl + org.apache.logging.log4j + test diff --git a/structured-event-log/pom.xml b/structured-event-log/pom.xml index d09637fca76ae..cc34f921dc2bb 100644 --- a/structured-event-log/pom.xml +++ b/structured-event-log/pom.xml @@ -51,7 +51,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl test diff --git a/tiered-storage/file-system/pom.xml b/tiered-storage/file-system/pom.xml index 040d17a8b40da..8df8aa21c42f6 100644 --- a/tiered-storage/file-system/pom.xml +++ b/tiered-storage/file-system/pom.xml @@ -48,7 +48,7 @@ org.slf4j - slf4j-log4j12 + * @@ -118,6 +118,10 @@ org.bouncycastle * + + org.slf4j + * + From 997c8b95e1798cee08c56d92b77eb70056dfca8f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 25 Apr 2024 14:12:33 +0300 Subject: [PATCH 111/580] [fix][broker] Fix BufferOverflowException and EOFException bugs in /metrics gzip compression (#22576) --- .../PrometheusMetricsGenerator.java | 74 +++++++++++----- .../PrometheusMetricsGeneratorTest.java | 85 +++++++++++++++++++ 2 files changed, 138 insertions(+), 21 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java index 8cd68caf1ee26..6b4d08c359d42 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java @@ -24,6 +24,7 @@ import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.PooledByteBufAllocator; +import io.netty.buffer.Unpooled; import io.prometheus.client.Collector; import java.io.BufferedOutputStream; import java.io.IOException; @@ -191,8 +192,8 @@ private static class GzipByteBufferWriter { crc = new CRC32(); this.bufferSize = Math.max(Math.min(resolveChunkSize(bufAllocator), readableBytes), 8192); this.bufAllocator = bufAllocator; - this.resultBuffer = bufAllocator.compositeDirectBuffer(readableBytes / bufferSize + 1); - allocateBuffer(); + this.resultBuffer = bufAllocator.compositeDirectBuffer(readableBytes / bufferSize + 2); + allocateCompressBuffer(); } /** @@ -217,37 +218,66 @@ private void compressAndAppend(ByteBuffer nioBuffer, boolean isFirst, boolean is // write gzip header compressBuffer.put(GZIP_HEADER); } + // update the CRC32 checksum calculation nioBuffer.mark(); crc.update(nioBuffer); nioBuffer.reset(); + // pass the input buffer to the deflater deflater.setInput(nioBuffer); + // when the input buffer is the last one, set the flag to finish the deflater if (isLast) { deflater.finish(); } - while (!deflater.needsInput() && !deflater.finished()) { - int written = deflater.deflate(compressBuffer); - if (written == 0 && !compressBuffer.hasRemaining()) { - backingCompressBuffer.setIndex(0, compressBuffer.position()); - resultBuffer.addComponent(true, backingCompressBuffer); - allocateBuffer(); + int written = -1; + // the deflater may need multiple calls to deflate the input buffer + // the completion is checked by the deflater.needsInput() method for buffers that aren't the last buffer + // for the last buffer, the completion is checked by the deflater.finished() method + while (!isLast && !deflater.needsInput() || isLast && !deflater.finished()) { + // when the previous deflater.deflate call returns 0 (and needsInput/finished returns false), + // it means that the output buffer is full. + // append the compressed buffer to the result buffer and allocate a new buffer. + if (written == 0) { + if (compressBuffer.position() > 0) { + appendCompressBufferToResultBuffer(); + allocateCompressBuffer(); + } else { + // this is an unexpected case, throw an exception to prevent an infinite loop + throw new IllegalStateException( + "Deflater didn't write any bytes while the compress buffer is empty."); + } } + written = deflater.deflate(compressBuffer); } if (isLast) { - // write gzip footer, integer values are in little endian byte order - compressBuffer.order(ByteOrder.LITTLE_ENDIAN); - // write CRC32 checksum - compressBuffer.putInt((int) crc.getValue()); - // write uncompressed size - compressBuffer.putInt(deflater.getTotalIn()); - // append the last compressed buffer - backingCompressBuffer.setIndex(0, compressBuffer.position()); - resultBuffer.addComponent(true, backingCompressBuffer); + // append the last compressed buffer when it is not empty + if (compressBuffer.position() > 0) { + appendCompressBufferToResultBuffer(); + } else { + // release an unused empty buffer + backingCompressBuffer.release(); + } backingCompressBuffer = null; compressBuffer = null; + + // write gzip trailer, 2 integers (CRC32 checksum and uncompressed size) + ByteBuffer trailerBuf = ByteBuffer.allocate(2 * Integer.BYTES); + // integer values are in little endian byte order + trailerBuf.order(ByteOrder.LITTLE_ENDIAN); + // write CRC32 checksum + trailerBuf.putInt((int) crc.getValue()); + // write uncompressed size + trailerBuf.putInt(deflater.getTotalIn()); + trailerBuf.flip(); + resultBuffer.addComponent(true, Unpooled.wrappedBuffer(trailerBuf)); } } - private void allocateBuffer() { + private void appendCompressBufferToResultBuffer() { + backingCompressBuffer.setIndex(0, compressBuffer.position()); + resultBuffer.addComponent(true, backingCompressBuffer); + } + + private void allocateCompressBuffer() { backingCompressBuffer = bufAllocator.directBuffer(bufferSize); compressBuffer = backingCompressBuffer.nioBuffer(0, bufferSize); } @@ -282,7 +312,7 @@ public PrometheusMetricsGenerator(PulsarService pulsar, boolean includeTopicMetr this.clock = clock; } - private ByteBuf generate0(List metricsProviders) { + protected ByteBuf generateMetrics(List metricsProviders) { ByteBuf buf = allocateMultipartCompositeDirectBuffer(); boolean exceptionHappens = false; //Used in namespace/topic and transaction aggregators as share metric names @@ -342,7 +372,9 @@ private ByteBuf allocateMultipartCompositeDirectBuffer() { int totalLen = 0; while (totalLen < initialBufferSize) { totalLen += chunkSize; - buf.addComponent(false, byteBufAllocator.directBuffer(chunkSize)); + // increase the capacity in increments of chunkSize to preallocate the buffers + // in the composite buffer + buf.capacity(totalLen); } return buf; } @@ -492,7 +524,7 @@ public MetricsBuffer renderToBuffer(Executor executor, List bufferFuture = newMetricsBuffer.getBufferFuture(); executor.execute(() -> { try { - bufferFuture.complete(new ResponseBuffer(generate0(metricsProviders))); + bufferFuture.complete(new ResponseBuffer(generateMetrics(metricsProviders))); } catch (Exception e) { bufferFuture.completeExceptionally(e); } finally { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorTest.java new file mode 100644 index 0000000000000..ed5c5a6335ceb --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorTest.java @@ -0,0 +1,85 @@ +/* + * 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.stats.prometheus; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import com.google.common.util.concurrent.MoreExecutors; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.time.Clock; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.concurrent.ExecutionException; +import java.util.zip.GZIPInputStream; +import org.apache.commons.io.IOUtils; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.testng.annotations.Test; + +public class PrometheusMetricsGeneratorTest { + + // reproduce issue #22575 + @Test + public void testReproducingBufferOverflowExceptionAndEOFExceptionBugsInGzipCompression() + throws ExecutionException, InterruptedException, IOException { + PulsarService pulsar = mock(PulsarService.class); + ServiceConfiguration serviceConfiguration = new ServiceConfiguration(); + when(pulsar.getConfiguration()).thenReturn(serviceConfiguration); + + // generate a random byte buffer which is 8 bytes less than the minimum compress buffer size limit + // this will trigger the BufferOverflowException bug in writing the gzip trailer + // it will also trigger another bug in finishing the gzip compression stream when the compress buffer is full + // which results in EOFException + Random random = new Random(); + byte[] inputBytes = new byte[8192 - 8]; + random.nextBytes(inputBytes); + ByteBuf byteBuf = Unpooled.wrappedBuffer(inputBytes); + + PrometheusMetricsGenerator generator = + new PrometheusMetricsGenerator(pulsar, false, false, false, false, Clock.systemUTC()) { + // override the generateMetrics method to return the random byte buffer for gzip compression + // instead of the actual metrics + @Override + protected ByteBuf generateMetrics(List metricsProviders) { + return byteBuf; + } + }; + + PrometheusMetricsGenerator.MetricsBuffer metricsBuffer = + generator.renderToBuffer(MoreExecutors.directExecutor(), Collections.emptyList()); + try { + PrometheusMetricsGenerator.ResponseBuffer responseBuffer = metricsBuffer.getBufferFuture().get(); + + ByteBuf compressed = responseBuffer.getCompressedBuffer(MoreExecutors.directExecutor()).get(); + byte[] compressedBytes = new byte[compressed.readableBytes()]; + compressed.readBytes(compressedBytes); + try (GZIPInputStream gzipInputStream = new GZIPInputStream(new ByteArrayInputStream(compressedBytes))) { + byte[] uncompressedBytes = IOUtils.toByteArray(gzipInputStream); + assertEquals(uncompressedBytes, inputBytes); + } + } finally { + metricsBuffer.release(); + } + } +} \ No newline at end of file From 6a9423156f35fe1c2fedab94028f337276f5e7a3 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 25 Apr 2024 18:38:31 +0300 Subject: [PATCH 112/580] [improve][ci] Disable test that causes OOME until the problem has been resolved (#22586) --- .../pulsar/broker/service/ReplicatorSubscriptionTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java index 4cc3a9ada7d04..8aeb902211db2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java @@ -757,7 +757,8 @@ private Object[][] isTopicPolicyEnabled() { * similar to step 1. *

*/ - @Test(dataProvider = "isTopicPolicyEnabled") + // TODO: this test causes OOME in the CI, need to investigate + @Test(dataProvider = "isTopicPolicyEnabled", enabled = false) public void testWriteMarkerTaskOfReplicateSubscriptions(boolean isTopicPolicyEnabled) throws Exception { // 1. Prepare resource and use proper configuration. String namespace = BrokerTestUtil.newUniqueName("pulsar/testReplicateSubBackLog"); From 3de14c55de138770ed61d1a14cd883048ea1915c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 25 Apr 2024 21:54:55 +0300 Subject: [PATCH 113/580] [fix][test] Clear fields in test cleanup to reduce memory consumption (#22583) --- .../broker/MultiBrokerTestZKBaseTest.java | 1 + .../pulsar/broker/SLAMonitoringTest.java | 21 ++++++-- .../LeaderElectionServiceTest.java | 5 +- .../broker/loadbalance/LoadBalancerTest.java | 15 ++++-- .../SimpleLoadManagerImplTest.java | 30 ++++++++--- .../extensions/BrokerRegistryTest.java | 10 +++- .../ExtensibleLoadManagerImplBaseTest.java | 9 +++- .../impl/BundleSplitterTaskTest.java | 10 +++- .../impl/ModularLoadManagerImplTest.java | 35 ++++++++---- .../broker/service/AdvertisedAddressTest.java | 10 +++- .../broker/service/BkEnsemblesTestBase.java | 15 ++++-- .../service/BrokerBookieIsolationTest.java | 6 ++- ...econnectZKClientPulsarServiceBaseTest.java | 24 ++++++--- .../broker/service/MaxMessageSizeTest.java | 15 ++++-- .../service/OneWayReplicatorTestBase.java | 44 +++++++++++---- .../pulsar/broker/service/ReplicatorTest.java | 8 +-- .../broker/service/ReplicatorTestBase.java | 53 ++++++++++++++----- .../pulsar/broker/service/TopicOwnerTest.java | 15 ++++-- .../TransactionMetaStoreTestBase.java | 17 +++--- .../api/ClientDeduplicationFailureTest.java | 20 +++++-- .../worker/PulsarFunctionE2ESecurityTest.java | 25 +++++++-- .../worker/PulsarFunctionPublishTest.java | 25 +++++++-- .../worker/PulsarFunctionTlsTest.java | 8 ++- .../worker/PulsarWorkerAssignmentTest.java | 25 +++++++-- .../pulsar/io/AbstractPulsarE2ETest.java | 42 ++++++++------- .../pulsar/io/PulsarFunctionAdminTest.java | 25 +++++++-- .../pulsar/io/PulsarFunctionTlsTest.java | 20 +++++-- .../pulsar/zookeeper/ZookeeperServerTest.java | 13 +++-- .../apache/pulsar/metadata/TestZKServer.java | 1 + 29 files changed, 417 insertions(+), 130 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/MultiBrokerTestZKBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/MultiBrokerTestZKBaseTest.java index e5b80c0af33ab..a78254df4aae0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/MultiBrokerTestZKBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/MultiBrokerTestZKBaseTest.java @@ -59,6 +59,7 @@ protected void onCleanup() { } catch (Exception e) { log.error("Error in stopping ZK server", e); } + testZKServer = null; } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java index 4a6524bf24521..941229fc3d96c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/SLAMonitoringTest.java @@ -126,15 +126,26 @@ private void createTenant(PulsarAdmin pulsarAdmin) @AfterClass(alwaysRun = true) public void shutdown() throws Exception { log.info("--- Shutting down ---"); - executor.shutdownNow(); - executor = null; + if (executor != null) { + executor.shutdownNow(); + executor = null; + } for (int i = 0; i < BROKER_COUNT; i++) { - pulsarAdmins[i].close(); - pulsarServices[i].close(); + if (pulsarAdmins[i] != null) { + pulsarAdmins[i].close(); + pulsarAdmins[i] = null; + } + if (pulsarServices[i] != null) { + pulsarServices[i].close(); + pulsarServices[i] = null; + } } - bkEnsemble.stop(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java index ded4ee8e58d53..358410f1f28e3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java @@ -59,7 +59,10 @@ public void setup() throws Exception { @AfterMethod(alwaysRun = true) void shutdown() throws Exception { - bkEnsemble.stop(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } log.info("---- bk stopped ----"); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java index 7a2314b01a3d1..95aafd84ae406 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java @@ -147,11 +147,20 @@ void shutdown() throws Exception { log.info("--- Shutting down ---"); for (int i = 0; i < BROKER_COUNT; i++) { - pulsarAdmins[i].close(); - pulsarServices[i].close(); + if (pulsarAdmins[i] != null) { + pulsarAdmins[i].close(); + pulsarAdmins[i] = null; + } + if (pulsarServices[i] != null) { + pulsarServices[i].close(); + pulsarServices[i] = null; + } } - bkEnsemble.stop(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } private void loopUntilLeaderChangesForAllBroker(List activePulsars, LeaderBroker oldLeader) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java index f6154e3ec8e30..8f7aa17d0d7bf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java @@ -164,15 +164,33 @@ void setup() throws Exception { @AfterMethod(alwaysRun = true) void shutdown() throws Exception { log.info("--- Shutting down ---"); - executor.shutdownNow(); + if (executor != null) { + executor.shutdownNow(); + executor = null; + } - admin1.close(); - admin2.close(); + if (admin1 != null) { + admin1.close(); + admin1 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } - pulsar2.close(); - pulsar1.close(); + if (pulsar2 != null) { + pulsar2.close(); + pulsar2 = null; + } + if (pulsar1 != null) { + pulsar1.close(); + pulsar1 = null; + } - bkEnsemble.stop(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } private void createNamespacePolicies(PulsarService pulsar) throws Exception { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java index fdd1eb7272c30..42600a4203551 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java @@ -192,8 +192,14 @@ private BrokerRegistryImpl createBrokerRegistryImpl(PulsarService pulsar) { @AfterClass(alwaysRun = true) void shutdown() throws Exception { - executor.shutdownNow(); - bkEnsemble.stop(); + if (executor != null) { + executor.shutdownNow(); + executor = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } @AfterMethod(alwaysRun = true) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java index 4f2c1ae6607bb..32b7c5027281e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java @@ -107,8 +107,15 @@ protected void setup() throws Exception { @Override @AfterClass(alwaysRun = true) protected void cleanup() throws Exception { - this.additionalPulsarTestContext.close(); + if (additionalPulsarTestContext != null) { + additionalPulsarTestContext.close(); + additionalPulsarTestContext = null; + } super.internalCleanup(); + pulsar1 = pulsar2 = null; + primaryLoadManager = secondaryLoadManager = null; + channel1 = channel2 = null; + lookupService = null; } @BeforeMethod(alwaysRun = true) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTaskTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTaskTest.java index 3173987a3c8a8..bc49352f41d21 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTaskTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTaskTest.java @@ -150,8 +150,14 @@ public void testLoadBalancerNamespaceMaximumBundles() throws Exception { @AfterMethod(alwaysRun = true) void shutdown() throws Exception { log.info("--- Shutting down ---"); - pulsar.close(); - bkEnsemble.stop(); + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java index 824291c52da77..1f9cd806e19b5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java @@ -228,19 +228,36 @@ void setup() throws Exception { @AfterMethod(alwaysRun = true) void shutdown() throws Exception { log.info("--- Shutting down ---"); - executor.shutdownNow(); - - admin1.close(); - admin2.close(); + if (executor != null) { + executor.shutdownNow(); + executor = null; + } - pulsar2.close(); - pulsar1.close(); + if (admin1 != null) { + admin1.close(); + admin1 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } - if (pulsar3.isRunning()) { + if (pulsar2 != null) { + pulsar2.close(); + pulsar2 = null; + } + if (pulsar1 != null) { + pulsar1.close(); + pulsar1 = null; + } + if (pulsar3 != null && pulsar3.isRunning()) { pulsar3.close(); } - - bkEnsemble.stop(); + pulsar3 = null; + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } private NamespaceBundle makeBundle(final String property, final String cluster, final String namespace) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AdvertisedAddressTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AdvertisedAddressTest.java index 19e40ebf9960f..a60d6599e8f76 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AdvertisedAddressTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AdvertisedAddressTest.java @@ -66,8 +66,14 @@ public void setup() throws Exception { @AfterMethod(alwaysRun = true) public void shutdown() throws Exception { - pulsar.close(); - bkEnsemble.stop(); + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesTestBase.java index 3d9ba658f770e..71c5a995643c6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BkEnsemblesTestBase.java @@ -119,9 +119,18 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { config = null; markCurrentSetupNumberCleaned(); - admin.close(); - pulsar.close(); - bkEnsemble.stop(); + if (admin != null) { + admin.close(); + admin = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java index 19aa3ae0bd1c9..d7272fcffa964 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java @@ -105,8 +105,12 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { if (pulsarService != null) { pulsarService.close(); + pulsarService = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; } - bkEnsemble.stop(); } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java index bc6df685ffcd7..a1cb4abc4c30b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java @@ -43,7 +43,6 @@ @Slf4j public abstract class CanReconnectZKClientPulsarServiceBaseTest extends TestRetrySupport { - protected final String defaultTenant = "public"; protected final String defaultNamespace = defaultTenant + "/default"; protected int numberOfBookies = 3; @@ -60,6 +59,7 @@ public abstract class CanReconnectZKClientPulsarServiceBaseTest extends TestRetr protected ZooKeeper localZkOfBroker; protected Object localMetaDataStoreClientCnx; protected final AtomicBoolean LocalMetadataStoreInReconnectFinishSignal = new AtomicBoolean(); + protected void startZKAndBK() throws Exception { // Start ZK. brokerConfigZk = new ZookeeperServerTest(0); @@ -198,18 +198,30 @@ protected void cleanup() throws Exception { stopLocalMetadataStoreAlwaysReconnect(); // Stop brokers. - client.close(); - admin.close(); + if (client != null) { + client.close(); + client = null; + } + if (admin != null) { + admin.close(); + admin = null; + } if (pulsar != null) { pulsar.close(); + pulsar = null; } // Stop ZK and BK. - bkEnsemble.stop(); - brokerConfigZk.stop(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } + if (brokerConfigZk != null) { + brokerConfigZk.stop(); + brokerConfigZk = null; + } // Reset configs. config = new ServiceConfiguration(); - setConfigDefaults(config, clusterName, bkEnsemble, brokerConfigZk); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MaxMessageSizeTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MaxMessageSizeTest.java index 780d33de521b3..84543a82d7725 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MaxMessageSizeTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MaxMessageSizeTest.java @@ -91,9 +91,18 @@ void setup() { @AfterMethod(alwaysRun = true) void shutdown() { try { - pulsar.close(); - bkEnsemble.stop(); - admin.close(); + if (admin != null) { + admin.close(); + admin = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } catch (Throwable t) { t.printStackTrace(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index 95f976f965a0d..b4eed00c4470f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -240,27 +240,51 @@ protected void cleanup() throws Exception { log.info("--- Shutting down ---"); // Stop brokers. - client1.close(); - client2.close(); - admin1.close(); - admin2.close(); + if (client1 != null) { + client1.close(); + client1 = null; + } + if (client2 != null) { + client2.close(); + client2 = null; + } + if (admin1 != null) { + admin1.close(); + admin1 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } if (pulsar2 != null) { pulsar2.close(); + pulsar2 = null; } if (pulsar1 != null) { pulsar1.close(); + pulsar1 = null; } // Stop ZK and BK. - bkEnsemble1.stop(); - bkEnsemble2.stop(); - brokerConfigZk1.stop(); - brokerConfigZk2.stop(); + if (bkEnsemble1 != null) { + bkEnsemble1.stop(); + bkEnsemble1 = null; + } + if (bkEnsemble2 != null) { + bkEnsemble2.stop(); + bkEnsemble2 = null; + } + if (brokerConfigZk1 != null) { + brokerConfigZk1.stop(); + brokerConfigZk1 = null; + } + if (brokerConfigZk2 != null) { + brokerConfigZk2.stop(); + brokerConfigZk2 = null; + } // Reset configs. config1 = new ServiceConfiguration(); - setConfigDefaults(config1, cluster1, bkEnsemble1, brokerConfigZk1); config2 = new ServiceConfiguration(); - setConfigDefaults(config2, cluster2, bkEnsemble2, brokerConfigZk2); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index 0bfcdf563d632..fa12eba1c6611 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -130,9 +130,11 @@ public class ReplicatorTest extends ReplicatorTestBase { @BeforeMethod(alwaysRun = true) public void beforeMethod(Method m) throws Exception { methodName = m.getName(); - admin1.namespaces().removeBacklogQuota("pulsar/ns"); - admin1.namespaces().removeBacklogQuota("pulsar/ns1"); - admin1.namespaces().removeBacklogQuota("pulsar/global/ns"); + if (admin1 != null) { + admin1.namespaces().removeBacklogQuota("pulsar/ns"); + admin1.namespaces().removeBacklogQuota("pulsar/ns1"); + admin1.namespaces().removeBacklogQuota("pulsar/global/ns"); + } } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java index ba9f850ff0cc1..d87f896e31a1c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java @@ -355,22 +355,18 @@ private void setConfigDefaults(ServiceConfiguration config, String clusterName, public void resetConfig1() { config1 = new ServiceConfiguration(); - setConfig1DefaultValue(); } public void resetConfig2() { config2 = new ServiceConfiguration(); - setConfig2DefaultValue(); } public void resetConfig3() { config3 = new ServiceConfiguration(); - setConfig3DefaultValue(); } public void resetConfig4() { config4 = new ServiceConfiguration(); - setConfig4DefaultValue(); } private int inSec(int time, TimeUnit unit) { @@ -386,29 +382,60 @@ protected void cleanup() throws Exception { executor = null; } - admin1.close(); - admin2.close(); - admin3.close(); - admin4.close(); + if (admin1 != null) { + admin1.close(); + admin1 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } + if (admin3 != null) { + admin3.close(); + admin3 = null; + } + if (admin4 != null) { + admin4.close(); + admin4 = null; + } if (pulsar4 != null) { pulsar4.close(); + pulsar4 = null; } if (pulsar3 != null) { pulsar3.close(); + pulsar3 = null; } if (pulsar2 != null) { pulsar2.close(); + pulsar2 = null; } if (pulsar1 != null) { pulsar1.close(); + pulsar1 = null; } - bkEnsemble1.stop(); - bkEnsemble2.stop(); - bkEnsemble3.stop(); - bkEnsemble4.stop(); - globalZkS.stop(); + if (bkEnsemble1 != null) { + bkEnsemble1.stop(); + bkEnsemble1 = null; + } + if (bkEnsemble2 != null) { + bkEnsemble2.stop(); + bkEnsemble2 = null; + } + if (bkEnsemble3 != null) { + bkEnsemble3.stop(); + bkEnsemble3 = null; + } + if (bkEnsemble4 != null) { + bkEnsemble4.stop(); + bkEnsemble4 = null; + } + if (globalZkS != null) { + globalZkS.stop(); + globalZkS = null; + } resetConfig1(); resetConfig2(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicOwnerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicOwnerTest.java index 5a8fd34c9cdba..521d68cebe599 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicOwnerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicOwnerTest.java @@ -117,10 +117,19 @@ void setup() throws Exception { @AfterMethod(alwaysRun = true) void tearDown() throws Exception { for (int i = 0; i < BROKER_COUNT; i++) { - pulsarServices[i].close(); - pulsarAdmins[i].close(); + if (pulsarAdmins[i] != null) { + pulsarAdmins[i].close(); + pulsarAdmins[i] = null; + } + if (pulsarServices[i] != null) { + pulsarServices[i].close(); + pulsarServices[i] = null; + } + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; } - bkEnsemble.stop(); } @SuppressWarnings("unchecked") diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/coordinator/TransactionMetaStoreTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/coordinator/TransactionMetaStoreTestBase.java index 7a0fb48f91150..5bf48932f3687 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/coordinator/TransactionMetaStoreTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/coordinator/TransactionMetaStoreTestBase.java @@ -122,22 +122,27 @@ public final void shutdownAll() throws Exception { protected void cleanup() throws Exception { if (transactionCoordinatorClient != null) { transactionCoordinatorClient.close(); + transactionCoordinatorClient = null; } - for (PulsarAdmin admin : pulsarAdmins) { - if (admin != null) { - admin.close(); + for (int i = 0; i < BROKER_COUNT; i++) { + if (pulsarAdmins[i] != null) { + pulsarAdmins[i].close(); + pulsarAdmins[i] = null; } } if (pulsarClient != null) { pulsarClient.close(); + pulsarClient = null; } - for (PulsarService service : pulsarServices) { - if (service != null) { - service.close(); + for (int i = 0; i < BROKER_COUNT; i++) { + if (pulsarServices[i] != null) { + pulsarServices[i].close(); + pulsarServices[i] = null; } } if (bkEnsemble != null) { bkEnsemble.stop(); + bkEnsemble = null; } Mockito.reset(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationFailureTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationFailureTest.java index 6b3b05405baea..601a8d76aaacd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationFailureTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientDeduplicationFailureTest.java @@ -127,10 +127,22 @@ void setup(Method method) throws Exception { @AfterMethod(alwaysRun = true) void shutdown() throws Exception { log.info("--- Shutting down ---"); - pulsarClient.close(); - admin.close(); - pulsar.close(); - bkEnsemble.stop(); + if (pulsarClient != null) { + pulsarClient.close(); + pulsar = null; + } + if (admin != null) { + admin.close(); + admin = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } private static class ProducerThread implements Runnable { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java index cbf2f28b0b50b..e9b3531c7c2e2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java @@ -216,11 +216,26 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { void shutdown() throws Exception { try { log.info("--- Shutting down ---"); - pulsarClient.close(); - superUserAdmin.close(); - functionsWorkerService.stop(); - pulsar.close(); - bkEnsemble.stop(); + if (pulsarClient != null) { + pulsarClient.close(); + pulsarClient = null; + } + if (superUserAdmin != null) { + superUserAdmin.close(); + superUserAdmin = null; + } + if (functionsWorkerService != null) { + functionsWorkerService.stop(); + functionsWorkerService = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } finally { if (tempDirectory != null) { tempDirectory.delete(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java index 569c2d36ff3a7..50dc39a3a79d2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java @@ -219,11 +219,26 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { void shutdown() throws Exception { try { log.info("--- Shutting down ---"); - pulsarClient.close(); - admin.close(); - functionsWorkerService.stop(); - pulsar.close(); - bkEnsemble.stop(); + if (pulsarClient != null) { + pulsarClient.close(); + pulsarClient = null; + } + if (admin != null) { + admin.close(); + admin = null; + } + if (functionsWorkerService != null) { + functionsWorkerService.stop(); + functionsWorkerService = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } finally { if (tempDirectory != null) { tempDirectory.delete(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java index 3508cf0bfc7e6..3be16357d332b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java @@ -202,11 +202,13 @@ void tearDown() throws Exception { for (int i = 0; i < BROKER_COUNT; i++) { if (pulsarAdmins[i] != null) { pulsarAdmins[i].close(); + pulsarAdmins[i] = null; } } for (int i = 0; i < BROKER_COUNT; i++) { if (fnWorkerServices[i] != null) { fnWorkerServices[i].stop(); + fnWorkerServices[i] = null; } } for (int i = 0; i < BROKER_COUNT; i++) { @@ -221,9 +223,13 @@ void tearDown() throws Exception { getBrokerServicePort().ifPresent(PortManager::releaseLockedPort); pulsarServices[i].getConfiguration() .getWebServicePort().ifPresent(PortManager::releaseLockedPort); + pulsarServices[i] = null; } } - bkEnsemble.stop(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } finally { for (int i = 0; i < BROKER_COUNT; i++) { if (tempDirectories[i] != null) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java index 6226fa904885c..9c137e37095ed 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java @@ -131,11 +131,26 @@ void setup(Method method) throws Exception { void shutdown() { log.info("--- Shutting down ---"); try { - pulsarClient.close(); - admin.close(); - functionsWorkerService.stop(); - pulsar.close(); - bkEnsemble.stop(); + if (pulsarClient != null) { + pulsarClient.close(); + pulsarClient = null; + } + if (admin != null) { + admin.close(); + admin = null; + } + if (functionsWorkerService != null) { + functionsWorkerService.stop(); + functionsWorkerService = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } catch (Exception e) { log.warn("Encountered errors at shutting down PulsarWorkerAssignmentTest", e); } finally { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java index 3c0dd0822b7dc..d27e27639048e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java @@ -239,29 +239,35 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { void shutdown() throws Exception { log.info("--- Shutting down ---"); try { - if (fileServer != null) { - fileServer.stop(); - } + if (fileServer != null) { + fileServer.stop(); + fileServer = null; + } - if (pulsarClient != null) { - pulsarClient.close(); - } + if (pulsarClient != null) { + pulsarClient.close(); + pulsarClient = null; + } - if (admin != null) { - admin.close(); - } + if (admin != null) { + admin.close(); + admin = null; + } - if (functionsWorkerService != null) { - functionsWorkerService.stop(); - } + if (functionsWorkerService != null) { + functionsWorkerService.stop(); + functionsWorkerService = null; + } - if (pulsar != null) { - pulsar.close(); - } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } - if (bkEnsemble != null) { - bkEnsemble.stop(); - } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } finally { if (tempDirectory != null) { tempDirectory.delete(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java index 22b9ad0df3a69..aafd82d339a1d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java @@ -172,11 +172,26 @@ && isNotBlank(workerConfig.getBrokerClientAuthenticationParameters())) { @AfterMethod(alwaysRun = true) void shutdown() throws Exception { log.info("--- Shutting down ---"); - pulsarClient.close(); - admin.close(); - functionsWorkerService.stop(); - pulsar.close(); - bkEnsemble.stop(); + if (pulsarClient != null) { + pulsarClient.close(); + pulsarClient = null; + } + if (admin != null) { + admin.close(); + admin = null; + } + if (functionsWorkerService != null) { + functionsWorkerService.stop(); + functionsWorkerService = null; + } + if (pulsar != null) { + pulsar.close(); + pulsar = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration config) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java index 810ac69ac3eb3..da479321b8bc2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java @@ -180,10 +180,22 @@ void setup(Method method) throws Exception { void shutdown() throws Exception { log.info("--- Shutting down ---"); try { - functionAdmin.close(); - functionsWorkerService.stop(); - workerServer.stop(); - bkEnsemble.stop(); + if (functionAdmin != null) { + functionAdmin.close(); + functionAdmin = null; + } + if (functionsWorkerService != null) { + functionsWorkerService.stop(); + functionsWorkerService = null; + } + if (workerServer != null) { + workerServer.stop(); + workerServer = null; + } + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } } finally { if (tempDirectory != null) { tempDirectory.delete(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/ZookeeperServerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/ZookeeperServerTest.java index eec61b9144e47..355d2a0b1dbe1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/ZookeeperServerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/ZookeeperServerTest.java @@ -68,15 +68,20 @@ public void start() throws IOException { } public void stop() throws IOException { - zks.shutdown(); - serverFactory.shutdown(); + if (zks != null) { + zks.shutdown(); + zks = null; + } + if (serverFactory != null) { + serverFactory.shutdown(); + serverFactory = null; + } log.info("Stopped ZK server at {}", hostPort); } @Override public void close() throws IOException { - zks.shutdown(); - serverFactory.shutdown(); + stop(); zkTmpDir.delete(); } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/TestZKServer.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/TestZKServer.java index 33034ddb3fe0f..0d01d9c56abc8 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/TestZKServer.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/TestZKServer.java @@ -131,6 +131,7 @@ public void checkContainers() throws Exception { public void stop() throws Exception { if (zooKeeperServerEmbedded != null) { zooKeeperServerEmbedded.close(); + zooKeeperServerEmbedded = null; } log.info("Stopped test ZK server"); } From f25776d7fe6812f11b17226995d989c5a2364920 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Fri, 26 Apr 2024 09:18:27 +0800 Subject: [PATCH 114/580] [fix][admin] Fix namespace admin api exception response (#22587) --- .../broker/admin/impl/NamespacesBase.java | 5 +- .../broker/admin/NamespaceAuthZTest.java | 60 ++++++++++++++----- 2 files changed, 48 insertions(+), 17 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 bbadc7bb3316d..5f2dccc3e9c24 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 @@ -2019,7 +2019,7 @@ protected void internalSetMaxUnackedMessagesPerConsumer(Integer maxUnackedMessag } protected void internalSetMaxSubscriptionsPerTopic(Integer maxSubscriptionsPerTopic){ - validateNamespacePolicyOperationAsync(namespaceName, PolicyName.MAX_SUBSCRIPTIONS, PolicyOperation.WRITE); + validateNamespacePolicyOperation(namespaceName, PolicyName.MAX_SUBSCRIPTIONS, PolicyOperation.WRITE); validatePoliciesReadOnlyAccess(); if (maxSubscriptionsPerTopic != null && maxSubscriptionsPerTopic < 0) { throw new RestException(Status.PRECONDITION_FAILED, @@ -2125,9 +2125,10 @@ protected CompletableFuture internalSetOffloadThresholdInSecondsAsync(long f.complete(null); }) .exceptionally(t -> { + Throwable cause = FutureUtil.unwrapCompletionException(t); log.error("[{}] Failed to update offloadThresholdInSeconds configuration for namespace {}", clientAppId(), namespaceName, t); - f.completeExceptionally(new RestException(t)); + f.completeExceptionally(new RestException(cause)); return null; }); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java index d5a0468f340c9..5358295b78568 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.admin; +import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.deleteNamespaceWithRetry; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -58,7 +59,6 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @Test(groups = "broker-admin") @@ -72,8 +72,6 @@ public class NamespaceAuthZTest extends MockedPulsarStandalone { private AuthorizationService authorizationService; - private AuthorizationService orignalAuthorizationService; - private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); private static final String TENANT_ADMIN_TOKEN = Jwts.builder() .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); @@ -100,6 +98,9 @@ public void setup() { .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) .build(); this.pulsarClient = super.getPulsarService().getClient(); + this.authorizationService = Mockito.spy(getPulsarService().getBrokerService().getAuthorizationService()); + FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", + authorizationService, true); } @@ -115,19 +116,9 @@ public void cleanup() { close(); } - @BeforeMethod - public void before() throws IllegalAccessException { - orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); - authorizationService = Mockito.spy(orignalAuthorizationService); - FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", - authorizationService, true); - } - @AfterMethod - public void after() throws IllegalAccessException, PulsarAdminException { - FieldUtils.writeField(getPulsarService().getBrokerService(), "authorizationService", - orignalAuthorizationService, true); - superUserAdmin.namespaces().deleteNamespace("public/default", true); + public void after() throws Exception { + deleteNamespaceWithRetry("public/default", true, superUserAdmin); superUserAdmin.namespaces().createNamespace("public/default"); } @@ -1028,4 +1019,43 @@ public void testPackageAPI() throws Exception { superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); } } + + @Test + @SneakyThrows + public void testOffloadThresholdInSeconds() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getOffloadThresholdInSeconds(namespace)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setOffloadThresholdInSeconds(namespace, 10000)); + } + + @Test + @SneakyThrows + public void testMaxSubscriptionsPerTopic() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getMaxSubscriptionsPerTopic(namespace)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setMaxSubscriptionsPerTopic(namespace, 100)); + + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeMaxSubscriptionsPerTopic(namespace)); + } } From 69a600e86bb5110a118d836125411e941b83764d Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Fri, 26 Apr 2024 14:05:30 +0800 Subject: [PATCH 115/580] [improve][admin] Check if the topic existed before the permission operations (#22547) --- .../broker/admin/impl/PersistentTopicsBase.java | 9 ++++++--- .../broker/admin/AdminApiSchemaWithAuthTest.java | 1 + .../apache/pulsar/broker/admin/AdminApiTest.java | 12 ++++++++++++ .../pulsar/broker/admin/PersistentTopicsTest.java | 10 ++++++++-- .../pulsar/broker/auth/AuthorizationTest.java | 13 ++++++++----- .../api/AuthenticatedProducerConsumerTest.java | 4 +++- .../api/AuthorizationProducerConsumerTest.java | 2 ++ .../websocket/proxy/ProxyAuthorizationTest.java | 8 +++++--- 8 files changed, 45 insertions(+), 14 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 63ea987bb07fe..682f41dcdb61f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -205,6 +205,7 @@ protected CompletableFuture> internalGetPartitionedTopicListAsync() protected CompletableFuture>> internalGetPermissionsOnTopic() { // This operation should be reading from zookeeper and it should be allowed without having admin privileges return validateAdminAccessForTenantAsync(namespaceName.getTenant()) + .thenCompose(__ -> internalCheckTopicExists(topicName)) .thenCompose(__ -> getAuthorizationService().getPermissionsAsync(topicName)); } @@ -256,9 +257,10 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse Set actions) { // This operation should be reading from zookeeper and it should be allowed without having admin privileges validateAdminAccessForTenantAsync(namespaceName.getTenant()) - .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync().thenCompose(unused1 -> - grantPermissionsAsync(topicName, role, actions) - .thenAccept(unused -> asyncResponse.resume(Response.noContent().build())))) + .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + .thenCompose(__ -> internalCheckTopicExists(topicName)) + .thenCompose(unused1 -> grantPermissionsAsync(topicName, role, actions)) + .thenAccept(unused -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { Throwable realCause = FutureUtil.unwrapCompletionException(ex); log.error("[{}] Failed to get permissions for topic {}", clientAppId(), topicName, realCause); @@ -270,6 +272,7 @@ protected void internalGrantPermissionsOnTopic(final AsyncResponse asyncResponse protected void internalRevokePermissionsOnTopic(AsyncResponse asyncResponse, String role) { // This operation should be reading from zookeeper and it should be allowed without having admin privileges validateAdminAccessForTenantAsync(namespaceName.getTenant()) + .thenCompose(__ -> internalCheckTopicExists(topicName)) .thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, true, false) .thenCompose(metadata -> { int numPartitions = metadata.partitions; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaWithAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaWithAuthTest.java index e89b4ff5e83ec..2dcb930fbe719 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaWithAuthTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaWithAuthTest.java @@ -120,6 +120,7 @@ public void testGetCreateDeleteSchema() throws Exception { .serviceHttpUrl(brokerUrl != null ? brokerUrl.toString() : brokerUrlTls.toString()) .authentication(AuthenticationToken.class.getName(), PRODUCE_TOKEN) .build(); + admin.topics().createNonPartitionedTopic(topicName); admin.topics().grantPermission(topicName, "consumer", EnumSet.of(AuthAction.consume)); admin.topics().grantPermission(topicName, "producer", EnumSet.of(AuthAction.produce)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index b28cfc98fdb07..635b2c25bc1d0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -3698,4 +3698,16 @@ public void testRetentionAndBacklogQuotaCheck() throws PulsarAdminException { }); } + + @Test + @SneakyThrows + public void testPermissions() { + String namespace = "prop-xyz/ns1/"; + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://" + namespace + random; + final String subject = UUID.randomUUID().toString(); + assertThrows(NotFoundException.class, () -> admin.topics().getPermissions(topic)); + assertThrows(NotFoundException.class, () -> admin.topics().grantPermission(topic, subject, Set.of(AuthAction.produce))); + assertThrows(NotFoundException.class, () -> admin.topics().revokePermissions(topic, subject)); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index c588051a0feff..55b4c6e1c6f59 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -890,12 +890,15 @@ public void testGetList() throws Exception { public void testGrantNonPartitionedTopic() { final String topicName = "non-partitioned-topic"; AsyncResponse response = mock(AsyncResponse.class); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); persistentTopics.createNonPartitionedTopic(response, testTenant, testNamespace, topicName, true, null); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); String role = "role"; Set expectActions = new HashSet<>(); expectActions.add(AuthAction.produce); response = mock(AsyncResponse.class); - ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); persistentTopics.grantPermissionsOnTopic(response, testTenant, testNamespace, topicName, role, expectActions); verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); @@ -957,12 +960,15 @@ public void testGrantPartitionedTopic() { public void testRevokeNonPartitionedTopic() { final String topicName = "non-partitioned-topic"; AsyncResponse response = mock(AsyncResponse.class); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); persistentTopics.createNonPartitionedTopic(response, testTenant, testNamespace, topicName, true, null); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); String role = "role"; Set expectActions = new HashSet<>(); expectActions.add(AuthAction.produce); response = mock(AsyncResponse.class); - ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); persistentTopics.grantPermissionsOnTopic(response, testTenant, testNamespace, topicName, role, expectActions); verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java index f59f9d480b8c2..6c913d4290897 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java @@ -58,6 +58,7 @@ public AuthorizationTest() { public void setup() throws Exception { conf.setClusterName("c1"); conf.setSystemTopicEnabled(false); + conf.setForceDeleteNamespaceAllowed(true); conf.setAuthenticationEnabled(true); conf.setForceDeleteNamespaceAllowed(true); conf.setForceDeleteTenantAllowed(true); @@ -107,8 +108,9 @@ public void simple() throws Exception { assertTrue(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null)); assertTrue(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null)); - admin.topics().grantPermission("persistent://p1/c1/ns1/ds2", "other-role", - EnumSet.of(AuthAction.consume)); + String topic = "persistent://p1/c1/ns1/ds2"; + admin.topics().createNonPartitionedTopic(topic); + admin.topics().grantPermission(topic, "other-role", EnumSet.of(AuthAction.consume)); waitForChange(); assertTrue(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null)); @@ -178,8 +180,9 @@ public void simple() throws Exception { assertFalse(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "my.role.1", null)); assertFalse(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "my.role.2", null)); - admin.topics().grantPermission("persistent://p1/c1/ns1/ds1", "my.*", - EnumSet.of(AuthAction.produce)); + String topic1 = "persistent://p1/c1/ns1/ds1"; + admin.topics().createNonPartitionedTopic(topic1); + admin.topics().grantPermission(topic1, "my.*", EnumSet.of(AuthAction.produce)); waitForChange(); assertTrue(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my.role.1", null)); @@ -242,7 +245,7 @@ public void simple() throws Exception { assertTrue(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "role2", null, "role2-sub2")); assertTrue(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "pulsar.super_user", null, "role3-sub1")); - admin.namespaces().deleteNamespace("p1/c1/ns1"); + admin.namespaces().deleteNamespace("p1/c1/ns1", true); admin.tenants().deleteTenant("p1"); admin.clusters().deleteCluster("c1"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java index f9aa17ea3c451..c46f4744cd5df 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java @@ -263,7 +263,9 @@ public void testAnonymousSyncProducerAndConsumer(int batchMessageDelayMs) throws closeAdmin(); admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrl.toString()).build()); admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); - admin.topics().grantPermission("persistent://my-property/my-ns/my-topic", "anonymousUser", + String topic = "persistent://my-property/my-ns/my-topic"; + admin.topics().createNonPartitionedTopic(topic); + admin.topics().grantPermission(topic, "anonymousUser", EnumSet.allOf(AuthAction.class)); // setup the client diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java index 769486054ab04..3ead51ad7fc92 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java @@ -234,6 +234,7 @@ public void testSubscriberPermission() throws Exception { } // grant topic consume authorization to the subscriptionRole + tenantAdmin.topics().createNonPartitionedTopic(topicName); tenantAdmin.topics().grantPermission(topicName, subscriptionRole, Collections.singleton(AuthAction.consume)); @@ -773,6 +774,7 @@ public void testPermissionForProducerCreateInitialSubscription() throws Exceptio admin.tenants().createTenant("my-property", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); admin.namespaces().createNamespace("my-property/my-ns", Sets.newHashSet("test")); + admin.topics().createNonPartitionedTopic(topic); admin.topics().grantPermission(topic, invalidRole, Collections.singleton(AuthAction.produce)); admin.topics().grantPermission(topic, producerRole, Sets.newHashSet(AuthAction.produce, AuthAction.consume)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java index d4f7c72bed016..2d00e15a13f19 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/websocket/proxy/ProxyAuthorizationTest.java @@ -55,6 +55,7 @@ public ProxyAuthorizationTest() { @Override protected void setup() throws Exception { conf.setClusterName(configClusterName); + conf.setForceDeleteNamespaceAllowed(true); internalSetup(); WebSocketProxyConfiguration config = new WebSocketProxyConfiguration(); @@ -99,8 +100,9 @@ public void test() throws Exception { assertTrue(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null)); assertTrue(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null)); - admin.topics().grantPermission("persistent://p1/c1/ns1/ds2", "other-role", - EnumSet.of(AuthAction.consume)); + String topic = "persistent://p1/c1/ns1/ds2"; + admin.topics().createNonPartitionedTopic(topic); + admin.topics().grantPermission(topic, "other-role", EnumSet.of(AuthAction.consume)); waitForChange(); assertTrue(auth.canLookup(TopicName.get("persistent://p1/c1/ns1/ds2"), "other-role", null)); @@ -117,7 +119,7 @@ public void test() throws Exception { assertTrue(auth.canProduce(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null)); assertTrue(auth.canConsume(TopicName.get("persistent://p1/c1/ns1/ds1"), "my-role", null, null)); - admin.namespaces().deleteNamespace("p1/c1/ns1"); + admin.namespaces().deleteNamespace("p1/c1/ns1", true); admin.tenants().deleteTenant("p1"); admin.clusters().deleteCluster("c1"); } From d19860c706e47b3f2525678da5edfad3f6adafd1 Mon Sep 17 00:00:00 2001 From: thetumbled <52550727+thetumbled@users.noreply.github.com> Date: Fri, 26 Apr 2024 19:21:45 +0800 Subject: [PATCH 116/580] [cleanup] [test] remove useless TestAuthorizationProvider2 (#22595) --- .../AuthorizationProducerConsumerTest.java | 28 ++----------------- 1 file changed, 2 insertions(+), 26 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java index 3ead51ad7fc92..2638709abc5e2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java @@ -350,7 +350,8 @@ public void testSubscriberPermission() throws Exception { } catch (Exception e) { // my-sub1 has no msg backlog, so expire message won't be issued on that subscription assertTrue(e.getMessage().startsWith("Expire message by timestamp not issued on topic")); - } sub1Admin.topics().peekMessages(topicName, subscriptionName, 1); + } + sub1Admin.topics().peekMessages(topicName, subscriptionName, 1); sub1Admin.topics().resetCursor(topicName, subscriptionName, 10); sub1Admin.topics().resetCursor(topicName, subscriptionName, MessageId.earliest); @@ -992,31 +993,6 @@ public CompletableFuture allowTopicOperationAsync( } } - /** - * This provider always fails authorization on consumer and passes on producer - * - */ - public static class TestAuthorizationProvider2 extends TestAuthorizationProvider { - - @Override - public CompletableFuture canProduceAsync(TopicName topicName, String role, - AuthenticationDataSource authenticationData) { - return CompletableFuture.completedFuture(true); - } - - @Override - public CompletableFuture canConsumeAsync(TopicName topicName, String role, - AuthenticationDataSource authenticationData, String subscription) { - return CompletableFuture.completedFuture(false); - } - - @Override - public CompletableFuture canLookupAsync(TopicName topicName, String role, - AuthenticationDataSource authenticationData) { - return CompletableFuture.completedFuture(true); - } - } - public static class TestAuthorizationProviderWithSubscriptionPrefix extends TestAuthorizationProvider { @Override public CompletableFuture allowTopicOperationAsync(TopicName topic, From 69839c72f1375d141b56734bc5e041c13e366c57 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 26 Apr 2024 15:16:47 +0300 Subject: [PATCH 117/580] [improve][meta] Log a warning when ZK batch fails with connectionloss (#22566) --- .../pulsar/metadata/impl/ZKMetadataStore.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index 079ae3e2ae5c3..2e88cb3332467 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -192,7 +192,20 @@ protected void batchOperation(List ops) { Code code = Code.get(rc); if (code == Code.CONNECTIONLOSS) { // There is the chance that we caused a connection reset by sending or requesting a batch - // that passed the max ZK limit. Retry with the individual operations + // that passed the max ZK limit. + + // Build the log warning message + // summarize the operations by type + String countsByType = ops.stream().collect( + Collectors.groupingBy(MetadataOp::getType, Collectors.summingInt(op -> 1))) + .entrySet().stream().map(e -> e.getValue() + " " + e.getKey().name() + " entries") + .collect(Collectors.joining(", ")); + Long totalSize = ops.stream().collect(Collectors.summingLong(MetadataOp::size)); + log.warn("Connection loss while executing batch operation of {} " + + "of total data size of {}. " + + "Retrying individual operations one-by-one.", countsByType, totalSize); + + // Retry with the individual operations executor.schedule(() -> { ops.forEach(o -> batchOperation(Collections.singletonList(o))); }, 100, TimeUnit.MILLISECONDS); From 3b9602c04db5a6577e2dc2fabddbf7a6e1d1a4a2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 26 Apr 2024 16:23:36 +0300 Subject: [PATCH 118/580] [improve][broker] Propagate cause exception in TopicBusyException when applicable (#22596) --- .../apache/pulsar/broker/service/BrokerServiceException.java | 4 ++++ .../broker/service/nonpersistent/NonPersistentTopic.java | 3 ++- .../pulsar/broker/service/persistent/PersistentTopic.java | 3 ++- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java index 831d6068e2097..6abe40f811d1d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java @@ -146,6 +146,10 @@ public static class TopicBusyException extends BrokerServiceException { public TopicBusyException(String msg) { super(msg); } + + public TopicBusyException(String msg, Throwable t) { + super(msg, t); + } } public static class TopicNotFoundException extends BrokerServiceException { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 586fcd76151e4..8cb8394440f33 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -477,7 +477,8 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, boolean c } }).exceptionally(ex -> { deleteFuture.completeExceptionally( - new TopicBusyException("Failed to close clients before deleting topic.")); + new TopicBusyException("Failed to close clients before deleting topic.", + FutureUtil.unwrapCompletionException(ex))); return null; }); } finally { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index c1a75d67e3c4e..c7d762d595c33 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -1506,7 +1506,8 @@ public void deleteLedgerComplete(Object ctx) { }).exceptionally(ex->{ unfenceTopicToResume(); deleteFuture.completeExceptionally( - new TopicBusyException("Failed to close clients before deleting topic.")); + new TopicBusyException("Failed to close clients before deleting topic.", + FutureUtil.unwrapCompletionException(ex))); return null; }); From f411e3c0f26eef98382c7d06ea1676781247149b Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 26 Apr 2024 21:30:15 +0800 Subject: [PATCH 119/580] [fix][broker] Avoid being stuck when closing the broker with extensible load manager (#22573) --- .../apache/pulsar/broker/PulsarService.java | 3 + .../store/TableViewLoadDataStoreImpl.java | 6 +- .../pulsar/broker/service/BrokerService.java | 11 ++ .../ExtensibleLoadManagerCloseTest.java | 107 ++++++++++++++++++ 4 files changed, 122 insertions(+), 5 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 7613a13db22de..c21c7dc771eae 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -444,6 +444,9 @@ public CompletableFuture closeAsync() { return closeFuture; } LOG.info("Closing PulsarService"); + if (brokerService != null) { + brokerService.unloadNamespaceBundlesGracefully(); + } state = State.Closing; // close the service in reverse order v.s. in which they are started diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java index d916e91716223..81cf33b4a55d2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java @@ -161,12 +161,8 @@ public synchronized void init() throws IOException { } private void validateProducer() { - if (producer == null || !producer.isConnected()) { + if (producer == null) { try { - if (producer != null) { - producer.close(); - } - producer = null; startProducer(); log.info("Restarted producer on {}", topic); } catch (Exception e) { 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 295a9a2954126..1f0cb12258e1d 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 @@ -309,6 +309,7 @@ public class BrokerService implements Closeable { private Set brokerEntryPayloadProcessors; private final TopicEventsDispatcher topicEventsDispatcher = new TopicEventsDispatcher(); + private volatile boolean unloaded = false; public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws Exception { this.pulsar = pulsar; @@ -926,9 +927,13 @@ public void unloadNamespaceBundlesGracefully() { } public void unloadNamespaceBundlesGracefully(int maxConcurrentUnload, boolean closeWithoutWaitingClientDisconnect) { + if (unloaded) { + return; + } try { log.info("Unloading namespace-bundles..."); // make broker-node unavailable from the cluster + long disableBrokerStartTime = System.nanoTime(); if (pulsar.getLoadManager() != null && pulsar.getLoadManager().get() != null) { try { pulsar.getLoadManager().get().disableBroker(); @@ -937,6 +942,10 @@ public void unloadNamespaceBundlesGracefully(int maxConcurrentUnload, boolean cl // still continue and release bundle ownership as broker's registration node doesn't exist. } } + double disableBrokerTimeSeconds = + TimeUnit.NANOSECONDS.toMillis((System.nanoTime() - disableBrokerStartTime)) + / 1000.0; + log.info("Disable broker in load manager completed in {} seconds", disableBrokerTimeSeconds); // unload all namespace-bundles gracefully long closeTopicsStartTime = System.nanoTime(); @@ -966,6 +975,8 @@ public void unloadNamespaceBundlesGracefully(int maxConcurrentUnload, boolean cl } } catch (Exception e) { log.error("Failed to disable broker from loadbalancer list {}", e.getMessage(), e); + } finally { + unloaded = true; } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java new file mode 100644 index 0000000000000..41413f3e3a913 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java @@ -0,0 +1,107 @@ +/* + * 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.loadbalance.extensions; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +public class ExtensibleLoadManagerCloseTest { + + private static final String clusterName = "test"; + private final LocalBookkeeperEnsemble bk = new LocalBookkeeperEnsemble(1, 0, () -> 0); + private final List brokers = new ArrayList<>(); + private PulsarAdmin admin; + + @BeforeClass(alwaysRun = true) + public void setup() throws Exception { + bk.start(); + for (int i = 0; i < 3; i++) { + final var broker = new PulsarService(brokerConfig()); + broker.start(); + brokers.add(broker); + } + admin = brokers.get(0).getAdminClient(); + admin.clusters().createCluster(clusterName, ClusterData.builder().build()); + admin.tenants().createTenant("public", TenantInfo.builder() + .allowedClusters(Collections.singleton(clusterName)).build()); + admin.namespaces().createNamespace("public/default"); + } + + + @AfterClass(alwaysRun = true, timeOut = 30000) + public void cleanup() throws Exception { + bk.stop(); + } + + private ServiceConfiguration brokerConfig() { + final var config = new ServiceConfiguration(); + config.setClusterName(clusterName); + config.setAdvertisedAddress("localhost"); + config.setBrokerServicePort(Optional.of(0)); + config.setWebServicePort(Optional.of(0)); + config.setMetadataStoreUrl("zk:127.0.0.1:" + bk.getZookeeperPort()); + config.setManagedLedgerDefaultWriteQuorum(1); + config.setManagedLedgerDefaultAckQuorum(1); + config.setManagedLedgerDefaultEnsembleSize(1); + config.setDefaultNumberOfNamespaceBundles(16); + config.setLoadBalancerAutoBundleSplitEnabled(false); + config.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + config.setLoadBalancerDebugModeEnabled(true); + config.setBrokerShutdownTimeoutMs(100); + return config; + } + + + @Test + public void testCloseAfterLoadingBundles() throws Exception { + final var topic = "test"; + admin.topics().createPartitionedTopic(topic, 20); + admin.lookups().lookupPartitionedTopic(topic); + final var client = PulsarClient.builder().serviceUrl(brokers.get(0).getBrokerServiceUrl()).build(); + final var producer = client.newProducer().topic(topic).create(); + producer.close(); + client.close(); + + final var closeTimeMsList = new ArrayList(); + for (var broker : brokers) { + final var startTimeMs = System.currentTimeMillis(); + broker.close(); + closeTimeMsList.add(System.currentTimeMillis() - startTimeMs); + } + log.info("Brokers close time: {}", closeTimeMsList); + for (var closeTimeMs : closeTimeMsList) { + Assert.assertTrue(closeTimeMs < 5000L); + } + } +} From f8f256cfbdcd780c81442dc5566b6ed071141645 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 26 Apr 2024 23:16:54 +0300 Subject: [PATCH 120/580] [fix][broker] Continue closing even when executor is shut down (#22599) --- .../broker/service/persistent/PersistentTopic.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index c7d762d595c33..155b67778820b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -45,6 +45,7 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -1429,7 +1430,14 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, } FutureUtil.waitForAll(futures).thenRunAsync(() -> { closeClientFuture.complete(null); - }, getOrderedExecutor()).exceptionally(ex -> { + }, command -> { + try { + getOrderedExecutor().execute(command); + } catch (RejectedExecutionException e) { + // executor has been shut down, execute in current thread + command.run(); + } + }).exceptionally(ex -> { log.error("[{}] Error closing clients", topic, ex); unfenceTopicToResume(); closeClientFuture.completeExceptionally(ex); From 7a44c801f86c4276533b0f008e768fb8deba4abc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 26 Apr 2024 23:17:18 +0300 Subject: [PATCH 121/580] [improve][broker] Close TopicPoliciesService to allow Pulsar broker graceful shutdown (#22589) --- .../apache/pulsar/broker/PulsarService.java | 5 ++ .../SystemTopicBasedTopicPoliciesService.java | 63 +++++++++++++++++-- .../SystemTopicTxnBufferSnapshotService.java | 20 +++++- .../broker/service/TopicPoliciesService.java | 7 ++- 4 files changed, 87 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index c21c7dc771eae..51dffc20d076e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -565,6 +565,11 @@ public CompletableFuture closeAsync() { transactionBufferClient.close(); } + if (topicPoliciesService != null) { + topicPoliciesService.close(); + topicPoliciesService = null; + } + if (client != null) { client.close(); client = null; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index 0449e5c885cd3..6d18d6d61b08e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -32,6 +32,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nonnull; import org.apache.commons.lang3.concurrent.ConcurrentInitializer; @@ -72,6 +73,7 @@ public class SystemTopicBasedTopicPoliciesService implements TopicPoliciesServic private final PulsarService pulsarService; private final HashSet localCluster; private final String clusterName; + private final AtomicBoolean closed = new AtomicBoolean(false); private final ConcurrentInitializer namespaceEventsSystemTopicFactoryLazyInitializer = new LazyInitializer<>() { @@ -110,12 +112,18 @@ public SystemTopicBasedTopicPoliciesService(PulsarService pulsarService) { this.writerCaches = Caffeine.newBuilder() .expireAfterAccess(5, TimeUnit.MINUTES) .removalListener((namespaceName, writer, cause) -> { - ((SystemTopicClient.Writer) writer).closeAsync().exceptionally(ex -> { - log.error("[{}] Close writer error.", namespaceName, ex); - return null; - }); + try { + ((SystemTopicClient.Writer) writer).close(); + } catch (Exception e) { + log.error("[{}] Close writer error.", namespaceName, e); + } }) + .executor(pulsarService.getExecutor()) .buildAsync((namespaceName, executor) -> { + if (closed.get()) { + return CompletableFuture.failedFuture( + new BrokerServiceException(getClass().getName() + " is closed.")); + } SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(namespaceName); return systemTopicClient.newWriterAsync(); @@ -382,6 +390,10 @@ public CompletableFuture addOwnedNamespaceBundleAsync(NamespaceBundle name protected CompletableFuture> createSystemTopicClient( NamespaceName namespace) { + if (closed.get()) { + return CompletableFuture.failedFuture( + new BrokerServiceException(getClass().getName() + " is closed.")); + } try { createSystemTopicFactoryIfNeeded(); } catch (PulsarServerException ex) { @@ -430,6 +442,11 @@ public boolean test(NamespaceBundle namespaceBundle) { } private void initPolicesCache(SystemTopicClient.Reader reader, CompletableFuture future) { + if (closed.get()) { + future.completeExceptionally(new BrokerServiceException(getClass().getName() + " is closed.")); + cleanCacheAndCloseReader(reader.getSystemTopic().getTopicName().getNamespaceObject(), false); + return; + } reader.hasMoreEventsAsync().whenComplete((hasMore, ex) -> { if (ex != null) { log.error("[{}] Failed to check the move events for the system topic", @@ -511,6 +528,10 @@ private void cleanCacheAndCloseReader(@Nonnull NamespaceName namespace, boolean * #{@link SystemTopicBasedTopicPoliciesService#getTopicPoliciesAsync(TopicName)} method to block loading topic. */ private void readMorePoliciesAsync(SystemTopicClient.Reader reader) { + if (closed.get()) { + cleanCacheAndCloseReader(reader.getSystemTopic().getTopicName().getNamespaceObject(), false); + return; + } reader.readNextAsync() .thenAccept(msg -> { refreshTopicPoliciesCache(msg); @@ -628,11 +649,20 @@ private NamespaceEventsSystemTopicFactory getNamespaceEventsSystemTopicFactory() private void fetchTopicPoliciesAsyncAndCloseReader(SystemTopicClient.Reader reader, TopicName topicName, TopicPolicies policies, CompletableFuture future) { + if (closed.get()) { + future.completeExceptionally(new BrokerServiceException(getClass().getName() + " is closed.")); + reader.closeAsync().whenComplete((v, e) -> { + if (e != null) { + log.error("[{}] Close reader error.", topicName, e); + } + }); + return; + } reader.hasMoreEventsAsync().whenComplete((hasMore, ex) -> { if (ex != null) { future.completeExceptionally(ex); } - if (hasMore) { + if (hasMore != null && hasMore) { reader.readNextAsync().whenComplete((msg, e) -> { if (e != null) { future.completeExceptionally(e); @@ -656,7 +686,9 @@ private void fetchTopicPoliciesAsyncAndCloseReader(SystemTopicClient.Reader { if (e != null) { log.error("[{}] Close reader error.", topicName, e); @@ -740,4 +772,23 @@ protected AsyncLoadingCache } private static final Logger log = LoggerFactory.getLogger(SystemTopicBasedTopicPoliciesService.class); + + @Override + public void close() throws Exception { + if (closed.compareAndSet(false, true)) { + writerCaches.synchronous().invalidateAll(); + readerCaches.values().forEach(future -> { + if (future != null && !future.isCompletedExceptionally()) { + future.thenAccept(reader -> { + try { + reader.close(); + } catch (Exception e) { + log.error("Failed to close reader.", e); + } + }); + } + }); + readerCaches.clear(); + } + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java index 332d754cf97d2..bd1b90981695e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java @@ -142,8 +142,26 @@ private SystemTopicClient getTransactionBufferSystemTopicClient(NamespaceName public void close() throws Exception { for (Map.Entry> entry : clients.entrySet()) { - entry.getValue().close(); + try { + entry.getValue().close(); + } catch (Exception e) { + log.error("Failed to close system topic client for namespace {}", entry.getKey(), e); + } + } + clients.clear(); + for (Map.Entry> entry : refCountedWriterMap.entrySet()) { + CompletableFuture> future = entry.getValue().getFuture(); + if (!future.isCompletedExceptionally()) { + future.thenAccept(writer -> { + try { + writer.close(); + } catch (Exception e) { + log.error("Failed to close writer for namespace {}", entry.getKey(), e); + } + }); + } } + refCountedWriterMap.clear(); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java index aa3a6aaeff29f..41fecb3b87ed4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java @@ -38,7 +38,7 @@ * Topic policies service. */ @InterfaceStability.Evolving -public interface TopicPoliciesService { +public interface TopicPoliciesService extends AutoCloseable { TopicPoliciesService DISABLED = new TopicPoliciesServiceDisabled(); long DEFAULT_GET_TOPIC_POLICY_TIMEOUT = 30_000; @@ -239,5 +239,10 @@ public void registerListener(TopicName topicName, TopicPolicyListener listener) { //No-op } + + @Override + public void close() { + //No-op + } } } From 8323a3c49912976aee723787fa67bee4d7d8d846 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 26 Apr 2024 23:17:51 +0300 Subject: [PATCH 122/580] [improve][broker] Don't use forkjoin pool by default for deleting partitioned topics (#22598) --- .../broker/resources/NamespaceResources.java | 14 +++++++++++--- .../pulsar/broker/resources/PulsarResources.java | 12 ++++++++++-- .../org/apache/pulsar/broker/PulsarService.java | 2 +- 3 files changed, 22 insertions(+), 6 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java index 1ba353dccaa1c..975b23192f949 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java @@ -24,6 +24,8 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.function.Supplier; @@ -54,10 +56,14 @@ public class NamespaceResources extends BaseResources { private static final String NAMESPACE_BASE_PATH = "/namespace"; public NamespaceResources(MetadataStore configurationStore, int operationTimeoutSec) { + this(configurationStore, operationTimeoutSec, ForkJoinPool.commonPool()); + } + + public NamespaceResources(MetadataStore configurationStore, int operationTimeoutSec, Executor executor) { super(configurationStore, Policies.class, operationTimeoutSec); this.configurationStore = configurationStore; isolationPolicies = new IsolationPolicyResources(configurationStore, operationTimeoutSec); - partitionedTopicResources = new PartitionedTopicResources(configurationStore, operationTimeoutSec); + partitionedTopicResources = new PartitionedTopicResources(configurationStore, operationTimeoutSec, executor); } public CompletableFuture> listNamespacesAsync(String tenant) { @@ -234,9 +240,11 @@ public void setIsolationDataWithCreate(String cluster, public static class PartitionedTopicResources extends BaseResources { private static final String PARTITIONED_TOPIC_PATH = "/admin/partitioned-topics"; + private final Executor executor; - public PartitionedTopicResources(MetadataStore configurationStore, int operationTimeoutSec) { + public PartitionedTopicResources(MetadataStore configurationStore, int operationTimeoutSec, Executor executor) { super(configurationStore, PartitionedTopicMetadata.class, operationTimeoutSec); + this.executor = executor; } public CompletableFuture updatePartitionedTopicAsync(TopicName tn, Function runWithMarkDeleteAsync(TopicName topic, future.complete(deleteResult); } }); - }); + }, executor); return future; } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java index fe7ffe0bc7b43..cc64eeb52f6eb 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.resources; import java.util.Optional; +import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; import lombok.Getter; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreConfig; @@ -57,13 +59,19 @@ public class PulsarResources { public PulsarResources(MetadataStore localMetadataStore, MetadataStore configurationMetadataStore) { this(localMetadataStore, configurationMetadataStore, DEFAULT_OPERATION_TIMEOUT_SEC); } + + public PulsarResources(MetadataStore localMetadataStore, MetadataStore configurationMetadataStore, + int operationTimeoutSec) { + this(localMetadataStore, configurationMetadataStore, operationTimeoutSec, ForkJoinPool.commonPool()); + } + public PulsarResources(MetadataStore localMetadataStore, MetadataStore configurationMetadataStore, - int operationTimeoutSec) { + int operationTimeoutSec, Executor executor) { if (configurationMetadataStore != null) { tenantResources = new TenantResources(configurationMetadataStore, operationTimeoutSec); clusterResources = new ClusterResources(localMetadataStore, configurationMetadataStore, operationTimeoutSec); - namespaceResources = new NamespaceResources(configurationMetadataStore, operationTimeoutSec); + namespaceResources = new NamespaceResources(configurationMetadataStore, operationTimeoutSec, executor); resourcegroupResources = new ResourceGroupResources(configurationMetadataStore, operationTimeoutSec); } else { tenantResources = null; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 51dffc20d076e..96f3653ea9966 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -1000,7 +1000,7 @@ protected ManagedLedgerStorage newManagedLedgerClientFactory() throws Exception @VisibleForTesting protected PulsarResources newPulsarResources() { PulsarResources pulsarResources = new PulsarResources(localMetadataStore, configurationMetadataStore, - config.getMetadataStoreOperationTimeoutSeconds()); + config.getMetadataStoreOperationTimeoutSeconds(), getExecutor()); pulsarResources.getClusterResources().getStore().registerListener(this::handleDeleteCluster); return pulsarResources; From bf5d6aac1b62d195c544a486bcefec676948a3a4 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov <8622884+dlg99@users.noreply.github.com> Date: Fri, 26 Apr 2024 13:22:19 -0700 Subject: [PATCH 123/580] [improve][broker] perf: Reduce stickyHash calculations of non-persistent topics in SHARED subscriptions (#22536) --- .../pulsar/broker/service/Consumer.java | 21 ++++++++++--- ...tStickyKeyDispatcherMultipleConsumers.java | 30 +++++++++++++++---- ...ckyKeyDispatcherMultipleConsumersTest.java | 8 ++--- 3 files changed, 45 insertions(+), 14 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index 6b2028095e205..b1c3687b3a0f6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -286,16 +286,29 @@ public Future sendMessages(final List entries, EntryBatch totalChunkedMessages, redeliveryTracker, DEFAULT_CONSUMER_EPOCH); } + public Future sendMessages(final List entries, EntryBatchSizes batchSizes, + EntryBatchIndexesAcks batchIndexesAcks, + int totalMessages, long totalBytes, long totalChunkedMessages, + RedeliveryTracker redeliveryTracker, long epoch) { + return sendMessages(entries, null, batchSizes, batchIndexesAcks, totalMessages, totalBytes, + totalChunkedMessages, redeliveryTracker, epoch); + } + /** * Dispatch a list of entries to the consumer.
* It is also responsible to release entries data and recycle entries object. * * @return a SendMessageInfo object that contains the detail of what was sent to consumer */ - public Future sendMessages(final List entries, EntryBatchSizes batchSizes, + public Future sendMessages(final List entries, + final List stickyKeyHashes, + EntryBatchSizes batchSizes, EntryBatchIndexesAcks batchIndexesAcks, - int totalMessages, long totalBytes, long totalChunkedMessages, - RedeliveryTracker redeliveryTracker, long epoch) { + int totalMessages, + long totalBytes, + long totalChunkedMessages, + RedeliveryTracker redeliveryTracker, + long epoch) { this.lastConsumedTimestamp = System.currentTimeMillis(); if (entries.isEmpty() || totalMessages == 0) { @@ -323,7 +336,7 @@ public Future sendMessages(final List entries, EntryBatch // because this consumer is possible to disconnect at this time. if (pendingAcks != null) { int batchSize = batchSizes.getBatchSize(i); - int stickyKeyHash = getStickyKeyHash(entry); + int stickyKeyHash = stickyKeyHashes == null ? getStickyKeyHash(entry) : stickyKeyHashes.get(i); long[] ackSet = batchIndexesAcks == null ? null : batchIndexesAcks.getAckSet(i); if (ackSet != null) { unackedMessages -= (batchSize - BitSet.valueOf(ackSet).cardinality()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java index 2cad253f96ee2..fb7bd22de94a7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java @@ -126,6 +126,14 @@ protected Map> initialValue() throws Exception { } }; + private static final FastThreadLocal>> localGroupedStickyKeyHashes = + new FastThreadLocal>>() { + @Override + protected Map> initialValue() throws Exception { + return new HashMap<>(); + } + }; + @Override public void sendMessages(List entries) { if (entries.isEmpty()) { @@ -139,28 +147,38 @@ public void sendMessages(List entries) { final Map> groupedEntries = localGroupedEntries.get(); groupedEntries.clear(); + final Map> consumerStickyKeyHashesMap = localGroupedStickyKeyHashes.get(); + consumerStickyKeyHashesMap.clear(); for (Entry entry : entries) { - Consumer consumer = selector.select(peekStickyKey(entry.getDataBuffer())); + byte[] stickyKey = peekStickyKey(entry.getDataBuffer()); + int stickyKeyHash = StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey); + + Consumer consumer = selector.select(stickyKeyHash); if (consumer != null) { - groupedEntries.computeIfAbsent(consumer, k -> new ArrayList<>()).add(entry); + int startingSize = Math.max(10, entries.size() / (2 * consumerSet.size())); + groupedEntries.computeIfAbsent(consumer, k -> new ArrayList<>(startingSize)).add(entry); + consumerStickyKeyHashesMap + .computeIfAbsent(consumer, k -> new ArrayList<>(startingSize)).add(stickyKeyHash); } else { entry.release(); } } for (Map.Entry> entriesByConsumer : groupedEntries.entrySet()) { - Consumer consumer = entriesByConsumer.getKey(); - List entriesForConsumer = entriesByConsumer.getValue(); + final Consumer consumer = entriesByConsumer.getKey(); + final List entriesForConsumer = entriesByConsumer.getValue(); + final List stickyKeysForConsumer = consumerStickyKeyHashesMap.get(consumer); SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); EntryBatchSizes batchSizes = EntryBatchSizes.get(entriesForConsumer.size()); filterEntriesForConsumer(entriesForConsumer, batchSizes, sendMessageInfo, null, null, false, consumer); if (consumer.getAvailablePermits() > 0 && consumer.isWritable()) { - consumer.sendMessages(entriesForConsumer, batchSizes, null, sendMessageInfo.getTotalMessages(), + consumer.sendMessages(entriesForConsumer, stickyKeysForConsumer, batchSizes, + null, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), - getRedeliveryTracker()); + getRedeliveryTracker(), Commands.DEFAULT_CONSUMER_EPOCH); TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -sendMessageInfo.getTotalMessages()); } else { entriesForConsumer.forEach(e -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumersTest.java index b2638d53ab1c3..6b0f48a57cfe3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -128,15 +128,15 @@ public void testSendMessage() throws BrokerServiceException { assertEquals(byteBuf.toString(UTF_8), "message" + index); }; return mockPromise; - }).when(consumerMock).sendMessages(any(List.class), any(EntryBatchSizes.class), any(), - anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); + }).when(consumerMock).sendMessages(any(List.class), any(List.class), any(EntryBatchSizes.class), any(), + anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class), anyLong()); try { nonpersistentDispatcher.sendMessages(entries); } catch (Exception e) { fail("Failed to sendMessages.", e); } - verify(consumerMock, times(1)).sendMessages(any(List.class), any(EntryBatchSizes.class), - eq(null), anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); + verify(consumerMock, times(1)).sendMessages(any(List.class), any(List.class), any(EntryBatchSizes.class), + eq(null), anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class), anyLong()); } @Test(timeOut = 10000) From 5d9ccd48520e0a0aa0aeb7c918f7b8ee6c866471 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Sat, 27 Apr 2024 19:16:31 +0800 Subject: [PATCH 124/580] [fix][test] Flaky-test: ManagedLedgerTest.testTimestampOnWorkingLedger (#22600) --- .../mledger/impl/ManagedLedgerTest.java | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 22cf4d8b7a7ca..e983523c1b62e 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -2446,7 +2446,7 @@ public void testRetentionSize() throws Exception { }); } - @Test(groups = "flaky") + @Test public void testTimestampOnWorkingLedger() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); conf.setMaxEntriesPerLedger(1); @@ -2473,19 +2473,18 @@ public void testTimestampOnWorkingLedger() throws Exception { ml.addEntry("msg02".getBytes()); + // reopen a new ml2 ml.close(); - // Thread.sleep(1000); - iter = ml.getLedgersInfoAsList().iterator(); - ts = -1; - while (iter.hasNext()) { - LedgerInfo i = iter.next(); - if (iter.hasNext()) { - assertTrue(ts <= i.getTimestamp(), i.toString()); - ts = i.getTimestamp(); - } else { - assertTrue(i.getTimestamp() > 0, "well closed LedgerInfo should set a timestamp > 0"); - } - } + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) factory.open("my_test_ledger", conf); + + List ledgers = ml2.getLedgersInfoAsList(); + // after reopen ledgers will be 2 + 1(new open, not contain any entries) + assertEquals(ledgers.size(), 3); + + // the last closed ledger should be the penultimate one. + LedgerInfo lastClosedLeger = ledgers.get(ledgers.size() - 2); + assertTrue(lastClosedLeger.getTimestamp() > 0, "well closed LedgerInfo should set a timestamp > 0"); + ml2.close(); } @Test From 1bb9378b50aa891834b64cd39f55ae0e32a055bb Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Sun, 28 Apr 2024 10:37:37 +0800 Subject: [PATCH 125/580] [improve][test] Add policy authentication test for namespace API (#22593) --- .../broker/admin/NamespaceAuthZTest.java | 1248 +++++++++++++++-- 1 file changed, 1140 insertions(+), 108 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java index 5358295b78568..ec6a122f7df80 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java @@ -20,9 +20,11 @@ package org.apache.pulsar.broker.admin; import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.deleteNamespaceWithRetry; +import static org.apache.pulsar.common.policies.data.SchemaAutoUpdateCompatibilityStrategy.AutoUpdateDisabled; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; +import com.google.common.collect.Sets; import io.jsonwebtoken.Jwts; import java.io.File; import java.util.ArrayList; @@ -32,6 +34,8 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import lombok.Cleanup; import lombok.SneakyThrows; import org.apache.commons.lang3.StringUtils; @@ -44,17 +48,33 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.AutoSubscriptionCreationOverride; +import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; +import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.BookieAffinityGroupData; import org.apache.pulsar.common.policies.data.BundlesData; +import org.apache.pulsar.common.policies.data.DispatchRate; +import org.apache.pulsar.common.policies.data.EntryFilters; +import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode; +import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.OffloadPolicies; +import org.apache.pulsar.common.policies.data.PersistencePolicies; import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.PolicyName; +import org.apache.pulsar.common.policies.data.PolicyOperation; +import org.apache.pulsar.common.policies.data.PublishRate; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.packages.management.core.MockedPackagesStorageProvider; import org.apache.pulsar.packages.management.core.common.PackageMetadata; import org.apache.pulsar.security.MockedPulsarStandalone; import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; @@ -72,7 +92,7 @@ public class NamespaceAuthZTest extends MockedPulsarStandalone { private AuthorizationService authorizationService; - private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); + private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); private static final String TENANT_ADMIN_TOKEN = Jwts.builder() .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); @@ -122,16 +142,46 @@ public void after() throws Exception { superUserAdmin.namespaces().createNamespace("public/default"); } - private void setAuthorizationOperationChecker(String role, NamespaceOperation operation) { + private AtomicBoolean setAuthorizationOperationChecker(String role, NamespaceOperation operation) { + AtomicBoolean execFlag = new AtomicBoolean(false); Mockito.doAnswer(invocationOnMock -> { String role_ = invocationOnMock.getArgument(2); if (role.equals(role_)) { NamespaceOperation operation_ = invocationOnMock.getArgument(1); Assert.assertEquals(operation_, operation); } + execFlag.set(true); return invocationOnMock.callRealMethod(); }).when(authorizationService).allowNamespaceOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), Mockito.any()); + return execFlag; + } + + private void clearAuthorizationOperationChecker() { + Mockito.doAnswer(InvocationOnMock::callRealMethod).when(authorizationService) + .allowNamespaceOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any()); + } + + private AtomicBoolean setAuthorizationPolicyOperationChecker(String role, Object policyName, Object operation) { + AtomicBoolean execFlag = new AtomicBoolean(false); + if (operation instanceof PolicyOperation) { + Mockito.doAnswer(invocationOnMock -> { + String role_ = invocationOnMock.getArgument(3); + if (role.equals(role_)) { + PolicyName policyName_ = invocationOnMock.getArgument(1); + PolicyOperation operation_ = invocationOnMock.getArgument(2); + assertEquals(operation_, operation); + assertEquals(policyName_, policyName); + } + execFlag.set(true); + return invocationOnMock.callRealMethod(); + }).when(authorizationService).allowNamespacePolicyOperationAsync(Mockito.any(), Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any()); + } else { + throw new IllegalArgumentException(""); + } + return execFlag; } @SneakyThrows @@ -140,13 +190,12 @@ public void testProperties() { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; final String topic = "persistent://public/default/" + random; - final String subject = UUID.randomUUID().toString(); + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -214,18 +263,17 @@ public void testProperties() { superUserAdmin.topics().delete(topic, true); } - @Test + @Test public void testTopics() throws Exception { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; final String topic = "persistent://" + namespace + "/" + random; - final String subject = UUID.randomUUID().toString(); + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -236,10 +284,10 @@ public void testTopics() throws Exception { // test tenant manager tenantManagerAdmin.namespaces().getTopics(namespace); + AtomicBoolean execFlag = setAuthorizationOperationChecker(subject, NamespaceOperation.GET_TOPICS); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().getTopics(namespace)); - - setAuthorizationOperationChecker(subject, NamespaceOperation.GET_TOPICS); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); @@ -260,13 +308,12 @@ public void testBookieAffinityGroup() throws Exception { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; final String topic = "persistent://" + namespace + "/" + random; - final String subject = UUID.randomUUID().toString(); + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -302,11 +349,11 @@ public void testBookieAffinityGroup() throws Exception { for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().setBookieAffinityGroup(namespace, bookieAffinityGroupData)); + () -> subAdmin.namespaces().setBookieAffinityGroup(namespace, bookieAffinityGroupData)); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().getBookieAffinityGroup(namespace)); + () -> subAdmin.namespaces().getBookieAffinityGroup(namespace)); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().deleteBookieAffinityGroup(namespace)); + () -> subAdmin.namespaces().deleteBookieAffinityGroup(namespace)); superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); } @@ -319,20 +366,19 @@ public void testGetBundles() throws Exception { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; final String topic = "persistent://" + namespace + "/" + random; - final String subject = UUID.randomUUID().toString(); + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); Producer producer = pulsarClient.newProducer(Schema.BYTES) - .topic(topic) - .enableBatching(false) - .messageRoutingMode(MessageRoutingMode.SinglePartition) - .create(); + .topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); producer.send("message".getBytes()); - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -343,10 +389,10 @@ public void testGetBundles() throws Exception { tenantManagerAdmin.namespaces().getBundles(namespace); // test nobody + AtomicBoolean execFlag = setAuthorizationOperationChecker(subject, NamespaceOperation.GET_BUNDLE); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().getBundles(namespace)); - - setAuthorizationOperationChecker(subject, NamespaceOperation.GET_BUNDLE); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); @@ -367,20 +413,19 @@ public void testUnloadBundles() throws Exception { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; final String topic = "persistent://" + namespace + "/" + random; - final String subject = UUID.randomUUID().toString(); + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); Producer producer = pulsarClient.newProducer(Schema.BYTES) - .topic(topic) - .enableBatching(false) - .messageRoutingMode(MessageRoutingMode.SinglePartition) - .create(); + .topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); producer.send("message".getBytes()); - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -401,7 +446,7 @@ public void testUnloadBundles() throws Exception { for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().unloadNamespaceBundle(namespace, defaultBundle)); + () -> subAdmin.namespaces().unloadNamespaceBundle(namespace, defaultBundle)); superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); } @@ -413,20 +458,19 @@ public void testSplitBundles() throws Exception { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; final String topic = "persistent://" + namespace + "/" + random; - final String subject = UUID.randomUUID().toString(); + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); Producer producer = pulsarClient.newProducer(Schema.BYTES) - .topic(topic) - .enableBatching(false) - .messageRoutingMode(MessageRoutingMode.SinglePartition) - .create(); + .topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); producer.send("message".getBytes()); - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -447,7 +491,7 @@ public void testSplitBundles() throws Exception { for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().splitNamespaceBundle(namespace, defaultBundle, false, null)); + () -> subAdmin.namespaces().splitNamespaceBundle(namespace, defaultBundle, false, null)); superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); } @@ -459,13 +503,12 @@ public void testDeleteBundles() throws Exception { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; final String topic = "persistent://" + namespace + "/" + random; - final String subject = UUID.randomUUID().toString(); + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -478,7 +521,8 @@ public void testDeleteBundles() throws Exception { producer.send("message".getBytes()); for (int i = 0; i < 3; i++) { - superUserAdmin.namespaces().splitNamespaceBundle(namespace, Policies.BundleType.LARGEST.toString(), false, null); + superUserAdmin.namespaces() + .splitNamespaceBundle(namespace, Policies.BundleType.LARGEST.toString(), false, null); } BundlesData bundles = superUserAdmin.namespaces().getBundles(namespace); @@ -490,7 +534,7 @@ public void testDeleteBundles() throws Exception { for (int i = 0; i < boundaries.size() - 1; i++) { String bundleRange = boundaries.get(i) + "_" + boundaries.get(i + 1); List allTopicsFromNamespaceBundle = getPulsarService().getBrokerService() - .getAllTopicsFromNamespaceBundle(namespace, namespace + "/" + bundleRange); + .getAllTopicsFromNamespaceBundle(namespace, namespace + "/" + bundleRange); System.out.println(StringUtils.join(allTopicsFromNamespaceBundle)); if (allTopicsFromNamespaceBundle.isEmpty()) { bundleRanges.add(bundleRange); @@ -504,15 +548,15 @@ public void testDeleteBundles() throws Exception { tenantManagerAdmin.namespaces().deleteNamespaceBundle(namespace, bundleRanges.get(1)); // test nobody + AtomicBoolean execFlag = setAuthorizationOperationChecker(subject, NamespaceOperation.DELETE_BUNDLE); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().deleteNamespaceBundle(namespace, bundleRanges.get(1))); - - setAuthorizationOperationChecker(subject, NamespaceOperation.DELETE_BUNDLE); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().deleteNamespaceBundle(namespace, bundleRanges.get(1))); + () -> subAdmin.namespaces().deleteNamespaceBundle(namespace, bundleRanges.get(1))); superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); } } @@ -522,7 +566,7 @@ public void testPermission() throws Exception { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; final String topic = "persistent://" + namespace + "/" + random; - final String subject = UUID.randomUUID().toString(); + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); @@ -530,13 +574,11 @@ public void testPermission() throws Exception { final String role = "sub"; final AuthAction testAction = AuthAction.consume; - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); - // test super admin superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, role, Set.of(testAction)); Map> permissions = superUserAdmin.namespaces().getPermissions(namespace); @@ -554,25 +596,33 @@ public void testPermission() throws Exception { Assert.assertTrue(permissions.isEmpty()); // test nobody + AtomicBoolean execFlag = + setAuthorizationOperationChecker(subject, NamespaceOperation.GRANT_PERMISSION); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().grantPermissionOnNamespace(namespace, role, Set.of(testAction))); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationOperationChecker(subject, NamespaceOperation.GET_PERMISSION); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().getPermissions(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = + setAuthorizationOperationChecker(subject, NamespaceOperation.REVOKE_PERMISSION); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().revokePermissionsOnNamespace(namespace, role)); + Assert.assertTrue(execFlag.get()); + clearAuthorizationOperationChecker(); for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); - setAuthorizationOperationChecker(subject, NamespaceOperation.GRANT_PERMISSION); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().grantPermissionOnNamespace(namespace, role, Set.of(testAction))); - setAuthorizationOperationChecker(subject, NamespaceOperation.GET_PERMISSION); + () -> subAdmin.namespaces().grantPermissionOnNamespace(namespace, role, Set.of(testAction))); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().getPermissions(namespace)); - setAuthorizationOperationChecker(subject, NamespaceOperation.REVOKE_PERMISSION); + () -> subAdmin.namespaces().getPermissions(namespace)); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().revokePermissionsOnNamespace(namespace, role)); + () -> subAdmin.namespaces().revokePermissionsOnNamespace(namespace, role)); superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); } @@ -584,13 +634,12 @@ public void testPermissionOnSubscription() throws Exception { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; final String topic = "persistent://" + namespace + "/" + random; - final String subject = UUID.randomUUID().toString(); + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -604,7 +653,8 @@ public void testPermissionOnSubscription() throws Exception { // test super admin superUserAdmin.namespaces().grantPermissionOnSubscription(namespace, subscription, Set.of(role)); - Map> permissionOnSubscription = superUserAdmin.namespaces().getPermissionOnSubscription(namespace); + Map> permissionOnSubscription = + superUserAdmin.namespaces().getPermissionOnSubscription(namespace); Assert.assertEquals(permissionOnSubscription.get(subscription), Set.of(role)); superUserAdmin.namespaces().revokePermissionOnSubscription(namespace, subscription, role); permissionOnSubscription = superUserAdmin.namespaces().getPermissionOnSubscription(namespace); @@ -619,25 +669,29 @@ public void testPermissionOnSubscription() throws Exception { Assert.assertTrue(permissionOnSubscription.isEmpty()); // test nobody + AtomicBoolean execFlag = setAuthorizationOperationChecker(subject, NamespaceOperation.GRANT_PERMISSION); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().grantPermissionOnSubscription(namespace, subscription, Set.of(role))); + Assert.assertTrue(execFlag.get()); + execFlag = setAuthorizationOperationChecker(subject, NamespaceOperation.GET_PERMISSION); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().getPermissionOnSubscription(namespace)); + Assert.assertTrue(execFlag.get()); + execFlag = setAuthorizationOperationChecker(subject, NamespaceOperation.REVOKE_PERMISSION); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().revokePermissionOnSubscription(namespace, subscription, role)); + Assert.assertTrue(execFlag.get()); + clearAuthorizationOperationChecker(); for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); - setAuthorizationOperationChecker(subject, NamespaceOperation.GRANT_PERMISSION); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().grantPermissionOnSubscription(namespace, subscription, Set.of(role))); - setAuthorizationOperationChecker(subject, NamespaceOperation.GET_PERMISSION); + () -> subAdmin.namespaces().grantPermissionOnSubscription(namespace, subscription, Set.of(role))); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().getPermissionOnSubscription(namespace)); - setAuthorizationOperationChecker(subject, NamespaceOperation.REVOKE_PERMISSION); + () -> subAdmin.namespaces().getPermissionOnSubscription(namespace)); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().revokePermissionOnSubscription(namespace, subscription, role)); + () -> subAdmin.namespaces().revokePermissionOnSubscription(namespace, subscription, role)); superUserAdmin.namespaces().revokePermissionsOnNamespace(namespace, subject); } @@ -649,12 +703,11 @@ public void testClearBacklog() throws Exception { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; final String topic = "persistent://" + namespace + "/" + random; - final String subject = UUID.randomUUID().toString(); + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -665,10 +718,10 @@ public void testClearBacklog() throws Exception { tenantManagerAdmin.namespaces().clearNamespaceBacklog(namespace); // test nobody + AtomicBoolean execFlag = setAuthorizationOperationChecker(subject, NamespaceOperation.CLEAR_BACKLOG); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().clearNamespaceBacklog(namespace)); - - setAuthorizationOperationChecker(subject, NamespaceOperation.CLEAR_BACKLOG); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); @@ -684,17 +737,16 @@ public void testClearBacklog() throws Exception { superUserAdmin.topics().delete(topic, true); } - @Test + @Test public void testClearNamespaceBundleBacklog() throws Exception { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; final String topic = "persistent://" + namespace + "/" + random; - final String subject = UUID.randomUUID().toString(); + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -706,17 +758,17 @@ public void testClearNamespaceBundleBacklog() throws Exception { final String defaultBundle = "0x00000000_0xffffffff"; - // test super admin + // test super admin superUserAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle); // test tenant manager tenantManagerAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle); // test nobody - Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + AtomicBoolean execFlag = setAuthorizationOperationChecker(subject, NamespaceOperation.CLEAR_BACKLOG); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().clearNamespaceBundleBacklog(namespace, defaultBundle)); - - setAuthorizationOperationChecker(subject, NamespaceOperation.CLEAR_BACKLOG); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); @@ -737,12 +789,11 @@ public void testUnsubscribeNamespace() throws Exception { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; final String topic = "persistent://" + namespace + "/" + random; - final String subject = UUID.randomUUID().toString(); + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -756,17 +807,17 @@ public void testUnsubscribeNamespace() throws Exception { .subscriptionName("sub") .subscribe().close(); - // test super admin + // test super admin superUserAdmin.namespaces().unsubscribeNamespace(namespace, "sub"); // test tenant manager tenantManagerAdmin.namespaces().unsubscribeNamespace(namespace, "sub"); // test nobody - Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + AtomicBoolean execFlag = setAuthorizationOperationChecker(subject, NamespaceOperation.UNSUBSCRIBE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().unsubscribeNamespace(namespace, "sub")); - - setAuthorizationOperationChecker(subject, NamespaceOperation.UNSUBSCRIBE); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); @@ -786,13 +837,12 @@ public void testUnsubscribeNamespace() throws Exception { public void testUnsubscribeNamespaceBundle() throws Exception { final String random = UUID.randomUUID().toString(); final String namespace = "public/default"; - final String topic = "persistent://" + namespace + "/" + random; - final String subject = UUID.randomUUID().toString(); + final String topic = "persistent://" + namespace + "/" + random ; + final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() .claim("sub", subject).signWith(SECRET_KEY).compact(); superUserAdmin.topics().createNonPartitionedTopic(topic); - @Cleanup - final PulsarAdmin subAdmin = PulsarAdmin.builder() + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); @@ -808,17 +858,17 @@ public void testUnsubscribeNamespaceBundle() throws Exception { final String defaultBundle = "0x00000000_0xffffffff"; - // test super admin + // test super admin superUserAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub"); // test tenant manager tenantManagerAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub"); // test nobody - Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + AtomicBoolean execFlag = setAuthorizationOperationChecker(subject, NamespaceOperation.UNSUBSCRIBE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.namespaces().unsubscribeNamespaceBundle(namespace, defaultBundle, "sub")); - - setAuthorizationOperationChecker(subject, NamespaceOperation.UNSUBSCRIBE); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); @@ -921,6 +971,7 @@ public void testPackageAPI() throws Exception { tenantManagerAdmin.packages().updateMetadata(packageName, updatedMetadata); // ---- test nobody --- + AtomicBoolean execFlag = setAuthorizationOperationChecker(subject, NamespaceOperation.PACKAGES); File file3 = File.createTempFile("package-api-test", ".package"); @@ -954,9 +1005,7 @@ public void testPackageAPI() throws Exception { updatedMetadata3.setProperties(Collections.singletonMap("key", "value")); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, () -> subAdmin.packages().updateMetadata(packageName3, updatedMetadata3)); - - - setAuthorizationOperationChecker(subject, NamespaceOperation.PACKAGES); + Assert.assertTrue(execFlag.get()); for (AuthAction action : AuthAction.values()) { superUserAdmin.namespaces().grantPermissionOnNamespace(namespace, subject, Set.of(action)); @@ -1022,7 +1071,7 @@ public void testPackageAPI() throws Exception { @Test @SneakyThrows - public void testOffloadThresholdInSeconds() { + public void testDispatchRate() { final String namespace = "public/default"; final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() @@ -1031,16 +1080,27 @@ public void testOffloadThresholdInSeconds() { .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().getOffloadThresholdInSeconds(namespace)); + () -> subAdmin.namespaces().getDispatchRate(namespace)); + Assert.assertTrue(execFlag.get()); + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + DispatchRate dispatchRate = + DispatchRate.builder().dispatchThrottlingRateInByte(10).dispatchThrottlingRateInMsg(10).build(); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().setOffloadThresholdInSeconds(namespace, 10000)); + () -> subAdmin.namespaces().setDispatchRate(namespace, dispatchRate)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeDispatchRate(namespace)); + Assert.assertTrue(execFlag.get()); } @Test @SneakyThrows - public void testMaxSubscriptionsPerTopic() { + public void testSubscribeRate() { final String namespace = "public/default"; final String subject = UUID.randomUUID().toString(); final String token = Jwts.builder() @@ -1049,13 +1109,985 @@ public void testMaxSubscriptionsPerTopic() { .serviceHttpUrl(getPulsarService().getWebServiceAddress()) .authentication(new AuthenticationToken(token)) .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().getMaxSubscriptionsPerTopic(namespace)); + () -> subAdmin.namespaces().getSubscribeRate(namespace)); + Assert.assertTrue(execFlag.get()); + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().setMaxSubscriptionsPerTopic(namespace, 100)); + () -> subAdmin.namespaces().setSubscribeRate(namespace, new SubscribeRate())); + Assert.assertTrue(execFlag.get()); + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, - () -> subAdmin.namespaces().removeMaxSubscriptionsPerTopic(namespace)); + () -> subAdmin.namespaces().removeSubscribeRate(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testPublishRate() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getPublishRate(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setPublishRate(namespace, new PublishRate(10, 10))); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removePublishRate(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testSubscriptionDispatchRate() { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String topic = "persistent://" + namespace + "/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getSubscriptionDispatchRate(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + DispatchRate dispatchRate = DispatchRate.builder().dispatchThrottlingRateInMsg(10).dispatchThrottlingRateInByte(10).build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setSubscriptionDispatchRate(namespace, dispatchRate)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeSubscriptionDispatchRate(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testCompactionThreshold() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.COMPACTION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getCompactionThreshold(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.COMPACTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setCompactionThreshold(namespace, 100L * 1024L *1024L)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.COMPACTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeCompactionThreshold(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testAutoTopicCreation() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_TOPIC_CREATION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getAutoTopicCreation(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_TOPIC_CREATION, PolicyOperation.WRITE); + AutoTopicCreationOverride build = AutoTopicCreationOverride.builder().allowAutoTopicCreation(true).build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setAutoTopicCreation(namespace, build)); + Assert.assertTrue(execFlag.get()); + + execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_TOPIC_CREATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeAutoTopicCreation(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testAutoSubscriptionCreation() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_SUBSCRIPTION_CREATION, + PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getAutoSubscriptionCreation(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_SUBSCRIPTION_CREATION, + PolicyOperation.WRITE); + AutoSubscriptionCreationOverride build = + AutoSubscriptionCreationOverride.builder().allowAutoSubscriptionCreation(true).build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setAutoSubscriptionCreation(namespace, build)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.AUTO_SUBSCRIPTION_CREATION, + PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeAutoSubscriptionCreation(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testMaxUnackedMessagesPerConsumer() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, + PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getMaxUnackedMessagesPerConsumer(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, + PolicyOperation.WRITE); + AutoSubscriptionCreationOverride build = + AutoSubscriptionCreationOverride.builder().allowAutoSubscriptionCreation(true).build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setMaxUnackedMessagesPerConsumer(namespace, 100)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, + PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeMaxUnackedMessagesPerConsumer(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testMaxUnackedMessagesPerSubscription() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, + PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getMaxUnackedMessagesPerSubscription(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, + PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setMaxUnackedMessagesPerSubscription(namespace, 100)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_UNACKED, + PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeMaxUnackedMessagesPerSubscription(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testNamespaceResourceGroup() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RESOURCEGROUP, + PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getNamespaceResourceGroup(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RESOURCEGROUP, + PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setNamespaceResourceGroup(namespace, "test-group")); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RESOURCEGROUP, + PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeNamespaceResourceGroup(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testDispatcherPauseOnAckStatePersistent() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.DISPATCHER_PAUSE_ON_ACK_STATE_PERSISTENT, + PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getDispatcherPauseOnAckStatePersistent(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DISPATCHER_PAUSE_ON_ACK_STATE_PERSISTENT, + PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setDispatcherPauseOnAckStatePersistent(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DISPATCHER_PAUSE_ON_ACK_STATE_PERSISTENT, + PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeDispatcherPauseOnAckStatePersistent(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testBacklogQuota() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.BACKLOG, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getBacklogQuotaMap(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.BACKLOG, PolicyOperation.WRITE); + BacklogQuota backlogQuota = BacklogQuota.builder().limitTime(10).limitSize(10).build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setBacklogQuota(namespace, backlogQuota)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.BACKLOG, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeBacklogQuota(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testDeduplicationSnapshotInterval() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getDeduplicationSnapshotInterval(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setDeduplicationSnapshotInterval(namespace, 100)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION_SNAPSHOT, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeDeduplicationSnapshotInterval(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testMaxSubscriptionsPerTopic() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_SUBSCRIPTIONS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getMaxSubscriptionsPerTopic(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_SUBSCRIPTIONS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setMaxSubscriptionsPerTopic(namespace, 10)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_SUBSCRIPTIONS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeMaxSubscriptionsPerTopic(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testMaxProducersPerTopic() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_PRODUCERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getMaxProducersPerTopic(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_PRODUCERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setMaxProducersPerTopic(namespace, 10)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_PRODUCERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeMaxProducersPerTopic(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testMaxConsumersPerTopic() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getMaxConsumersPerTopic(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setMaxConsumersPerTopic(namespace, 10)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeMaxConsumersPerTopic(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testNamespaceReplicationClusters() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getNamespaceReplicationClusters(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("test"))); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testReplicatorDispatchRate() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION_RATE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getReplicatorDispatchRate(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION_RATE, PolicyOperation.WRITE); + DispatchRate build = + DispatchRate.builder().dispatchThrottlingRateInByte(10).dispatchThrottlingRateInMsg(10).build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setReplicatorDispatchRate(namespace, build)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.REPLICATION_RATE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeReplicatorDispatchRate(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testMaxConsumersPerSubscription() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getMaxConsumersPerSubscription(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setMaxConsumersPerSubscription(namespace, 10)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_CONSUMERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeMaxConsumersPerSubscription(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testOffloadThreshold() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getOffloadThreshold(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setOffloadThreshold(namespace, 10)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testOffloadPolicies() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getOffloadPolicies(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.WRITE); + OffloadPolicies offloadPolicies = OffloadPolicies.builder().managedLedgerOffloadThresholdInBytes(10L).build(); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setOffloadPolicies(namespace, offloadPolicies)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeOffloadPolicies(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testMaxTopicsPerNamespace() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_TOPICS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getMaxTopicsPerNamespace(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_TOPICS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setMaxTopicsPerNamespace(namespace, 10)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.MAX_TOPICS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeMaxTopicsPerNamespace(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testDeduplicationStatus() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getDeduplicationStatus(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setDeduplicationStatus(namespace, true)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.DEDUPLICATION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeDeduplicationStatus(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testPersistence() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.PERSISTENCE, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getPersistence(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.PERSISTENCE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setPersistence(namespace, new PersistencePolicies(10, 10, 10, 10))); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.PERSISTENCE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removePersistence(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testNamespaceMessageTTL() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.TTL, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getNamespaceMessageTTL(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.TTL, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setNamespaceMessageTTL(namespace, 10)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.TTL, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeNamespaceMessageTTL(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testSubscriptionExpirationTime() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_EXPIRATION_TIME, + PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getSubscriptionExpirationTime(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_EXPIRATION_TIME, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setSubscriptionExpirationTime(namespace, 10)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_EXPIRATION_TIME, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeSubscriptionExpirationTime(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testDelayedDeliveryMessages() { + final String random = UUID.randomUUID().toString(); + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.DELAYED_DELIVERY, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getDelayedDelivery(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testRetention() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RETENTION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getRetention(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RETENTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setRetention(namespace, new RetentionPolicies(10, 10))); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.RETENTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeRetention(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testInactiveTopicPolicies() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.INACTIVE_TOPIC, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getInactiveTopicPolicies(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.INACTIVE_TOPIC, PolicyOperation.WRITE); + InactiveTopicPolicies inactiveTopicPolicies = new InactiveTopicPolicies( + InactiveTopicDeleteMode.delete_when_no_subscriptions, 10, false); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setInactiveTopicPolicies(namespace, inactiveTopicPolicies)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.INACTIVE_TOPIC, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeInactiveTopicPolicies(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testNamespaceAntiAffinityGroup() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ANTI_AFFINITY, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getNamespaceAntiAffinityGroup(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ANTI_AFFINITY, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setNamespaceAntiAffinityGroup(namespace, "invalid-group")); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testOffloadDeleteLagMs() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getOffloadDeleteLagMs(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setOffloadDeleteLag(namespace, 100, TimeUnit.HOURS)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testOffloadThresholdInSeconds() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = + setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getOffloadThresholdInSeconds(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.OFFLOAD, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setOffloadThresholdInSeconds(namespace, 10000)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testNamespaceEntryFilters() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ENTRY_FILTERS, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getNamespaceEntryFilters(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setNamespaceEntryFilters(namespace, new EntryFilters("filter1"))); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ENTRY_FILTERS, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeNamespaceEntryFilters(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testEncryptionRequiredStatus() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ENCRYPTION, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getEncryptionRequiredStatus(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.ENCRYPTION, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setEncryptionRequiredStatus(namespace, false)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testSubscriptionTypesEnabled() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_AUTH_MODE, + PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getSubscriptionTypesEnabled(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setSubscriptionTypesEnabled(namespace, Sets.newHashSet(SubscriptionType.Failover))); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SUBSCRIPTION_AUTH_MODE, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().removeSubscriptionTypesEnabled(namespace)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testIsAllowAutoUpdateSchema() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getIsAllowAutoUpdateSchema(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setIsAllowAutoUpdateSchema(namespace, true)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testSchemaAutoUpdateCompatibilityStrategy() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getSchemaAutoUpdateCompatibilityStrategy(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setSchemaAutoUpdateCompatibilityStrategy(namespace, AutoUpdateDisabled)); + Assert.assertTrue(execFlag.get()); + } + + @Test + @SneakyThrows + public void testSchemaValidationEnforced() { + final String namespace = "public/default"; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + @Cleanup final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + AtomicBoolean execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.READ); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().getSchemaValidationEnforced(namespace)); + Assert.assertTrue(execFlag.get()); + + execFlag = setAuthorizationPolicyOperationChecker(subject, PolicyName.SCHEMA_COMPATIBILITY_STRATEGY, PolicyOperation.WRITE); + Assert.assertThrows(PulsarAdminException.NotAuthorizedException.class, + () -> subAdmin.namespaces().setSchemaValidationEnforced(namespace, true)); + Assert.assertTrue(execFlag.get()); } } From a761b97b733142b1ade525e1d1c06785e98face1 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Sun, 28 Apr 2024 20:22:09 +0800 Subject: [PATCH 126/580] [fix][broker] Reader stuck after call hasMessageAvailable when enable replicateSubscriptionState (#22572) --- .../mledger/util/ManagedLedgerImplUtils.java | 82 +++++++++++++++++++ .../util/ManagedLedgerImplUtilsTest.java | 74 +++++++++++++++++ .../pulsar/broker/service/ServerCnx.java | 48 +++++------ .../apache/pulsar/broker/service/Topic.java | 7 ++ .../service/persistent/PersistentTopic.java | 18 ++++ .../service/ReplicatorSubscriptionTest.java | 77 +++++++++++++++++ ...icatorSubscriptionWithTransactionTest.java | 53 ++++++++++++ .../buffer/TopicTransactionBufferTest.java | 22 +++-- 8 files changed, 350 insertions(+), 31 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtils.java create mode 100644 managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionWithTransactionTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtils.java new file mode 100644 index 0000000000000..cd8671b0e6289 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtils.java @@ -0,0 +1,82 @@ +/* + * 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.bookkeeper.mledger.util; + +import java.util.concurrent.CompletableFuture; +import java.util.function.Predicate; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.common.classification.InterfaceStability; + +@InterfaceStability.Evolving +public class ManagedLedgerImplUtils { + + /** + * Reverse find last valid position one-entry by one-entry. + */ + public static CompletableFuture asyncGetLastValidPosition(final ManagedLedgerImpl ledger, + final Predicate predicate, + final PositionImpl startPosition) { + CompletableFuture future = new CompletableFuture<>(); + if (!ledger.isValidPosition(startPosition)) { + future.complete(startPosition); + } else { + internalAsyncReverseFindPositionOneByOne(ledger, predicate, startPosition, future); + } + return future; + } + + private static void internalAsyncReverseFindPositionOneByOne(final ManagedLedgerImpl ledger, + final Predicate predicate, + final PositionImpl position, + final CompletableFuture future) { + ledger.asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { + @Override + public void readEntryComplete(Entry entry, Object ctx) { + final Position position = entry.getPosition(); + try { + if (predicate.test(entry)) { + future.complete(position); + return; + } + PositionImpl previousPosition = ledger.getPreviousPosition((PositionImpl) position); + if (!ledger.isValidPosition(previousPosition)) { + future.complete(previousPosition); + } else { + internalAsyncReverseFindPositionOneByOne(ledger, predicate, + ledger.getPreviousPosition((PositionImpl) position), future); + } + } catch (Exception e) { + future.completeExceptionally(e); + } finally { + entry.release(); + } + } + + @Override + public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + }, null); + } +} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java new file mode 100644 index 0000000000000..f13d23c05296f --- /dev/null +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java @@ -0,0 +1,74 @@ +/* + * 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.bookkeeper.mledger.util; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.testng.Assert.assertEquals; +import java.nio.charset.StandardCharsets; +import java.util.function.Predicate; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.test.MockedBookKeeperTestCase; +import org.testng.annotations.Test; + +@Slf4j +public class ManagedLedgerImplUtilsTest extends MockedBookKeeperTestCase { + + @Test + public void testGetLastValidPosition() throws Exception { + final int maxEntriesPerLedger = 5; + + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setMaxEntriesPerLedger(maxEntriesPerLedger); + ManagedLedger ledger = factory.open("testReverseFindPositionOneByOne", managedLedgerConfig); + + String matchEntry = "match-entry"; + String noMatchEntry = "nomatch-entry"; + Predicate predicate = entry -> { + String entryValue = entry.getDataBuffer().toString(UTF_8); + return matchEntry.equals(entryValue); + }; + + // New ledger will return the last position, regardless of whether the conditions are met or not. + Position position = ManagedLedgerImplUtils.asyncGetLastValidPosition((ManagedLedgerImpl) ledger, + predicate, (PositionImpl) ledger.getLastConfirmedEntry()).get(); + assertEquals(ledger.getLastConfirmedEntry(), position); + + for (int i = 0; i < maxEntriesPerLedger - 1; i++) { + ledger.addEntry(matchEntry.getBytes(StandardCharsets.UTF_8)); + } + Position lastMatchPosition = ledger.addEntry(matchEntry.getBytes(StandardCharsets.UTF_8)); + for (int i = 0; i < maxEntriesPerLedger; i++) { + ledger.addEntry(noMatchEntry.getBytes(StandardCharsets.UTF_8)); + } + + // Returns last position of entry is "match-entry" + position = ManagedLedgerImplUtils.asyncGetLastValidPosition((ManagedLedgerImpl) ledger, + predicate, (PositionImpl) ledger.getLastConfirmedEntry()).get(); + assertEquals(position, lastMatchPosition); + + ledger.close(); + } + +} 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 a60f1d805ceb6..5ccdbfbe715c5 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 @@ -2174,29 +2174,31 @@ protected void handleGetLastMessageId(CommandGetLastMessageId getLastMessageId) long requestId = getLastMessageId.getRequestId(); Topic topic = consumer.getSubscription().getTopic(); - topic.checkIfTransactionBufferRecoverCompletely(true).thenRun(() -> { - Position lastPosition = ((PersistentTopic) topic).getMaxReadPosition(); - int partitionIndex = TopicName.getPartitionIndex(topic.getName()); - - Position markDeletePosition = null; - if (consumer.getSubscription() instanceof PersistentSubscription) { - markDeletePosition = ((PersistentSubscription) consumer.getSubscription()).getCursor() - .getMarkDeletedPosition(); - } - - getLargestBatchIndexWhenPossible( - topic, - (PositionImpl) lastPosition, - (PositionImpl) markDeletePosition, - partitionIndex, - requestId, - consumer.getSubscription().getName(), - consumer.readCompacted()); - }).exceptionally(e -> { - writeAndFlush(Commands.newError(getLastMessageId.getRequestId(), - ServerError.UnknownError, "Failed to recover Transaction Buffer.")); - return null; - }); + topic.checkIfTransactionBufferRecoverCompletely(true) + .thenCompose(__ -> topic.getLastDispatchablePosition()) + .thenApply(lastPosition -> { + int partitionIndex = TopicName.getPartitionIndex(topic.getName()); + + Position markDeletePosition = null; + if (consumer.getSubscription() instanceof PersistentSubscription) { + markDeletePosition = ((PersistentSubscription) consumer.getSubscription()).getCursor() + .getMarkDeletedPosition(); + } + + getLargestBatchIndexWhenPossible( + topic, + (PositionImpl) lastPosition, + (PositionImpl) markDeletePosition, + partitionIndex, + requestId, + consumer.getSubscription().getName(), + consumer.readCompacted()); + return null; + }).exceptionally(e -> { + writeAndFlush(Commands.newError(getLastMessageId.getRequestId(), + ServerError.UnknownError, "Failed to recover Transaction Buffer.")); + return null; + }); } else { writeAndFlush(Commands.newError(getLastMessageId.getRequestId(), ServerError.MetadataError, "Consumer not found")); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java index a296052a41191..37696d7a7c53c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java @@ -275,6 +275,13 @@ CompletableFuture asyncGetStats(boolean getPreciseBack Position getLastPosition(); + /** + * Get the last message position that can be dispatch. + */ + default CompletableFuture getLastDispatchablePosition() { + throw new UnsupportedOperationException("getLastDispatchablePosition is not supported by default"); + } + CompletableFuture getLastMessageId(); /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 155b67778820b..95a2b64908a73 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -83,6 +83,7 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.impl.ShadowManagedLedgerImpl; import org.apache.bookkeeper.mledger.util.Futures; +import org.apache.bookkeeper.mledger.util.ManagedLedgerImplUtils; import org.apache.bookkeeper.net.BookieId; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -174,6 +175,7 @@ import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.protocol.Markers; import org.apache.pulsar.common.protocol.schema.SchemaData; import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.schema.SchemaType; @@ -3634,6 +3636,22 @@ public Position getLastPosition() { return ledger.getLastConfirmedEntry(); } + @Override + public CompletableFuture getLastDispatchablePosition() { + PositionImpl maxReadPosition = getMaxReadPosition(); + // If `maxReadPosition` is not equal to `LastPosition`. It means that there are uncommitted transactions. + // so return `maxRedPosition` directly. + if (maxReadPosition.compareTo((PositionImpl) getLastPosition()) != 0) { + return CompletableFuture.completedFuture(maxReadPosition); + } else { + return ManagedLedgerImplUtils.asyncGetLastValidPosition((ManagedLedgerImpl) ledger, entry -> { + MessageMetadata md = Commands.parseMessageMetadata(entry.getDataBuffer()); + // If a messages has marker will filter by AbstractBaseDispatcher.filterEntriesForConsumer + return !Markers.isServerOnlyMarker(md); + }, maxReadPosition); + } + } + @Override public CompletableFuture getLastMessageId() { CompletableFuture completableFuture = new CompletableFuture<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java index 8aeb902211db2..25b09f965498d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java @@ -51,6 +51,7 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionType; @@ -167,6 +168,82 @@ public void testReplicatedSubscriptionAcrossTwoRegions() throws Exception { "messages don't match."); } + /** + * Tests replicated subscriptions across two regions and can read successful. + */ + @Test + public void testReplicatedSubscriptionAcrossTwoRegionsGetLastMessage() throws Exception { + String namespace = BrokerTestUtil.newUniqueName("pulsar/replicatedsubscriptionlastmessage"); + String topicName = "persistent://" + namespace + "/mytopic"; + String subscriptionName = "cluster-subscription"; + // this setting can be used to manually run the test with subscription replication disabled + // it shows that subscription replication has no impact in behavior for this test case + boolean replicateSubscriptionState = true; + + admin1.namespaces().createNamespace(namespace); + admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r2")); + + @Cleanup + PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()) + .statsInterval(0, TimeUnit.SECONDS) + .build(); + + // create subscription in r1 + createReplicatedSubscription(client1, topicName, subscriptionName, replicateSubscriptionState); + + @Cleanup + PulsarClient client2 = PulsarClient.builder().serviceUrl(url2.toString()) + .statsInterval(0, TimeUnit.SECONDS) + .build(); + + // create subscription in r2 + createReplicatedSubscription(client2, topicName, subscriptionName, replicateSubscriptionState); + + Set sentMessages = new LinkedHashSet<>(); + + // send messages in r1 + @Cleanup + Producer producer = client1.newProducer().topic(topicName) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + int numMessages = 6; + for (int i = 0; i < numMessages; i++) { + String body = "message" + i; + producer.send(body.getBytes(StandardCharsets.UTF_8)); + sentMessages.add(body); + } + producer.close(); + + + // consume 3 messages in r1 + Set receivedMessages = new LinkedHashSet<>(); + try (Consumer consumer1 = client1.newConsumer() + .topic(topicName) + .subscriptionName(subscriptionName) + .replicateSubscriptionState(replicateSubscriptionState) + .subscribe()) { + readMessages(consumer1, receivedMessages, 3, false); + } + + // wait for subscription to be replicated + Thread.sleep(2 * config1.getReplicatedSubscriptionsSnapshotFrequencyMillis()); + + // create a reader in r2 + Reader reader = client2.newReader().topic(topicName) + .subscriptionName("new-sub") + .startMessageId(MessageId.earliest) + .create(); + int readNum = 0; + while (reader.hasMessageAvailable()) { + Message message = reader.readNext(10, TimeUnit.SECONDS); + assertNotNull(message); + log.info("Receive message: " + new String(message.getValue()) + " msgId: " + message.getMessageId()); + readNum++; + } + assertEquals(readNum, numMessages); + } + @Test public void testReplicatedSubscribeAndSwitchToStandbyCluster() throws Exception { final String namespace = BrokerTestUtil.newUniqueName("pulsar/ns_"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionWithTransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionWithTransactionTest.java new file mode 100644 index 0000000000000..93a22a851f160 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionWithTransactionTest.java @@ -0,0 +1,53 @@ +/* + * 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.service; + +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +/** + * Tests replicated subscriptions with transaction (PIP-33) + */ +@Test(groups = "broker") +public class ReplicatorSubscriptionWithTransactionTest extends ReplicatorSubscriptionTest { + + @Override + @BeforeClass(timeOut = 300000) + public void setup() throws Exception { + config1.setTransactionCoordinatorEnabled(true); + config2.setTransactionCoordinatorEnabled(true); + config3.setTransactionCoordinatorEnabled(true); + config4.setTransactionCoordinatorEnabled(true); + super.setup(); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + @DataProvider(name = "isTopicPolicyEnabled") + private Object[][] isTopicPolicyEnabled() { + // Todo: fix replication can not be enabled at topic level. + return new Object[][] { { Boolean.FALSE } }; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java index fad785cc882ff..b0903b00be380 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java @@ -280,9 +280,9 @@ public void testGetLastMessageIdsWithOngoingTransactions() throws Exception { for (int i = 0; i < 3; i++) { expectedLastMessageID = (MessageIdImpl) producer.newMessage().send(); } - assertMessageId(consumer, expectedLastMessageID, 0); + assertMessageId(consumer, expectedLastMessageID); // 2.2 Case2: send 2 ongoing transactional messages and 2 original messages. - // |1:0|1:1|1:2|txn1->1:3|1:4|txn2->1:5|1:6|. + // |1:0|1:1|1:2|txn1:start->1:3|1:4|txn2:start->1:5|1:6|. Transaction txn1 = pulsarClient.newTransaction() .withTransactionTimeout(5, TimeUnit.HOURS) .build() @@ -292,18 +292,24 @@ public void testGetLastMessageIdsWithOngoingTransactions() throws Exception { .build() .get(); producer.newMessage(txn1).send(); + // expectedLastMessageID1 == 1:4 MessageIdImpl expectedLastMessageID1 = (MessageIdImpl) producer.newMessage().send(); producer.newMessage(txn2).send(); + // expectedLastMessageID2 == 1:6 MessageIdImpl expectedLastMessageID2 = (MessageIdImpl) producer.newMessage().send(); + // 2.2.1 Last message ID will not change when txn1 and txn2 do not end. - assertMessageId(consumer, expectedLastMessageID, 0); + assertMessageId(consumer, expectedLastMessageID); + // 2.2.2 Last message ID will update to 1:4 when txn1 committed. + // |1:0|1:1|1:2|txn1:start->1:3|1:4|txn2:start->1:5|1:6|tx1:commit->1:7| txn1.commit().get(5, TimeUnit.SECONDS); - assertMessageId(consumer, expectedLastMessageID1, 0); + assertMessageId(consumer, expectedLastMessageID1); + // 2.2.3 Last message ID will update to 1:6 when txn2 aborted. + // |1:0|1:1|1:2|txn1:start->1:3|1:4|txn2:start->1:5|1:6|tx1:commit->1:7|tx2:abort->1:8| txn2.abort().get(5, TimeUnit.SECONDS); - // Todo: We can not ignore the marker's position in this fix. - assertMessageId(consumer, expectedLastMessageID2, 2); + assertMessageId(consumer, expectedLastMessageID2); } /** @@ -362,9 +368,9 @@ private void triggerLedgerSwitch(String topicName) throws Exception{ }); } - private void assertMessageId(Consumer consumer, MessageIdImpl expected, int entryOffset) throws Exception { + private void assertMessageId(Consumer consumer, MessageIdImpl expected) throws Exception { TopicMessageIdImpl actual = (TopicMessageIdImpl) consumer.getLastMessageIds().get(0); - assertEquals(expected.getEntryId(), actual.getEntryId() - entryOffset); + assertEquals(expected.getEntryId(), actual.getEntryId()); assertEquals(expected.getLedgerId(), actual.getLedgerId()); } From 264722f1da9ab806c9a79196c091bfe4d03b3090 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Mon, 29 Apr 2024 11:49:28 +0800 Subject: [PATCH 127/580] [fix][test] Fix the flaky tests of ManagedLedgerImplUtilsTest (#22611) --- .../bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java index f13d23c05296f..cce593fbb38ec 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java @@ -21,6 +21,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.testng.Assert.assertEquals; import java.nio.charset.StandardCharsets; +import java.util.concurrent.TimeUnit; import java.util.function.Predicate; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; @@ -41,6 +42,8 @@ public void testGetLastValidPosition() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); managedLedgerConfig.setMaxEntriesPerLedger(maxEntriesPerLedger); + managedLedgerConfig.setRetentionSizeInMB(10); + managedLedgerConfig.setRetentionTime(5, TimeUnit.MINUTES); ManagedLedger ledger = factory.open("testReverseFindPositionOneByOne", managedLedgerConfig); String matchEntry = "match-entry"; From 93afd89b047ac56d3b7e476f578993197cf41935 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 29 Apr 2024 13:40:18 +0800 Subject: [PATCH 128/580] [fix][broker] One topic can be closed multiple times concurrently (#17524) --- .../service/persistent/PersistentTopic.java | 199 ++++++++++++++---- .../broker/service/OneWayReplicatorTest.java | 21 +- .../persistent/PersistentTopicTest.java | 81 ++++++- .../apache/pulsar/common/util/FutureUtil.java | 33 ++- 4 files changed, 290 insertions(+), 44 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 95a2b64908a73..22041326ba240 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -50,6 +50,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.function.BiFunction; import java.util.stream.Collectors; @@ -276,6 +277,8 @@ protected TopicStatsHelper initialValue() { @Getter private final ExecutorService orderedExecutor; + private volatile CloseFutures closeFutures; + @Getter private final PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); @@ -299,6 +302,50 @@ private static class EstimateTimeBasedBacklogQuotaCheckResult { Long estimatedOldestUnacknowledgedMessageTimestamp; } + /*** + * We use 3 futures to prevent a new closing if there is an in-progress deletion or closing. We make Pulsar return + * the in-progress one when it is called the second time. + * + * The topic closing will be called the below scenarios: + * 1. Calling "pulsar-admin topics unload". Relate to {@link CloseFutures#waitDisconnectClients}. + * 2. Namespace bundle transfer or unloading. + * a. The unloading topic triggered by unloading namespace bundles will not wait for clients disconnect. Relate + * to {@link CloseFutures#notWaitDisconnectClients}. + * b. The unloading topic triggered by unloading namespace bundles was seperated to two steps when using + * {@link ExtensibleLoadManagerImpl}. + * b-1. step-1: fence the topic on the original Broker, and do not trigger reconnections of clients. Relate + * to {@link CloseFutures#transferring}. This step is a half closing. + * b-2. step-2: send the owner broker information to clients and disconnect clients. Relate + * to {@link CloseFutures#notWaitDisconnectClients}. + * + * The three futures will be setting as the below rule: + * Event: Topic close. + * - If the first one closing is called by "close and not disconnect clients": + * - {@link CloseFutures#transferring} will be initialized as "close and not disconnect clients". + * - {@link CloseFutures#waitDisconnectClients} ang {@link CloseFutures#notWaitDisconnectClients} will be empty, + * the second closing will do a new close after {@link CloseFutures#transferring} is completed. + * - If the first one closing is called by "close and not wait for clients disconnect": + * - {@link CloseFutures#waitDisconnectClients} will be initialized as "waiting for clients disconnect". + * - {@link CloseFutures#notWaitDisconnectClients} ang {@link CloseFutures#transferring} will be + * initialized as "not waiting for clients disconnect" . + * - If the first one closing is called by "close and wait for clients disconnect", the three futures will be + * initialized as "waiting for clients disconnect". + * Event: Topic delete. + * the three futures will be initialized as "waiting for clients disconnect". + */ + private class CloseFutures { + private final CompletableFuture transferring; + private final CompletableFuture notWaitDisconnectClients; + private final CompletableFuture waitDisconnectClients; + + public CloseFutures(CompletableFuture transferring, CompletableFuture waitDisconnectClients, + CompletableFuture notWaitDisconnectClients) { + this.transferring = transferring; + this.waitDisconnectClients = waitDisconnectClients; + this.notWaitDisconnectClients = notWaitDisconnectClients; + } + } + private static class TopicStatsHelper { public double averageMsgSize; public double aggMsgRateIn; @@ -1417,8 +1464,11 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, } fenceTopicToCloseOrDelete(); // Avoid clients reconnections while deleting + // Mark the progress of close to prevent close calling concurrently. + this.closeFutures = + new CloseFutures(new CompletableFuture(), new CompletableFuture(), new CompletableFuture()); - return getBrokerService().getPulsar().getPulsarResources().getNamespaceResources() + CompletableFuture res = getBrokerService().getPulsar().getPulsarResources().getNamespaceResources() .getPartitionedTopicResources().runWithMarkDeleteAsync(TopicName.get(topic), () -> { CompletableFuture deleteFuture = new CompletableFuture<>(); @@ -1528,6 +1578,11 @@ public void deleteLedgerComplete(Object ctx) { unfenceTopicToResume(); } }); + + FutureUtil.completeAfter(closeFutures.transferring, res); + FutureUtil.completeAfter(closeFutures.notWaitDisconnectClients, res); + FutureUtil.completeAfter(closeFutures.waitDisconnectClients, res); + return res; } finally { lock.writeLock().unlock(); } @@ -1543,6 +1598,12 @@ public CompletableFuture close(boolean closeWithoutWaitingClientDisconnect return close(true, closeWithoutWaitingClientDisconnect); } + private enum CloseTypes { + transferring, + notWaitDisconnectClients, + waitDisconnectClients; + } + /** * Close this topic - close all producers and subscriptions associated with this topic. * @@ -1553,32 +1614,57 @@ public CompletableFuture close(boolean closeWithoutWaitingClientDisconnect @Override public CompletableFuture close( boolean disconnectClients, boolean closeWithoutWaitingClientDisconnect) { - CompletableFuture closeFuture = new CompletableFuture<>(); - lock.writeLock().lock(); - try { - if (!disconnectClients) { - transferring = true; - } + // Choose the close type. + CloseTypes closeType; + if (!disconnectClients) { + closeType = CloseTypes.transferring; + } else if (closeWithoutWaitingClientDisconnect) { + closeType = CloseTypes.notWaitDisconnectClients; + } else { // closing managed-ledger waits until all producers/consumers/replicators get closed. Sometimes, broker // forcefully wants to close managed-ledger without waiting all resources to be closed. - if (!isClosingOrDeleting || closeWithoutWaitingClientDisconnect) { - fenceTopicToCloseOrDelete(); + closeType = CloseTypes.waitDisconnectClients; + } + /** Maybe there is a in-progress half closing task. see the section 2-b-1 of {@link CloseFutures}. **/ + CompletableFuture inProgressTransferCloseTask = null; + try { + // Return in-progress future if exists. + if (isClosingOrDeleting) { + if (closeType == CloseTypes.transferring) { + return closeFutures.transferring; + } + if (closeType == CloseTypes.notWaitDisconnectClients && closeFutures.notWaitDisconnectClients != null) { + return closeFutures.notWaitDisconnectClients; + } + if (closeType == CloseTypes.waitDisconnectClients && closeFutures.waitDisconnectClients != null) { + return closeFutures.waitDisconnectClients; + } + if (transferring) { + inProgressTransferCloseTask = closeFutures.transferring; + } + } + fenceTopicToCloseOrDelete(); + if (closeType == CloseTypes.transferring) { + transferring = true; + this.closeFutures = new CloseFutures(new CompletableFuture(), null, null); } else { - log.warn("[{}] Topic is already being closed or deleted", topic); - closeFuture.completeExceptionally(new TopicFencedException("Topic is already fenced")); - return closeFuture; + this.closeFutures = + new CloseFutures(new CompletableFuture(), new CompletableFuture(), new CompletableFuture()); } } finally { lock.writeLock().unlock(); } List> futures = new ArrayList<>(); + if (inProgressTransferCloseTask != null) { + futures.add(inProgressTransferCloseTask); + } futures.add(transactionBuffer.closeAsync()); replicators.forEach((cluster, replicator) -> futures.add(replicator.terminate())); shadowReplicators.forEach((__, replicator) -> futures.add(replicator.terminate())); - if (disconnectClients) { + if (closeType != CloseTypes.transferring) { futures.add(ExtensibleLoadManagerImpl.getAssignedBrokerLookupData( brokerService.getPulsar(), topic).thenAccept(lookupData -> { producers.values().forEach(producer -> futures.add(producer.disconnect(lookupData))); @@ -1616,40 +1702,79 @@ public CompletableFuture close( } } - CompletableFuture clientCloseFuture = closeWithoutWaitingClientDisconnect - ? CompletableFuture.completedFuture(null) - : FutureUtil.waitForAll(futures); + CompletableFuture disconnectClientsInCurrentCall = null; + // Note: "disconnectClientsToCache" is a non-able value, it is null when close type is transferring. + AtomicReference> disconnectClientsToCache = new AtomicReference<>(); + switch (closeType) { + case transferring -> { + disconnectClientsInCurrentCall = FutureUtil.waitForAll(futures); + break; + } + case notWaitDisconnectClients -> { + disconnectClientsInCurrentCall = CompletableFuture.completedFuture(null); + disconnectClientsToCache.set(FutureUtil.waitForAll(futures)); + break; + } + case waitDisconnectClients -> { + disconnectClientsInCurrentCall = FutureUtil.waitForAll(futures); + disconnectClientsToCache.set(disconnectClientsInCurrentCall); + } + } - clientCloseFuture.thenRun(() -> { - // After having disconnected all producers/consumers, close the managed ledger - ledger.asyncClose(new CloseCallback() { - @Override - public void closeComplete(Object ctx) { - if (disconnectClients) { - // Everything is now closed, remove the topic from map - disposeTopic(closeFuture); - } else { - closeFuture.complete(null); - } + CompletableFuture closeFuture = new CompletableFuture<>(); + Runnable closeLedgerAfterCloseClients = (() -> ledger.asyncClose(new CloseCallback() { + @Override + public void closeComplete(Object ctx) { + if (closeType != CloseTypes.transferring) { + // Everything is now closed, remove the topic from map + disposeTopic(closeFuture); + } else { + closeFuture.complete(null); } + } - @Override - public void closeFailed(ManagedLedgerException exception, Object ctx) { - log.error("[{}] Failed to close managed ledger, proceeding anyway.", topic, exception); - if (disconnectClients) { - disposeTopic(closeFuture); - } else { - closeFuture.complete(null); - } + @Override + public void closeFailed(ManagedLedgerException exception, Object ctx) { + log.error("[{}] Failed to close managed ledger, proceeding anyway.", topic, exception); + if (closeType != CloseTypes.transferring) { + disposeTopic(closeFuture); + } else { + closeFuture.complete(null); } - }, null); - }).exceptionally(exception -> { + } + }, null)); + + disconnectClientsInCurrentCall.thenRun(closeLedgerAfterCloseClients).exceptionally(exception -> { log.error("[{}] Error closing topic", topic, exception); unfenceTopicToResume(); closeFuture.completeExceptionally(exception); return null; }); + switch (closeType) { + case transferring -> { + FutureUtil.completeAfterAll(closeFutures.transferring, closeFuture); + break; + } + case notWaitDisconnectClients -> { + FutureUtil.completeAfterAll(closeFutures.transferring, closeFuture); + FutureUtil.completeAfter(closeFutures.notWaitDisconnectClients, closeFuture); + FutureUtil.completeAfterAll(closeFutures.waitDisconnectClients, + closeFuture.thenCompose(ignore -> disconnectClientsToCache.get().exceptionally(ex -> { + // Since the managed ledger has been closed, eat the error of clients disconnection. + log.error("[{}] Closed managed ledger, but disconnect clients failed," + + " this topic will be marked closed", topic, ex); + return null; + }))); + break; + } + case waitDisconnectClients -> { + FutureUtil.completeAfterAll(closeFutures.transferring, closeFuture); + FutureUtil.completeAfter(closeFutures.notWaitDisconnectClients, closeFuture); + FutureUtil.completeAfterAll(closeFutures.waitDisconnectClients, closeFuture); + } + } + return closeFuture; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index 9b8b567af081b..eb31c13b0d528 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -25,6 +25,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.google.common.collect.Sets; @@ -226,7 +227,7 @@ public void testTopicCloseWhenInternalProducerCloseErrorOnce() throws Exception }); } - private void injectMockReplicatorProducerBuilder( + private Runnable injectMockReplicatorProducerBuilder( BiFunction producerDecorator) throws Exception { String cluster2 = pulsar2.getConfig().getClusterName(); @@ -246,7 +247,8 @@ private void injectMockReplicatorProducerBuilder( replicationClients = WhiteboxImpl.getInternalState(brokerService, "replicationClients"); PulsarClientImpl internalClient = (PulsarClientImpl) replicationClients.get(cluster2); PulsarClient spyClient = spy(internalClient); - replicationClients.put(cluster2, spyClient); + assertTrue(replicationClients.remove(cluster2, internalClient)); + assertNull(replicationClients.putIfAbsent(cluster2, spyClient)); // Inject producer decorator. doAnswer(invocation -> { @@ -275,6 +277,12 @@ private void injectMockReplicatorProducerBuilder( }).when(spyProducerBuilder).createAsync(); return spyProducerBuilder; }).when(spyClient).newProducer(any(Schema.class)); + + // Return a cleanup injection task; + return () -> { + assertTrue(replicationClients.remove(cluster2, spyClient)); + assertNull(replicationClients.putIfAbsent(cluster2, internalClient)); + }; } private SpyCursor spyCursor(PersistentTopic persistentTopic, String cursorName) throws Exception { @@ -368,7 +376,7 @@ public void testConcurrencyOfUnloadBundleAndRecreateProducer() throws Exception // If the retry counter is larger than 6, the next creation will be slow enough to close Replicator. final AtomicInteger createProducerCounter = new AtomicInteger(); final int failTimes = 6; - injectMockReplicatorProducerBuilder((producerCnf, originalProducer) -> { + Runnable taskToClearInjection = injectMockReplicatorProducerBuilder((producerCnf, originalProducer) -> { if (topicName.equals(producerCnf.getTopicName())) { // There is a switch to determine create producer successfully or not. if (createProducerCounter.incrementAndGet() > failTimes) { @@ -427,6 +435,7 @@ public void testConcurrencyOfUnloadBundleAndRecreateProducer() throws Exception }); // cleanup. + taskToClearInjection.run(); cleanupTopics(() -> { admin1.topics().delete(topicName); admin2.topics().delete(topicName); @@ -531,7 +540,7 @@ public void testConcurrencyOfUnloadBundleAndRecreateProducer2() throws Exception // If the retry counter is larger than 6, the next creation will be slow enough to close Replicator. final AtomicInteger createProducerCounter = new AtomicInteger(); final int failTimes = 6; - injectMockReplicatorProducerBuilder((producerCnf, originalProducer) -> { + Runnable taskToClearInjection = injectMockReplicatorProducerBuilder((producerCnf, originalProducer) -> { if (topicName.equals(producerCnf.getTopicName())) { // There is a switch to determine create producer successfully or not. if (createProducerCounter.incrementAndGet() > failTimes) { @@ -593,6 +602,7 @@ public void testConcurrencyOfUnloadBundleAndRecreateProducer2() throws Exception }); // cleanup. + taskToClearInjection.run(); cleanupTopics(namespaceName, () -> { admin1.topics().delete(topicName); admin2.topics().delete(topicName); @@ -644,8 +654,9 @@ public void testUnFenceTopicToReuse() throws Exception { assertTrue(replicator2.producer != null && replicator2.producer.isConnected()); }); - // cleanup. + // cleanup the injection. persistentTopic.getProducers().remove(mockProducerName, mockProducer); + // cleanup. producer1.close(); cleanupTopics(() -> { admin1.topics().delete(topicName); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index 44d24668cc381..d523586c2e2d3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -49,6 +49,8 @@ import java.util.List; import java.util.Optional; import java.util.Set; +import java.util.Map; +import java.util.Objects; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; @@ -56,6 +58,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Supplier; +import java.util.stream.Collectors; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.LedgerHandle; @@ -322,6 +325,83 @@ public void testPersistentPartitionedTopicUnload() throws Exception { } } + @DataProvider(name = "closeWithoutWaitingClientDisconnectInFirstBatch") + public Object[][] closeWithoutWaitingClientDisconnectInFirstBatch() { + return new Object[][]{ + new Object[] {true}, + new Object[] {false}, + }; + } + + @Test(dataProvider = "closeWithoutWaitingClientDisconnectInFirstBatch") + public void testConcurrentClose(boolean closeWithoutWaitingClientDisconnectInFirstBatch) throws Exception { + final String topicName = "persistent://prop/ns/concurrentClose"; + final String ns = "prop/ns"; + admin.namespaces().createNamespace(ns, 1); + admin.topics().createNonPartitionedTopic(topicName); + final Topic topic = pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); + List> futureList = + make2ConcurrentBatchesOfClose(topic, 10, closeWithoutWaitingClientDisconnectInFirstBatch); + Map>> futureMap = + futureList.stream().collect(Collectors.groupingBy(Objects::hashCode)); + /** + * The first call: get the return value of "topic.close". + * The other 19 calls: get the cached value which related {@link PersistentTopic#closeFutures}. + */ + assertTrue(futureMap.size() <= 3); + for (List list : futureMap.values()){ + if (list.size() == 1){ + // This is the first call, the future is the return value of `topic.close`. + } else { + // Two types future list: wait client close or not. + assertTrue(list.size() >= 9 && list.size() <= 10); + } + } + } + + private List> make2ConcurrentBatchesOfClose(Topic topic, int tryTimes, + boolean closeWithoutWaitingClientDisconnectInFirstBatch){ + final List> futureList = Collections.synchronizedList(new ArrayList<>()); + final List taskList = new ArrayList<>(); + CountDownLatch allTaskBeginLatch = new CountDownLatch(1); + // Call a batch of close. + for (int i = 0; i < tryTimes; i++) { + Thread thread = new Thread(() -> { + try { + allTaskBeginLatch.await(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + futureList.add(topic.close(closeWithoutWaitingClientDisconnectInFirstBatch)); + }); + thread.start(); + taskList.add(thread); + } + // Call another batch of close. + for (int i = 0; i < tryTimes; i++) { + Thread thread = new Thread(() -> { + try { + allTaskBeginLatch.await(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + futureList.add(topic.close(!closeWithoutWaitingClientDisconnectInFirstBatch)); + }); + thread.start(); + taskList.add(thread); + } + // Wait close task executed. + allTaskBeginLatch.countDown(); + Awaitility.await().atMost(5, TimeUnit.SECONDS).until(()->{ + for (Thread thread : taskList){ + if (thread.isAlive()){ + return false; + } + } + return true; + }); + return futureList; + } @DataProvider(name = "topicAndMetricsLevel") public Object[][] indexPatternTestData() { @@ -331,7 +411,6 @@ public Object[][] indexPatternTestData() { }; } - @Test(dataProvider = "topicAndMetricsLevel") public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean exposeTopicLevelMetrics) throws Exception { PulsarClient client = pulsar.getClient(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java index 6f62589853593..f6fcb12f35939 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java @@ -21,6 +21,7 @@ import com.google.common.util.concurrent.MoreExecutors; import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Objects; @@ -69,6 +70,36 @@ public static CompletableFuture> waitForAll(Stream void completeAfter(final CompletableFuture dest, CompletableFuture src) { + src.whenComplete((v, ex) -> { + if (ex != null) { + dest.completeExceptionally(ex); + } else { + dest.complete(v); + } + }); + } + + /** + * Make the dest future complete after others. {@param dest} is will be completed with a {@link Void} value + * if all the futures of {@param src} is completed, or be completed exceptionally with the same error as the first + * one completed exceptionally future of {@param src}. + */ + public static void completeAfterAll(final CompletableFuture dest, + CompletableFuture... src) { + FutureUtil.waitForAll(Arrays.asList(src)).whenComplete((ignore, ex) -> { + if (ex != null) { + dest.completeExceptionally(ex); + } else { + dest.complete(null); + } + }); + } + /** * Return a future that represents the completion of any future in the provided Collection. * @@ -131,7 +162,7 @@ public static CompletableFuture> waitForAny(Collection waitForAllAndSupportCancel( - Collection> futures) { + Collection> futures) { CompletableFuture[] futuresArray = futures.toArray(new CompletableFuture[0]); CompletableFuture combinedFuture = CompletableFuture.allOf(futuresArray); whenCancelledOrTimedOut(combinedFuture, () -> { From 6fdc0e31bff906446e70965531671389d57e6cda Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 29 Apr 2024 17:24:41 +0800 Subject: [PATCH 129/580] [fix] [test] Fix flaky test ReplicatorTest (#22594) --- .../service/ReplicatorGlobalNSTest.java | 129 +++++++++++++----- .../pulsar/broker/service/ReplicatorTest.java | 121 +++++----------- 2 files changed, 130 insertions(+), 120 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java index eed849ef1a01e..514e0207fbfb1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java @@ -18,18 +18,24 @@ */ package org.apache.pulsar.broker.service; +import static org.testng.Assert.fail; import com.google.common.collect.Sets; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.Future; import lombok.Cleanup; -import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.ProducerImpl; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -41,6 +47,11 @@ import java.lang.reflect.Method; import java.util.concurrent.TimeUnit; +/** + * The tests in this class should be denied in a production pulsar cluster. they are very dangerous, which leads to + * a lot of topic deletion and makes namespace policies being incorrect. + */ +@Slf4j @Test(groups = "broker-impl") public class ReplicatorGlobalNSTest extends ReplicatorTestBase { @@ -81,7 +92,7 @@ public void cleanup() throws Exception { * * @throws Exception */ - @Test + @Test(priority = Integer.MAX_VALUE) public void testRemoveLocalClusterOnGlobalNamespace() throws Exception { log.info("--- Starting ReplicatorTest::testRemoveLocalClusterOnGlobalNamespace ---"); @@ -115,32 +126,88 @@ public void testRemoveLocalClusterOnGlobalNamespace() throws Exception { }); } - @Test - public void testForcefullyTopicDeletion() throws Exception { - log.info("--- Starting ReplicatorTest::testForcefullyTopicDeletion ---"); - - final String namespace = "pulsar/removeClusterTest"; - admin1.namespaces().createNamespace(namespace); - admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1")); - - final String topicName = "persistent://" + namespace + "/topic"; - - @Cleanup - PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()).statsInterval(0, TimeUnit.SECONDS) - .build(); - - ProducerImpl producer1 = (ProducerImpl) client1.newProducer().topic(topicName) - .enableBatching(false).messageRoutingMode(MessageRoutingMode.SinglePartition).create(); - producer1.close(); - - admin1.topics().delete(topicName, true); - - MockedPulsarServiceBaseTest - .retryStrategically((test) -> !pulsar1.getBrokerService().getTopics().containsKey(topicName), 50, 150); - - Assert.assertFalse(pulsar1.getBrokerService().getTopics().containsKey(topicName)); + /** + * This is not a formal operation and can cause serious problems if call it in a production environment. + */ + @Test(priority = Integer.MAX_VALUE - 1) + public void testConfigChange() throws Exception { + log.info("--- Starting ReplicatorTest::testConfigChange ---"); + // This test is to verify that the config change on global namespace is successfully applied in broker during + // runtime. + // Run a set of producer tasks to create the topics + List> results = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + final TopicName dest = TopicName.get(BrokerTestUtil.newUniqueName("persistent://pulsar/ns/topic-" + i)); + + results.add(executor.submit(new Callable() { + @Override + public Void call() throws Exception { + + @Cleanup + MessageProducer producer = new MessageProducer(url1, dest); + log.info("--- Starting producer --- " + url1); + + @Cleanup + MessageConsumer consumer = new MessageConsumer(url1, dest); + log.info("--- Starting Consumer --- " + url1); + + producer.produce(2); + consumer.receive(2); + return null; + } + })); + } + + for (Future result : results) { + try { + result.get(); + } catch (Exception e) { + log.error("exception in getting future result ", e); + fail(String.format("replication test failed with %s exception", e.getMessage())); + } + } + + Thread.sleep(1000L); + // Make sure that the internal replicators map contains remote cluster info + ConcurrentOpenHashMap replicationClients1 = ns1.getReplicationClients(); + ConcurrentOpenHashMap replicationClients2 = ns2.getReplicationClients(); + ConcurrentOpenHashMap replicationClients3 = ns3.getReplicationClients(); + + Assert.assertNotNull(replicationClients1.get("r2")); + Assert.assertNotNull(replicationClients1.get("r3")); + Assert.assertNotNull(replicationClients2.get("r1")); + Assert.assertNotNull(replicationClients2.get("r3")); + Assert.assertNotNull(replicationClients3.get("r1")); + Assert.assertNotNull(replicationClients3.get("r2")); + + // Case 1: Update the global namespace replication configuration to only contains the local cluster itself + admin1.namespaces().setNamespaceReplicationClusters("pulsar/ns", Sets.newHashSet("r1")); + + // Wait for config changes to be updated. + Thread.sleep(1000L); + + // Make sure that the internal replicators map still contains remote cluster info + Assert.assertNotNull(replicationClients1.get("r2")); + Assert.assertNotNull(replicationClients1.get("r3")); + Assert.assertNotNull(replicationClients2.get("r1")); + Assert.assertNotNull(replicationClients2.get("r3")); + Assert.assertNotNull(replicationClients3.get("r1")); + Assert.assertNotNull(replicationClients3.get("r2")); + + // Case 2: Update the configuration back + admin1.namespaces().setNamespaceReplicationClusters("pulsar/ns", Sets.newHashSet("r1", "r2", "r3")); + + // Wait for config changes to be updated. + Thread.sleep(1000L); + + // Make sure that the internal replicators map still contains remote cluster info + Assert.assertNotNull(replicationClients1.get("r2")); + Assert.assertNotNull(replicationClients1.get("r3")); + Assert.assertNotNull(replicationClients2.get("r1")); + Assert.assertNotNull(replicationClients2.get("r3")); + Assert.assertNotNull(replicationClients3.get("r1")); + Assert.assertNotNull(replicationClients3.get("r2")); + + // Case 3: TODO: Once automatic cleanup is implemented, add tests case to verify auto removal of clusters } - - private static final Logger log = LoggerFactory.getLogger(ReplicatorGlobalNSTest.class); - } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index fa12eba1c6611..765727aeac319 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -44,13 +44,11 @@ import java.util.SortedSet; import java.util.TreeSet; import java.util.UUID; -import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -68,6 +66,7 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.BrokerServiceException.NamingException; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -154,88 +153,6 @@ public Object[][] partitionedTopicProvider() { return new Object[][] { { Boolean.TRUE }, { Boolean.FALSE } }; } - @Test(priority = Integer.MAX_VALUE) - public void testConfigChange() throws Exception { - log.info("--- Starting ReplicatorTest::testConfigChange ---"); - // This test is to verify that the config change on global namespace is successfully applied in broker during - // runtime. - // Run a set of producer tasks to create the topics - List> results = new ArrayList<>(); - for (int i = 0; i < 10; i++) { - final TopicName dest = TopicName.get(BrokerTestUtil.newUniqueName("persistent://pulsar/ns/topic-" + i)); - - results.add(executor.submit(new Callable() { - @Override - public Void call() throws Exception { - - @Cleanup - MessageProducer producer = new MessageProducer(url1, dest); - log.info("--- Starting producer --- " + url1); - - @Cleanup - MessageConsumer consumer = new MessageConsumer(url1, dest); - log.info("--- Starting Consumer --- " + url1); - - producer.produce(2); - consumer.receive(2); - return null; - } - })); - } - - for (Future result : results) { - try { - result.get(); - } catch (Exception e) { - log.error("exception in getting future result ", e); - fail(String.format("replication test failed with %s exception", e.getMessage())); - } - } - - Thread.sleep(1000L); - // Make sure that the internal replicators map contains remote cluster info - ConcurrentOpenHashMap replicationClients1 = ns1.getReplicationClients(); - ConcurrentOpenHashMap replicationClients2 = ns2.getReplicationClients(); - ConcurrentOpenHashMap replicationClients3 = ns3.getReplicationClients(); - - Assert.assertNotNull(replicationClients1.get("r2")); - Assert.assertNotNull(replicationClients1.get("r3")); - Assert.assertNotNull(replicationClients2.get("r1")); - Assert.assertNotNull(replicationClients2.get("r3")); - Assert.assertNotNull(replicationClients3.get("r1")); - Assert.assertNotNull(replicationClients3.get("r2")); - - // Case 1: Update the global namespace replication configuration to only contains the local cluster itself - admin1.namespaces().setNamespaceReplicationClusters("pulsar/ns", Sets.newHashSet("r1")); - - // Wait for config changes to be updated. - Thread.sleep(1000L); - - // Make sure that the internal replicators map still contains remote cluster info - Assert.assertNotNull(replicationClients1.get("r2")); - Assert.assertNotNull(replicationClients1.get("r3")); - Assert.assertNotNull(replicationClients2.get("r1")); - Assert.assertNotNull(replicationClients2.get("r3")); - Assert.assertNotNull(replicationClients3.get("r1")); - Assert.assertNotNull(replicationClients3.get("r2")); - - // Case 2: Update the configuration back - admin1.namespaces().setNamespaceReplicationClusters("pulsar/ns", Sets.newHashSet("r1", "r2", "r3")); - - // Wait for config changes to be updated. - Thread.sleep(1000L); - - // Make sure that the internal replicators map still contains remote cluster info - Assert.assertNotNull(replicationClients1.get("r2")); - Assert.assertNotNull(replicationClients1.get("r3")); - Assert.assertNotNull(replicationClients2.get("r1")); - Assert.assertNotNull(replicationClients2.get("r3")); - Assert.assertNotNull(replicationClients3.get("r1")); - Assert.assertNotNull(replicationClients3.get("r2")); - - // Case 3: TODO: Once automatic cleanup is implemented, add tests case to verify auto removal of clusters - } - @Test(timeOut = 10000) public void activeBrokerParse() throws Exception { pulsar1.getConfiguration().setAuthorizationEnabled(true); @@ -253,6 +170,32 @@ public void activeBrokerParse() throws Exception { pulsar1.getConfiguration().setAuthorizationEnabled(false); } + @Test + public void testForcefullyTopicDeletion() throws Exception { + log.info("--- Starting ReplicatorTest::testForcefullyTopicDeletion ---"); + + final String namespace = BrokerTestUtil.newUniqueName("pulsar/removeClusterTest"); + admin1.namespaces().createNamespace(namespace); + admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1")); + + final String topicName = "persistent://" + namespace + "/topic"; + + @Cleanup + PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()).statsInterval(0, TimeUnit.SECONDS) + .build(); + + ProducerImpl producer1 = (ProducerImpl) client1.newProducer().topic(topicName) + .enableBatching(false).messageRoutingMode(MessageRoutingMode.SinglePartition).create(); + producer1.close(); + + admin1.topics().delete(topicName, true); + + MockedPulsarServiceBaseTest + .retryStrategically((test) -> !pulsar1.getBrokerService().getTopics().containsKey(topicName), 50, 150); + + Assert.assertFalse(pulsar1.getBrokerService().getTopics().containsKey(topicName)); + } + @SuppressWarnings("unchecked") @Test(timeOut = 30000) public void testConcurrentReplicator() throws Exception { @@ -1270,7 +1213,7 @@ public void testReplicatedCluster() throws Exception { log.info("--- Starting ReplicatorTest::testReplicatedCluster ---"); - final String namespace = "pulsar/global/repl"; + final String namespace = BrokerTestUtil.newUniqueName("pulsar/global/repl"); final String topicName = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/topic1"); admin1.namespaces().createNamespace(namespace); admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r2", "r3")); @@ -1677,7 +1620,7 @@ public void testReplicatorWithFailedAck() throws Exception { log.info("--- Starting ReplicatorTest::testReplication ---"); - String namespace = "pulsar/global/ns2"; + String namespace = BrokerTestUtil.newUniqueName("pulsar/global/ns"); admin1.namespaces().createNamespace(namespace, Sets.newHashSet("r1")); final TopicName dest = TopicName .get(BrokerTestUtil.newUniqueName("persistent://" + namespace + "/ackFailedTopic")); @@ -1749,7 +1692,7 @@ public void testReplicatorWithFailedAck() throws Exception { @Test public void testWhenUpdateReplicationCluster() throws Exception { log.info("--- testWhenUpdateReplicationCluster ---"); - String namespace = "pulsar/ns2"; + String namespace = BrokerTestUtil.newUniqueName("pulsar/ns");; admin1.namespaces().createNamespace(namespace); admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r2")); final TopicName dest = TopicName.get( @@ -1778,12 +1721,12 @@ public void testWhenUpdateReplicationCluster() throws Exception { @Test public void testReplicatorProducerNotExceed() throws Exception { log.info("--- testReplicatorProducerNotExceed ---"); - String namespace1 = "pulsar/ns11"; + String namespace1 = BrokerTestUtil.newUniqueName("pulsar/ns1"); admin1.namespaces().createNamespace(namespace1); admin1.namespaces().setNamespaceReplicationClusters(namespace1, Sets.newHashSet("r1", "r2")); final TopicName dest1 = TopicName.get( BrokerTestUtil.newUniqueName("persistent://" + namespace1 + "/testReplicatorProducerNotExceed1")); - String namespace2 = "pulsar/ns22"; + String namespace2 = BrokerTestUtil.newUniqueName("pulsar/ns2"); admin2.namespaces().createNamespace(namespace2); admin2.namespaces().setNamespaceReplicationClusters(namespace2, Sets.newHashSet("r1", "r2")); final TopicName dest2 = TopicName.get( From 340d60df0be32ed26586f292a8d24a8a6663aba2 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 29 Apr 2024 20:46:43 +0800 Subject: [PATCH 130/580] [fix] [broker] Fix metrics pulsar_topic_load_failed_count is 0 when load non-persistent topic fails and fix the flaky test testBrokerStatsTopicLoadFailed (#22580) --- .../pulsar/broker/service/BrokerService.java | 3 +- .../broker/service/BrokerServiceTest.java | 159 ++++++++++-------- 2 files changed, 91 insertions(+), 71 deletions(-) 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 1f0cb12258e1d..b08b1a472ca20 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 @@ -1265,7 +1265,8 @@ private CompletableFuture> createNonPersistentTopic(String topic nonPersistentTopic = newTopic(topic, null, this, NonPersistentTopic.class); } catch (Throwable e) { log.warn("Failed to create topic {}", topic, e); - return FutureUtil.failedFuture(e); + topicFuture.completeExceptionally(e); + return topicFuture; } CompletableFuture isOwner = checkTopicNsOwnership(topic); isOwner.thenRun(() -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 8ebba5c9aeabd..5fbe147638026 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -20,20 +20,23 @@ import static org.apache.pulsar.common.naming.SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN; import static org.apache.pulsar.common.naming.SystemTopicNames.TRANSACTION_COORDINATOR_LOG; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +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.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import com.google.gson.Gson; import com.google.gson.JsonArray; import com.google.gson.JsonObject; -import com.google.gson.JsonPrimitive; import io.netty.buffer.ByteBuf; import io.netty.channel.EventLoopGroup; import io.netty.util.concurrent.DefaultThreadFactory; @@ -79,6 +82,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceException; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; import org.apache.pulsar.client.admin.BrokerStats; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -113,7 +117,11 @@ import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.apache.pulsar.compaction.Compactor; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.MockZooKeeper; import org.awaitility.Awaitility; +import org.glassfish.jersey.client.JerseyClient; +import org.glassfish.jersey.client.JerseyClientBuilder; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -1589,82 +1597,93 @@ public void testDynamicConfigurationsForceDeleteTenantAllowed() throws Exception }); } - // this test is disabled since it is flaky - @Test(enabled = false) - public void testBrokerStatsTopicLoadFailed() throws Exception { - admin.namespaces().createNamespace("prop/ns-test"); - - String persistentTopic = "persistent://prop/ns-test/topic1_" + UUID.randomUUID(); - String nonPersistentTopic = "non-persistent://prop/ns-test/topic2_" + UUID.randomUUID(); - - BrokerService brokerService = pulsar.getBrokerService(); - brokerService = Mockito.spy(brokerService); - // mock create persistent topic failed - Mockito - .doAnswer(invocation -> { - CompletableFuture f = new CompletableFuture<>(); - f.completeExceptionally(new RuntimeException("This is an exception")); - return f; - }) - .when(brokerService).getManagedLedgerConfig(Mockito.eq(TopicName.get(persistentTopic))); - - // mock create non-persistent topic failed - Mockito - .doAnswer(inv -> { - CompletableFuture f = new CompletableFuture<>(); - f.completeExceptionally(new RuntimeException("This is an exception")); - return f; - }) - .when(brokerService).checkTopicNsOwnership(Mockito.eq(nonPersistentTopic)); - - - PulsarService pulsarService = pulsar; - Field field = PulsarService.class.getDeclaredField("brokerService"); - field.setAccessible(true); - field.set(pulsarService, brokerService); - - CompletableFuture> producer = pulsarClient.newProducer(Schema.STRING) - .topic(persistentTopic) - .createAsync(); - CompletableFuture> producer1 = pulsarClient.newProducer(Schema.STRING) - .topic(nonPersistentTopic) - .createAsync(); - - producer.whenComplete((v, t) -> { - if (t == null) { - try { - v.close(); - } catch (PulsarClientException e) { - // ignore - } + @Test + public void testMetricsPersistentTopicLoadFails() throws Exception { + final String namespace = "prop/" + UUID.randomUUID().toString().replaceAll("-", ""); + String topic = "persistent://" + namespace + "/topic1_" + UUID.randomUUID(); + admin.namespaces().createNamespace(namespace); + admin.topics().createNonPartitionedTopic(topic); + admin.topics().unload(topic); + + // Inject an error that makes the topic load fails. + AtomicBoolean failMarker = new AtomicBoolean(true); + mockZooKeeper.failConditional(KeeperException.Code.NODEEXISTS, (op, path) -> { + if (failMarker.get() && op.equals(MockZooKeeper.Op.SET) && + path.endsWith(TopicName.get(topic).getPersistenceNamingEncoding())) { + return true; } + return false; }); - producer1.whenComplete((v, t) -> { - if (t == null) { - try { - v.close(); - } catch (PulsarClientException e) { - // ignore - } + + // Do test + CompletableFuture> producer = pulsarClient.newProducer().topic(topic).createAsync(); + JerseyClient httpClient = JerseyClientBuilder.createClient(); + Awaitility.await().until(() -> { + String response = httpClient.target(pulsar.getWebServiceAddress()).path("/metrics/") + .request().get(String.class); + Multimap metricMap = PrometheusMetricsClient.parseMetrics(response); + if (!metricMap.containsKey("pulsar_topic_load_failed_count")) { + return false; + } + double topic_load_failed_count = 0; + for (PrometheusMetricsClient.Metric metric : metricMap.get("pulsar_topic_load_failed_count")) { + topic_load_failed_count += metric.value; } + return topic_load_failed_count >= 1D; }); - Awaitility.waitAtMost(2, TimeUnit.MINUTES).until(() -> { - String json = admin.brokerStats().getMetrics(); - JsonArray metrics = new Gson().fromJson(json, JsonArray.class); - AtomicBoolean flag = new AtomicBoolean(false); - - metrics.forEach(ele -> { - JsonObject obj = ((JsonObject) ele); - JsonObject metrics0 = (JsonObject) obj.get("metrics"); - JsonPrimitive v = (JsonPrimitive) metrics0.get("brk_topic_load_failed_count"); - if (null != v && v.getAsDouble() >= 2D) { - flag.set(true); - } - }); + // Remove the injection. + failMarker.set(false); + // cleanup. + httpClient.close(); + producer.join().close(); + admin.topics().delete(topic); + admin.namespaces().deleteNamespace(namespace); + } - return flag.get(); + @Test + public void testMetricsNonPersistentTopicLoadFails() throws Exception { + final String namespace = "prop/" + UUID.randomUUID().toString().replaceAll("-", ""); + String topic = "non-persistent://" + namespace + "/topic1_" + UUID.randomUUID(); + admin.namespaces().createNamespace(namespace); + + // Inject an error that makes the topic load fails. + // Since we did not set a topic factory name, the "topicFactory" variable is null, inject a mocked + // "topicFactory". + Field fieldTopicFactory = BrokerService.class.getDeclaredField("topicFactory"); + fieldTopicFactory.setAccessible(true); + TopicFactory originalTopicFactory = (TopicFactory) fieldTopicFactory.get(pulsar.getBrokerService()); + assertNull(originalTopicFactory); + TopicFactory mockedTopicFactory = mock(TopicFactory.class); + when(mockedTopicFactory.create(anyString(), any(), any(), any())) + .thenThrow(new RuntimeException("mocked error")); + fieldTopicFactory.set(pulsar.getBrokerService(), mockedTopicFactory); + + // Do test. + CompletableFuture> producer = pulsarClient.newProducer().topic(topic).createAsync(); + JerseyClient httpClient = JerseyClientBuilder.createClient(); + Awaitility.await().until(() -> { + String response = httpClient.target(pulsar.getWebServiceAddress()).path("/metrics/") + .request().get(String.class); + Multimap metricMap = PrometheusMetricsClient.parseMetrics(response); + if (!metricMap.containsKey("pulsar_topic_load_failed_count")) { + return false; + } + double topic_load_failed_count = 0; + for (PrometheusMetricsClient.Metric metric : metricMap.get("pulsar_topic_load_failed_count")) { + topic_load_failed_count += metric.value; + } + return topic_load_failed_count >= 1D; }); + + // Remove the injection. + fieldTopicFactory.set(pulsar.getBrokerService(), null); + + // cleanup. + httpClient.close(); + producer.join().close(); + admin.topics().delete(topic); + admin.namespaces().deleteNamespace(namespace); } @Test From 0fb1a71fcf51e80f235f4b47dada92ff57f17280 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 30 Apr 2024 17:57:36 +0300 Subject: [PATCH 131/580] [improve][ci] Upgrade deprecated GitHub Actions to supported versions (#22620) --- .github/workflows/ci-go-functions.yaml | 2 +- .github/workflows/ci-semantic-pull-request.yml | 2 +- .github/workflows/labeler.yml | 2 +- .github/workflows/pulsar-ci.yaml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/ci-go-functions.yaml b/.github/workflows/ci-go-functions.yaml index 9aa2c896547a9..655503849b1c3 100644 --- a/.github/workflows/ci-go-functions.yaml +++ b/.github/workflows/ci-go-functions.yaml @@ -85,7 +85,7 @@ jobs: uses: ./.github/actions/tune-runner-vm - name: Set up Go - uses: actions/setup-go@v2 + uses: actions/setup-go@v5 with: go-version: ${{ matrix.go-version }} id: go diff --git a/.github/workflows/ci-semantic-pull-request.yml b/.github/workflows/ci-semantic-pull-request.yml index ba421405d5790..15ac85090243c 100644 --- a/.github/workflows/ci-semantic-pull-request.yml +++ b/.github/workflows/ci-semantic-pull-request.yml @@ -34,7 +34,7 @@ jobs: name: Check pull request title runs-on: ubuntu-latest steps: - - uses: amannn/action-semantic-pull-request@v5.0.2 + - uses: amannn/action-semantic-pull-request@v5.5.2 env: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} with: diff --git a/.github/workflows/labeler.yml b/.github/workflows/labeler.yml index 94b148a743443..f10e61c8fd20e 100644 --- a/.github/workflows/labeler.yml +++ b/.github/workflows/labeler.yml @@ -26,4 +26,4 @@ jobs: pull-requests: write runs-on: ubuntu-latest steps: - - uses: actions/labeler@v4 + - uses: actions/labeler@v5 diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 1642b54337fc0..c15d51f9cfcf6 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -888,7 +888,7 @@ jobs: output: 'trivy-results.sarif' - name: Upload Trivy scan results to GitHub Security tab - uses: github/codeql-action/upload-sarif@v2 + uses: github/codeql-action/upload-sarif@v3 if: ${{ github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} with: sarif_file: 'trivy-results.sarif' From d067efcc67f761babd056e1db2b9c7c1dc419a1b Mon Sep 17 00:00:00 2001 From: Rui Fu Date: Wed, 1 May 2024 13:18:05 +0800 Subject: [PATCH 132/580] [fix][fn]make sure the classloader for ContextImpl is `functionClassLoader` in different runtimes (#22501) --- .../pulsar/functions/instance/JavaInstanceRunnable.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java index 21f125d349738..f1b9af00f9d0b 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java @@ -283,13 +283,19 @@ ContextImpl setupContext() throws PulsarClientException { Logger instanceLog = LoggerFactory.getILoggerFactory().getLogger( "function-" + instanceConfig.getFunctionDetails().getName()); Thread currentThread = Thread.currentThread(); + ClassLoader clsLoader = currentThread.getContextClassLoader(); Consumer fatalHandler = throwable -> { this.deathException = throwable; currentThread.interrupt(); }; - return new ContextImpl(instanceConfig, instanceLog, client, secretsProvider, + try { + Thread.currentThread().setContextClassLoader(functionClassLoader); + return new ContextImpl(instanceConfig, instanceLog, client, secretsProvider, collectorRegistry, metricsLabels, this.componentType, this.stats, stateManager, pulsarAdmin, clientBuilder, fatalHandler); + } finally { + Thread.currentThread().setContextClassLoader(clsLoader); + } } public interface AsyncResultConsumer { From 084daf016294ee56496ae36e298d4e8758dc8906 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 1 May 2024 10:29:11 +0200 Subject: [PATCH 133/580] [improve][storage] Periodically rollover Cursor ledgers (#22622) --- .../bookkeeper/mledger/ManagedCursor.java | 8 ++++ .../bookkeeper/mledger/ManagedLedger.java | 5 +++ .../mledger/impl/ManagedCursorImpl.java | 36 +++++++++++++++--- .../mledger/impl/ManagedLedgerImpl.java | 9 +++++ .../mledger/impl/ManagedLedgerBkTest.java | 37 +++++++++++++++++++ .../pulsar/broker/service/BrokerService.java | 1 + 6 files changed, 90 insertions(+), 6 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java index 8372592c851d1..227b5429abf77 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java @@ -870,4 +870,12 @@ default void skipNonRecoverableLedger(long ledgerId){} default boolean isCursorDataFullyPersistable() { return true; } + + /** + * Called by the system to trigger periodic rollover in absence of activity. + */ + default boolean periodicRollover() { + return false; + } + } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java index f91d9ec3f5a02..955a0d7850275 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java @@ -631,6 +631,11 @@ void asyncSetProperties(Map properties, AsyncCallbacks.UpdatePro */ void trimConsumedLedgersInBackground(CompletableFuture promise); + /** + * Rollover cursors in background if needed. + */ + default void rolloverCursorsInBackground() {} + /** * If a ledger is lost, this ledger will be skipped after enabled "autoSkipNonRecoverableData", and the method is * used to delete information about this ledger in the ManagedCursor. diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 69b130a98c869..c2f33639c3d26 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -3113,12 +3113,7 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin lh1.getId()); } - if (shouldCloseLedger(lh1)) { - if (log.isDebugEnabled()) { - log.debug("[{}] Need to create new metadata ledger for cursor {}", ledger.getName(), name); - } - startCreatingNewMetadataLedger(); - } + rolloverLedgerIfNeeded(lh1); mbean.persistToLedger(true); mbean.addWriteCursorLedgerSize(data.length); @@ -3136,6 +3131,35 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin }, null); } + public boolean periodicRollover() { + LedgerHandle lh = cursorLedger; + if (State.Open.equals(STATE_UPDATER.get(this)) + && lh != null && lh.getLength() > 0) { + boolean triggered = rolloverLedgerIfNeeded(lh); + if (triggered) { + log.info("[{}] Periodic rollover triggered for cursor {} (length={} bytes)", + ledger.getName(), name, lh.getLength()); + } else { + log.debug("[{}] Periodic rollover skipped for cursor {} (length={} bytes)", + ledger.getName(), name, lh.getLength()); + + } + return triggered; + } + return false; + } + + boolean rolloverLedgerIfNeeded(LedgerHandle lh1) { + if (shouldCloseLedger(lh1)) { + if (log.isDebugEnabled()) { + log.debug("[{}] Need to create new metadata ledger for cursor {}", ledger.getName(), name); + } + startCreatingNewMetadataLedger(); + return true; + } + return false; + } + void persistPositionToMetaStore(MarkDeleteEntry mdEntry, final VoidCallback callback) { final PositionImpl newPosition = mdEntry.newPosition; STATE_UPDATER.compareAndSet(ManagedCursorImpl.this, State.Open, State.NoLedger); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 70d3c1f74cab3..e5e163127f7b6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -2809,6 +2809,15 @@ public void operationFailed(MetaStoreException e) { } } + @Override + public void rolloverCursorsInBackground() { + if (cursors.hasDurableCursors()) { + executor.execute(() -> { + cursors.forEach(ManagedCursor::periodicRollover); + }); + } + } + /** * @param ledgerId the ledger handle which maybe will be released. * @return if the ledger handle was released. diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java index 0281c8cdd88e3..bb505200ba75e 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java @@ -53,6 +53,7 @@ import org.apache.bookkeeper.mledger.util.ThrowableToStringUtil; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; +import org.awaitility.Awaitility; import org.testng.annotations.Test; public class ManagedLedgerBkTest extends BookKeeperClusterTestCase { @@ -548,6 +549,42 @@ public void testChangeCrcType() throws Exception { } } + @Test + public void testPeriodicRollover() throws Exception { + ManagedLedgerFactoryConfig factoryConf = new ManagedLedgerFactoryConfig(); + factoryConf.setMaxCacheSize(0); + + int rolloverTimeForCursorInSeconds = 5; + + @Cleanup("shutdown") + ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConf); + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setEnsembleSize(1).setWriteQuorumSize(1).setAckQuorumSize(1).setMetadataEnsembleSize(1) + .setMetadataAckQuorumSize(1) + .setLedgerRolloverTimeout(rolloverTimeForCursorInSeconds); + ManagedLedger ledger = factory.open("my-ledger" + testName, config); + ManagedCursor cursor = ledger.openCursor("c1"); + + Position pos = ledger.addEntry("entry-0".getBytes()); + ledger.addEntry("entry-1".getBytes()); + + List entries = cursor.readEntries(2); + assertEquals(2, entries.size()); + entries.forEach(Entry::release); + ManagedCursorImpl cursorImpl = (ManagedCursorImpl) cursor; + assertEquals(ManagedCursorImpl.State.NoLedger, cursorImpl.state); + + // this creates the ledger + cursor.delete(pos); + + Awaitility.await().until(() -> cursorImpl.state == ManagedCursorImpl.State.Open); + + Thread.sleep(rolloverTimeForCursorInSeconds * 1000 + 1000); + + long currentLedgerId = cursorImpl.getCursorLedger(); + assertTrue(cursor.periodicRollover()); + Awaitility.await().until(() -> cursorImpl.getCursorLedger() != currentLedgerId); + } } 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 b08b1a472ca20..dff6c40054060 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 @@ -2118,6 +2118,7 @@ private void checkConsumedLedgers() { Optional.ofNullable(((PersistentTopic) t).getManagedLedger()).ifPresent( managedLedger -> { managedLedger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); + managedLedger.rolloverCursorsInBackground(); } ); } From 7daebaabc0c33ac206a10da68ed548cc6c74bf82 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 1 May 2024 19:13:28 +0300 Subject: [PATCH 134/580] [fix][ci] Fix labeler GitHub Actions workflow, adapt to v5 configuration format (#22628) --- .github/labeler.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/labeler.yml b/.github/labeler.yml index 4fc556900926a..851dd2ed27219 100644 --- a/.github/labeler.yml +++ b/.github/labeler.yml @@ -16,4 +16,5 @@ # under the License. PIP: -- 'pip/**' + - changed-files: + - any-glob-to-any-file: 'pip/**' From a9048639c1c9b60b67fc96e4a40d168bcf86c0b4 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 1 May 2024 11:15:43 -0700 Subject: [PATCH 135/580] [fix] Test was leaving client instance to null (#22631) --- .../pulsar/client/api/SimpleProducerConsumerTest.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index 691f501777eda..70214fe6e3b87 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -4329,10 +4329,6 @@ public static Object[] avroSchemaProvider() { public void testAccessAvroSchemaMetadata(Schema schema) throws Exception { log.info("-- Starting {} test --", methodName); - if (pulsarClient == null) { - pulsarClient = newPulsarClient(lookupUrl.toString(), 0); - } - final String topic = "persistent://my-property/my-ns/accessSchema"; Consumer consumer = pulsarClient.newConsumer(Schema.AUTO_CONSUME()) .topic(topic) @@ -4382,7 +4378,7 @@ public void testAccessAvroSchemaMetadata(Schema schema) throws Exception fail(); } finally { pulsarClient.shutdown(); - pulsarClient = null; + pulsarClient = newPulsarClient(lookupUrl.toString(), 0); admin.schemas().deleteSchema(topic); } } From 4f3cc6c5d277b334b3a6868f9fc641648cd952a3 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 1 May 2024 11:17:19 -0700 Subject: [PATCH 136/580] [feat][broker] PIP-264: Add topic messaging metrics (#22467) --- .../mledger/ManagedLedgerMXBean.java | 10 + .../mledger/impl/ManagedLedgerMBeanImpl.java | 10 + .../mledger/impl/ManagedLedgerMBeanTest.java | 8 + .../apache/pulsar/broker/PulsarService.java | 8 + .../pulsar/broker/service/AbstractTopic.java | 8 + .../broker/stats/OpenTelemetryTopicStats.java | 490 ++++++++++++++++++ .../broker/stats/prometheus/TopicStats.java | 17 + .../pulsar/compaction/CompactionRecord.java | 12 + .../broker/admin/AdminApiOffloadTest.java | 31 +- .../auth/MockedPulsarServiceBaseTest.java | 10 + .../service/BacklogQuotaManagerTest.java | 59 ++- .../service/BrokerServiceThrottlingTest.java | 16 +- .../persistent/DelayedDeliveryTest.java | 21 + .../stats/BrokerOpenTelemetryTestUtil.java | 92 ++++ .../stats/OpenTelemetryTopicStatsTest.java | 145 ++++++ .../broker/testcontext/PulsarTestContext.java | 10 +- .../broker/transaction/TransactionTest.java | 34 +- .../transaction/TransactionTestBase.java | 1 + .../client/api/BrokerServiceLookupTest.java | 3 + .../pulsar/compaction/CompactorTest.java | 45 +- .../OpenTelemetryAttributes.java | 40 ++ 21 files changed, 1039 insertions(+), 31 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStats.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStatsTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java index cb6d3700afe3a..44345c430b7fb 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java @@ -85,6 +85,11 @@ public interface ManagedLedgerMXBean { */ long getAddEntrySucceed(); + /** + * @return the total number of addEntry requests that succeeded + */ + long getAddEntrySucceedTotal(); + /** * @return the number of addEntry requests that failed */ @@ -100,6 +105,11 @@ public interface ManagedLedgerMXBean { */ long getReadEntriesSucceeded(); + /** + * @return the total number of readEntries requests that succeeded + */ + long getReadEntriesSucceededTotal(); + /** * @return the number of readEntries requests that failed */ diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java index 3935828ff3d80..5e5161a29ca79 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java @@ -230,6 +230,11 @@ public long getAddEntrySucceed() { return addEntryOps.getCount(); } + @Override + public long getAddEntrySucceedTotal() { + return addEntryOps.getTotalCount(); + } + @Override public long getAddEntryErrors() { return addEntryOpsFailed.getCount(); @@ -240,6 +245,11 @@ public long getReadEntriesSucceeded() { return readEntriesOps.getCount(); } + @Override + public long getReadEntriesSucceededTotal() { + return readEntriesOps.getTotalCount(); + } + @Override public long getReadEntriesErrors() { return readEntriesOpsFailed.getCount(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanTest.java index 2505db6ec55d7..5f6bd0b7ae64d 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanTest.java @@ -77,10 +77,12 @@ public void simple() throws Exception { assertEquals(mbean.getAddEntryWithReplicasBytesRate(), 0.0); assertEquals(mbean.getAddEntryMessagesRate(), 0.0); assertEquals(mbean.getAddEntrySucceed(), 0); + assertEquals(mbean.getAddEntrySucceedTotal(), 0); assertEquals(mbean.getAddEntryErrors(), 0); assertEquals(mbean.getReadEntriesBytesRate(), 0.0); assertEquals(mbean.getReadEntriesRate(), 0.0); assertEquals(mbean.getReadEntriesSucceeded(), 0); + assertEquals(mbean.getReadEntriesSucceededTotal(), 0); assertEquals(mbean.getReadEntriesErrors(), 0); assertEquals(mbean.getMarkDeleteRate(), 0.0); @@ -105,10 +107,12 @@ public void simple() throws Exception { assertEquals(mbean.getAddEntryWithReplicasBytesRate(), 1600.0); assertEquals(mbean.getAddEntryMessagesRate(), 2.0); assertEquals(mbean.getAddEntrySucceed(), 2); + assertEquals(mbean.getAddEntrySucceedTotal(), 2); assertEquals(mbean.getAddEntryErrors(), 0); assertEquals(mbean.getReadEntriesBytesRate(), 0.0); assertEquals(mbean.getReadEntriesRate(), 0.0); assertEquals(mbean.getReadEntriesSucceeded(), 0); + assertEquals(mbean.getReadEntriesSucceededTotal(), 0); assertEquals(mbean.getReadEntriesErrors(), 0); assertTrue(mbean.getMarkDeleteRate() > 0.0); @@ -134,10 +138,14 @@ public void simple() throws Exception { assertEquals(mbean.getReadEntriesBytesRate(), 600.0); assertEquals(mbean.getReadEntriesRate(), 1.0); assertEquals(mbean.getReadEntriesSucceeded(), 1); + assertEquals(mbean.getReadEntriesSucceededTotal(), 1); assertEquals(mbean.getReadEntriesErrors(), 0); assertEquals(mbean.getNumberOfMessagesInBacklog(), 1); assertEquals(mbean.getMarkDeleteRate(), 0.0); + assertEquals(mbean.getAddEntrySucceed(), 0); + assertEquals(mbean.getAddEntrySucceedTotal(), 2); + factory.shutdown(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 96f3653ea9966..8c910fb91e109 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -109,6 +109,7 @@ import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.SchemaStorageFactory; import org.apache.pulsar.broker.stats.MetricsGenerator; +import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; import org.apache.pulsar.broker.stats.PulsarBrokerOpenTelemetry; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; @@ -252,6 +253,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private MetricsGenerator metricsGenerator; private final PulsarBrokerOpenTelemetry openTelemetry; + private OpenTelemetryTopicStats openTelemetryTopicStats; private TransactionMetadataStoreService transactionMetadataStoreService; private TransactionBufferProvider transactionBufferProvider; @@ -631,6 +633,10 @@ public CompletableFuture closeAsync() { brokerClientSharedTimer.stop(); monotonicSnapshotClock.close(); + if (openTelemetryTopicStats != null) { + openTelemetryTopicStats.close(); + } + asyncCloseFutures.add(EventLoopUtil.shutdownGracefully(ioEventLoopGroup)); @@ -771,6 +777,8 @@ public void start() throws PulsarServerException { config.getDefaultRetentionTimeInMinutes() * 60)); } + openTelemetryTopicStats = new OpenTelemetryTopicStats(this); + localMetadataSynchronizer = StringUtils.isNotBlank(config.getMetadataSyncEventTopic()) ? new PulsarMetadataEventSynchronizer(this, config.getMetadataSyncEventTopic()) : null; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index 44a4ca42cea46..b6ce43b060c6f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -133,6 +133,9 @@ public abstract class AbstractTopic implements Topic, TopicPolicyListener RATE_LIMITED_UPDATER = AtomicLongFieldUpdater.newUpdater(AbstractTopic.class, "publishRateLimitedTimes"); protected volatile long publishRateLimitedTimes = 0L; + private static final AtomicLongFieldUpdater TOTAL_RATE_LIMITED_UPDATER = + AtomicLongFieldUpdater.newUpdater(AbstractTopic.class, "totalPublishRateLimitedCounter"); + protected volatile long totalPublishRateLimitedCounter = 0L; private static final AtomicIntegerFieldUpdater USER_CREATED_PRODUCER_COUNTER_UPDATER = AtomicIntegerFieldUpdater.newUpdater(AbstractTopic.class, "userCreatedProducerCount"); @@ -897,6 +900,7 @@ public void recordAddLatency(long latency, TimeUnit unit) { @Override public long increasePublishLimitedTimes() { + TOTAL_RATE_LIMITED_UPDATER.incrementAndGet(this); return RATE_LIMITED_UPDATER.incrementAndGet(this); } @@ -1185,6 +1189,10 @@ public long getBytesOutCounter() { + sumSubscriptions(AbstractSubscription::getBytesOutCounter); } + public long getTotalPublishRateLimitCounter() { + return TOTAL_RATE_LIMITED_UPDATER.get(this); + } + private long sumSubscriptions(ToLongFunction toCounter) { return getSubscriptions().values().stream() .map(AbstractSubscription.class::cast) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStats.java new file mode 100644 index 0000000000000..1f0735c0ec1f7 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStats.java @@ -0,0 +1,490 @@ +/* + * 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.stats; + +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.BatchCallback; +import io.opentelemetry.api.metrics.ObservableDoubleMeasurement; +import io.opentelemetry.api.metrics.ObservableLongMeasurement; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.AbstractTopic; +import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.common.stats.MetricsUtil; +import org.apache.pulsar.compaction.CompactedTopicContext; +import org.apache.pulsar.compaction.Compactor; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; + +public class OpenTelemetryTopicStats implements AutoCloseable { + + // Replaces pulsar_subscriptions_count + public static final String SUBSCRIPTION_COUNTER = "pulsar.broker.topic.subscription.count"; + private final ObservableLongMeasurement subscriptionCounter; + + // Replaces pulsar_producers_count + public static final String PRODUCER_COUNTER = "pulsar.broker.topic.producer.count"; + private final ObservableLongMeasurement producerCounter; + + // Replaces pulsar_consumers_count + public static final String CONSUMER_COUNTER = "pulsar.broker.topic.consumer.count"; + private final ObservableLongMeasurement consumerCounter; + + // Replaces ['pulsar_rate_in', 'pulsar_in_messages_total'] + public static final String MESSAGE_IN_COUNTER = "pulsar.broker.topic.message.incoming.count"; + private final ObservableLongMeasurement messageInCounter; + + // Replaces ['pulsar_rate_out', 'pulsar_out_messages_total'] + public static final String MESSAGE_OUT_COUNTER = "pulsar.broker.topic.message.outgoing.count"; + private final ObservableLongMeasurement messageOutCounter; + + // Replaces ['pulsar_throughput_in', 'pulsar_in_bytes_total'] + public static final String BYTES_IN_COUNTER = "pulsar.broker.topic.message.incoming.size"; + private final ObservableLongMeasurement bytesInCounter; + + // Replaces ['pulsar_throughput_out', 'pulsar_out_bytes_total'] + public static final String BYTES_OUT_COUNTER = "pulsar.broker.topic.message.outgoing.size"; + private final ObservableLongMeasurement bytesOutCounter; + + // Replaces pulsar_publish_rate_limit_times + public static final String PUBLISH_RATE_LIMIT_HIT_COUNTER = "pulsar.broker.topic.publish.rate.limit.count"; + private final ObservableLongMeasurement publishRateLimitHitCounter; + + // Omitted: pulsar_consumer_msg_ack_rate + + // Replaces pulsar_storage_size + public static final String STORAGE_COUNTER = "pulsar.broker.topic.storage.size"; + private final ObservableLongMeasurement storageCounter; + + // Replaces pulsar_storage_logical_size + public static final String STORAGE_LOGICAL_COUNTER = "pulsar.broker.topic.storage.logical.size"; + private final ObservableLongMeasurement storageLogicalCounter; + + // Replaces pulsar_storage_backlog_size + public static final String STORAGE_BACKLOG_COUNTER = "pulsar.broker.topic.storage.backlog.size"; + private final ObservableLongMeasurement storageBacklogCounter; + + // Replaces pulsar_storage_offloaded_size + public static final String STORAGE_OFFLOADED_COUNTER = "pulsar.broker.topic.storage.offloaded.size"; + private final ObservableLongMeasurement storageOffloadedCounter; + + // Replaces pulsar_storage_backlog_quota_limit + public static final String BACKLOG_QUOTA_LIMIT_SIZE = "pulsar.broker.topic.storage.backlog.quota.limit.size"; + private final ObservableLongMeasurement backlogQuotaLimitSize; + + // Replaces pulsar_storage_backlog_quota_limit_time + public static final String BACKLOG_QUOTA_LIMIT_TIME = "pulsar.broker.topic.storage.backlog.quota.limit.time"; + private final ObservableLongMeasurement backlogQuotaLimitTime; + + // Replaces pulsar_storage_backlog_quota_exceeded_evictions_total + public static final String BACKLOG_EVICTION_COUNTER = "pulsar.broker.topic.storage.backlog.quota.eviction.count"; + private final ObservableLongMeasurement backlogEvictionCounter; + + // Replaces pulsar_storage_backlog_age_seconds + public static final String BACKLOG_QUOTA_AGE = "pulsar.broker.topic.storage.backlog.age"; + private final ObservableLongMeasurement backlogQuotaAge; + + // Replaces pulsar_storage_write_rate + public static final String STORAGE_OUT_COUNTER = "pulsar.broker.topic.storage.entry.outgoing.count"; + private final ObservableLongMeasurement storageOutCounter; + + // Replaces pulsar_storage_read_rate + public static final String STORAGE_IN_COUNTER = "pulsar.broker.topic.storage.entry.incoming.count"; + private final ObservableLongMeasurement storageInCounter; + + // Omitted: pulsar_storage_write_latency_le_* + + // Omitted: pulsar_entry_size_le_* + + // Replaces pulsar_compaction_removed_event_count + public static final String COMPACTION_REMOVED_COUNTER = "pulsar.broker.topic.compaction.removed.message.count"; + private final ObservableLongMeasurement compactionRemovedCounter; + + // Replaces ['pulsar_compaction_succeed_count', 'pulsar_compaction_failed_count'] + public static final String COMPACTION_OPERATION_COUNTER = "pulsar.broker.topic.compaction.operation.count"; + private final ObservableLongMeasurement compactionOperationCounter; + + // Replaces pulsar_compaction_duration_time_in_mills + public static final String COMPACTION_DURATION_SECONDS = "pulsar.broker.topic.compaction.duration"; + private final ObservableDoubleMeasurement compactionDurationSeconds; + + // Replaces pulsar_compaction_read_throughput + public static final String COMPACTION_BYTES_IN_COUNTER = "pulsar.broker.topic.compaction.incoming.size"; + private final ObservableLongMeasurement compactionBytesInCounter; + + // Replaces pulsar_compaction_write_throughput + public static final String COMPACTION_BYTES_OUT_COUNTER = "pulsar.broker.topic.compaction.outgoing.size"; + private final ObservableLongMeasurement compactionBytesOutCounter; + + // Omitted: pulsar_compaction_latency_le_* + + // Replaces pulsar_compaction_compacted_entries_count + public static final String COMPACTION_ENTRIES_COUNTER = "pulsar.broker.topic.compaction.compacted.entry.count"; + private final ObservableLongMeasurement compactionEntriesCounter; + + // Replaces pulsar_compaction_compacted_entries_size + public static final String COMPACTION_BYTES_COUNTER = "pulsar.broker.topic.compaction.compacted.entry.size"; + private final ObservableLongMeasurement compactionBytesCounter; + + // Replaces ['pulsar_txn_tb_active_total', 'pulsar_txn_tb_aborted_total', 'pulsar_txn_tb_committed_total'] + public static final String TRANSACTION_COUNTER = "pulsar.broker.topic.transaction.count"; + private final ObservableLongMeasurement transactionCounter; + + // Replaces pulsar_subscription_delayed + public static final String DELAYED_SUBSCRIPTION_COUNTER = "pulsar.broker.topic.subscription.delayed.entry.count"; + private final ObservableLongMeasurement delayedSubscriptionCounter; + + // Omitted: pulsar_delayed_message_index_size_bytes + + // Omitted: pulsar_delayed_message_index_bucket_total + + // Omitted: pulsar_delayed_message_index_loaded + + // Omitted: pulsar_delayed_message_index_bucket_snapshot_size_bytes + + // Omitted: pulsar_delayed_message_index_bucket_op_count + + // Omitted: pulsar_delayed_message_index_bucket_op_latency_ms + + + private final BatchCallback batchCallback; + private final PulsarService pulsar; + + public OpenTelemetryTopicStats(PulsarService pulsar) { + this.pulsar = pulsar; + var meter = pulsar.getOpenTelemetry().getMeter(); + + subscriptionCounter = meter + .upDownCounterBuilder(SUBSCRIPTION_COUNTER) + .setUnit("{subscription}") + .setDescription("The number of Pulsar subscriptions of the topic served by this broker.") + .buildObserver(); + + producerCounter = meter + .upDownCounterBuilder(PRODUCER_COUNTER) + .setUnit("{producer}") + .setDescription("The number of active producers of the topic connected to this broker.") + .buildObserver(); + + consumerCounter = meter + .upDownCounterBuilder(CONSUMER_COUNTER) + .setUnit("{consumer}") + .setDescription("The number of active consumers of the topic connected to this broker.") + .buildObserver(); + + messageInCounter = meter + .counterBuilder(MESSAGE_IN_COUNTER) + .setUnit("{message}") + .setDescription("The total number of messages received for this topic.") + .buildObserver(); + + messageOutCounter = meter + .counterBuilder(MESSAGE_OUT_COUNTER) + .setUnit("{message}") + .setDescription("The total number of messages read from this topic.") + .buildObserver(); + + bytesInCounter = meter + .counterBuilder(BYTES_IN_COUNTER) + .setUnit("By") + .setDescription("The total number of messages bytes received for this topic.") + .buildObserver(); + + bytesOutCounter = meter + .counterBuilder(BYTES_OUT_COUNTER) + .setUnit("By") + .setDescription("The total number of messages bytes read from this topic.") + .buildObserver(); + + publishRateLimitHitCounter = meter + .counterBuilder(PUBLISH_RATE_LIMIT_HIT_COUNTER) + .setUnit("{event}") + .setDescription("The number of times the publish rate limit is triggered.") + .buildObserver(); + + storageCounter = meter + .upDownCounterBuilder(STORAGE_COUNTER) + .setUnit("By") + .setDescription( + "The total storage size of the messages in this topic, including storage used by replicas.") + .buildObserver(); + + storageLogicalCounter = meter + .upDownCounterBuilder(STORAGE_LOGICAL_COUNTER) + .setUnit("By") + .setDescription("The storage size of the messages in this topic, excluding storage used by replicas.") + .buildObserver(); + + storageBacklogCounter = meter + .upDownCounterBuilder(STORAGE_BACKLOG_COUNTER) + .setUnit("By") + .setDescription("The size of the backlog storage for this topic.") + .buildObserver(); + + storageOffloadedCounter = meter + .upDownCounterBuilder(STORAGE_OFFLOADED_COUNTER) + .setUnit("By") + .setDescription("The total amount of the data in this topic offloaded to the tiered storage.") + .buildObserver(); + + backlogQuotaLimitSize = meter + .upDownCounterBuilder(BACKLOG_QUOTA_LIMIT_SIZE) + .setUnit("By") + .setDescription("The size based backlog quota limit for this topic.") + .buildObserver(); + + backlogQuotaLimitTime = meter + .gaugeBuilder(BACKLOG_QUOTA_LIMIT_TIME) + .ofLongs() + .setUnit("s") + .setDescription("The time based backlog quota limit for this topic.") + .buildObserver(); + + backlogEvictionCounter = meter + .counterBuilder(BACKLOG_EVICTION_COUNTER) + .setUnit("{eviction}") + .setDescription("The number of times a backlog was evicted since it has exceeded its quota.") + .buildObserver(); + + backlogQuotaAge = meter + .gaugeBuilder(BACKLOG_QUOTA_AGE) + .ofLongs() + .setUnit("s") + .setDescription("The age of the oldest unacknowledged message (backlog).") + .buildObserver(); + + storageOutCounter = meter + .counterBuilder(STORAGE_OUT_COUNTER) + .setUnit("{entry}") + .setDescription("The total message batches (entries) written to the storage for this topic.") + .buildObserver(); + + storageInCounter = meter + .counterBuilder(STORAGE_IN_COUNTER) + .setUnit("{entry}") + .setDescription("The total message batches (entries) read from the storage for this topic.") + .buildObserver(); + + compactionRemovedCounter = meter + .counterBuilder(COMPACTION_REMOVED_COUNTER) + .setUnit("{message}") + .setDescription("The total number of messages removed by compaction.") + .buildObserver(); + + compactionOperationCounter = meter + .counterBuilder(COMPACTION_OPERATION_COUNTER) + .setUnit("{operation}") + .setDescription("The total number of compaction operations.") + .buildObserver(); + + compactionDurationSeconds = meter + .upDownCounterBuilder(COMPACTION_DURATION_SECONDS) + .ofDoubles() + .setUnit("s") + .setDescription("The total time duration of compaction operations on the topic.") + .buildObserver(); + + compactionBytesInCounter = meter + .counterBuilder(COMPACTION_BYTES_IN_COUNTER) + .setUnit("By") + .setDescription("The total count of bytes read by the compaction process for this topic.") + .buildObserver(); + + compactionBytesOutCounter = meter + .counterBuilder(COMPACTION_BYTES_OUT_COUNTER) + .setUnit("By") + .setDescription("The total count of bytes written by the compaction process for this topic.") + .buildObserver(); + + compactionEntriesCounter = meter + .counterBuilder(COMPACTION_ENTRIES_COUNTER) + .setUnit("{entry}") + .setDescription("The total number of compacted entries.") + .buildObserver(); + + compactionBytesCounter = meter + .counterBuilder(COMPACTION_BYTES_COUNTER) + .setUnit("By") + .setDescription("The total size of the compacted entries.") + .buildObserver(); + + transactionCounter = meter + .upDownCounterBuilder(TRANSACTION_COUNTER) + .setUnit("{transaction}") + .setDescription("The number of transactions on this topic.") + .buildObserver(); + + delayedSubscriptionCounter = meter + .upDownCounterBuilder(DELAYED_SUBSCRIPTION_COUNTER) + .setUnit("{entry}") + .setDescription("The total number of message batches (entries) delayed for dispatching.") + .buildObserver(); + + batchCallback = meter.batchCallback(() -> pulsar.getBrokerService() + .getTopics() + .values() + .stream() + .map(topicFuture -> topicFuture.getNow(Optional.empty())) + .forEach(topic -> topic.ifPresent(this::recordMetricsForTopic)), + subscriptionCounter, + producerCounter, + consumerCounter, + messageInCounter, + messageOutCounter, + bytesInCounter, + bytesOutCounter, + publishRateLimitHitCounter, + storageCounter, + storageLogicalCounter, + storageBacklogCounter, + storageOffloadedCounter, + backlogQuotaLimitSize, + backlogQuotaLimitTime, + backlogEvictionCounter, + backlogQuotaAge, + storageOutCounter, + storageInCounter, + compactionRemovedCounter, + compactionOperationCounter, + compactionDurationSeconds, + compactionBytesInCounter, + compactionBytesOutCounter, + compactionEntriesCounter, + compactionBytesCounter, + transactionCounter, + delayedSubscriptionCounter); + } + + @Override + public void close() { + batchCallback.close(); + } + + private void recordMetricsForTopic(Topic topic) { + var topicName = TopicName.get(topic.getName()); + var builder = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, topicName.getDomain().toString()) + .put(OpenTelemetryAttributes.PULSAR_TENANT, topicName.getTenant()) + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, topicName.getNamespace()) + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName.getPartitionedTopicName()); + if (topicName.isPartitioned()) { + builder.put(OpenTelemetryAttributes.PULSAR_PARTITION_INDEX, topicName.getPartitionIndex()); + } + var attributes = builder.build(); + + if (topic instanceof AbstractTopic abstractTopic) { + subscriptionCounter.record(abstractTopic.getSubscriptions().size(), attributes); + producerCounter.record(abstractTopic.getProducers().size(), attributes); + consumerCounter.record(abstractTopic.getNumberOfConsumers(), attributes); + + messageInCounter.record(abstractTopic.getMsgInCounter(), attributes); + messageOutCounter.record(abstractTopic.getMsgOutCounter(), attributes); + bytesInCounter.record(abstractTopic.getBytesInCounter(), attributes); + bytesOutCounter.record(abstractTopic.getBytesOutCounter(), attributes); + + publishRateLimitHitCounter.record(abstractTopic.getTotalPublishRateLimitCounter(), attributes); + + // Omitted: consumerMsgAckCounter + } + + if (topic instanceof PersistentTopic persistentTopic) { + var managedLedger = persistentTopic.getManagedLedger(); + var managedLedgerStats = persistentTopic.getManagedLedger().getStats(); + storageCounter.record(managedLedgerStats.getStoredMessagesSize(), attributes); + storageLogicalCounter.record(managedLedgerStats.getStoredMessagesLogicalSize(), attributes); + storageBacklogCounter.record(managedLedger.getEstimatedBacklogSize(), attributes); + storageOffloadedCounter.record(managedLedger.getOffloadedSize(), attributes); + storageInCounter.record(managedLedgerStats.getReadEntriesSucceededTotal(), attributes); + storageOutCounter.record(managedLedgerStats.getAddEntrySucceedTotal(), attributes); + + backlogQuotaLimitSize.record( + topic.getBacklogQuota(BacklogQuota.BacklogQuotaType.destination_storage).getLimitSize(), + attributes); + backlogQuotaLimitTime.record( + topic.getBacklogQuota(BacklogQuota.BacklogQuotaType.message_age).getLimitTime(), + attributes); + backlogQuotaAge.record(topic.getBestEffortOldestUnacknowledgedMessageAgeSeconds(), attributes); + var backlogQuotaMetrics = persistentTopic.getPersistentTopicMetrics().getBacklogQuotaMetrics(); + backlogEvictionCounter.record(backlogQuotaMetrics.getSizeBasedBacklogQuotaExceededEvictionCount(), + Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_BACKLOG_QUOTA_TYPE, "size") + .build()); + backlogEvictionCounter.record(backlogQuotaMetrics.getTimeBasedBacklogQuotaExceededEvictionCount(), + Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_BACKLOG_QUOTA_TYPE, "time") + .build()); + + var txnBuffer = persistentTopic.getTransactionBuffer(); + transactionCounter.record(txnBuffer.getOngoingTxnCount(), Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "active") + .build()); + transactionCounter.record(txnBuffer.getCommittedTxnCount(), Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "committed") + .build()); + transactionCounter.record(txnBuffer.getAbortedTxnCount(), Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "aborted") + .build()); + + Optional.ofNullable(pulsar.getNullableCompactor()) + .map(Compactor::getStats) + .flatMap(compactorMXBean -> compactorMXBean.getCompactionRecordForTopic(topic.getName())) + .ifPresent(compactionRecord -> { + compactionRemovedCounter.record(compactionRecord.getCompactionRemovedEventCount(), attributes); + compactionOperationCounter.record(compactionRecord.getCompactionSucceedCount(), + Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_COMPACTION_STATUS, "success") + .build()); + compactionOperationCounter.record(compactionRecord.getCompactionFailedCount(), + Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_COMPACTION_STATUS, "failure") + .build()); + compactionDurationSeconds.record(MetricsUtil.convertToSeconds( + compactionRecord.getCompactionDurationTimeInMills(), TimeUnit.MILLISECONDS), attributes); + compactionBytesInCounter.record(compactionRecord.getCompactionReadBytes(), attributes); + compactionBytesOutCounter.record(compactionRecord.getCompactionWriteBytes(), attributes); + + persistentTopic.getCompactedTopicContext().map(CompactedTopicContext::getLedger) + .ifPresent(ledger -> { + compactionEntriesCounter.record(ledger.getLastAddConfirmed() + 1, attributes); + compactionBytesCounter.record(ledger.getLength(), attributes); + }); + }); + + var delayedMessages = topic.getSubscriptions().values().stream() + .map(Subscription::getDispatcher) + .filter(Objects::nonNull) + .mapToLong(Dispatcher::getNumberOfDelayedMessages) + .sum(); + delayedSubscriptionCounter.record(delayedMessages, attributes); + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java index 27288291d2969..e8ab7b095dc3c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java @@ -25,28 +25,45 @@ import org.apache.bookkeeper.mledger.util.StatsBuckets; import org.apache.commons.lang3.ArrayUtils; import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; import org.apache.pulsar.broker.stats.prometheus.metrics.PrometheusLabels; import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; import org.apache.pulsar.compaction.CompactionRecord; import org.apache.pulsar.compaction.CompactorMXBean; +import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric; class TopicStats { + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.SUBSCRIPTION_COUNTER) int subscriptionsCount; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.PRODUCER_COUNTER) int producersCount; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.CONSUMER_COUNTER) int consumersCount; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.MESSAGE_IN_COUNTER) double rateIn; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.MESSAGE_OUT_COUNTER) double rateOut; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.BYTES_IN_COUNTER) double throughputIn; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.BYTES_OUT_COUNTER) double throughputOut; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.MESSAGE_IN_COUNTER) long msgInCounter; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.BYTES_IN_COUNTER) long bytesInCounter; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.MESSAGE_OUT_COUNTER) long msgOutCounter; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.BYTES_OUT_COUNTER) long bytesOutCounter; + @PulsarDeprecatedMetric // Can be derived from MESSAGE_IN_COUNTER and BYTES_IN_COUNTER double averageMsgSize; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.TRANSACTION_COUNTER) long ongoingTxnCount; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.TRANSACTION_COUNTER) long abortedTxnCount; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.TRANSACTION_COUNTER) long committedTxnCount; public long msgBacklog; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactionRecord.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactionRecord.java index 09f9f9b00abab..1d2af6638c33a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactionRecord.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactionRecord.java @@ -45,6 +45,8 @@ public class CompactionRecord { private final LongAdder compactionSucceedCount = new LongAdder(); private final LongAdder compactionFailedCount = new LongAdder(); private final LongAdder compactionDurationTimeInMills = new LongAdder(); + private final LongAdder compactionReadBytes = new LongAdder(); + private final LongAdder compactionWriteBytes = new LongAdder(); public final StatsBuckets writeLatencyStats = new StatsBuckets(WRITE_LATENCY_BUCKETS_USEC); public final Rate writeRate = new Rate(); public final Rate readRate = new Rate(); @@ -83,10 +85,12 @@ public void addCompactionEndOp(boolean succeed) { public void addCompactionReadOp(long readableBytes) { readRate.recordEvent(readableBytes); + compactionReadBytes.add(readableBytes); } public void addCompactionWriteOp(long writeableBytes) { writeRate.recordEvent(writeableBytes); + compactionWriteBytes.add(writeableBytes); } public void addCompactionLatencyOp(long latency, TimeUnit unit) { @@ -123,8 +127,16 @@ public double getCompactionReadThroughput() { return readRate.getValueRate(); } + public long getCompactionReadBytes() { + return compactionReadBytes.sum(); + } + public double getCompactionWriteThroughput() { writeRate.calculateRate(); return writeRate.getValueRate(); } + + public long getCompactionWriteBytes() { + return compactionWriteBytes.sum(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java index 95b0d48c69a6c..eac816bd81089 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java @@ -37,6 +37,7 @@ */ package org.apache.pulsar.broker.admin; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -48,6 +49,7 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import io.opentelemetry.api.common.Attributes; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -59,6 +61,9 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil; +import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdminException.ConflictException; import org.apache.pulsar.client.api.MessageId; @@ -71,6 +76,7 @@ import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.OffloadedReadPriority; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -100,6 +106,12 @@ public void setup() throws Exception { admin.namespaces().createNamespace(myNamespace, Set.of("test")); } + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder pulsarTestContextBuilder) { + super.customizeMainPulsarTestContextBuilder(pulsarTestContextBuilder); + pulsarTestContextBuilder.enableOpenTelemetry(true); + } + @AfterMethod(alwaysRun = true) @Override public void cleanup() throws Exception { @@ -125,8 +137,18 @@ private void testOffload(String topicName, String mlName) throws Exception { ManagedLedgerInfo info = pulsar.getManagedLedgerFactory().getManagedLedgerInfo(mlName); assertEquals(info.ledgers.size(), 2); - assertEquals(admin.topics().offloadStatus(topicName).getStatus(), - LongRunningProcessStatus.Status.NOT_RUN); + assertEquals(admin.topics().offloadStatus(topicName).getStatus(), LongRunningProcessStatus.Status.NOT_RUN); + var topicNameObject = TopicName.get(topicName); + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, topicNameObject.getDomain().toString()) + .put(OpenTelemetryAttributes.PULSAR_TENANT, topicNameObject.getTenant()) + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, topicNameObject.getNamespace()) + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicNameObject.getPartitionedTopicName()) + .build(); + // Verify the respective metric is 0 before the offload begins. + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + BrokerOpenTelemetryTestUtil.assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.STORAGE_OFFLOADED_COUNTER, + attributes, actual -> assertThat(actual).isZero()); admin.topics().triggerOffload(topicName, currentId); @@ -164,6 +186,11 @@ private void testOffload(String topicName, String mlName) throws Exception { assertEquals(firstUnoffloadedMessage.getEntryId(), 0); verify(offloader, times(2)).offload(any(), any(), any()); + + // Verify the metrics have been updated. + metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + BrokerOpenTelemetryTestUtil.assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.STORAGE_OFFLOADED_COUNTER, + attributes, actual -> assertThat(actual).isPositive()); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index 0bf096fb5d76a..10d56ce2245f9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -19,8 +19,10 @@ package org.apache.pulsar.broker.auth; import static org.apache.pulsar.broker.BrokerTestUtil.spyWithoutRecordingInvocations; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import com.google.common.collect.Sets; +import io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions; import java.lang.reflect.Field; import java.lang.reflect.Method; import java.net.InetSocketAddress; @@ -739,5 +741,13 @@ protected void reconnectAllConnections() throws Exception { reconnectAllConnections((PulsarClientImpl) pulsarClient); } + protected void assertOtelMetricLongSumValue(String metricName, int value) { + assertThat(pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics()) + .anySatisfy(metric -> OpenTelemetryAssertions.assertThat(metric) + .hasName(metricName) + .hasLongSumSatisfying( + sum -> sum.hasPointsSatisfying(point -> point.hasValue(value)))); + } + private static final Logger log = LoggerFactory.getLogger(MockedPulsarServiceBaseTest.class); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java index f30b7f12b01eb..6be7023b161f1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java @@ -21,6 +21,8 @@ import static java.util.Map.entry; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongGaugeValue; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType.destination_storage; import static org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType.message_age; import static org.assertj.core.api.Assertions.assertThat; @@ -30,6 +32,8 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.google.common.collect.Sets; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; import java.net.URL; import java.time.Duration; import java.util.ArrayList; @@ -45,10 +49,13 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil; +import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metrics; @@ -70,6 +77,8 @@ import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; +import org.apache.pulsar.functions.worker.WorkerConfig; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.awaitility.Awaitility; import org.slf4j.Logger; @@ -94,6 +103,7 @@ public class BacklogQuotaManagerTest { LocalBookkeeperEnsemble bkEnsemble; PrometheusMetricsClient prometheusMetricsClient; + InMemoryMetricReader openTelemetryMetricReader; private static final int TIME_TO_CHECK_BACKLOG_QUOTA = 2; private static final int MAX_ENTRIES_PER_LEDGER = 5; @@ -145,7 +155,9 @@ void setup() throws Exception { config.setTopicLevelPoliciesEnabled(true); config.setForceDeleteNamespaceAllowed(true); - pulsar = new PulsarService(config); + openTelemetryMetricReader = InMemoryMetricReader.create(); + pulsar = new PulsarService(config, new WorkerConfig(), Optional.empty(), exitCode -> { + }, BrokerOpenTelemetryTestUtil.getOpenTelemetrySdkBuilderConsumer(openTelemetryMetricReader)); pulsar.start(); adminUrl = new URL("http://127.0.0.1" + ":" + pulsar.getListenPortHTTP().get()); @@ -709,16 +721,17 @@ public void testTriggerBacklogTimeQuotaWithReader() throws Exception { public void testConsumerBacklogEvictionSizeQuota() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), new HashMap<>()); + var backlogSizeLimit = 10 * 1024; admin.namespaces().setBacklogQuota("prop/ns-quota", BacklogQuota.builder() - .limitSize(10 * 1024) + .limitSize(backlogSizeLimit) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) .build()); @Cleanup PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); - final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); + final String topic1 = BrokerTestUtil.newUniqueName("persistent://prop/ns-quota/topic2"); final String subName1 = "c1"; final String subName2 = "c2"; final int numMsgs = 20; @@ -740,6 +753,21 @@ public void testConsumerBacklogEvictionSizeQuota() throws Exception { assertTrue(stats.getBacklogSize() < 10 * 1024, "Storage size is [" + stats.getStorageSize() + "]"); assertThat(evictionCountMetric("prop/ns-quota", topic1, "size")).isEqualTo(1); assertThat(evictionCountMetric("size")).isEqualTo(1); + + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, "persistent") + .put(OpenTelemetryAttributes.PULSAR_TENANT, "prop") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "prop/ns-quota") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topic1) + .build(); + var metrics = openTelemetryMetricReader.collectAllMetrics(); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.BACKLOG_QUOTA_LIMIT_SIZE, attributes, + backlogSizeLimit); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.BACKLOG_EVICTION_COUNTER, Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_BACKLOG_QUOTA_TYPE, "size") + .build(), + 1); } @Test @@ -812,16 +840,17 @@ private long evictionCountMetric(String quotaType) { public void testConsumerBacklogEvictionTimeQuota() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), new HashMap<>()); + var backlogTimeLimit = TIME_TO_CHECK_BACKLOG_QUOTA; admin.namespaces().setBacklogQuota("prop/ns-quota", BacklogQuota.builder() - .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) + .limitTime(backlogTimeLimit) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) .build(), message_age); @Cleanup PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); - final String topic1 = "persistent://prop/ns-quota/topic3" + UUID.randomUUID(); + final String topic1 = BrokerTestUtil.newUniqueName("persistent://prop/ns-quota/topic3"); final String subName1 = "c1"; final String subName2 = "c2"; final int numMsgs = 14; @@ -844,7 +873,8 @@ public void testConsumerBacklogEvictionTimeQuota() throws Exception { ManagedLedgerImpl ml = (ManagedLedgerImpl) topic1Reference.getManagedLedger(); Position slowConsumerReadPos = ml.getSlowestConsumer().getReadPosition(); - Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA * 2) * 1000); + var delaySeconds = backlogTimeLimit * 2; + Thread.sleep(delaySeconds * 1000); rolloverStats(); TopicStats stats2 = getTopicStats(topic1); @@ -856,6 +886,23 @@ public void testConsumerBacklogEvictionTimeQuota() throws Exception { }); assertEquals(ml.getSlowestConsumer().getReadPosition(), slowConsumerReadPos); + + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, "persistent") + .put(OpenTelemetryAttributes.PULSAR_TENANT, "prop") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "prop/ns-quota") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topic1) + .build(); + var metrics = openTelemetryMetricReader.collectAllMetrics(); + assertMetricLongGaugeValue(metrics, OpenTelemetryTopicStats.BACKLOG_QUOTA_LIMIT_TIME, attributes, + backlogTimeLimit); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.BACKLOG_EVICTION_COUNTER, Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_BACKLOG_QUOTA_TYPE, "time") + .build(), + 1); + assertMetricLongGaugeValue(metrics, OpenTelemetryTopicStats.BACKLOG_QUOTA_AGE, attributes, + value -> assertThat(value).isGreaterThanOrEqualTo(delaySeconds)); } @Test(timeOut = 60000) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java index 0d517c014b315..312bfe0fc8ad7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java @@ -86,11 +86,11 @@ public void testThrottlingLookupRequestSemaphore() throws Exception { var metricName = BrokerService.TOPIC_LOOKUP_LIMIT_METRIC_NAME; // Validate that the configuration has not been overridden. assertThat(admin.brokers().getAllDynamicConfigurations()).doesNotContainKey(configName); - assertLongSumValue(metricName, 50_000); + assertOtelMetricLongSumValue(metricName, 50_000); assertThat(lookupRequestSemaphore.get().availablePermits()).isNotEqualTo(0); admin.brokers().updateDynamicConfiguration(configName, Integer.toString(0)); waitAtMost(1, TimeUnit.SECONDS).until(() -> lookupRequestSemaphore.get().availablePermits() == 0); - assertLongSumValue(metricName, 0); + assertOtelMetricLongSumValue(metricName, 0); } /** @@ -104,19 +104,11 @@ public void testThrottlingTopicLoadRequestSemaphore() throws Exception { var metricName = BrokerService.TOPIC_LOAD_LIMIT_METRIC_NAME; // Validate that the configuration has not been overridden. assertThat(admin.brokers().getAllDynamicConfigurations()).doesNotContainKey(configName); - assertLongSumValue(metricName, 5_000); + assertOtelMetricLongSumValue(metricName, 5_000); assertThat(topicLoadRequestSemaphore.get().availablePermits()).isNotEqualTo(0); admin.brokers().updateDynamicConfiguration(configName, Integer.toString(0)); waitAtMost(1, TimeUnit.SECONDS).until(() -> topicLoadRequestSemaphore.get().availablePermits() == 0); - assertLongSumValue(metricName, 0); - } - - private void assertLongSumValue(String metricName, int value) { - assertThat(pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics()) - .anySatisfy(metric -> assertThat(metric) - .hasName(metricName) - .hasLongSumSatisfying( - sum -> sum.hasPointsSatisfying(point -> point.hasValue(value)))); + assertOtelMetricLongSumValue(metricName, 0); } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java index ae7edde449631..3ca966d210886 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java @@ -24,6 +24,7 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import io.opentelemetry.api.common.Attributes; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -36,6 +37,9 @@ import org.apache.bookkeeper.client.BKException; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil; +import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -45,6 +49,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -69,6 +74,12 @@ public void cleanup() throws Exception { super.internalCleanup(); } + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder pulsarTestContextBuilder) { + super.customizeMainPulsarTestContextBuilder(pulsarTestContextBuilder); + pulsarTestContextBuilder.enableOpenTelemetry(true); + } + @Test public void testDelayedDelivery() throws Exception { String topic = BrokerTestUtil.newUniqueName("testNegativeAcks"); @@ -106,6 +117,16 @@ public void testDelayedDelivery() throws Exception { Message msg = sharedConsumer.receive(100, TimeUnit.MILLISECONDS); assertNull(msg); + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, "persistent") + .put(OpenTelemetryAttributes.PULSAR_TENANT, "public") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "public/default") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, "persistent://public/default/" + topic) + .build(); + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + BrokerOpenTelemetryTestUtil.assertMetricLongSumValue(metrics, + OpenTelemetryTopicStats.DELAYED_SUBSCRIPTION_COUNTER, attributes, 10); + for (int i = 0; i < 10; i++) { msg = failoverConsumer.receive(100, TimeUnit.MILLISECONDS); assertEquals(msg.getValue(), "msg-" + i); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java new file mode 100644 index 0000000000000..cb61677ab953d --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java @@ -0,0 +1,92 @@ +/* + * 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.stats; + +import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; +import io.opentelemetry.sdk.metrics.data.MetricData; +import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; +import java.util.Collection; +import java.util.Map; +import java.util.function.Consumer; +import org.apache.pulsar.opentelemetry.OpenTelemetryService; + +public class BrokerOpenTelemetryTestUtil { + // Creates an OpenTelemetrySdkBuilder customizer for use in tests. + public static Consumer getOpenTelemetrySdkBuilderConsumer( + InMemoryMetricReader reader) { + return sdkBuilder -> { + sdkBuilder.addMeterProviderCustomizer( + (meterProviderBuilder, __) -> meterProviderBuilder.registerMetricReader(reader)); + sdkBuilder.addPropertiesSupplier( + () -> Map.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false", + "otel.java.enabled.resource.providers", "none")); + }; + } + + public static void assertMetricDoubleSumValue(Collection metrics, String metricName, + Attributes attributes, Consumer valueConsumer) { + assertThat(metrics) + .anySatisfy(metric -> assertThat(metric) + .hasName(metricName) + .hasDoubleSumSatisfying(sum -> sum.satisfies( + sumData -> assertThat(sumData.getPoints()).anySatisfy( + point -> { + assertThat(point.getAttributes()).isEqualTo(attributes); + valueConsumer.accept(point.getValue()); + })))); + } + + public static void assertMetricLongSumValue(Collection metrics, String metricName, + Attributes attributes, long expected) { + assertMetricLongSumValue(metrics, metricName, attributes, actual -> assertThat(actual).isEqualTo(expected)); + } + + public static void assertMetricLongSumValue(Collection metrics, String metricName, + Attributes attributes, Consumer valueConsumer) { + assertThat(metrics) + .anySatisfy(metric -> assertThat(metric) + .hasName(metricName) + .hasLongSumSatisfying(sum -> sum.satisfies( + sumData -> assertThat(sumData.getPoints()).anySatisfy( + point -> { + assertThat(point.getAttributes()).isEqualTo(attributes); + valueConsumer.accept(point.getValue()); + })))); + } + + public static void assertMetricLongGaugeValue(Collection metrics, String metricName, + Attributes attributes, long expected) { + assertMetricLongGaugeValue(metrics, metricName, attributes, actual -> assertThat(actual).isEqualTo(expected)); + } + + public static void assertMetricLongGaugeValue(Collection metrics, String metricName, + Attributes attributes, Consumer valueConsumer) { + assertThat(metrics) + .anySatisfy(metric -> assertThat(metric) + .hasName(metricName) + .hasLongGaugeSatisfying(gauge -> gauge.satisfies( + pointData -> assertThat(pointData.getPoints()).anySatisfy( + point -> { + assertThat(point.getAttributes()).isEqualTo(attributes); + valueConsumer.accept(point.getValue()); + })))); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStatsTest.java new file mode 100644 index 0000000000000..c6d07c018c806 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStatsTest.java @@ -0,0 +1,145 @@ +/* + * 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.stats; + +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; +import static org.assertj.core.api.Assertions.assertThat; +import io.opentelemetry.api.common.Attributes; +import java.util.Objects; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.common.policies.data.PublishRate; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class OpenTelemetryTopicStatsTest extends BrokerTestBase { + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.baseSetup(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder builder) { + super.customizeMainPulsarTestContextBuilder(builder); + builder.enableOpenTelemetry(true); + } + + @Test(timeOut = 30_000) + public void testMessagingMetrics() throws Exception { + var topicName = BrokerTestUtil.newUniqueName("persistent://prop/ns-abc/testMessagingMetrics"); + admin.topics().createNonPartitionedTopic(topicName); + + var producerCount = 5; + var messagesPerProducer = 2; + var consumerCount = 3; + var messageCount = producerCount * messagesPerProducer; + + for (int i = 0; i < producerCount; i++) { + var producer = registerCloseable(pulsarClient.newProducer().topic(topicName).create()); + for (int j = 0; j < messagesPerProducer; j++) { + producer.send(String.format("producer-%d-msg-%d", i, j).getBytes()); + } + } + + var cdl = new CountDownLatch(consumerCount); + for (int i = 0; i < consumerCount; i++) { + var consumer = registerCloseable(pulsarClient.newConsumer().topic(topicName) + .subscriptionName("test") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionType(SubscriptionType.Shared) + .subscribe()); + consumer.receiveAsync().orTimeout(100, TimeUnit.MILLISECONDS).handle((__, ex) -> { + cdl.countDown(); + return null; + }); + } + cdl.await(); + + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, "persistent") + .put(OpenTelemetryAttributes.PULSAR_TENANT, "prop") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "prop/ns-abc") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName) + .build(); + + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.SUBSCRIPTION_COUNTER, attributes, 1); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.PRODUCER_COUNTER, attributes, producerCount); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.CONSUMER_COUNTER, attributes, consumerCount); + + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.MESSAGE_IN_COUNTER, attributes, messageCount); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.MESSAGE_OUT_COUNTER, attributes, messageCount); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.BYTES_IN_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.BYTES_OUT_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.STORAGE_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.STORAGE_LOGICAL_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.STORAGE_BACKLOG_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.STORAGE_OUT_COUNTER, attributes, messageCount); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.STORAGE_IN_COUNTER, attributes, messageCount); + } + + @Test(timeOut = 30_000) + public void testPublishRateLimitMetric() throws Exception { + var topicName = BrokerTestUtil.newUniqueName("persistent://prop/ns-abc/testPublishRateLimitMetric"); + admin.topics().createNonPartitionedTopic(topicName); + + var publishRate = new PublishRate(1, -1); + admin.topicPolicies().setPublishRate(topicName, publishRate); + Awaitility.await().until(() -> Objects.equals(publishRate, admin.topicPolicies().getPublishRate(topicName))); + + @Cleanup + var producer = pulsarClient.newProducer().topic(topicName).create(); + producer.send("msg".getBytes()); + + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, "persistent") + .put(OpenTelemetryAttributes.PULSAR_TENANT, "prop") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "prop/ns-abc") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName) + .build(); + + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.PUBLISH_RATE_LIMIT_HIT_COUNTER, attributes, 1); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java index 13209ccfce7d3..dceb18cbeaa9a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java @@ -28,7 +28,6 @@ import java.time.Duration; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.function.Consumer; import java.util.function.Function; @@ -55,6 +54,7 @@ import org.apache.pulsar.broker.resources.TopicResources; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.ServerCnx; +import org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil; import org.apache.pulsar.broker.storage.ManagedLedgerStorage; import org.apache.pulsar.common.util.GracefulExecutorServicesShutdown; import org.apache.pulsar.common.util.PortManager; @@ -67,7 +67,6 @@ import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.apache.pulsar.metadata.impl.MetadataStoreFactoryImpl; import org.apache.pulsar.metadata.impl.ZKMetadataStore; -import org.apache.pulsar.opentelemetry.OpenTelemetryService; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.MockZooKeeper; import org.apache.zookeeper.MockZooKeeperSession; @@ -746,13 +745,8 @@ protected void initializePulsarServices(SpyConfig spyConfig, Builder builder) { Consumer openTelemetrySdkBuilderCustomizer; if (builder.enableOpenTelemetry) { var reader = InMemoryMetricReader.create(); - openTelemetrySdkBuilderCustomizer = sdkBuilder -> { - sdkBuilder.addMeterProviderCustomizer( - (meterProviderBuilder, __) -> meterProviderBuilder.registerMetricReader(reader)); - sdkBuilder.addPropertiesSupplier( - () -> Map.of(OpenTelemetryService.OTEL_SDK_DISABLED_KEY, "false")); - }; openTelemetryMetricReader(reader); + openTelemetrySdkBuilderCustomizer = BrokerOpenTelemetryTestUtil.getOpenTelemetrySdkBuilderConsumer(reader); } else { openTelemetrySdkBuilderCustomizer = null; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index e45924e8bb4f2..ed1b74c46e0f0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -40,6 +40,7 @@ import io.netty.util.HashedWheelTimer; import io.netty.util.Timeout; import io.netty.util.concurrent.DefaultThreadFactory; +import io.opentelemetry.api.common.Attributes; import java.lang.reflect.Field; import java.lang.reflect.Method; import java.nio.charset.StandardCharsets; @@ -79,6 +80,7 @@ import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.reflect.MethodUtils; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.intercept.CounterBrokerInterceptor; @@ -91,6 +93,8 @@ import org.apache.pulsar.broker.service.TransactionBufferSnapshotServiceFactory; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil; +import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; @@ -142,6 +146,7 @@ import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.compaction.CompactionServiceFactory; import org.apache.pulsar.compaction.PulsarCompactionServiceFactory; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStore; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStoreState; @@ -182,7 +187,7 @@ protected void cleanup() throws Exception { @Test public void testTopicTransactionMetrics() throws Exception { - final String topic = "persistent://tnx/ns1/test_transaction_topic"; + final String topic = BrokerTestUtil.newUniqueName("persistent://tnx/ns1/test_transaction_topic"); @Cleanup Producer producer = this.pulsarClient.newProducer() @@ -216,6 +221,33 @@ public void testTopicTransactionMetrics() throws Exception { assertEquals(stats.committedTxnCount, 1); assertEquals(stats.abortedTxnCount, 1); assertEquals(stats.ongoingTxnCount, 1); + + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, "persistent") + .put(OpenTelemetryAttributes.PULSAR_TENANT, "tnx") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "tnx/ns1") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topic) + .build(); + + var metrics = pulsarTestContexts.get(0).getOpenTelemetryMetricReader().collectAllMetrics(); + BrokerOpenTelemetryTestUtil.assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.TRANSACTION_COUNTER, + Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "committed") + .build(), + 1); + BrokerOpenTelemetryTestUtil.assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.TRANSACTION_COUNTER, + Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "aborted") + .build(), + 1); + BrokerOpenTelemetryTestUtil.assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.TRANSACTION_COUNTER, + Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "active") + .build(), + 1); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java index 1ff835732aab5..4ab886492a4eb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java @@ -169,6 +169,7 @@ protected void startBroker() throws Exception { PulsarTestContext.builder() .brokerInterceptor(new CounterBrokerInterceptor()) .spyByDefault() + .enableOpenTelemetry(true) .config(conf); if (i > 0) { testContextBuilder.reuseMockBookkeeperAndMetadataStores(pulsarTestContexts.get(0)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index 7a527a16889e0..2d2019b38eddf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -189,6 +189,9 @@ public void testMultipleBrokerLookup() throws Exception { doReturn(Optional.of(resourceUnit)).when(loadManager2).getLeastLoaded(any(ServiceUnitId.class)); loadManagerField.set(pulsar.getNamespaceService(), new AtomicReference<>(loadManager1)); + // Disable collecting topic stats during this test, as it deadlocks on access to map BrokerService.topics. + pulsar2.getOpenTelemetryTopicStats().close(); + var metricReader = pulsarTestContext.getOpenTelemetryMetricReader(); var lookupRequestSemaphoreField = BrokerService.class.getDeclaredField("lookupRequestSemaphore"); lookupRequestSemaphoreField.setAccessible(true); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java index 71700ef83a443..debc3dd5e3f98 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java @@ -18,13 +18,17 @@ */ package org.apache.pulsar.compaction; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricDoubleSumValue; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.apache.pulsar.client.impl.RawReaderTest.extractKey; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.netty.buffer.ByteBuf; +import io.opentelemetry.api.common.Attributes; import java.util.ArrayList; import java.util.Enumeration; import java.util.HashMap; @@ -37,7 +41,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; - import lombok.Cleanup; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerEntry; @@ -45,9 +48,12 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; @@ -63,6 +69,7 @@ import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.awaitility.Awaitility; import org.mockito.Mockito; import org.testng.Assert; @@ -106,6 +113,12 @@ public void cleanup() throws Exception { compactionScheduler.shutdownNow(); } + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder pulsarTestContextBuilder) { + super.customizeMainPulsarTestContextBuilder(pulsarTestContextBuilder); + pulsarTestContextBuilder.enableOpenTelemetry(true); + } + protected long compact(String topic) throws ExecutionException, InterruptedException { return compactor.compact(topic).get(); } @@ -186,7 +199,7 @@ public void testCompaction() throws Exception { @Test public void testAllCompactedOut() throws Exception { - String topicName = "persistent://my-property/use/my-ns/testAllCompactedOut"; + String topicName = BrokerTestUtil.newUniqueName("persistent://my-property/use/my-ns/testAllCompactedOut"); // set retain null key to true boolean oldRetainNullKey = pulsar.getConfig().isTopicCompactionRetainNullKey(); pulsar.getConfig().setTopicCompactionRetainNullKey(true); @@ -208,6 +221,34 @@ public void testAllCompactedOut() throws Exception { LongRunningProcessStatus.Status.SUCCESS); }); + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, "persistent") + .put(OpenTelemetryAttributes.PULSAR_TENANT, "my-property") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "my-property/use/my-ns") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName) + .build(); + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_REMOVED_COUNTER, attributes, 1); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_OPERATION_COUNTER, Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_COMPACTION_STATUS, "success") + .build(), + 1); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_OPERATION_COUNTER, Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_COMPACTION_STATUS, "failure") + .build(), + 0); + assertMetricDoubleSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_DURATION_SECONDS, attributes, + actual -> assertThat(actual).isPositive()); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_BYTES_IN_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_BYTES_OUT_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_ENTRIES_COUNTER, attributes, 1); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_BYTES_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + producer.newMessage().key("K1").value(null).sendAsync(); producer.flush(); diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index bdb002cb359ff..6088f52f72c61 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -29,4 +29,44 @@ public interface OpenTelemetryAttributes { * {@link OpenTelemetryService}. */ AttributeKey PULSAR_CLUSTER = AttributeKey.stringKey("pulsar.cluster"); + + /** + * The name of the Pulsar namespace. + */ + AttributeKey PULSAR_NAMESPACE = AttributeKey.stringKey("pulsar.namespace"); + + /** + * The name of the Pulsar tenant. + */ + AttributeKey PULSAR_TENANT = AttributeKey.stringKey("pulsar.tenant"); + + /** + * The Pulsar topic domain. + */ + AttributeKey PULSAR_DOMAIN = AttributeKey.stringKey("pulsar.domain"); + + /** + * The name of the Pulsar topic. + */ + AttributeKey PULSAR_TOPIC = AttributeKey.stringKey("pulsar.topic"); + + /** + * The partition index of a Pulsar topic. + */ + AttributeKey PULSAR_PARTITION_INDEX = AttributeKey.longKey("pulsar.partition.index"); + + /** + * The status of the Pulsar transaction. + */ + AttributeKey PULSAR_TRANSACTION_STATUS = AttributeKey.stringKey("pulsar.transaction.status"); + + /** + * The status of the Pulsar compaction operation. + */ + AttributeKey PULSAR_COMPACTION_STATUS = AttributeKey.stringKey("pulsar.compaction.status"); + + /** + * The type of the backlog quota. + */ + AttributeKey PULSAR_BACKLOG_QUOTA_TYPE = AttributeKey.stringKey("pulsar.backlog.quota.type"); } From 6f75569f960e82a4ccb4b95a3297727eb311865f Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Thu, 2 May 2024 21:08:39 +0800 Subject: [PATCH 137/580] [improve][cli] PIP-343: Refactor pulsar-perf to subcommand (#22388) Co-authored-by: Zixuan Liu --- bin/pulsar-perf | 65 +-- bin/pulsar-perf.cmd | 84 +-- .../socket/client/PerformanceClient.java | 209 ++++---- .../pulsar/testclient/BrokerMonitor.java | 62 +-- .../org/apache/pulsar/testclient/CmdBase.java | 80 +++ .../testclient/CmdGenerateDocumentation.java | 66 +-- .../testclient/LoadSimulationClient.java | 89 ++-- .../testclient/LoadSimulationController.java | 88 ++- .../testclient/ManagedLedgerWriter.java | 163 +++--- .../testclient/PerformanceBaseArguments.java | 153 +----- .../testclient/PerformanceConsumer.java | 499 +++++++++--------- .../testclient/PerformanceProducer.java | 413 +++++++-------- .../pulsar/testclient/PerformanceReader.java | 109 ++-- .../PerformanceTopicListArguments.java | 4 + .../testclient/PerformanceTransaction.java | 236 ++++----- .../testclient/ProxyProtocolConverter.java | 35 ++ .../pulsar/testclient/PulsarPerfTestTool.java | 133 +++++ .../socket/client/PerformanceClientTest.java | 31 +- .../testclient/GenerateDocumentionTest.java | 4 +- .../Oauth2PerformanceTransactionTest.java | 2 +- .../testclient/PerfClientUtilsTest.java | 38 +- .../PerformanceBaseArgumentsTest.java | 153 +++--- .../testclient/PerformanceProducerTest.java | 34 +- .../PerformanceTransactionTest.java | 6 +- 24 files changed, 1320 insertions(+), 1436 deletions(-) create mode 100644 pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdBase.java create mode 100644 pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ProxyProtocolConverter.java create mode 100644 pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PulsarPerfTestTool.java diff --git a/bin/pulsar-perf b/bin/pulsar-perf index bdc1dc1ed8b8c..1f6ce97476b4e 100755 --- a/bin/pulsar-perf +++ b/bin/pulsar-perf @@ -84,37 +84,6 @@ add_maven_deps_to_classpath() { fi PULSAR_CLASSPATH=${CLASSPATH}:`cat "${f}"` } -pulsar_help() { - cat < -where command is one of: - produce Run a producer - consume Run a consumer - transaction Run a transaction repeatedly - read Run a topic reader - - websocket-producer Run a websocket producer - - managed-ledger Write directly on managed-ledgers - monitor-brokers Continuously receive broker data and/or load reports - simulation-client Run a simulation server acting as a Pulsar client - simulation-controller Run a simulation controller to give commands to servers - - gen-doc Generate documentation automatically. - - help This help message - -or command is the full name of a class with a defined main() method. - -Environment variables: - PULSAR_LOG_CONF Log4j configuration file (default $DEFAULT_LOG_CONF) - PULSAR_CLIENT_CONF Configuration file for client (default: $DEFAULT_CLIENT_CONF) - PULSAR_EXTRA_OPTS Extra options to be passed to the jvm - PULSAR_EXTRA_CLASSPATH Add extra paths to the pulsar classpath - -These variable can also be set in conf/pulsar_env.sh -EOF -} if [ -d "$PULSAR_HOME/lib" ]; then PULSAR_CLASSPATH="$PULSAR_CLASSPATH:$PULSAR_HOME/lib/*" @@ -162,36 +131,4 @@ OPTS="$OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE" #Change to PULSAR_HOME to support relative paths cd "$PULSAR_HOME" -# if no args specified, show usage -if [ $# = 0 ]; then - pulsar_help; - exit 1; -fi - -# get arguments -COMMAND=$1 -shift - -if [ "$COMMAND" == "produce" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.PerformanceProducer --conf-file $PULSAR_PERFTEST_CONF "$@" -elif [ "$COMMAND" == "consume" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.PerformanceConsumer --conf-file $PULSAR_PERFTEST_CONF "$@" -elif [ "$COMMAND" == "transaction" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.PerformanceTransaction --conf-file $PULSAR_PERFTEST_CONF "$@" -elif [ "$COMMAND" == "read" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.PerformanceReader --conf-file $PULSAR_PERFTEST_CONF "$@" -elif [ "$COMMAND" == "monitor-brokers" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.BrokerMonitor "$@" -elif [ "$COMMAND" == "simulation-client" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.LoadSimulationClient "$@" -elif [ "$COMMAND" == "simulation-controller" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.LoadSimulationController "$@" -elif [ "$COMMAND" == "websocket-producer" ]; then - exec $JAVA $OPTS org.apache.pulsar.proxy.socket.client.PerformanceClient "$@" -elif [ "$COMMAND" == "managed-ledger" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.ManagedLedgerWriter "$@" -elif [ "$COMMAND" == "gen-doc" ]; then - exec $JAVA $OPTS org.apache.pulsar.testclient.CmdGenerateDocumentation "$@" -else - pulsar_help; -fi +exec $JAVA $OPTS org.apache.pulsar.testclient.PulsarPerfTestTool $PULSAR_PERFTEST_CONF "$@" diff --git a/bin/pulsar-perf.cmd b/bin/pulsar-perf.cmd index cf6c25b77e59d..f2b33ef6eb88e 100644 --- a/bin/pulsar-perf.cmd +++ b/bin/pulsar-perf.cmd @@ -72,67 +72,7 @@ set "OPTS=%OPTS% -Dpulsar.log.level=%PULSAR_LOG_LEVEL%" set "OPTS=%OPTS% -Dpulsar.log.root.level=%PULSAR_LOG_ROOT_LEVEL%" set "OPTS=%OPTS% -Dpulsar.log.immediateFlush=%PULSAR_LOG_IMMEDIATE_FLUSH%" -set "COMMAND=%1" - -for /f "tokens=1,* delims= " %%a in ("%*") do set "_args=%%b" - -if "%COMMAND%" == "produce" ( - call :execCmdWithConfigFile org.apache.pulsar.testclient.PerformanceProducer - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "consume" ( - call :execCmdWithConfigFile org.apache.pulsar.testclient.PerformanceConsumer - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "transaction" ( - call :execCmdWithConfigFile org.apache.pulsar.testclient.PerformanceTransaction - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "read" ( - call :execCmdWithConfigFile org.apache.pulsar.testclient.PerformanceReader - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "monitor-brokers" ( - call :execCmd org.apache.pulsar.testclient.BrokerMonitor - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "simulation-client" ( - call :execCmd org.apache.pulsar.testclient.LoadSimulationClient - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "simulation-controller" ( - call :execCmd org.apache.pulsar.testclient.LoadSimulationController - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "websocket-producer" ( - call :execCmd org.apache.pulsar.proxy.socket.client.PerformanceClient - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "managed-ledger" ( - call :execCmd org.apache.pulsar.testclient.ManagedLedgerWriter - exit /B %ERROR_CODE% -) -if "%COMMAND%" == "gen-doc" ( - call :execCmd org.apache.pulsar.testclient.CmdGenerateDocumentation - exit /B %ERROR_CODE% -) - -call :usage -exit /B %ERROR_CODE% - -:execCmdWithConfigFile -"%JAVACMD%" %OPTS% %1 --conf-file "%PULSAR_PERFTEST_CONF%" %_args% -if ERRORLEVEL 1 ( - call :error -) -goto :eof - -:execCmd -"%JAVACMD%" %OPTS% %1 %_args% -if ERRORLEVEL 1 ( - call :error -) -goto :eof +"%JAVACMD%" %OPTS% org.apache.pulsar.testclient.PulsarPerfTestTool "%PULSAR_PERFTEST_CONF%" %* @@ -142,25 +82,3 @@ goto :eof - -:usage -echo Usage: pulsar-perf COMMAND -echo where command is one of: -echo produce Run a producer -echo consume Run a consumer -echo transaction Run a transaction repeatedly -echo read Run a topic reader -echo websocket-producer Run a websocket producer -echo managed-ledger Write directly on managed-ledgers -echo monitor-brokers Continuously receive broker data and/or load reports -echo simulation-client Run a simulation server acting as a Pulsar client -echo simulation-controller Run a simulation controller to give commands to servers -echo gen-doc Generate documentation automatically. -echo help This help message -echo or command is the full name of a class with a defined main() method. -echo Environment variables: -echo PULSAR_LOG_CONF Log4j configuration file (default %PULSAR_HOME%\logs) -echo PULSAR_CLIENT_CONF Configuration file for client (default: %PULSAR_HOME%\conf\client.conf) -echo PULSAR_EXTRA_OPTS Extra options to be passed to the jvm -echo PULSAR_EXTRA_CLASSPATH Add extra paths to the pulsar classpath -goto error diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java b/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java index 9d95d0b74a284..4d73fd9f9b4e3 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java @@ -49,6 +49,7 @@ import org.apache.pulsar.client.api.AuthenticationDataProvider; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.testclient.CmdBase; import org.apache.pulsar.testclient.IMessageFormatter; import org.apache.pulsar.testclient.PerfClientUtils; import org.apache.pulsar.testclient.PositiveNumberParameterConvert; @@ -60,173 +61,159 @@ import org.slf4j.LoggerFactory; import picocli.CommandLine; import picocli.CommandLine.Command; +import picocli.CommandLine.Model.CommandSpec; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; import picocli.CommandLine.Parameters; +import picocli.CommandLine.Spec; -public class PerformanceClient { +@Command(name = "websocket-producer", description = "Test pulsar websocket producer performance.") +public class PerformanceClient extends CmdBase { private static final LongAdder messagesSent = new LongAdder(); private static final LongAdder bytesSent = new LongAdder(); private static final LongAdder totalMessagesSent = new LongAdder(); private static final LongAdder totalBytesSent = new LongAdder(); private static IMessageFormatter messageFormatter = null; - private CommandLine commander; - @Command(description = "Test pulsar websocket producer performance.") - static class Arguments { + @Option(names = { "-cf", "--conf-file" }, description = "Configuration file") + public String confFile; - @Option(names = { "-h", "--help" }, description = "Help message", help = true) - boolean help; + @Option(names = { "-u", "--proxy-url" }, description = "Pulsar Proxy URL, e.g., \"ws://localhost:8080/\"") + public String proxyURL; - @Option(names = { "-cf", "--conf-file" }, description = "Configuration file") - public String confFile; + @Parameters(description = "persistent://tenant/ns/my-topic", arity = "1") + public List topics; - @Option(names = { "-u", "--proxy-url" }, description = "Pulsar Proxy URL, e.g., \"ws://localhost:8080/\"") - public String proxyURL; + @Option(names = { "-r", "--rate" }, description = "Publish rate msg/s across topics") + public int msgRate = 100; - @Parameters(description = "persistent://tenant/ns/my-topic", arity = "1") - public List topics; + @Option(names = { "-s", "--size" }, description = "Message size in byte") + public int msgSize = 1024; - @Option(names = { "-r", "--rate" }, description = "Publish rate msg/s across topics") - public int msgRate = 100; + @Option(names = { "-t", "--num-topic" }, description = "Number of topics", + converter = PositiveNumberParameterConvert.class + ) + public int numTopics = 1; - @Option(names = { "-s", "--size" }, description = "Message size in byte") - public int msgSize = 1024; + @Option(names = { "--auth_plugin" }, description = "Authentication plugin class name", hidden = true) + public String deprecatedAuthPluginClassName; - @Option(names = { "-t", "--num-topic" }, description = "Number of topics", - converter = PositiveNumberParameterConvert.class - ) - public int numTopics = 1; + @Option(names = { "--auth-plugin" }, description = "Authentication plugin class name") + public String authPluginClassName; - @Option(names = { "--auth_plugin" }, description = "Authentication plugin class name", hidden = true) - public String deprecatedAuthPluginClassName; + @Option( + names = { "--auth-params" }, + description = "Authentication parameters, whose format is determined by the implementation " + + "of method `configure` in authentication plugin class, for example \"key1:val1,key2:val2\" " + + "or \"{\"key1\":\"val1\",\"key2\":\"val2\"}\".") + public String authParams; - @Option(names = { "--auth-plugin" }, description = "Authentication plugin class name") - public String authPluginClassName; + @Option(names = { "-m", + "--num-messages" }, description = "Number of messages to publish in total. If <= 0, it will keep" + + " publishing") + public long numMessages = 0; - @Option( - names = { "--auth-params" }, - description = "Authentication parameters, whose format is determined by the implementation " - + "of method `configure` in authentication plugin class, for example \"key1:val1,key2:val2\" " - + "or \"{\"key1\":\"val1\",\"key2\":\"val2\"}\".") - public String authParams; + @Option(names = { "-f", "--payload-file" }, description = "Use payload from a file instead of empty buffer") + public String payloadFilename = null; - @Option(names = { "-m", - "--num-messages" }, description = "Number of messages to publish in total. If <= 0, it will keep" - + " publishing") - public long numMessages = 0; + @Option(names = { "-e", "--payload-delimiter" }, + description = "The delimiter used to split lines when using payload from a file") + // here escaping \n since default value will be printed with the help text + public String payloadDelimiter = "\\n"; - @Option(names = { "-f", "--payload-file" }, description = "Use payload from a file instead of empty buffer") - public String payloadFilename = null; + @Option(names = { "-fp", "--format-payload" }, + description = "Format %%i as a message index in the stream from producer and/or %%t as the timestamp" + + " nanoseconds") + public boolean formatPayload = false; - @Option(names = { "-e", "--payload-delimiter" }, - description = "The delimiter used to split lines when using payload from a file") - // here escaping \n since default value will be printed with the help text - public String payloadDelimiter = "\\n"; + @Option(names = {"-fc", "--format-class"}, description = "Custom Formatter class name") + public String formatterClass = "org.apache.pulsar.testclient.DefaultMessageFormatter"; - @Option(names = { "-fp", "--format-payload" }, - description = "Format %i as a message index in the stream from producer and/or %t as the timestamp" - + " nanoseconds") - public boolean formatPayload = false; + @Option(names = { "-time", + "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep publishing") + public long testTime = 0; - @Option(names = {"-fc", "--format-class"}, description = "Custom Formatter class name") - public String formatterClass = "org.apache.pulsar.testclient.DefaultMessageFormatter"; - - @Option(names = { "-time", - "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep publishing") - public long testTime = 0; + public PerformanceClient() { + super("websocket-producer"); } - public Arguments loadArguments(String[] args) { - Arguments arguments = new Arguments(); - commander = new CommandLine(arguments); - commander.setCommandName("pulsar-perf websocket-producer"); - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } - if (arguments.help) { - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } + @Spec + CommandSpec spec; - if (isBlank(arguments.authPluginClassName) && !isBlank(arguments.deprecatedAuthPluginClassName)) { - arguments.authPluginClassName = arguments.deprecatedAuthPluginClassName; + public void loadArguments() { + CommandLine commander = spec.commandLine(); + + if (isBlank(this.authPluginClassName) && !isBlank(this.deprecatedAuthPluginClassName)) { + this.authPluginClassName = this.deprecatedAuthPluginClassName; } - if (arguments.topics.size() != 1) { + if (this.topics.size() != 1) { System.err.println("Only one topic name is allowed"); commander.usage(commander.getOut()); PerfClientUtils.exit(1); } - if (arguments.confFile != null) { + if (this.confFile != null) { Properties prop = new Properties(System.getProperties()); try { - prop.load(new FileInputStream(arguments.confFile)); + prop.load(new FileInputStream(this.confFile)); } catch (IOException e) { log.error("Error in loading config file"); commander.usage(commander.getOut()); PerfClientUtils.exit(1); } - if (isBlank(arguments.proxyURL)) { + if (isBlank(this.proxyURL)) { String webSocketServiceUrl = prop.getProperty("webSocketServiceUrl"); if (isNotBlank(webSocketServiceUrl)) { - arguments.proxyURL = webSocketServiceUrl; + this.proxyURL = webSocketServiceUrl; } else { String webServiceUrl = isNotBlank(prop.getProperty("webServiceUrl")) ? prop.getProperty("webServiceUrl") : prop.getProperty("serviceUrl"); if (isNotBlank(webServiceUrl)) { if (webServiceUrl.startsWith("ws://") || webServiceUrl.startsWith("wss://")) { - arguments.proxyURL = webServiceUrl; + this.proxyURL = webServiceUrl; } else if (webServiceUrl.startsWith("http://") || webServiceUrl.startsWith("https://")) { - arguments.proxyURL = webServiceUrl.replaceFirst("^http", "ws"); + this.proxyURL = webServiceUrl.replaceFirst("^http", "ws"); } } } } - if (arguments.authPluginClassName == null) { - arguments.authPluginClassName = prop.getProperty("authPlugin", null); + if (this.authPluginClassName == null) { + this.authPluginClassName = prop.getProperty("authPlugin", null); } - if (arguments.authParams == null) { - arguments.authParams = prop.getProperty("authParams", null); + if (this.authParams == null) { + this.authParams = prop.getProperty("authParams", null); } } - if (isBlank(arguments.proxyURL)) { - arguments.proxyURL = "ws://localhost:8080/"; + if (isBlank(this.proxyURL)) { + this.proxyURL = "ws://localhost:8080/"; } - if (!arguments.proxyURL.endsWith("/")) { - arguments.proxyURL += "/"; + if (!this.proxyURL.endsWith("/")) { + this.proxyURL += "/"; } - return arguments; - } - public void runPerformanceTest(Arguments arguments) throws InterruptedException, IOException { + public void runPerformanceTest() throws InterruptedException, IOException { // Read payload data from file if needed - final byte[] payloadBytes = new byte[arguments.msgSize]; + final byte[] payloadBytes = new byte[this.msgSize]; Random random = new Random(0); List payloadByteList = new ArrayList<>(); - if (arguments.payloadFilename != null) { - Path payloadFilePath = Paths.get(arguments.payloadFilename); + if (this.payloadFilename != null) { + Path payloadFilePath = Paths.get(this.payloadFilename); if (Files.notExists(payloadFilePath) || Files.size(payloadFilePath) == 0) { throw new IllegalArgumentException("Payload file doesn't exist or it is empty."); } // here escaping the default payload delimiter to correct value - String delimiter = arguments.payloadDelimiter.equals("\\n") ? "\n" : arguments.payloadDelimiter; + String delimiter = this.payloadDelimiter.equals("\\n") ? "\n" : this.payloadDelimiter; String[] payloadList = new String(Files.readAllBytes(payloadFilePath), StandardCharsets.UTF_8) .split(delimiter); log.info("Reading payloads from {} and {} records read", payloadFilePath.toAbsolutePath(), @@ -235,8 +222,8 @@ public void runPerformanceTest(Arguments arguments) throws InterruptedException, payloadByteList.add(payload.getBytes(StandardCharsets.UTF_8)); } - if (arguments.formatPayload) { - messageFormatter = getMessageFormatter(arguments.formatterClass); + if (this.formatPayload) { + messageFormatter = getMessageFormatter(this.formatterClass); } } else { for (int i = 0; i < payloadBytes.length; ++i) { @@ -248,21 +235,21 @@ public void runPerformanceTest(Arguments arguments) throws InterruptedException, ExecutorService executor = Executors.newCachedThreadPool( new DefaultThreadFactory("pulsar-perf-producer-exec")); HashMap producersMap = new HashMap<>(); - String topicName = arguments.topics.get(0); + String topicName = this.topics.get(0); String restPath = TopicName.get(topicName).getRestPath(); String produceBaseEndPoint = TopicName.get(topicName).isV2() - ? arguments.proxyURL + "ws/v2/producer/" + restPath : arguments.proxyURL + "ws/producer/" + restPath; - for (int i = 0; i < arguments.numTopics; i++) { - String topic = arguments.numTopics > 1 ? produceBaseEndPoint + i : produceBaseEndPoint; + ? this.proxyURL + "ws/v2/producer/" + restPath : this.proxyURL + "ws/producer/" + restPath; + for (int i = 0; i < this.numTopics; i++) { + String topic = this.numTopics > 1 ? produceBaseEndPoint + i : produceBaseEndPoint; URI produceUri = URI.create(topic); WebSocketClient produceClient = new WebSocketClient(new SslContextFactory(true)); ClientUpgradeRequest produceRequest = new ClientUpgradeRequest(); - if (StringUtils.isNotBlank(arguments.authPluginClassName) && StringUtils.isNotBlank(arguments.authParams)) { + if (StringUtils.isNotBlank(this.authPluginClassName) && StringUtils.isNotBlank(this.authParams)) { try { - Authentication auth = AuthenticationFactory.create(arguments.authPluginClassName, - arguments.authParams); + Authentication auth = AuthenticationFactory.create(this.authPluginClassName, + this.authParams); auth.start(); AuthenticationDataProvider authData = auth.getAuthData(); if (authData.hasDataForHttp()) { @@ -296,23 +283,23 @@ public void runPerformanceTest(Arguments arguments) throws InterruptedException, executor.submit(() -> { try { - RateLimiter rateLimiter = RateLimiter.create(arguments.msgRate); + RateLimiter rateLimiter = RateLimiter.create(this.msgRate); long startTime = System.nanoTime(); - long testEndTime = startTime + (long) (arguments.testTime * 1e9); + long testEndTime = startTime + (long) (this.testTime * 1e9); // Send messages on all topics/producers long totalSent = 0; while (true) { for (String topic : producersMap.keySet()) { - if (arguments.testTime > 0 && System.nanoTime() > testEndTime) { + if (this.testTime > 0 && System.nanoTime() > testEndTime) { log.info("------------- DONE (reached the maximum duration: [{} seconds] of production) " - + "--------------", arguments.testTime); + + "--------------", this.testTime); PerfClientUtils.exit(0); } - if (arguments.numMessages > 0) { - if (totalSent >= arguments.numMessages) { + if (this.numMessages > 0) { + if (totalSent >= this.numMessages) { log.trace("------------- DONE (reached the maximum number: [{}] of production) " - + "--------------", arguments.numMessages); + + "--------------", this.numMessages); Thread.sleep(10000); PerfClientUtils.exit(0); } @@ -326,7 +313,7 @@ public void runPerformanceTest(Arguments arguments) throws InterruptedException, } byte[] payloadData; - if (arguments.payloadFilename != null) { + if (this.payloadFilename != null) { if (messageFormatter != null) { payloadData = messageFormatter.formatMessage("", totalSent, payloadByteList.get(random.nextInt(payloadByteList.size()))); @@ -416,16 +403,16 @@ static IMessageFormatter getMessageFormatter(String formatterClass) { } } - public static void main(String[] args) throws Exception { - PerformanceClient test = new PerformanceClient(); - Arguments arguments = test.loadArguments(args); + @Override + public void run() throws Exception { + loadArguments(); PerfClientUtils.printJVMInformation(log); long start = System.nanoTime(); Runtime.getRuntime().addShutdownHook(new Thread(() -> { printAggregatedThroughput(start); printAggregatedStats(); })); - test.runPerformanceTest(arguments); + runPerformanceTest(); } private class Tuple { diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/BrokerMonitor.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/BrokerMonitor.java index d195e8fd45695..a2f5b382c7b8f 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/BrokerMonitor.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/BrokerMonitor.java @@ -46,17 +46,17 @@ import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; -import picocli.CommandLine.ScopeType; /** * Monitors brokers and prints to the console information about their system resource usages, their topic and bundle * counts, their message rates, and other metrics. */ -public class BrokerMonitor { +@Command(name = "monitor-brokers", + description = "Monitors brokers and prints to the console information about their system " + + "resource usages, \ntheir topic and bundle counts, their message rates, and other metrics.") +public class BrokerMonitor extends CmdBase { private static final Logger log = LoggerFactory.getLogger(BrokerMonitor.class); private static final String BROKER_ROOT = "/loadbalance/brokers"; @@ -88,6 +88,7 @@ public class BrokerMonitor { private Map loadData; private static final FixedColumnLengthTableMaker localTableMaker = new FixedColumnLengthTableMaker(); + static { // Makes the table length about 120. localTableMaker.elementLength = 14; @@ -95,6 +96,7 @@ public class BrokerMonitor { } private static final FixedColumnLengthTableMaker globalTableMaker = new FixedColumnLengthTableMaker(); + static { globalTableMaker.decimalFormatter = "%.2f"; globalTableMaker.topBorder = '*'; @@ -126,7 +128,7 @@ private static void initRow(final Object[] row, final Object... elements) { // Helper method to initialize rows which hold message data. private static void initMessageRow(final Object[] row, final double messageRateIn, final double messageRateOut, - final double messageThroughputIn, final double messageThroughputOut) { + final double messageThroughputIn, final double messageThroughputOut) { initRow(row, messageRateIn, messageRateOut, messageRateIn + messageRateOut, messageThroughputIn / 1024, messageThroughputOut / 1024, (messageThroughputIn + messageThroughputOut) / 1024); @@ -391,7 +393,7 @@ private synchronized void printLoadReport(final String broker, final LoadReport // Print the broker data in a tabular form for a broker using ModularLoadManagerImpl. private synchronized void printBrokerData(final String broker, final LocalBrokerData localBrokerData, - final TimeAverageBrokerData timeAverageData) { + final TimeAverageBrokerData timeAverageData) { loadData.put(broker, localBrokerData); // Initialize the constant rows. @@ -435,19 +437,15 @@ private synchronized void printBrokerData(final String broker, final LocalBroker } } - // picocli arguments class. - @Command(description = "Monitors brokers and prints to the console information about their system " - + "resource usages, \ntheir topic and bundle counts, their message rates, and other metrics.", - showDefaultValues = true, scope = ScopeType.INHERIT) - private static class Arguments { - @Option(names = { "-h", "--help" }, description = "Help message", help = true) - boolean help; + @Option(names = {"--connect-string"}, description = "Zookeeper or broker connect string", required = true) + public String connectString = null; + + @Option(names = {"--extensions"}, description = "true to monitor Load Balance Extensions.") + boolean extensions = false; - @Option(names = { "--connect-string" }, description = "Zookeeper or broker connect string", required = true) - public String connectString = null; - @Option(names = { "--extensions" }, description = "true to monitor Load Balance Extensions.") - boolean extensions = false; + public BrokerMonitor() { + super("monitor-brokers"); } /** @@ -456,6 +454,7 @@ private static class Arguments { * @param zkClient Client to create this from. */ public BrokerMonitor(final ZooKeeper zkClient) { + super("monitor-brokers"); loadData = new ConcurrentHashMap<>(); this.zkClient = zkClient; } @@ -479,6 +478,7 @@ public void start() { private TableView brokerLoadDataTableView; private BrokerMonitor(String brokerServiceUrl) { + super("monitor-brokers"); try { PulsarClient client = PulsarClient.builder() .memoryLimit(0, SizeUnit.BYTES) @@ -541,32 +541,16 @@ private void startBrokerLoadDataStoreMonitor() { } } - /** - * Run a monitor from command line arguments. - * - * @param args Arguments for the monitor. - */ - public static void main(String[] args) throws Exception { - final Arguments arguments = new Arguments(); - final CommandLine commander = new CommandLine(arguments); - commander.setCommandName("pulsar-perf monitor-brokers"); - - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } - - - if (arguments.extensions) { - final BrokerMonitor monitor = new BrokerMonitor(arguments.connectString); + @Override + public void run() throws Exception { + if (this.extensions) { + final BrokerMonitor monitor = new BrokerMonitor(this.connectString); monitor.startBrokerLoadDataStoreMonitor(); } else { - final ZooKeeper zkClient = new ZooKeeper(arguments.connectString, ZOOKEEPER_TIMEOUT_MILLIS, null); + final ZooKeeper zkClient = new ZooKeeper(this.connectString, ZOOKEEPER_TIMEOUT_MILLIS, null); final BrokerMonitor monitor = new BrokerMonitor(zkClient); monitor.start(); } } + } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdBase.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdBase.java new file mode 100644 index 0000000000000..6d5796ad5dda7 --- /dev/null +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdBase.java @@ -0,0 +1,80 @@ +/* + * 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.testclient; + +import java.util.concurrent.Callable; +import picocli.CommandLine; + +public abstract class CmdBase implements Callable { + private final CommandLine commander; + + public CmdBase(String cmdName) { + commander = new CommandLine(this); + commander.setCommandName(cmdName); + } + + public boolean run(String[] args) { + return commander.execute(args) == 0; + } + + public void parse(String[] args) { + commander.parseArgs(args); + } + + /** + * Validate the CLI arguments. Default implementation provides validation for the common arguments. + * Each subclass should call super.validate() and provide validation code specific to the sub-command. + * @throws Exception + */ + public void validate() throws Exception { + } + + // Picocli entrypoint. + @Override + public Integer call() throws Exception { + validate(); + run(); + return 0; + } + + public abstract void run() throws Exception; + + + protected CommandLine getCommander() { + return commander; + } + + protected void addCommand(String name, Object cmd) { + commander.addSubcommand(name, cmd); + } + + protected void addCommand(String name, Object cmd, String... aliases) { + commander.addSubcommand(name, cmd, aliases); + } + + protected class ParameterException extends CommandLine.ParameterException { + public ParameterException(String msg) { + super(commander, msg); + } + + public ParameterException(String msg, Throwable e) { + super(commander, msg, e); + } + } +} diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdGenerateDocumentation.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdGenerateDocumentation.java index 6ff0ab296a684..d2e08e2cc8664 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdGenerateDocumentation.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/CmdGenerateDocumentation.java @@ -24,57 +24,41 @@ import java.util.List; import java.util.Map; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.proxy.socket.client.PerformanceClient; import picocli.CommandLine; import picocli.CommandLine.Command; +import picocli.CommandLine.Model.CommandSpec; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; -import picocli.CommandLine.ScopeType; +import picocli.CommandLine.Spec; @Slf4j -public class CmdGenerateDocumentation { +@Command(name = "gen-doc", description = "Generate documentation automatically.") +public class CmdGenerateDocumentation extends CmdBase{ - @Command(description = "Generate documentation automatically.", showDefaultValues = true, scope = ScopeType.INHERIT) - static class Arguments { - - @Option(names = {"-h", "--help"}, description = "Help message", help = true) - boolean help; - - @Option(names = {"-n", "--command-names"}, description = "List of command names") - private List commandNames = new ArrayList<>(); + @Option(names = {"-n", "--command-names"}, description = "List of command names") + private List commandNames = new ArrayList<>(); + public CmdGenerateDocumentation() { + super("gen-doc"); } - public static void main(String[] args) throws Exception { - final Arguments arguments = new Arguments(); - CommandLine commander = new CommandLine(arguments); - commander.setCommandName("pulsar-perf gen-doc"); - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } + @Spec + CommandSpec spec; - - if (arguments.help) { - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } + @Override + public void run() throws Exception { + CommandLine commander = spec.commandLine(); Map> cmdClassMap = new LinkedHashMap<>(); - cmdClassMap.put("produce", Class.forName("org.apache.pulsar.testclient.PerformanceProducer$Arguments")); - cmdClassMap.put("consume", Class.forName("org.apache.pulsar.testclient.PerformanceConsumer$Arguments")); - cmdClassMap.put("transaction", Class.forName("org.apache.pulsar.testclient.PerformanceTransaction$Arguments")); - cmdClassMap.put("read", Class.forName("org.apache.pulsar.testclient.PerformanceReader$Arguments")); - cmdClassMap.put("monitor-brokers", Class.forName("org.apache.pulsar.testclient.BrokerMonitor$Arguments")); - cmdClassMap.put("simulation-client", - Class.forName("org.apache.pulsar.testclient.LoadSimulationClient$MainArguments")); - cmdClassMap.put("simulation-controller", - Class.forName("org.apache.pulsar.testclient.LoadSimulationController$MainArguments")); - cmdClassMap.put("websocket-producer", - Class.forName("org.apache.pulsar.proxy.socket.client.PerformanceClient$Arguments")); - cmdClassMap.put("managed-ledger", Class.forName("org.apache.pulsar.testclient.ManagedLedgerWriter$Arguments")); + cmdClassMap.put("produce", PerformanceProducer.class); + cmdClassMap.put("consume", PerformanceConsumer.class); + cmdClassMap.put("transaction", PerformanceTransaction.class); + cmdClassMap.put("read", PerformanceReader.class); + cmdClassMap.put("monitor-brokers", BrokerMonitor.class); + cmdClassMap.put("simulation-client", LoadSimulationClient.class); + cmdClassMap.put("simulation-controller", LoadSimulationController.class); + cmdClassMap.put("websocket-producer", PerformanceClient.class); + cmdClassMap.put("managed-ledger", ManagedLedgerWriter.class); for (Map.Entry> entry : cmdClassMap.entrySet()) { String cmd = entry.getKey(); @@ -84,12 +68,12 @@ public static void main(String[] args) throws Exception { commander.addSubcommand(cmd, constructor.newInstance()); } - if (arguments.commandNames.size() == 0) { + if (this.commandNames.size() == 0) { for (Map.Entry cmd : commander.getSubcommands().entrySet()) { generateDocument(cmd.getKey(), commander); } } else { - for (String commandName : arguments.commandNames) { + for (String commandName : this.commandNames) { generateDocument(commandName, commander); } } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java index 42d2f0dd5143e..c58de64056a66 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java @@ -41,20 +41,20 @@ import org.apache.pulsar.client.api.MessageListener; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SizeUnit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; -import picocli.CommandLine.ScopeType; /** * LoadSimulationClient is used to simulate client load by maintaining producers and consumers for topics. Instances of * this class are controlled across a network via LoadSimulationController. */ -public class LoadSimulationClient { +@Command(name = "simulation-client", + description = "Simulate client load by maintaining producers and consumers for topics.") +public class LoadSimulationClient extends CmdBase{ private static final Logger log = LoggerFactory.getLogger(LoadSimulationClient.class); // Values for command encodings. @@ -65,7 +65,7 @@ public class LoadSimulationClient { public static final byte STOP_GROUP_COMMAND = 4; public static final byte FIND_COMMAND = 5; - private final ExecutorService executor; + private ExecutorService executor; // Map from a message size to a cached byte[] of that size. private final Map payloadCache; @@ -73,12 +73,10 @@ public class LoadSimulationClient { private final Map topicsToTradeUnits; // Pulsar admin to create namespaces with. - private final PulsarAdmin admin; + private PulsarAdmin admin; // Pulsar client to create producers and consumers with. - private final PulsarClient client; - - private final int port; + private PulsarClient client; // A TradeUnit is a Consumer and Producer pair. The rate of message // consumption as well as size may be changed at @@ -172,22 +170,17 @@ public void start() throws Exception { } // picocli arguments for starting a LoadSimulationClient. - @Command(description = "Simulate client load by maintaining producers and consumers for topics.", - showDefaultValues = true, scope = ScopeType.INHERIT) - private static class MainArguments { - @Option(names = { "-h", "--help" }, description = "Help message", help = true) - boolean help; - @Option(names = { "--port" }, description = "Port to listen on for controller", required = true) - public int port; + @Option(names = { "--port" }, description = "Port to listen on for controller", required = true) + public int port; - @Option(names = { "--service-url" }, description = "Pulsar Service URL", required = true) - public String serviceURL; + @Option(names = { "--service-url" }, description = "Pulsar Service URL", required = true) + public String serviceURL; + + @Option(names = { "-ml", "--memory-limit", }, description = "Configure the Pulsar client memory limit " + + "(eg: 32M, 64M)", converter = ByteUnitToLongConverter.class) + public long memoryLimit = 0L; - @Option(names = { "-ml", "--memory-limit", }, description = "Configure the Pulsar client memory limit " - + "(eg: 32M, 64M)", converter = ByteUnitToLongConverter.class) - public long memoryLimit = 0L; - } // Configuration class for initializing or modifying TradeUnits. private static class TradeConfiguration { @@ -312,54 +305,40 @@ private void handle(final byte command, final DataInputStream inputStream, final private static final MessageListener ackListener = Consumer::acknowledgeAsync; /** - * Create a LoadSimulationClient with the given picocli arguments. + * Create a LoadSimulationClient with the given picocli this. * - * @param arguments - * Arguments to configure this from. */ - public LoadSimulationClient(final MainArguments arguments) throws Exception { + public LoadSimulationClient() throws PulsarClientException { + super("simulation-client"); payloadCache = new ConcurrentHashMap<>(); topicsToTradeUnits = new ConcurrentHashMap<>(); - - admin = PulsarAdmin.builder() - .serviceHttpUrl(arguments.serviceURL) - .build(); - client = PulsarClient.builder() - .memoryLimit(arguments.memoryLimit, SizeUnit.BYTES) - .serviceUrl(arguments.serviceURL) - .connectionsPerBroker(4) - .ioThreads(Runtime.getRuntime().availableProcessors()) - .statsInterval(0, TimeUnit.SECONDS) - .build(); - port = arguments.port; - executor = Executors.newCachedThreadPool(new DefaultThreadFactory("test-client")); } /** - * Start a client with command line arguments. + * Start a client with command line this. * - * @param args - * Command line arguments to pass in. */ - public static void main(String[] args) throws Exception { - final MainArguments mainArguments = new MainArguments(); - CommandLine commander = new CommandLine(mainArguments); - commander.setCommandName("pulsar-perf simulation-client"); - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } + @Override + public void run() throws Exception { + admin = PulsarAdmin.builder() + .serviceHttpUrl(this.serviceURL) + .build(); + client = PulsarClient.builder() + .memoryLimit(this.memoryLimit, SizeUnit.BYTES) + .serviceUrl(this.serviceURL) + .connectionsPerBroker(4) + .ioThreads(Runtime.getRuntime().availableProcessors()) + .statsInterval(0, TimeUnit.SECONDS) + .build(); + executor = Executors.newCachedThreadPool(new DefaultThreadFactory("test-client")); PerfClientUtils.printJVMInformation(log); - (new LoadSimulationClient(mainArguments)).run(); + this.start(); } /** * Start listening for controller commands to create producers and consumers. */ - public void run() throws Exception { + public void start() throws Exception { final ServerSocket serverSocket = new ServerSocket(port); while (true) { diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java index 94186c581ebe4..99f443f26d7d2 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java @@ -55,51 +55,41 @@ import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; import picocli.CommandLine.Parameters; -import picocli.CommandLine.ScopeType; /** * This class provides a shell for the user to dictate how simulation clients should incur load. */ -public class LoadSimulationController { +@Command(name = "simulation-controller", + description = "Provides a shell for the user to dictate how simulation clients should " + + "incur load.") +public class LoadSimulationController extends CmdBase{ private static final Logger log = LoggerFactory.getLogger(LoadSimulationController.class); // Input streams for each client to send commands through. - private final DataInputStream[] inputStreams; + private DataInputStream[] inputStreams; // Output streams for each client to receive information from. - private final DataOutputStream[] outputStreams; + private DataOutputStream[] outputStreams; // client host names. - private final String[] clients; + private String[] clients; - // Port clients are listening on. - private final int clientPort; - - // The ZooKeeper cluster to run on. - private final String cluster; - - private final Random random; + private Random random; private static final ExecutorService threadPool = Executors.newCachedThreadPool(); // picocli arguments for starting a controller via main. - @Command(description = "Provides a shell for the user to dictate how simulation clients should " - + "incur load.", showDefaultValues = true, scope = ScopeType.INHERIT) - private static class MainArguments { - @Option(names = { "-h", "--help" }, description = "Help message", help = true) - boolean help; - @Option(names = { "--cluster" }, description = "Cluster to test on", required = true) - String cluster; + @Option(names = { "--cluster" }, description = "Cluster to test on", required = true) + String cluster; - @Option(names = { "--clients" }, description = "Comma separated list of client hostnames", required = true) - String clientHostNames; + @Option(names = { "--clients" }, description = "Comma separated list of client hostnames", required = true) + String clientHostNames; + + @Option(names = { "--client-port" }, description = "Port that the clients are listening on", required = true) + int clientPort; - @Option(names = { "--client-port" }, description = "Port that the clients are listening on", required = true) - int clientPort; - } // picocli arguments for accepting user input. private static class ShellArguments { @@ -216,24 +206,9 @@ public synchronized void process(final WatchedEvent event) { /** * Create a LoadSimulationController with the given picocli arguments. * - * @param arguments - * Arguments to create from. */ - public LoadSimulationController(final MainArguments arguments) throws Exception { - random = new Random(); - clientPort = arguments.clientPort; - cluster = arguments.cluster; - clients = arguments.clientHostNames.split(","); - final Socket[] sockets = new Socket[clients.length]; - inputStreams = new DataInputStream[clients.length]; - outputStreams = new DataOutputStream[clients.length]; - log.info("Found {} clients:", clients.length); - for (int i = 0; i < clients.length; ++i) { - sockets[i] = new Socket(clients[i], clientPort); - inputStreams[i] = new DataInputStream(sockets[i].getInputStream()); - outputStreams[i] = new DataOutputStream(sockets[i].getOutputStream()); - log.info("Connected to {}", clients[i]); - } + public LoadSimulationController() throws Exception { + super("simulation-controller"); } // Check that the expected number of application arguments matches the @@ -700,7 +675,7 @@ private void read(final String[] args) { /** * Create a shell for the user to send commands to clients. */ - public void run() throws Exception { + public void start() throws Exception { BufferedReader inReader = new BufferedReader(new InputStreamReader(System.in)); while (true) { // Print the very simple prompt. @@ -713,20 +688,21 @@ public void run() throws Exception { /** * Start a controller with command line arguments. * - * @param args - * Arguments to pass in. */ - public static void main(String[] args) throws Exception { - final MainArguments arguments = new MainArguments(); - final CommandLine commander = new CommandLine(arguments); - commander.setCommandName("pulsar-perf simulation-controller"); - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); + @Override + public void run() throws Exception { + random = new Random(); + clients = this.clientHostNames.split(","); + final Socket[] sockets = new Socket[clients.length]; + inputStreams = new DataInputStream[clients.length]; + outputStreams = new DataOutputStream[clients.length]; + log.info("Found {} clients:", clients.length); + for (int i = 0; i < clients.length; ++i) { + sockets[i] = new Socket(clients[i], clientPort); + inputStreams[i] = new DataInputStream(sockets[i].getInputStream()); + outputStreams[i] = new DataOutputStream(sockets[i].getOutputStream()); + log.info("Connected to {}", clients[i]); } - (new LoadSimulationController(arguments)).run(); + start(); } } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java index bad8e56a638b6..8913d17474279 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java @@ -62,11 +62,12 @@ import org.slf4j.LoggerFactory; import picocli.CommandLine; import picocli.CommandLine.Command; +import picocli.CommandLine.Model.CommandSpec; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; -import picocli.CommandLine.ScopeType; +import picocli.CommandLine.Spec; -public class ManagedLedgerWriter { +@Command(name = "managed-ledger", description = "Write directly on managed-ledgers") +public class ManagedLedgerWriter extends CmdBase{ private static final ExecutorService executor = Executors .newCachedThreadPool(new DefaultThreadFactory("pulsar-perf-managed-ledger-exec")); @@ -79,86 +80,72 @@ public class ManagedLedgerWriter { private static Recorder recorder = new Recorder(TimeUnit.SECONDS.toMillis(120000), 5); private static Recorder cumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMillis(120000), 5); - @Command(description = "Write directly on managed-ledgers", showDefaultValues = true, scope = ScopeType.INHERIT) - static class Arguments { - @Option(names = { "-h", "--help" }, description = "Help message", help = true) - boolean help; + @Option(names = { "-r", "--rate" }, description = "Write rate msg/s across managed ledgers") + public int msgRate = 100; - @Option(names = { "-r", "--rate" }, description = "Write rate msg/s across managed ledgers") - public int msgRate = 100; + @Option(names = { "-s", "--size" }, description = "Message size") + public int msgSize = 1024; - @Option(names = { "-s", "--size" }, description = "Message size") - public int msgSize = 1024; + @Option(names = { "-t", "--num-topic" }, + description = "Number of managed ledgers", converter = PositiveNumberParameterConvert.class) + public int numManagedLedgers = 1; - @Option(names = { "-t", "--num-topic" }, - description = "Number of managed ledgers", converter = PositiveNumberParameterConvert.class) - public int numManagedLedgers = 1; + @Option(names = { "--threads" }, + description = "Number of threads writing", converter = PositiveNumberParameterConvert.class) + public int numThreads = 1; - @Option(names = { "--threads" }, - description = "Number of threads writing", converter = PositiveNumberParameterConvert.class) - public int numThreads = 1; + @Deprecated + @Option(names = {"-zk", "--zookeeperServers"}, + description = "ZooKeeper connection string", + hidden = true) + public String zookeeperServers; - @Deprecated - @Option(names = {"-zk", "--zookeeperServers"}, - description = "ZooKeeper connection string", - hidden = true) - public String zookeeperServers; + @Option(names = {"-md", + "--metadata-store"}, description = "Metadata store service URL. For example: zk:my-zk:2181") + private String metadataStoreUrl; - @Option(names = {"-md", - "--metadata-store"}, description = "Metadata store service URL. For example: zk:my-zk:2181") - private String metadataStoreUrl; + @Option(names = { "-o", "--max-outstanding" }, description = "Max number of outstanding requests") + public int maxOutstanding = 1000; - @Option(names = { "-o", "--max-outstanding" }, description = "Max number of outstanding requests") - public int maxOutstanding = 1000; + @Option(names = { "-c", + "--max-connections" }, description = "Max number of TCP connections to a single bookie") + public int maxConnections = 1; - @Option(names = { "-c", - "--max-connections" }, description = "Max number of TCP connections to a single bookie") - public int maxConnections = 1; + @Option(names = { "-m", + "--num-messages" }, + description = "Number of messages to publish in total. If <= 0, it will keep publishing") + public long numMessages = 0; - @Option(names = { "-m", - "--num-messages" }, - description = "Number of messages to publish in total. If <= 0, it will keep publishing") - public long numMessages = 0; + @Option(names = { "-e", "--ensemble-size" }, description = "Ledger ensemble size") + public int ensembleSize = 1; - @Option(names = { "-e", "--ensemble-size" }, description = "Ledger ensemble size") - public int ensembleSize = 1; + @Option(names = { "-w", "--write-quorum" }, description = "Ledger write quorum") + public int writeQuorum = 1; - @Option(names = { "-w", "--write-quorum" }, description = "Ledger write quorum") - public int writeQuorum = 1; + @Option(names = { "-a", "--ack-quorum" }, description = "Ledger ack quorum") + public int ackQuorum = 1; - @Option(names = { "-a", "--ack-quorum" }, description = "Ledger ack quorum") - public int ackQuorum = 1; + @Option(names = { "-dt", "--digest-type" }, description = "BookKeeper digest type") + public DigestType digestType = DigestType.CRC32C; - @Option(names = { "-dt", "--digest-type" }, description = "BookKeeper digest type") - public DigestType digestType = DigestType.CRC32C; - - @Option(names = { "-time", - "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep publishing") - public long testTime = 0; + @Option(names = { "-time", + "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep publishing") + public long testTime = 0; + public ManagedLedgerWriter() { + super("managed-ledger"); } - public static void main(String[] args) throws Exception { - - final Arguments arguments = new Arguments(); - CommandLine commander = new CommandLine(arguments); - commander.setCommandName("pulsar-perf managed-ledger"); - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } + @Spec + CommandSpec spec; - if (arguments.help) { - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } + @Override + public void run() throws Exception { + CommandLine commander = spec.commandLine(); - if (arguments.metadataStoreUrl == null && arguments.zookeeperServers == null) { + if (this.metadataStoreUrl == null && this.zookeeperServers == null) { System.err.println("Metadata store address argument is required (--metadata-store)"); commander.usage(commander.getOut()); PerfClientUtils.exit(1); @@ -168,17 +155,17 @@ public static void main(String[] args) throws Exception { PerfClientUtils.printJVMInformation(log); ObjectMapper m = new ObjectMapper(); ObjectWriter w = m.writerWithDefaultPrettyPrinter(); - log.info("Starting Pulsar managed-ledger perf writer with config: {}", w.writeValueAsString(arguments)); + log.info("Starting Pulsar managed-ledger perf writer with config: {}", w.writeValueAsString(this)); - byte[] payloadData = new byte[arguments.msgSize]; - ByteBuf payloadBuffer = PulsarByteBufAllocator.DEFAULT.directBuffer(arguments.msgSize); - payloadBuffer.writerIndex(arguments.msgSize); + byte[] payloadData = new byte[this.msgSize]; + ByteBuf payloadBuffer = PulsarByteBufAllocator.DEFAULT.directBuffer(this.msgSize); + payloadBuffer.writerIndex(this.msgSize); // Now processing command line arguments String managedLedgerPrefix = "test-" + DigestUtils.sha1Hex(UUID.randomUUID().toString()).substring(0, 5); - if (arguments.metadataStoreUrl == null) { - arguments.metadataStoreUrl = arguments.zookeeperServers; + if (this.metadataStoreUrl == null) { + this.metadataStoreUrl = this.zookeeperServers; } ClientConfiguration bkConf = new ClientConfiguration(); @@ -186,31 +173,31 @@ public static void main(String[] args) throws Exception { bkConf.setAddEntryTimeout(30); bkConf.setReadEntryTimeout(30); bkConf.setThrottleValue(0); - bkConf.setNumChannelsPerBookie(arguments.maxConnections); - bkConf.setMetadataServiceUri(arguments.metadataStoreUrl); + bkConf.setNumChannelsPerBookie(this.maxConnections); + bkConf.setMetadataServiceUri(this.metadataStoreUrl); ManagedLedgerFactoryConfig mlFactoryConf = new ManagedLedgerFactoryConfig(); mlFactoryConf.setMaxCacheSize(0); @Cleanup - MetadataStoreExtended metadataStore = MetadataStoreExtended.create(arguments.metadataStoreUrl, + MetadataStoreExtended metadataStore = MetadataStoreExtended.create(this.metadataStoreUrl, MetadataStoreConfig.builder().metadataStoreName(MetadataStoreConfig.METADATA_STORE).build()); ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkConf, mlFactoryConf); ManagedLedgerConfig mlConf = new ManagedLedgerConfig(); - mlConf.setEnsembleSize(arguments.ensembleSize); - mlConf.setWriteQuorumSize(arguments.writeQuorum); - mlConf.setAckQuorumSize(arguments.ackQuorum); + mlConf.setEnsembleSize(this.ensembleSize); + mlConf.setWriteQuorumSize(this.writeQuorum); + mlConf.setAckQuorumSize(this.ackQuorum); mlConf.setMinimumRolloverTime(10, TimeUnit.MINUTES); - mlConf.setMetadataEnsembleSize(arguments.ensembleSize); - mlConf.setMetadataWriteQuorumSize(arguments.writeQuorum); - mlConf.setMetadataAckQuorumSize(arguments.ackQuorum); - mlConf.setDigestType(arguments.digestType); + mlConf.setMetadataEnsembleSize(this.ensembleSize); + mlConf.setMetadataWriteQuorumSize(this.writeQuorum); + mlConf.setMetadataAckQuorumSize(this.ackQuorum); + mlConf.setDigestType(this.digestType); mlConf.setMaxSizePerLedgerMb(2048); List> futures = new ArrayList<>(); - for (int i = 0; i < arguments.numManagedLedgers; i++) { + for (int i = 0; i < this.numManagedLedgers; i++) { String name = String.format("%s-%03d", managedLedgerPrefix, i); CompletableFuture future = new CompletableFuture<>(); futures.add(future); @@ -242,23 +229,23 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { AtomicBoolean isDone = new AtomicBoolean(); Map> managedLedgersPerThread = allocateToThreads(managedLedgers, - arguments.numThreads); + this.numThreads); - for (int i = 0; i < arguments.numThreads; i++) { + for (int i = 0; i < this.numThreads; i++) { List managedLedgersForThisThread = managedLedgersPerThread.get(i); int nunManagedLedgersForThisThread = managedLedgersForThisThread.size(); - long numMessagesForThisThread = arguments.numMessages / arguments.numThreads; - int maxOutstandingForThisThread = arguments.maxOutstanding; + long numMessagesForThisThread = this.numMessages / this.numThreads; + int maxOutstandingForThisThread = this.maxOutstanding; executor.submit(() -> { try { - final double msgRate = arguments.msgRate / (double) arguments.numThreads; + final double msgRate = this.msgRate / (double) this.numThreads; final RateLimiter rateLimiter = RateLimiter.create(msgRate); // Acquire 1 sec worth of messages to have a slower ramp-up rateLimiter.acquire((int) msgRate); final long startTime = System.nanoTime(); - final long testEndTime = startTime + (long) (arguments.testTime * 1e9); + final long testEndTime = startTime + (long) (this.testTime * 1e9); final Semaphore semaphore = new Semaphore(maxOutstandingForThisThread); @@ -289,10 +276,10 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { long totalSent = 0; while (true) { for (int j = 0; j < nunManagedLedgersForThisThread; j++) { - if (arguments.testTime > 0) { + if (this.testTime > 0) { if (System.nanoTime() > testEndTime) { log.info("------------- DONE (reached the maximum duration: [{} seconds] of " - + "production) --------------", arguments.testTime); + + "production) --------------", this.testTime); isDone.set(true); Thread.sleep(5000); PerfClientUtils.exit(0); diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceBaseArguments.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceBaseArguments.java index d320cafc1a08f..3c4b831332281 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceBaseArguments.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceBaseArguments.java @@ -19,53 +19,44 @@ package org.apache.pulsar.testclient; import static org.apache.commons.lang3.StringUtils.isBlank; -import static org.apache.pulsar.testclient.PerfClientUtils.exit; -import java.io.File; -import java.io.FileInputStream; -import java.util.Properties; -import lombok.SneakyThrows; -import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.cli.converters.picocli.ByteUnitToLongConverter; import org.apache.pulsar.client.api.ProxyProtocol; -import picocli.CommandLine; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; /** * PerformanceBaseArguments contains common CLI arguments and parsing logic available to all sub-commands. * Sub-commands should create Argument subclasses and override the `validate` method as necessary. */ -public abstract class PerformanceBaseArguments { +public abstract class PerformanceBaseArguments extends CmdBase{ - @Option(names = { "-h", "--help" }, description = "Print help message", help = true) - boolean help; - @Option(names = { "-cf", "--conf-file" }, description = "Pulsar configuration file") - public String confFile; - - @Option(names = { "-u", "--service-url" }, description = "Pulsar Service URL") + @Option(names = { "-u", "--service-url" }, description = "Pulsar Service URL", descriptionKey = "brokerServiceUrl") public String serviceURL; - @Option(names = { "--auth-plugin" }, description = "Authentication plugin class name") + @Option(names = { "--auth-plugin" }, description = "Authentication plugin class name", + descriptionKey = "authPlugin") public String authPluginClassName; @Option( names = { "--auth-params" }, description = "Authentication parameters, whose format is determined by the implementation " + "of method `configure` in authentication plugin class, for example \"key1:val1,key2:val2\" " - + "or \"{\"key1\":\"val1\",\"key2\":\"val2\"}\".") + + "or \"{\"key1\":\"val1\",\"key2\":\"val2\"}\".", descriptionKey = "authParams") public String authParams; @Option(names = { - "--trust-cert-file" }, description = "Path for the trusted TLS certificate file") + "--trust-cert-file" }, description = "Path for the trusted TLS certificate file", + descriptionKey = "tlsTrustCertsFilePath") public String tlsTrustCertsFilePath = ""; @Option(names = { - "--tls-allow-insecure" }, description = "Allow insecure TLS connection") + "--tls-allow-insecure" }, description = "Allow insecure TLS connection", + descriptionKey = "tlsAllowInsecureConnection") public Boolean tlsAllowInsecureConnection = null; @Option(names = { - "--tls-enable-hostname-verification" }, description = "Enable TLS hostname verification") + "--tls-enable-hostname-verification" }, description = "Enable TLS hostname verification", + descriptionKey = "tlsEnableHostnameVerification") public Boolean tlsHostnameVerificationEnable = null; @Option(names = { "-c", @@ -95,10 +86,12 @@ public abstract class PerformanceBaseArguments { + "on each broker connection to prevent overloading a broker") public int maxLookupRequest = 50000; - @Option(names = { "--proxy-url" }, description = "Proxy-server URL to which to connect.") + @Option(names = { "--proxy-url" }, description = "Proxy-server URL to which to connect.", + descriptionKey = "proxyServiceUrl") String proxyServiceURL = null; - @Option(names = { "--proxy-protocol" }, description = "Proxy protocol to select type of routing at proxy.") + @Option(names = { "--proxy-protocol" }, description = "Proxy protocol to select type of routing at proxy.", + descriptionKey = "proxyProtocol", converter = ProxyProtocolConverter.class) ProxyProtocol proxyProtocol = null; @Option(names = { "--auth_plugin" }, description = "Authentication plugin class name", hidden = true) @@ -107,129 +100,23 @@ public abstract class PerformanceBaseArguments { @Option(names = { "-ml", "--memory-limit", }, description = "Configure the Pulsar client memory limit " + "(eg: 32M, 64M)", converter = ByteUnitToLongConverter.class) public long memoryLimit; - - public abstract void fillArgumentsFromProperties(Properties prop); - - @SneakyThrows - public void fillArgumentsFromProperties() { - if (confFile == null) { - return; - } - - Properties prop = new Properties(System.getProperties()); - try (FileInputStream fis = new FileInputStream(confFile)) { - prop.load(fis); - } - - if (serviceURL == null) { - serviceURL = prop.getProperty("brokerServiceUrl"); - } - - if (serviceURL == null) { - serviceURL = prop.getProperty("webServiceUrl"); - } - - // fallback to previous-version serviceUrl property to maintain backward-compatibility - if (serviceURL == null) { - serviceURL = prop.getProperty("serviceUrl", "http://localhost:8080/"); - } - - if (authPluginClassName == null) { - authPluginClassName = prop.getProperty("authPlugin", null); - } - - if (authParams == null) { - authParams = prop.getProperty("authParams", null); - } - - if (isBlank(tlsTrustCertsFilePath)) { - tlsTrustCertsFilePath = prop.getProperty("tlsTrustCertsFilePath", ""); - } - - if (tlsAllowInsecureConnection == null) { - tlsAllowInsecureConnection = Boolean.parseBoolean(prop - .getProperty("tlsAllowInsecureConnection", "")); - } - - if (tlsHostnameVerificationEnable == null) { - tlsHostnameVerificationEnable = Boolean.parseBoolean(prop - .getProperty("tlsEnableHostnameVerification", "")); - - } - - if (proxyServiceURL == null) { - proxyServiceURL = StringUtils.trimToNull(prop.getProperty("proxyServiceUrl")); - } - - if (proxyProtocol == null) { - String proxyProtocolString = null; - try { - proxyProtocolString = StringUtils.trimToNull(prop.getProperty("proxyProtocol")); - if (proxyProtocolString != null) { - proxyProtocol = ProxyProtocol.valueOf(proxyProtocolString.toUpperCase()); - } - } catch (IllegalArgumentException e) { - System.out.println("Incorrect proxyProtocol name '" + proxyProtocolString + "'"); - e.printStackTrace(); - exit(1); - } - - } - - fillArgumentsFromProperties(prop); + public PerformanceBaseArguments(String cmdName) { + super(cmdName); } - /** - * Validate the CLI arguments. Default implementation provides validation for the common arguments. - * Each subclass should call super.validate() and provide validation code specific to the sub-command. - * @throws Exception - */ + @Override public void validate() throws Exception { - if (confFile != null && !confFile.isBlank()) { - File configFile = new File(confFile); - if (!configFile.exists()) { - throw new Exception("config file '" + confFile + "', does not exist"); - } - if (configFile.isDirectory()) { - throw new Exception("config file '" + confFile + "', is a directory"); - } - } + parseCLI(); } /** * Parse the command line args. - * @param cmdName used for the help message - * @param args String[] of CLI args * @throws ParameterException If there is a problem parsing the arguments */ - public void parseCLI(String cmdName, String[] args) { - CommandLine commander = new CommandLine(this); - commander.setCommandName(cmdName); - try { - commander.parseArgs(args); - } catch (ParameterException e) { - System.out.println(e.getMessage()); - commander.usage(commander.getOut()); - PerfClientUtils.exit(1); - } - - if (help) { - commander.usage(commander.getOut()); - PerfClientUtils.exit(0); - } - - fillArgumentsFromProperties(); - + public void parseCLI() { if (isBlank(authPluginClassName) && !isBlank(deprecatedAuthPluginClassName)) { authPluginClassName = deprecatedAuthPluginClassName; } - - try { - validate(); - } catch (Exception e) { - System.out.println("error: " + e.getMessage()); - PerfClientUtils.exit(1); - } } } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java index 7a2bc4382fd14..5126eefd9ca1e 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java @@ -29,7 +29,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Properties; import java.util.concurrent.Future; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; @@ -57,9 +56,9 @@ import org.slf4j.LoggerFactory; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import picocli.CommandLine.ScopeType; -public class PerformanceConsumer { +@Command(name = "consume", description = "Test pulsar consumer performance.") +public class PerformanceConsumer extends PerformanceTopicListArguments{ private static final LongAdder messagesReceived = new LongAdder(); private static final LongAdder bytesReceived = new LongAdder(); private static final DecimalFormat intFormat = new PaddingDecimalFormat("0", 7); @@ -83,325 +82,319 @@ public class PerformanceConsumer { private static final Recorder recorder = new Recorder(MAX_LATENCY, 5); private static final Recorder cumulativeRecorder = new Recorder(MAX_LATENCY, 5); - @Command(description = "Test pulsar consumer performance.", showDefaultValues = true, scope = ScopeType.INHERIT) - static class Arguments extends PerformanceTopicListArguments { + @Option(names = { "-n", "--num-consumers" }, description = "Number of consumers (per subscription), only " + + "one consumer is allowed when subscriptionType is Exclusive", + converter = PositiveNumberParameterConvert.class + ) + public int numConsumers = 1; - @Option(names = { "-n", "--num-consumers" }, description = "Number of consumers (per subscription), only " - + "one consumer is allowed when subscriptionType is Exclusive", - converter = PositiveNumberParameterConvert.class - ) - public int numConsumers = 1; + @Option(names = { "-ns", "--num-subscriptions" }, description = "Number of subscriptions (per topic)", + converter = PositiveNumberParameterConvert.class + ) + public int numSubscriptions = 1; - @Option(names = { "-ns", "--num-subscriptions" }, description = "Number of subscriptions (per topic)", - converter = PositiveNumberParameterConvert.class - ) - public int numSubscriptions = 1; + @Option(names = { "-s", "--subscriber-name" }, description = "Subscriber name prefix", hidden = true) + public String subscriberName; - @Option(names = { "-s", "--subscriber-name" }, description = "Subscriber name prefix", hidden = true) - public String subscriberName; + @Option(names = { "-ss", "--subscriptions" }, + description = "A list of subscriptions to consume (for example, sub1,sub2)") + public List subscriptions = Collections.singletonList("sub"); - @Option(names = { "-ss", "--subscriptions" }, - description = "A list of subscriptions to consume (for example, sub1,sub2)") - public List subscriptions = Collections.singletonList("sub"); + @Option(names = { "-st", "--subscription-type" }, description = "Subscription type") + public SubscriptionType subscriptionType = SubscriptionType.Exclusive; - @Option(names = { "-st", "--subscription-type" }, description = "Subscription type") - public SubscriptionType subscriptionType = SubscriptionType.Exclusive; + @Option(names = { "-sp", "--subscription-position" }, description = "Subscription position") + private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest; - @Option(names = { "-sp", "--subscription-position" }, description = "Subscription position") - private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest; + @Option(names = { "-r", "--rate" }, description = "Simulate a slow message consumer (rate in msg/s)") + public double rate = 0; - @Option(names = { "-r", "--rate" }, description = "Simulate a slow message consumer (rate in msg/s)") - public double rate = 0; + @Option(names = { "-q", "--receiver-queue-size" }, description = "Size of the receiver queue") + public int receiverQueueSize = 1000; - @Option(names = { "-q", "--receiver-queue-size" }, description = "Size of the receiver queue") - public int receiverQueueSize = 1000; + @Option(names = { "-p", "--receiver-queue-size-across-partitions" }, + description = "Max total size of the receiver queue across partitions") + public int maxTotalReceiverQueueSizeAcrossPartitions = 50000; - @Option(names = { "-p", "--receiver-queue-size-across-partitions" }, - description = "Max total size of the receiver queue across partitions") - public int maxTotalReceiverQueueSizeAcrossPartitions = 50000; + @Option(names = {"-aq", "--auto-scaled-receiver-queue-size"}, + description = "Enable autoScaledReceiverQueueSize") + public boolean autoScaledReceiverQueueSize = false; - @Option(names = {"-aq", "--auto-scaled-receiver-queue-size"}, - description = "Enable autoScaledReceiverQueueSize") - public boolean autoScaledReceiverQueueSize = false; + @Option(names = {"-rs", "--replicated" }, + description = "Whether the subscription status should be replicated") + public boolean replicatedSubscription = false; - @Option(names = {"-rs", "--replicated" }, - description = "Whether the subscription status should be replicated") - public boolean replicatedSubscription = false; + @Option(names = { "--acks-delay-millis" }, description = "Acknowledgements grouping delay in millis") + public int acknowledgmentsGroupingDelayMillis = 100; - @Option(names = { "--acks-delay-millis" }, description = "Acknowledgements grouping delay in millis") - public int acknowledgmentsGroupingDelayMillis = 100; + @Option(names = {"-m", + "--num-messages"}, + description = "Number of messages to consume in total. If <= 0, it will keep consuming") + public long numMessages = 0; - @Option(names = {"-m", - "--num-messages"}, - description = "Number of messages to consume in total. If <= 0, it will keep consuming") - public long numMessages = 0; + @Option(names = { "-mc", "--max_chunked_msg" }, description = "Max pending chunk messages") + private int maxPendingChunkedMessage = 0; - @Option(names = { "-mc", "--max_chunked_msg" }, description = "Max pending chunk messages") - private int maxPendingChunkedMessage = 0; + @Option(names = { "-ac", + "--auto_ack_chunk_q_full" }, description = "Auto ack for oldest message on queue is full") + private boolean autoAckOldestChunkedMessageOnQueueFull = false; - @Option(names = { "-ac", - "--auto_ack_chunk_q_full" }, description = "Auto ack for oldest message on queue is full") - private boolean autoAckOldestChunkedMessageOnQueueFull = false; + @Option(names = { "-e", + "--expire_time_incomplete_chunked_messages" }, + description = "Expire time in ms for incomplete chunk messages") + private long expireTimeOfIncompleteChunkedMessageMs = 0; - @Option(names = { "-e", - "--expire_time_incomplete_chunked_messages" }, - description = "Expire time in ms for incomplete chunk messages") - private long expireTimeOfIncompleteChunkedMessageMs = 0; + @Option(names = { "-v", + "--encryption-key-value-file" }, + description = "The file which contains the private key to decrypt payload") + public String encKeyFile = null; - @Option(names = { "-v", - "--encryption-key-value-file" }, - description = "The file which contains the private key to decrypt payload") - public String encKeyFile = null; + @Option(names = { "-time", + "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep consuming") + public long testTime = 0; - @Option(names = { "-time", - "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep consuming") - public long testTime = 0; + @Option(names = {"--batch-index-ack" }, description = "Enable or disable the batch index acknowledgment") + public boolean batchIndexAck = false; - @Option(names = {"--batch-index-ack" }, description = "Enable or disable the batch index acknowledgment") - public boolean batchIndexAck = false; + @Option(names = { "-pm", "--pool-messages" }, description = "Use the pooled message", arity = "1") + private boolean poolMessages = true; - @Option(names = { "-pm", "--pool-messages" }, description = "Use the pooled message", arity = "1") - private boolean poolMessages = true; + @Option(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout," + + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)") + public long transactionTimeout = 10; - @Option(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout," - + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)") - public long transactionTimeout = 10; + @Option(names = {"-nmt", "--numMessage-perTransaction"}, + description = "The number of messages acknowledged by a transaction. " + + "(After --txn-enable setting to true, -numMessage-perTransaction takes effect") + public int numMessagesPerTransaction = 50; - @Option(names = {"-nmt", "--numMessage-perTransaction"}, - description = "The number of messages acknowledged by a transaction. " - + "(After --txn-enable setting to true, -numMessage-perTransaction takes effect") - public int numMessagesPerTransaction = 50; + @Option(names = {"-txn", "--txn-enable"}, description = "Enable or disable the transaction") + public boolean isEnableTransaction = false; - @Option(names = {"-txn", "--txn-enable"}, description = "Enable or disable the transaction") - public boolean isEnableTransaction = false; + @Option(names = {"-ntxn"}, description = "The number of opened transactions, 0 means keeping open." + + "(After --txn-enable setting to true, -ntxn takes effect.)") + public long totalNumTxn = 0; - @Option(names = {"-ntxn"}, description = "The number of opened transactions, 0 means keeping open." - + "(After --txn-enable setting to true, -ntxn takes effect.)") - public long totalNumTxn = 0; + @Option(names = {"-abort"}, description = "Abort the transaction. (After --txn-enable " + + "setting to true, -abort takes effect)") + public boolean isAbortTransaction = false; - @Option(names = {"-abort"}, description = "Abort the transaction. (After --txn-enable " - + "setting to true, -abort takes effect)") - public boolean isAbortTransaction = false; + @Option(names = { "--histogram-file" }, description = "HdrHistogram output file") + public String histogramFile = null; - @Option(names = { "--histogram-file" }, description = "HdrHistogram output file") - public String histogramFile = null; + public PerformanceConsumer() { + super("consume"); + } - @Override - public void fillArgumentsFromProperties(Properties prop) { - } - @Override - public void validate() throws Exception { - super.validate(); - if (subscriptionType == SubscriptionType.Exclusive && numConsumers > 1) { - throw new Exception("Only one consumer is allowed when subscriptionType is Exclusive"); - } + @Override + public void validate() throws Exception { + super.validate(); + if (subscriptionType == SubscriptionType.Exclusive && numConsumers > 1) { + throw new Exception("Only one consumer is allowed when subscriptionType is Exclusive"); + } - if (subscriptions != null && subscriptions.size() != numSubscriptions) { - // keep compatibility with the previous version - if (subscriptions.size() == 1) { - if (subscriberName == null) { - subscriberName = subscriptions.get(0); - } - List defaultSubscriptions = new ArrayList<>(); - for (int i = 0; i < numSubscriptions; i++) { - defaultSubscriptions.add(String.format("%s-%d", subscriberName, i)); - } - subscriptions = defaultSubscriptions; - } else { - throw new Exception("The size of subscriptions list should be equal to --num-subscriptions"); + if (subscriptions != null && subscriptions.size() != numSubscriptions) { + // keep compatibility with the previous version + if (subscriptions.size() == 1) { + if (subscriberName == null) { + subscriberName = subscriptions.get(0); } + List defaultSubscriptions = new ArrayList<>(); + for (int i = 0; i < numSubscriptions; i++) { + defaultSubscriptions.add(String.format("%s-%d", subscriberName, i)); + } + subscriptions = defaultSubscriptions; + } else { + throw new Exception("The size of subscriptions list should be equal to --num-subscriptions"); } } } - - public static void main(String[] args) throws Exception { - final Arguments arguments = new Arguments(); - arguments.parseCLI("pulsar-perf consume", args); - + @Override + public void run() throws Exception { // Dump config variables PerfClientUtils.printJVMInformation(log); ObjectMapper m = new ObjectMapper(); ObjectWriter w = m.writerWithDefaultPrettyPrinter(); - log.info("Starting Pulsar performance consumer with config: {}", w.writeValueAsString(arguments)); + log.info("Starting Pulsar performance consumer with config: {}", w.writeValueAsString(this)); - final Recorder qRecorder = arguments.autoScaledReceiverQueueSize - ? new Recorder(arguments.receiverQueueSize, 5) : null; - final RateLimiter limiter = arguments.rate > 0 ? RateLimiter.create(arguments.rate) : null; + final Recorder qRecorder = this.autoScaledReceiverQueueSize + ? new Recorder(this.receiverQueueSize, 5) : null; + final RateLimiter limiter = this.rate > 0 ? RateLimiter.create(this.rate) : null; long startTime = System.nanoTime(); - long testEndTime = startTime + (long) (arguments.testTime * 1e9); + long testEndTime = startTime + (long) (this.testTime * 1e9); - ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) - .enableTransaction(arguments.isEnableTransaction); + ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(this) + .enableTransaction(this.isEnableTransaction); PulsarClient pulsarClient = clientBuilder.build(); AtomicReference atomicReference; - if (arguments.isEnableTransaction) { + if (this.isEnableTransaction) { atomicReference = new AtomicReference<>(pulsarClient.newTransaction() - .withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS).build().get()); + .withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS).build().get()); } else { atomicReference = new AtomicReference<>(null); } AtomicLong messageAckedCount = new AtomicLong(); - Semaphore messageReceiveLimiter = new Semaphore(arguments.numMessagesPerTransaction); + Semaphore messageReceiveLimiter = new Semaphore(this.numMessagesPerTransaction); Thread thread = Thread.currentThread(); MessageListener listener = (consumer, msg) -> { - if (arguments.testTime > 0) { - if (System.nanoTime() > testEndTime) { - log.info("------------------- DONE -----------------------"); - PerfClientUtils.exit(0); - thread.interrupt(); - } - } - if (arguments.totalNumTxn > 0) { - if (totalEndTxnOpFailNum.sum() + totalEndTxnOpSuccessNum.sum() >= arguments.totalNumTxn) { - log.info("------------------- DONE -----------------------"); - PerfClientUtils.exit(0); - thread.interrupt(); - } - } - if (qRecorder != null) { - qRecorder.recordValue(((ConsumerBase) consumer).getTotalIncomingMessages()); + if (this.testTime > 0) { + if (System.nanoTime() > testEndTime) { + log.info("------------------- DONE -----------------------"); + PerfClientUtils.exit(0); + thread.interrupt(); } - messagesReceived.increment(); - bytesReceived.add(msg.size()); - - totalMessagesReceived.increment(); - totalBytesReceived.add(msg.size()); - - if (arguments.numMessages > 0 && totalMessagesReceived.sum() >= arguments.numMessages) { + } + if (this.totalNumTxn > 0) { + if (totalEndTxnOpFailNum.sum() + totalEndTxnOpSuccessNum.sum() >= this.totalNumTxn) { log.info("------------------- DONE -----------------------"); PerfClientUtils.exit(0); thread.interrupt(); } + } + if (qRecorder != null) { + qRecorder.recordValue(((ConsumerBase) consumer).getTotalIncomingMessages()); + } + messagesReceived.increment(); + bytesReceived.add(msg.size()); - if (limiter != null) { - limiter.acquire(); - } + totalMessagesReceived.increment(); + totalBytesReceived.add(msg.size()); - long latencyMillis = System.currentTimeMillis() - msg.getPublishTime(); - if (latencyMillis >= 0) { - if (latencyMillis >= MAX_LATENCY) { - latencyMillis = MAX_LATENCY; - } - recorder.recordValue(latencyMillis); - cumulativeRecorder.recordValue(latencyMillis); - } - if (arguments.isEnableTransaction) { - try { - messageReceiveLimiter.acquire(); - } catch (InterruptedException e){ - log.error("Got error: ", e); - } - consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> { - totalMessageAck.increment(); - messageAck.increment(); - }).exceptionally(throwable ->{ - log.error("Ack message {} failed with exception", msg, throwable); - totalMessageAckFailed.increment(); - return null; - }); - } else { - consumer.acknowledgeAsync(msg).thenRun(()->{ - totalMessageAck.increment(); - messageAck.increment(); - } - ).exceptionally(throwable ->{ - log.error("Ack message {} failed with exception", msg, throwable); - totalMessageAckFailed.increment(); - return null; - } - ); + if (this.numMessages > 0 && totalMessagesReceived.sum() >= this.numMessages) { + log.info("------------------- DONE -----------------------"); + PerfClientUtils.exit(0); + thread.interrupt(); + } + + if (limiter != null) { + limiter.acquire(); + } + + long latencyMillis = System.currentTimeMillis() - msg.getPublishTime(); + if (latencyMillis >= 0) { + if (latencyMillis >= MAX_LATENCY) { + latencyMillis = MAX_LATENCY; } - if (arguments.poolMessages) { - msg.release(); + recorder.recordValue(latencyMillis); + cumulativeRecorder.recordValue(latencyMillis); + } + if (this.isEnableTransaction) { + try { + messageReceiveLimiter.acquire(); + } catch (InterruptedException e){ + log.error("Got error: ", e); } - if (arguments.isEnableTransaction - && messageAckedCount.incrementAndGet() == arguments.numMessagesPerTransaction) { - Transaction transaction = atomicReference.get(); - if (!arguments.isAbortTransaction) { - transaction.commit() - .thenRun(() -> { - if (log.isDebugEnabled()) { - log.debug("Commit transaction {}", transaction.getTxnID()); - } - totalEndTxnOpSuccessNum.increment(); - numTxnOpSuccess.increment(); - }) - .exceptionally(exception -> { - log.error("Commit transaction failed with exception : ", exception); - totalEndTxnOpFailNum.increment(); - return null; - }); - } else { - transaction.abort().thenRun(() -> { - if (log.isDebugEnabled()) { - log.debug("Abort transaction {}", transaction.getTxnID()); - } - totalEndTxnOpSuccessNum.increment(); - numTxnOpSuccess.increment(); - }).exceptionally(exception -> { - log.error("Abort transaction {} failed with exception", - transaction.getTxnID().toString(), - exception); - totalEndTxnOpFailNum.increment(); + consumer.acknowledgeAsync(msg.getMessageId(), atomicReference.get()).thenRun(() -> { + totalMessageAck.increment(); + messageAck.increment(); + }).exceptionally(throwable ->{ + log.error("Ack message {} failed with exception", msg, throwable); + totalMessageAckFailed.increment(); + return null; + }); + } else { + consumer.acknowledgeAsync(msg).thenRun(()->{ + totalMessageAck.increment(); + messageAck.increment(); + } + ).exceptionally(throwable ->{ + log.error("Ack message {} failed with exception", msg, throwable); + totalMessageAckFailed.increment(); return null; - }); - } - while (true) { - try { - Transaction newTransaction = pulsarClient.newTransaction() - .withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS) - .build().get(); - atomicReference.compareAndSet(transaction, newTransaction); - totalNumTxnOpenSuccess.increment(); - messageAckedCount.set(0); - messageReceiveLimiter.release(arguments.numMessagesPerTransaction); - break; - } catch (Exception e) { - log.error("Failed to new transaction with exception:", e); - totalNumTxnOpenFail.increment(); } + ); + } + if (this.poolMessages) { + msg.release(); + } + if (this.isEnableTransaction + && messageAckedCount.incrementAndGet() == this.numMessagesPerTransaction) { + Transaction transaction = atomicReference.get(); + if (!this.isAbortTransaction) { + transaction.commit() + .thenRun(() -> { + if (log.isDebugEnabled()) { + log.debug("Commit transaction {}", transaction.getTxnID()); + } + totalEndTxnOpSuccessNum.increment(); + numTxnOpSuccess.increment(); + }) + .exceptionally(exception -> { + log.error("Commit transaction failed with exception : ", exception); + totalEndTxnOpFailNum.increment(); + return null; + }); + } else { + transaction.abort().thenRun(() -> { + if (log.isDebugEnabled()) { + log.debug("Abort transaction {}", transaction.getTxnID()); + } + totalEndTxnOpSuccessNum.increment(); + numTxnOpSuccess.increment(); + }).exceptionally(exception -> { + log.error("Abort transaction {} failed with exception", + transaction.getTxnID().toString(), + exception); + totalEndTxnOpFailNum.increment(); + return null; + }); + } + while (true) { + try { + Transaction newTransaction = pulsarClient.newTransaction() + .withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS) + .build().get(); + atomicReference.compareAndSet(transaction, newTransaction); + totalNumTxnOpenSuccess.increment(); + messageAckedCount.set(0); + messageReceiveLimiter.release(this.numMessagesPerTransaction); + break; + } catch (Exception e) { + log.error("Failed to new transaction with exception:", e); + totalNumTxnOpenFail.increment(); } } + } }; List>> futures = new ArrayList<>(); ConsumerBuilder consumerBuilder = pulsarClient.newConsumer(Schema.BYTEBUFFER) // .messageListener(listener) // - .receiverQueueSize(arguments.receiverQueueSize) // - .maxTotalReceiverQueueSizeAcrossPartitions(arguments.maxTotalReceiverQueueSizeAcrossPartitions) - .acknowledgmentGroupTime(arguments.acknowledgmentsGroupingDelayMillis, TimeUnit.MILLISECONDS) // - .subscriptionType(arguments.subscriptionType) - .subscriptionInitialPosition(arguments.subscriptionInitialPosition) - .autoAckOldestChunkedMessageOnQueueFull(arguments.autoAckOldestChunkedMessageOnQueueFull) - .enableBatchIndexAcknowledgment(arguments.batchIndexAck) - .poolMessages(arguments.poolMessages) - .replicateSubscriptionState(arguments.replicatedSubscription) - .autoScaledReceiverQueueSizeEnabled(arguments.autoScaledReceiverQueueSize); - if (arguments.maxPendingChunkedMessage > 0) { - consumerBuilder.maxPendingChunkedMessage(arguments.maxPendingChunkedMessage); + .receiverQueueSize(this.receiverQueueSize) // + .maxTotalReceiverQueueSizeAcrossPartitions(this.maxTotalReceiverQueueSizeAcrossPartitions) + .acknowledgmentGroupTime(this.acknowledgmentsGroupingDelayMillis, TimeUnit.MILLISECONDS) // + .subscriptionType(this.subscriptionType) + .subscriptionInitialPosition(this.subscriptionInitialPosition) + .autoAckOldestChunkedMessageOnQueueFull(this.autoAckOldestChunkedMessageOnQueueFull) + .enableBatchIndexAcknowledgment(this.batchIndexAck) + .poolMessages(this.poolMessages) + .replicateSubscriptionState(this.replicatedSubscription) + .autoScaledReceiverQueueSizeEnabled(this.autoScaledReceiverQueueSize); + if (this.maxPendingChunkedMessage > 0) { + consumerBuilder.maxPendingChunkedMessage(this.maxPendingChunkedMessage); } - if (arguments.expireTimeOfIncompleteChunkedMessageMs > 0) { - consumerBuilder.expireTimeOfIncompleteChunkedMessage(arguments.expireTimeOfIncompleteChunkedMessageMs, + if (this.expireTimeOfIncompleteChunkedMessageMs > 0) { + consumerBuilder.expireTimeOfIncompleteChunkedMessage(this.expireTimeOfIncompleteChunkedMessageMs, TimeUnit.MILLISECONDS); } - if (isNotBlank(arguments.encKeyFile)) { - consumerBuilder.defaultCryptoKeyReader(arguments.encKeyFile); + if (isNotBlank(this.encKeyFile)) { + consumerBuilder.defaultCryptoKeyReader(this.encKeyFile); } - for (int i = 0; i < arguments.numTopics; i++) { - final TopicName topicName = TopicName.get(arguments.topics.get(i)); + for (int i = 0; i < this.numTopics; i++) { + final TopicName topicName = TopicName.get(this.topics.get(i)); - log.info("Adding {} consumers per subscription on topic {}", arguments.numConsumers, topicName); + log.info("Adding {} consumers per subscription on topic {}", this.numConsumers, topicName); - for (int j = 0; j < arguments.numSubscriptions; j++) { - String subscriberName = arguments.subscriptions.get(j); - for (int k = 0; k < arguments.numConsumers; k++) { + for (int j = 0; j < this.numSubscriptions; j++) { + String subscriberName = this.subscriptions.get(j); + for (int k = 0; k < this.numConsumers; k++) { futures.add(consumerBuilder.clone().topic(topicName.toString()).subscriptionName(subscriberName) .subscribeAsync()); } @@ -410,13 +403,13 @@ public static void main(String[] args) throws Exception { for (Future> future : futures) { future.get(); } - log.info("Start receiving from {} consumers per subscription on {} topics", arguments.numConsumers, - arguments.numTopics); + log.info("Start receiving from {} consumers per subscription on {} topics", this.numConsumers, + this.numTopics); long start = System.nanoTime(); Runtime.getRuntime().addShutdownHook(new Thread(() -> { - printAggregatedThroughput(start, arguments); + printAggregatedThroughput(start); printAggregatedStats(); })); @@ -427,8 +420,8 @@ public static void main(String[] args) throws Exception { Histogram qHistogram = null; HistogramLogWriter histogramLogWriter = null; - if (arguments.histogramFile != null) { - String statsFileName = arguments.histogramFile; + if (this.histogramFile != null) { + String statsFileName = this.histogramFile; log.info("Dumping latency stats to {}", statsFileName); PrintStream histogramLog = new PrintStream(new FileOutputStream(statsFileName), false); @@ -457,7 +450,7 @@ public static void main(String[] args) throws Exception { double rateOpenTxn = 0; reportHistogram = recorder.getIntervalHistogram(reportHistogram); - if (arguments.isEnableTransaction) { + if (this.isEnableTransaction) { totalTxnOpSuccessNum = totalEndTxnOpSuccessNum.sum(); totalTxnOpFailNum = totalEndTxnOpFailNum.sum(); rateOpenTxn = numTxnOpSuccess.sumThenReset() / elapsed; @@ -478,7 +471,7 @@ public static void main(String[] args) throws Exception { reportHistogram.getValueAtPercentile(99), reportHistogram.getValueAtPercentile(99.9), reportHistogram.getValueAtPercentile(99.99), reportHistogram.getMaxValue()); - if (arguments.autoScaledReceiverQueueSize && log.isDebugEnabled() && qRecorder != null) { + if (this.autoScaledReceiverQueueSize && log.isDebugEnabled() && qRecorder != null) { qHistogram = qRecorder.getIntervalHistogram(qHistogram); log.debug("ReceiverQueueUsage: cnt={},mean={}, min={},max={},25pct={},50pct={},75pct={}", qHistogram.getTotalCount(), dec.format(qHistogram.getMean()), @@ -507,7 +500,7 @@ public static void main(String[] args) throws Exception { reportHistogram.reset(); oldTime = now; - if (arguments.testTime > 0) { + if (this.testTime > 0) { if (now > testEndTime) { log.info("------------------- DONE -----------------------"); PerfClientUtils.exit(0); @@ -519,7 +512,7 @@ public static void main(String[] args) throws Exception { pulsarClient.close(); } - private static void printAggregatedThroughput(long start, Arguments arguments) { + private void printAggregatedThroughput(long start) { double elapsed = (System.nanoTime() - start) / 1e9; double rate = totalMessagesReceived.sum() / elapsed; double throughput = totalBytesReceived.sum() / elapsed * 8 / 1024 / 1024; @@ -530,7 +523,7 @@ private static void printAggregatedThroughput(long start, Arguments arguments) { long totalnumMessageAckFailed = 0; double rateAck = totalMessageAck.sum() / elapsed; double rateOpenTxn = 0; - if (arguments.isEnableTransaction) { + if (this.isEnableTransaction) { totalEndTxnSuccess = totalEndTxnOpSuccessNum.sum(); totalEndTxnFail = totalEndTxnOpFailNum.sum(); rateOpenTxn = (totalEndTxnSuccess + totalEndTxnFail) / elapsed; diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java index 0eb8d02f31efa..ba5be3a3c4566 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java @@ -20,7 +20,6 @@ import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.NANOSECONDS; -import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isNotBlank; import static org.apache.pulsar.client.impl.conf.ProducerConfigurationData.DEFAULT_BATCHING_MAX_MESSAGES; import static org.apache.pulsar.client.impl.conf.ProducerConfigurationData.DEFAULT_MAX_PENDING_MESSAGES; @@ -40,7 +39,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Properties; import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -75,13 +73,13 @@ import picocli.CommandLine.Command; import picocli.CommandLine.ITypeConverter; import picocli.CommandLine.Option; -import picocli.CommandLine.ScopeType; import picocli.CommandLine.TypeConversionException; /** * A client program to test pulsar producer performance. */ -public class PerformanceProducer { +@Command(name = "produce", description = "Test pulsar producer performance.") +public class PerformanceProducer extends PerformanceTopicListArguments{ private static final ExecutorService executor = Executors .newCachedThreadPool(new DefaultThreadFactory("pulsar-perf-producer-exec")); @@ -105,192 +103,171 @@ public class PerformanceProducer { private static IMessageFormatter messageFormatter = null; - @Command(description = "Test pulsar producer performance.", showDefaultValues = true, scope = ScopeType.INHERIT) - static class Arguments extends PerformanceTopicListArguments { + @Option(names = { "-threads", "--num-test-threads" }, description = "Number of test threads", + converter = PositiveNumberParameterConvert.class + ) + public int numTestThreads = 1; - @Option(names = { "-threads", "--num-test-threads" }, description = "Number of test threads", - converter = PositiveNumberParameterConvert.class - ) - public int numTestThreads = 1; + @Option(names = { "-r", "--rate" }, description = "Publish rate msg/s across topics") + public int msgRate = 100; - @Option(names = { "-r", "--rate" }, description = "Publish rate msg/s across topics") - public int msgRate = 100; + @Option(names = { "-s", "--size" }, description = "Message size (bytes)") + public int msgSize = 1024; - @Option(names = { "-s", "--size" }, description = "Message size (bytes)") - public int msgSize = 1024; + @Option(names = { "-n", "--num-producers" }, description = "Number of producers (per topic)", + converter = PositiveNumberParameterConvert.class + ) + public int numProducers = 1; - @Option(names = { "-n", "--num-producers" }, description = "Number of producers (per topic)", - converter = PositiveNumberParameterConvert.class - ) - public int numProducers = 1; + @Option(names = {"--separator"}, description = "Separator between the topic and topic number") + public String separator = "-"; - @Option(names = {"--separator"}, description = "Separator between the topic and topic number") - public String separator = "-"; + @Option(names = {"--send-timeout"}, description = "Set the sendTimeout value default 0 to keep " + + "compatibility with previous version of pulsar-perf") + public int sendTimeout = 0; - @Option(names = {"--send-timeout"}, description = "Set the sendTimeout value default 0 to keep " - + "compatibility with previous version of pulsar-perf") - public int sendTimeout = 0; + @Option(names = { "-pn", "--producer-name" }, description = "Producer Name") + public String producerName = null; - @Option(names = { "-pn", "--producer-name" }, description = "Producer Name") - public String producerName = null; + @Option(names = { "-au", "--admin-url" }, description = "Pulsar Admin URL", descriptionKey = "webServiceUrl") + public String adminURL; - @Option(names = { "-au", "--admin-url" }, description = "Pulsar Admin URL") - public String adminURL; + @Option(names = { "-ch", + "--chunking" }, description = "Should split the message and publish in chunks if message size is " + + "larger than allowed max size") + private boolean chunkingAllowed = false; - @Option(names = { "-ch", - "--chunking" }, description = "Should split the message and publish in chunks if message size is " - + "larger than allowed max size") - private boolean chunkingAllowed = false; + @Option(names = { "-o", "--max-outstanding" }, description = "Max number of outstanding messages") + public int maxOutstanding = DEFAULT_MAX_PENDING_MESSAGES; - @Option(names = { "-o", "--max-outstanding" }, description = "Max number of outstanding messages") - public int maxOutstanding = DEFAULT_MAX_PENDING_MESSAGES; + @Option(names = { "-p", "--max-outstanding-across-partitions" }, description = "Max number of outstanding " + + "messages across partitions") + public int maxPendingMessagesAcrossPartitions = DEFAULT_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS; - @Option(names = { "-p", "--max-outstanding-across-partitions" }, description = "Max number of outstanding " - + "messages across partitions") - public int maxPendingMessagesAcrossPartitions = DEFAULT_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS; + @Option(names = { "-np", "--partitions" }, description = "Create partitioned topics with the given number " + + "of partitions, set 0 to not try to create the topic") + public Integer partitions = null; - @Option(names = { "-np", "--partitions" }, description = "Create partitioned topics with the given number " - + "of partitions, set 0 to not try to create the topic") - public Integer partitions = null; + @Option(names = { "-m", + "--num-messages" }, description = "Number of messages to publish in total. If <= 0, it will keep " + + "publishing") + public long numMessages = 0; - @Option(names = { "-m", - "--num-messages" }, description = "Number of messages to publish in total. If <= 0, it will keep " - + "publishing") - public long numMessages = 0; + @Option(names = { "-z", "--compression" }, description = "Compress messages payload") + public CompressionType compression = CompressionType.NONE; - @Option(names = { "-z", "--compression" }, description = "Compress messages payload") - public CompressionType compression = CompressionType.NONE; + @Option(names = { "-f", "--payload-file" }, description = "Use payload from an UTF-8 encoded text file and " + + "a payload will be randomly selected when publishing messages") + public String payloadFilename = null; - @Option(names = { "-f", "--payload-file" }, description = "Use payload from an UTF-8 encoded text file and " - + "a payload will be randomly selected when publishing messages") - public String payloadFilename = null; + @Option(names = { "-e", "--payload-delimiter" }, description = "The delimiter used to split lines when " + + "using payload from a file") + // here escaping \n since default value will be printed with the help text + public String payloadDelimiter = "\\n"; - @Option(names = { "-e", "--payload-delimiter" }, description = "The delimiter used to split lines when " - + "using payload from a file") - // here escaping \n since default value will be printed with the help text - public String payloadDelimiter = "\\n"; + @Option(names = { "-b", + "--batch-time-window" }, description = "Batch messages in 'x' ms window (Default: 1ms)") + public double batchTimeMillis = 1.0; - @Option(names = { "-b", - "--batch-time-window" }, description = "Batch messages in 'x' ms window (Default: 1ms)") - public double batchTimeMillis = 1.0; + @Option(names = { "-db", + "--disable-batching" }, description = "Disable batching if true") + public boolean disableBatching; - @Option(names = { "-db", - "--disable-batching" }, description = "Disable batching if true") - public boolean disableBatching; - - @Option(names = { + @Option(names = { "-bm", "--batch-max-messages" - }, description = "Maximum number of messages per batch") - public int batchMaxMessages = DEFAULT_BATCHING_MAX_MESSAGES; + }, description = "Maximum number of messages per batch") + public int batchMaxMessages = DEFAULT_BATCHING_MAX_MESSAGES; - @Option(names = { + @Option(names = { "-bb", "--batch-max-bytes" - }, description = "Maximum number of bytes per batch") - public int batchMaxBytes = 4 * 1024 * 1024; - - @Option(names = { "-time", - "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep publishing") - public long testTime = 0; - - @Option(names = "--warmup-time", description = "Warm-up time in seconds (Default: 1 sec)") - public double warmupTimeSeconds = 1.0; + }, description = "Maximum number of bytes per batch") + public int batchMaxBytes = 4 * 1024 * 1024; - @Option(names = { "-k", "--encryption-key-name" }, description = "The public key name to encrypt payload") - public String encKeyName = null; + @Option(names = { "-time", + "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep publishing") + public long testTime = 0; - @Option(names = { "-v", - "--encryption-key-value-file" }, - description = "The file which contains the public key to encrypt payload") - public String encKeyFile = null; + @Option(names = "--warmup-time", description = "Warm-up time in seconds (Default: 1 sec)") + public double warmupTimeSeconds = 1.0; - @Option(names = { "-d", - "--delay" }, description = "Mark messages with a given delay in seconds") - public long delay = 0; + @Option(names = { "-k", "--encryption-key-name" }, description = "The public key name to encrypt payload") + public String encKeyName = null; - @Option(names = { "-dr", "--delay-range"}, description = "Mark messages with a given delay by a random" - + " number of seconds. this value between the specified origin (inclusive) and the specified bound" - + " (exclusive). e.g. 1,300", converter = RangeConvert.class) - public Range delayRange = null; + @Option(names = { "-v", + "--encryption-key-value-file" }, + description = "The file which contains the public key to encrypt payload") + public String encKeyFile = null; - @Option(names = { "-set", - "--set-event-time" }, description = "Set the eventTime on messages") - public boolean setEventTime = false; + @Option(names = { "-d", + "--delay" }, description = "Mark messages with a given delay in seconds") + public long delay = 0; - @Option(names = { "-ef", - "--exit-on-failure" }, description = "Exit from the process on publish failure (default: disable)") - public boolean exitOnFailure = false; + @Option(names = { "-dr", "--delay-range"}, description = "Mark messages with a given delay by a random" + + " number of seconds. this value between the specified origin (inclusive) and the specified bound" + + " (exclusive). e.g. 1,300", converter = RangeConvert.class) + public Range delayRange = null; - @Option(names = {"-mk", "--message-key-generation-mode"}, description = "The generation mode of message key" - + ", valid options are: [autoIncrement, random]") - public String messageKeyGenerationMode = null; + @Option(names = { "-set", + "--set-event-time" }, description = "Set the eventTime on messages") + public boolean setEventTime = false; - @Option(names = { "-am", "--access-mode" }, description = "Producer access mode") - public ProducerAccessMode producerAccessMode = ProducerAccessMode.Shared; + @Option(names = { "-ef", + "--exit-on-failure" }, description = "Exit from the process on publish failure (default: disable)") + public boolean exitOnFailure = false; - @Option(names = { "-fp", "--format-payload" }, - description = "Format %%i as a message index in the stream from producer and/or %%t as the timestamp" - + " nanoseconds.") - public boolean formatPayload = false; + @Option(names = {"-mk", "--message-key-generation-mode"}, description = "The generation mode of message key" + + ", valid options are: [autoIncrement, random]", descriptionKey = "messageKeyGenerationMode") + public String messageKeyGenerationMode = null; - @Option(names = {"-fc", "--format-class"}, description = "Custom Formatter class name") - public String formatterClass = "org.apache.pulsar.testclient.DefaultMessageFormatter"; + @Option(names = { "-am", "--access-mode" }, description = "Producer access mode") + public ProducerAccessMode producerAccessMode = ProducerAccessMode.Shared; - @Option(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout," - + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)") - public long transactionTimeout = 10; + @Option(names = { "-fp", "--format-payload" }, + description = "Format %%i as a message index in the stream from producer and/or %%t as the timestamp" + + " nanoseconds.") + public boolean formatPayload = false; - @Option(names = {"-nmt", "--numMessage-perTransaction"}, - description = "The number of messages sent by a transaction. " - + "(After --txn-enable setting to true, -nmt takes effect)") - public int numMessagesPerTransaction = 50; + @Option(names = {"-fc", "--format-class"}, description = "Custom Formatter class name") + public String formatterClass = "org.apache.pulsar.testclient.DefaultMessageFormatter"; - @Option(names = {"-txn", "--txn-enable"}, description = "Enable or disable the transaction") - public boolean isEnableTransaction = false; + @Option(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout," + + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)") + public long transactionTimeout = 10; - @Option(names = {"-abort"}, description = "Abort the transaction. (After --txn-enable " - + "setting to true, -abort takes effect)") - public boolean isAbortTransaction = false; - - @Option(names = { "--histogram-file" }, description = "HdrHistogram output file") - public String histogramFile = null; - - @Override - public void fillArgumentsFromProperties(Properties prop) { - if (adminURL == null) { - adminURL = prop.getProperty("webServiceUrl"); - } - if (adminURL == null) { - adminURL = prop.getProperty("adminURL", "http://localhost:8080/"); - } + @Option(names = {"-nmt", "--numMessage-perTransaction"}, + description = "The number of messages sent by a transaction. " + + "(After --txn-enable setting to true, -nmt takes effect)") + public int numMessagesPerTransaction = 50; - if (isBlank(messageKeyGenerationMode)) { - messageKeyGenerationMode = prop.getProperty("messageKeyGenerationMode", null); - } - } - } + @Option(names = {"-txn", "--txn-enable"}, description = "Enable or disable the transaction") + public boolean isEnableTransaction = false; - public static void main(String[] args) throws Exception { + @Option(names = {"-abort"}, description = "Abort the transaction. (After --txn-enable " + + "setting to true, -abort takes effect)") + public boolean isAbortTransaction = false; - final Arguments arguments = new Arguments(); - arguments.parseCLI("pulsar-perf produce", args); + @Option(names = { "--histogram-file" }, description = "HdrHistogram output file") + public String histogramFile = null; + @Override + public void run() throws Exception { // Dump config variables PerfClientUtils.printJVMInformation(log); ObjectMapper m = new ObjectMapper(); ObjectWriter w = m.writerWithDefaultPrettyPrinter(); - log.info("Starting Pulsar perf producer with config: {}", w.writeValueAsString(arguments)); + log.info("Starting Pulsar perf producer with config: {}", w.writeValueAsString(this)); // Read payload data from file if needed - final byte[] payloadBytes = new byte[arguments.msgSize]; + final byte[] payloadBytes = new byte[msgSize]; Random random = new Random(0); List payloadByteList = new ArrayList<>(); - if (arguments.payloadFilename != null) { - Path payloadFilePath = Paths.get(arguments.payloadFilename); + if (this.payloadFilename != null) { + Path payloadFilePath = Paths.get(this.payloadFilename); if (Files.notExists(payloadFilePath) || Files.size(payloadFilePath) == 0) { throw new IllegalArgumentException("Payload file doesn't exist or it is empty."); } // here escaping the default payload delimiter to correct value - String delimiter = arguments.payloadDelimiter.equals("\\n") ? "\n" : arguments.payloadDelimiter; + String delimiter = this.payloadDelimiter.equals("\\n") ? "\n" : this.payloadDelimiter; String[] payloadList = new String(Files.readAllBytes(payloadFilePath), StandardCharsets.UTF_8).split(delimiter); log.info("Reading payloads from {} and {} records read", payloadFilePath.toAbsolutePath(), @@ -299,8 +276,8 @@ public static void main(String[] args) throws Exception { payloadByteList.add(payload.getBytes(StandardCharsets.UTF_8)); } - if (arguments.formatPayload) { - messageFormatter = getMessageFormatter(arguments.formatterClass); + if (this.formatPayload) { + messageFormatter = getMessageFormatter(this.formatterClass); } } else { for (int i = 0; i < payloadBytes.length; ++i) { @@ -312,29 +289,29 @@ public static void main(String[] args) throws Exception { Runtime.getRuntime().addShutdownHook(new Thread(() -> { executorShutdownNow(); - printAggregatedThroughput(start, arguments); + printAggregatedThroughput(start); printAggregatedStats(); })); - if (arguments.partitions != null) { + if (this.partitions != null) { final PulsarAdminBuilder adminBuilder = PerfClientUtils - .createAdminBuilderFromArguments(arguments, arguments.adminURL); + .createAdminBuilderFromArguments(this, this.adminURL); try (PulsarAdmin adminClient = adminBuilder.build()) { - for (String topic : arguments.topics) { - log.info("Creating partitioned topic {} with {} partitions", topic, arguments.partitions); + for (String topic : this.topics) { + log.info("Creating partitioned topic {} with {} partitions", topic, this.partitions); try { - adminClient.topics().createPartitionedTopic(topic, arguments.partitions); + adminClient.topics().createPartitionedTopic(topic, this.partitions); } catch (PulsarAdminException.ConflictException alreadyExists) { if (log.isDebugEnabled()) { log.debug("Topic {} already exists: {}", topic, alreadyExists); } PartitionedTopicMetadata partitionedTopicMetadata = adminClient.topics() .getPartitionedTopicMetadata(topic); - if (partitionedTopicMetadata.partitions != arguments.partitions) { + if (partitionedTopicMetadata.partitions != this.partitions) { log.error("Topic {} already exists but it has a wrong number of partitions: {}, " + "expecting {}", - topic, partitionedTopicMetadata.partitions, arguments.partitions); + topic, partitionedTopicMetadata.partitions, this.partitions); PerfClientUtils.exit(1); } } @@ -342,23 +319,23 @@ public static void main(String[] args) throws Exception { } } - CountDownLatch doneLatch = new CountDownLatch(arguments.numTestThreads); + CountDownLatch doneLatch = new CountDownLatch(this.numTestThreads); - final long numMessagesPerThread = arguments.numMessages / arguments.numTestThreads; - final int msgRatePerThread = arguments.msgRate / arguments.numTestThreads; + final long numMessagesPerThread = this.numMessages / this.numTestThreads; + final int msgRatePerThread = this.msgRate / this.numTestThreads; - for (int i = 0; i < arguments.numTestThreads; i++) { + for (int i = 0; i < this.numTestThreads; i++) { final int threadIdx = i; executor.submit(() -> { log.info("Started performance test thread {}", threadIdx); runProducer( - threadIdx, - arguments, - numMessagesPerThread, - msgRatePerThread, - payloadByteList, - payloadBytes, - doneLatch + threadIdx, + this, + numMessagesPerThread, + msgRatePerThread, + payloadByteList, + payloadBytes, + doneLatch ); }); } @@ -369,8 +346,8 @@ public static void main(String[] args) throws Exception { Histogram reportHistogram = null; HistogramLogWriter histogramLogWriter = null; - if (arguments.histogramFile != null) { - String statsFileName = arguments.histogramFile; + if (this.histogramFile != null) { + String statsFileName = this.histogramFile; log.info("Dumping latency stats to {}", statsFileName); PrintStream histogramLog = new PrintStream(new FileOutputStream(statsFileName), false); @@ -404,7 +381,7 @@ public static void main(String[] args) throws Exception { reportHistogram = recorder.getIntervalHistogram(reportHistogram); - if (arguments.isEnableTransaction) { + if (this.isEnableTransaction) { totalTxnOpSuccess = totalEndTxnOpSuccessNum.sum(); totalTxnOpFail = totalEndTxnOpFailNum.sum(); rateOpenTxn = numTxnOpSuccess.sumThenReset() / elapsed; @@ -435,7 +412,9 @@ public static void main(String[] args) throws Exception { oldTime = now; } - PerfClientUtils.exit(0); + } + public PerformanceProducer() { + super("produce"); } private static void executorShutdownNow() { @@ -460,49 +439,49 @@ static IMessageFormatter getMessageFormatter(String formatterClass) { } } - static ProducerBuilder createProducerBuilder(PulsarClient client, Arguments arguments, int producerId) { + ProducerBuilder createProducerBuilder(PulsarClient client, int producerId) { ProducerBuilder producerBuilder = client.newProducer() // - .sendTimeout(arguments.sendTimeout, TimeUnit.SECONDS) // - .compressionType(arguments.compression) // - .maxPendingMessages(arguments.maxOutstanding) // - .accessMode(arguments.producerAccessMode) + .sendTimeout(this.sendTimeout, TimeUnit.SECONDS) // + .compressionType(this.compression) // + .maxPendingMessages(this.maxOutstanding) // + .accessMode(this.producerAccessMode) // enable round robin message routing if it is a partitioned topic .messageRoutingMode(MessageRoutingMode.RoundRobinPartition); - if (arguments.maxPendingMessagesAcrossPartitions > 0) { - producerBuilder.maxPendingMessagesAcrossPartitions(arguments.maxPendingMessagesAcrossPartitions); + if (this.maxPendingMessagesAcrossPartitions > 0) { + producerBuilder.maxPendingMessagesAcrossPartitions(this.maxPendingMessagesAcrossPartitions); } - if (arguments.producerName != null) { - String producerName = String.format("%s%s%d", arguments.producerName, arguments.separator, producerId); + if (this.producerName != null) { + String producerName = String.format("%s%s%d", this.producerName, this.separator, producerId); producerBuilder.producerName(producerName); } - if (arguments.disableBatching || (arguments.batchTimeMillis <= 0.0 && arguments.batchMaxMessages <= 0)) { + if (this.disableBatching || (this.batchTimeMillis <= 0.0 && this.batchMaxMessages <= 0)) { producerBuilder.enableBatching(false); } else { - long batchTimeUsec = (long) (arguments.batchTimeMillis * 1000); + long batchTimeUsec = (long) (this.batchTimeMillis * 1000); producerBuilder.batchingMaxPublishDelay(batchTimeUsec, TimeUnit.MICROSECONDS).enableBatching(true); } - if (arguments.batchMaxMessages > 0) { - producerBuilder.batchingMaxMessages(arguments.batchMaxMessages); + if (this.batchMaxMessages > 0) { + producerBuilder.batchingMaxMessages(this.batchMaxMessages); } - if (arguments.batchMaxBytes > 0) { - producerBuilder.batchingMaxBytes(arguments.batchMaxBytes); + if (this.batchMaxBytes > 0) { + producerBuilder.batchingMaxBytes(this.batchMaxBytes); } // Block if queue is full else we will start seeing errors in sendAsync producerBuilder.blockIfQueueFull(true); - if (isNotBlank(arguments.encKeyName) && isNotBlank(arguments.encKeyFile)) { - producerBuilder.addEncryptionKey(arguments.encKeyName); - producerBuilder.defaultCryptoKeyReader(arguments.encKeyFile); + if (isNotBlank(this.encKeyName) && isNotBlank(this.encKeyFile)) { + producerBuilder.addEncryptionKey(this.encKeyName); + producerBuilder.defaultCryptoKeyReader(this.encKeyFile); } return producerBuilder; } - private static void runProducer(int producerId, - Arguments arguments, + private void runProducer(int producerId, + PerformanceProducer arguments, long numMessages, int msgRate, List payloadByteList, @@ -516,31 +495,31 @@ private static void runProducer(int producerId, ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) - .enableTransaction(arguments.isEnableTransaction); + .enableTransaction(this.isEnableTransaction); client = clientBuilder.build(); - ProducerBuilder producerBuilder = createProducerBuilder(client, arguments, producerId); + ProducerBuilder producerBuilder = createProducerBuilder(client, producerId); AtomicReference transactionAtomicReference; - if (arguments.isEnableTransaction) { + if (this.isEnableTransaction) { producerBuilder.sendTimeout(0, TimeUnit.SECONDS); transactionAtomicReference = new AtomicReference<>(client.newTransaction() - .withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS) + .withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS) .build() .get()); } else { transactionAtomicReference = new AtomicReference<>(null); } - for (int i = 0; i < arguments.numTopics; i++) { + for (int i = 0; i < this.numTopics; i++) { - String topic = arguments.topics.get(i); - log.info("Adding {} publishers on topic {}", arguments.numProducers, topic); + String topic = this.topics.get(i); + log.info("Adding {} publishers on topic {}", this.numProducers, topic); - for (int j = 0; j < arguments.numProducers; j++) { + for (int j = 0; j < this.numProducers; j++) { ProducerBuilder prodBuilder = producerBuilder.clone().topic(topic); - if (arguments.chunkingAllowed) { + if (this.chunkingAllowed) { prodBuilder.enableChunking(true); prodBuilder.enableBatching(false); } @@ -559,12 +538,12 @@ private static void runProducer(int producerId, RateLimiter rateLimiter = RateLimiter.create(msgRate); long startTime = System.nanoTime(); - long warmupEndTime = startTime + (long) (arguments.warmupTimeSeconds * 1e9); - long testEndTime = startTime + (long) (arguments.testTime * 1e9); + long warmupEndTime = startTime + (long) (this.warmupTimeSeconds * 1e9); + long testEndTime = startTime + (long) (this.testTime * 1e9); MessageKeyGenerationMode msgKeyMode = null; - if (isNotBlank(arguments.messageKeyGenerationMode)) { + if (isNotBlank(this.messageKeyGenerationMode)) { try { - msgKeyMode = MessageKeyGenerationMode.valueOf(arguments.messageKeyGenerationMode); + msgKeyMode = MessageKeyGenerationMode.valueOf(this.messageKeyGenerationMode); } catch (IllegalArgumentException e) { throw new IllegalArgumentException("messageKeyGenerationMode only support [autoIncrement, random]"); } @@ -572,16 +551,16 @@ private static void runProducer(int producerId, // Send messages on all topics/producers AtomicLong totalSent = new AtomicLong(0); AtomicLong numMessageSend = new AtomicLong(0); - Semaphore numMsgPerTxnLimit = new Semaphore(arguments.numMessagesPerTransaction); + Semaphore numMsgPerTxnLimit = new Semaphore(this.numMessagesPerTransaction); while (true) { if (produceEnough) { break; } for (Producer producer : producers) { - if (arguments.testTime > 0) { + if (this.testTime > 0) { if (System.nanoTime() > testEndTime) { log.info("------------- DONE (reached the maximum duration: [{} seconds] of production) " - + "--------------", arguments.testTime); + + "--------------", this.testTime); doneLatch.countDown(); produceEnough = true; break; @@ -604,9 +583,9 @@ private static void runProducer(int producerId, byte[] payloadData; - if (arguments.payloadFilename != null) { + if (this.payloadFilename != null) { if (messageFormatter != null) { - payloadData = messageFormatter.formatMessage(arguments.producerName, totalSent.get(), + payloadData = messageFormatter.formatMessage(this.producerName, totalSent.get(), payloadByteList.get(ThreadLocalRandom.current().nextInt(payloadByteList.size()))); } else { payloadData = payloadByteList.get( @@ -616,8 +595,8 @@ private static void runProducer(int producerId, payloadData = payloadBytes; } TypedMessageBuilder messageBuilder; - if (arguments.isEnableTransaction) { - if (arguments.numMessagesPerTransaction > 0) { + if (this.isEnableTransaction) { + if (this.numMessagesPerTransaction > 0) { try { numMsgPerTxnLimit.acquire(); } catch (InterruptedException exception){ @@ -630,14 +609,14 @@ private static void runProducer(int producerId, messageBuilder = producer.newMessage() .value(payloadData); } - if (arguments.delay > 0) { - messageBuilder.deliverAfter(arguments.delay, TimeUnit.SECONDS); - } else if (arguments.delayRange != null) { + if (this.delay > 0) { + messageBuilder.deliverAfter(this.delay, TimeUnit.SECONDS); + } else if (this.delayRange != null) { final long deliverAfter = ThreadLocalRandom.current() - .nextLong(arguments.delayRange.lowerEndpoint(), arguments.delayRange.upperEndpoint()); + .nextLong(this.delayRange.lowerEndpoint(), this.delayRange.upperEndpoint()); messageBuilder.deliverAfter(deliverAfter, TimeUnit.SECONDS); } - if (arguments.setEventTime) { + if (this.setEventTime) { messageBuilder.eventTime(System.currentTimeMillis()); } //generate msg key @@ -668,14 +647,14 @@ private static void runProducer(int producerId, } log.warn("Write message error with exception", ex); messagesFailed.increment(); - if (arguments.exitOnFailure) { + if (this.exitOnFailure) { PerfClientUtils.exit(1); } return null; }); - if (arguments.isEnableTransaction - && numMessageSend.incrementAndGet() == arguments.numMessagesPerTransaction) { - if (!arguments.isAbortTransaction) { + if (this.isEnableTransaction + && numMessageSend.incrementAndGet() == this.numMessagesPerTransaction) { + if (!this.isAbortTransaction) { transaction.commit() .thenRun(() -> { if (log.isDebugEnabled()) { @@ -709,11 +688,11 @@ private static void runProducer(int producerId, while (true) { try { Transaction newTransaction = pulsarClient.newTransaction() - .withTransactionTimeout(arguments.transactionTimeout, + .withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS).build().get(); transactionAtomicReference.compareAndSet(transaction, newTransaction); numMessageSend.set(0); - numMsgPerTxnLimit.release(arguments.numMessagesPerTransaction); + numMsgPerTxnLimit.release(this.numMessagesPerTransaction); totalNumTxnOpenTxnSuccess.increment(); break; } catch (Exception e){ @@ -740,7 +719,7 @@ private static void runProducer(int producerId, } } - private static void printAggregatedThroughput(long start, Arguments arguments) { + private void printAggregatedThroughput(long start) { double elapsed = (System.nanoTime() - start) / 1e9; double rate = totalMessagesSent.sum() / elapsed; double throughput = totalBytesSent.sum() / elapsed / 1024 / 1024 * 8; @@ -750,7 +729,7 @@ private static void printAggregatedThroughput(long start, Arguments arguments) { long numTransactionOpenFailed = 0; long numTransactionOpenSuccess = 0; - if (arguments.isEnableTransaction) { + if (this.isEnableTransaction) { totalTxnSuccess = totalEndTxnOpSuccessNum.sum(); totalTxnFail = totalEndTxnOpFailNum.sum(); rateOpenTxn = elapsed / (totalTxnFail + totalTxnSuccess); diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceReader.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceReader.java index 3572cbde43cb7..3c6940b262f44 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceReader.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceReader.java @@ -24,7 +24,6 @@ import java.text.DecimalFormat; import java.util.ArrayList; import java.util.List; -import java.util.Properties; import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.CompletableFuture; @@ -46,9 +45,9 @@ import org.slf4j.LoggerFactory; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import picocli.CommandLine.ScopeType; -public class PerformanceReader { +@Command(name = "read", description = "Test pulsar reader performance.") +public class PerformanceReader extends PerformanceTopicListArguments { private static final LongAdder messagesReceived = new LongAdder(); private static final LongAdder bytesReceived = new LongAdder(); private static final DecimalFormat intFormat = new PaddingDecimalFormat("0", 7); @@ -60,62 +59,53 @@ public class PerformanceReader { private static Recorder recorder = new Recorder(TimeUnit.DAYS.toMillis(10), 5); private static Recorder cumulativeRecorder = new Recorder(TimeUnit.DAYS.toMillis(10), 5); - @Command(description = "Test pulsar reader performance.", showDefaultValues = true, scope = ScopeType.INHERIT) - static class Arguments extends PerformanceTopicListArguments { + @Option(names = {"-r", "--rate"}, description = "Simulate a slow message reader (rate in msg/s)") + public double rate = 0; - @Option(names = { "-r", "--rate" }, description = "Simulate a slow message reader (rate in msg/s)") - public double rate = 0; + @Option(names = {"-m", + "--start-message-id"}, description = "Start message id. This can be either 'earliest', " + + "'latest' or a specific message id by using 'lid:eid'") + public String startMessageId = "earliest"; - @Option(names = { "-m", - "--start-message-id" }, description = "Start message id. This can be either 'earliest', " - + "'latest' or a specific message id by using 'lid:eid'") - public String startMessageId = "earliest"; + @Option(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue") + public int receiverQueueSize = 1000; - @Option(names = { "-q", "--receiver-queue-size" }, description = "Size of the receiver queue") - public int receiverQueueSize = 1000; + @Option(names = {"-n", + "--num-messages"}, description = "Number of messages to consume in total. If <= 0, " + + "it will keep consuming") + public long numMessages = 0; - @Option(names = {"-n", - "--num-messages"}, description = "Number of messages to consume in total. If <= 0, " - + "it will keep consuming") - public long numMessages = 0; + @Option(names = { + "--use-tls"}, description = "Use TLS encryption on the connection", descriptionKey = "useTls") + public boolean useTls; - @Option(names = { - "--use-tls" }, description = "Use TLS encryption on the connection") - public boolean useTls; - - @Option(names = { "-time", - "--test-duration" }, description = "Test duration in secs. If <= 0, it will keep consuming") - public long testTime = 0; + @Option(names = {"-time", + "--test-duration"}, description = "Test duration in secs. If <= 0, it will keep consuming") + public long testTime = 0; + public PerformanceReader() { + super("read"); + } - @Override - public void fillArgumentsFromProperties(Properties prop) { - if (!useTls) { - useTls = Boolean.parseBoolean(prop.getProperty("useTls")); - } - } - @Override - public void validate() throws Exception { - super.validate(); - if (startMessageId != "earliest" && startMessageId != "latest" - && (startMessageId.split(":")).length != 2) { - String errMsg = String.format("invalid start message ID '%s', must be either either 'earliest', " - + "'latest' or a specific message id by using 'lid:eid'", startMessageId); - throw new Exception(errMsg); - } + @Override + public void validate() throws Exception { + super.validate(); + if (startMessageId != "earliest" && startMessageId != "latest" + && (startMessageId.split(":")).length != 2) { + String errMsg = String.format("invalid start message ID '%s', must be either either 'earliest', " + + "'latest' or a specific message id by using 'lid:eid'", startMessageId); + throw new Exception(errMsg); } } - public static void main(String[] args) throws Exception { - final Arguments arguments = new Arguments(); - arguments.parseCLI("pulsar-perf read", args); - + @Override + public void run() throws Exception { // Dump config variables PerfClientUtils.printJVMInformation(log); ObjectMapper m = new ObjectMapper(); ObjectWriter w = m.writerWithDefaultPrettyPrinter(); - log.info("Starting Pulsar performance reader with config: {}", w.writeValueAsString(arguments)); + log.info("Starting Pulsar performance reader with config: {}", w.writeValueAsString(this)); - final RateLimiter limiter = arguments.rate > 0 ? RateLimiter.create(arguments.rate) : null; + final RateLimiter limiter = this.rate > 0 ? RateLimiter.create(this.rate) : null; ReaderListener listener = (reader, msg) -> { messagesReceived.increment(); bytesReceived.add(msg.getData().length); @@ -123,9 +113,9 @@ public static void main(String[] args) throws Exception { totalMessagesReceived.increment(); totalBytesReceived.add(msg.getData().length); - if (arguments.numMessages > 0 && totalMessagesReceived.sum() >= arguments.numMessages) { + if (this.numMessages > 0 && totalMessagesReceived.sum() >= this.numMessages) { log.info("------------- DONE (reached the maximum number: [{}] of consumption) --------------", - arguments.numMessages); + this.numMessages); PerfClientUtils.exit(0); } @@ -140,37 +130,37 @@ public static void main(String[] args) throws Exception { } }; - ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) - .enableTls(arguments.useTls); + ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(this) + .enableTls(this.useTls); PulsarClient pulsarClient = clientBuilder.build(); List>> futures = new ArrayList<>(); MessageId startMessageId; - if ("earliest".equals(arguments.startMessageId)) { + if ("earliest".equals(this.startMessageId)) { startMessageId = MessageId.earliest; - } else if ("latest".equals(arguments.startMessageId)) { + } else if ("latest".equals(this.startMessageId)) { startMessageId = MessageId.latest; } else { - String[] parts = arguments.startMessageId.split(":"); + String[] parts = this.startMessageId.split(":"); startMessageId = new MessageIdImpl(Long.parseLong(parts[0]), Long.parseLong(parts[1]), -1); } ReaderBuilder readerBuilder = pulsarClient.newReader() // .readerListener(listener) // - .receiverQueueSize(arguments.receiverQueueSize) // + .receiverQueueSize(this.receiverQueueSize) // .startMessageId(startMessageId); - for (int i = 0; i < arguments.numTopics; i++) { - final TopicName topicName = TopicName.get(arguments.topics.get(i)); + for (int i = 0; i < this.numTopics; i++) { + final TopicName topicName = TopicName.get(this.topics.get(i)); futures.add(readerBuilder.clone().topic(topicName.toString()).createAsync()); } FutureUtil.waitForAll(futures).get(); - log.info("Start reading from {} topics", arguments.numTopics); + log.info("Start reading from {} topics", this.numTopics); final long start = System.nanoTime(); Runtime.getRuntime().addShutdownHook(new Thread(() -> { @@ -178,17 +168,17 @@ public static void main(String[] args) throws Exception { printAggregatedStats(); })); - if (arguments.testTime > 0) { + if (this.testTime > 0) { TimerTask timoutTask = new TimerTask() { @Override public void run() { log.info("------------- DONE (reached the maximum duration: [{} seconds] of consumption) " - + "--------------", arguments.testTime); + + "--------------", testTime); PerfClientUtils.exit(0); } }; Timer timer = new Timer(); - timer.schedule(timoutTask, arguments.testTime * 1000); + timer.schedule(timoutTask, this.testTime * 1000); } long oldTime = System.nanoTime(); @@ -223,7 +213,6 @@ public void run() { pulsarClient.close(); } - private static void printAggregatedThroughput(long start) { double elapsed = (System.nanoTime() - start) / 1e9; double rate = totalMessagesReceived.sum() / elapsed; diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTopicListArguments.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTopicListArguments.java index 9ac99d0abcca5..e4771c3652fb1 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTopicListArguments.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTopicListArguments.java @@ -39,6 +39,10 @@ public abstract class PerformanceTopicListArguments extends PerformanceBaseArgum ) public int numTopics = 1; + public PerformanceTopicListArguments(String cmdName) { + super(cmdName); + } + @Override public void validate() throws Exception { super.validate(); diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java index 02e50ab4e2bb9..943cfaf451032 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceTransaction.java @@ -23,14 +23,12 @@ import com.fasterxml.jackson.databind.ObjectWriter; import com.google.common.util.concurrent.RateLimiter; import java.io.FileOutputStream; -import java.io.IOException; import java.io.PrintStream; import java.text.DecimalFormat; import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Properties; import java.util.Random; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -65,9 +63,9 @@ import org.slf4j.LoggerFactory; import picocli.CommandLine.Command; import picocli.CommandLine.Option; -import picocli.CommandLine.ScopeType; -public class PerformanceTransaction { +@Command(name = "transaction", description = "Test pulsar transaction performance.") +public class PerformanceTransaction extends PerformanceBaseArguments{ private static final LongAdder totalNumEndTxnOpFailed = new LongAdder(); private static final LongAdder totalNumEndTxnOpSuccess = new LongAdder(); @@ -90,132 +88,120 @@ public class PerformanceTransaction { private static final Recorder messageSendRCumulativeRecorder = new Recorder(TimeUnit.SECONDS.toMicros(120000), 5); - @Command(description = "Test pulsar transaction performance.", showDefaultValues = true, scope = ScopeType.INHERIT) - static class Arguments extends PerformanceBaseArguments { + @Option(names = "--topics-c", description = "All topics that need ack for a transaction", required = + true) + public List consumerTopic = Collections.singletonList("test-consume"); - @Option(names = "--topics-c", description = "All topics that need ack for a transaction", required = - true) - public List consumerTopic = Collections.singletonList("test-consume"); + @Option(names = "--topics-p", description = "All topics that need produce for a transaction", + required = true) + public List producerTopic = Collections.singletonList("test-produce"); - @Option(names = "--topics-p", description = "All topics that need produce for a transaction", - required = true) - public List producerTopic = Collections.singletonList("test-produce"); + @Option(names = {"-threads", "--num-test-threads"}, description = "Number of test threads." + + "This thread is for a new transaction to ack messages from consumer topics and produce message to " + + "producer topics, and then commit or abort this transaction. " + + "Increasing the number of threads increases the parallelism of the performance test, " + + "thereby increasing the intensity of the stress test.") + public int numTestThreads = 1; - @Option(names = {"-threads", "--num-test-threads"}, description = "Number of test threads." - + "This thread is for a new transaction to ack messages from consumer topics and produce message to " - + "producer topics, and then commit or abort this transaction. " - + "Increasing the number of threads increases the parallelism of the performance test, " - + "thereby increasing the intensity of the stress test.") - public int numTestThreads = 1; + @Option(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL", descriptionKey = "webServiceUrl") + public String adminURL; - @Option(names = {"-au", "--admin-url"}, description = "Pulsar Admin URL") - public String adminURL; + @Option(names = {"-np", + "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means" + + "not trying to create a topic") + public Integer partitions = null; - @Option(names = {"-np", - "--partitions"}, description = "Create partitioned topics with a given number of partitions, 0 means" - + "not trying to create a topic") - public Integer partitions = null; + @Option(names = {"-time", + "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing") + public long testTime = 0; - @Option(names = {"-time", - "--test-duration"}, description = "Test duration (in second). 0 means keeping publishing") - public long testTime = 0; + @Option(names = {"-ss", + "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)") + public List subscriptions = Collections.singletonList("sub"); - @Option(names = {"-ss", - "--subscriptions"}, description = "A list of subscriptions to consume (for example, sub1,sub2)") - public List subscriptions = Collections.singletonList("sub"); + @Option(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)") + public int numSubscriptions = 1; - @Option(names = {"-ns", "--num-subscriptions"}, description = "Number of subscriptions (per topic)") - public int numSubscriptions = 1; + @Option(names = {"-sp", "--subscription-position"}, description = "Subscription position") + private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest; - @Option(names = {"-sp", "--subscription-position"}, description = "Subscription position") - private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Earliest; + @Option(names = {"-st", "--subscription-type"}, description = "Subscription type") + public SubscriptionType subscriptionType = SubscriptionType.Shared; - @Option(names = {"-st", "--subscription-type"}, description = "Subscription type") - public SubscriptionType subscriptionType = SubscriptionType.Shared; + @Option(names = {"-rs", "--replicated" }, + description = "Whether the subscription status should be replicated") + private boolean replicatedSubscription = false; - @Option(names = {"-rs", "--replicated" }, - description = "Whether the subscription status should be replicated") - private boolean replicatedSubscription = false; + @Option(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue") + public int receiverQueueSize = 1000; - @Option(names = {"-q", "--receiver-queue-size"}, description = "Size of the receiver queue") - public int receiverQueueSize = 1000; + @Option(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout," + + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)") + public long transactionTimeout = 5; - @Option(names = {"-tto", "--txn-timeout"}, description = "Set the time value of transaction timeout," - + " and the time unit is second. (After --txn-enable setting to true, --txn-timeout takes effect)") - public long transactionTimeout = 5; + @Option(names = {"-ntxn", + "--number-txn"}, description = "Set the number of transaction. 0 means keeping open." + + "If transaction disabled, it means the number of tasks. The task or transaction produces or " + + "consumes a specified number of messages.") + public long numTransactions = 0; - @Option(names = {"-ntxn", - "--number-txn"}, description = "Set the number of transaction. 0 means keeping open." - + "If transaction disabled, it means the number of tasks. The task or transaction produces or " - + "consumes a specified number of messages.") - public long numTransactions = 0; + @Option(names = {"-nmp", "--numMessage-perTransaction-produce"}, + description = "Set the number of messages produced in a transaction." + + "If transaction disabled, it means the number of messages produced in a task.") + public int numMessagesProducedPerTransaction = 1; - @Option(names = {"-nmp", "--numMessage-perTransaction-produce"}, - description = "Set the number of messages produced in a transaction." - + "If transaction disabled, it means the number of messages produced in a task.") - public int numMessagesProducedPerTransaction = 1; + @Option(names = {"-nmc", "--numMessage-perTransaction-consume"}, + description = "Set the number of messages consumed in a transaction." + + "If transaction disabled, it means the number of messages consumed in a task.") + public int numMessagesReceivedPerTransaction = 1; - @Option(names = {"-nmc", "--numMessage-perTransaction-consume"}, - description = "Set the number of messages consumed in a transaction." - + "If transaction disabled, it means the number of messages consumed in a task.") - public int numMessagesReceivedPerTransaction = 1; + @Option(names = {"--txn-disable"}, description = "Disable transaction") + public boolean isDisableTransaction = false; - @Option(names = {"--txn-disable"}, description = "Disable transaction") - public boolean isDisableTransaction = false; + @Option(names = {"-abort"}, description = "Abort the transaction. (After --txn-disEnable " + + "setting to false, -abort takes effect)") + public boolean isAbortTransaction = false; - @Option(names = {"-abort"}, description = "Abort the transaction. (After --txn-disEnable " - + "setting to false, -abort takes effect)") - public boolean isAbortTransaction = false; - - @Option(names = "-txnRate", description = "Set the rate of opened transaction or task. 0 means no limit") - public int openTxnRate = 0; - - @Override - public void fillArgumentsFromProperties(Properties prop) { - if (adminURL == null) { - adminURL = prop.getProperty("webServiceUrl"); - } - if (adminURL == null) { - adminURL = prop.getProperty("adminURL", "http://localhost:8080/"); - } - } + @Option(names = "-txnRate", description = "Set the rate of opened transaction or task. 0 means no limit") + public int openTxnRate = 0; + public PerformanceTransaction() { + super("transaction"); } - public static void main(String[] args) - throws IOException, PulsarAdminException, ExecutionException, InterruptedException { - final Arguments arguments = new Arguments(); - arguments.parseCLI("pulsar-perf transaction", args); + @Override + public void run() throws Exception { + super.parseCLI(); // Dump config variables PerfClientUtils.printJVMInformation(log); ObjectMapper m = new ObjectMapper(); ObjectWriter w = m.writerWithDefaultPrettyPrinter(); - log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(arguments)); + log.info("Starting Pulsar perf transaction with config: {}", w.writeValueAsString(this)); final byte[] payloadBytes = new byte[1024]; Random random = new Random(0); for (int i = 0; i < payloadBytes.length; ++i) { payloadBytes[i] = (byte) (random.nextInt(26) + 65); } - if (arguments.partitions != null) { + if (this.partitions != null) { final PulsarAdminBuilder adminBuilder = PerfClientUtils - .createAdminBuilderFromArguments(arguments, arguments.adminURL); + .createAdminBuilderFromArguments(this, this.adminURL); try (PulsarAdmin adminClient = adminBuilder.build()) { - for (String topic : arguments.producerTopic) { - log.info("Creating produce partitioned topic {} with {} partitions", topic, arguments.partitions); + for (String topic : this.producerTopic) { + log.info("Creating produce partitioned topic {} with {} partitions", topic, this.partitions); try { - adminClient.topics().createPartitionedTopic(topic, arguments.partitions); + adminClient.topics().createPartitionedTopic(topic, this.partitions); } catch (PulsarAdminException.ConflictException alreadyExists) { if (log.isDebugEnabled()) { log.debug("Topic {} already exists: {}", topic, alreadyExists); } PartitionedTopicMetadata partitionedTopicMetadata = adminClient.topics().getPartitionedTopicMetadata(topic); - if (partitionedTopicMetadata.partitions != arguments.partitions) { + if (partitionedTopicMetadata.partitions != this.partitions) { log.error( "Topic {} already exists but it has a wrong number of partitions: {}, expecting {}", - topic, partitionedTopicMetadata.partitions, arguments.partitions); + topic, partitionedTopicMetadata.partitions, this.partitions); PerfClientUtils.exit(1); } } @@ -223,21 +209,21 @@ public static void main(String[] args) } } - ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) - .enableTransaction(!arguments.isDisableTransaction); + ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(this) + .enableTransaction(!this.isDisableTransaction); try (PulsarClient client = clientBuilder.build()) { - ExecutorService executorService = new ThreadPoolExecutor(arguments.numTestThreads, - arguments.numTestThreads, + ExecutorService executorService = new ThreadPoolExecutor(this.numTestThreads, + this.numTestThreads, 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>()); long startTime = System.nanoTime(); - long testEndTime = startTime + (long) (arguments.testTime * 1e9); + long testEndTime = startTime + (long) (this.testTime * 1e9); Runtime.getRuntime().addShutdownHook(new Thread(() -> { - if (!arguments.isDisableTransaction) { + if (!this.isDisableTransaction) { printTxnAggregatedThroughput(startTime); } else { printAggregatedThroughput(startTime); @@ -248,10 +234,10 @@ public static void main(String[] args) // start perf test AtomicBoolean executing = new AtomicBoolean(true); - RateLimiter rateLimiter = arguments.openTxnRate > 0 - ? RateLimiter.create(arguments.openTxnRate) + RateLimiter rateLimiter = this.openTxnRate > 0 + ? RateLimiter.create(this.openTxnRate) : null; - for (int i = 0; i < arguments.numTestThreads; i++) { + for (int i = 0; i < this.numTestThreads; i++) { executorService.submit(() -> { //The producer and consumer clients are built in advance, and then this thread is //responsible for the production and consumption tasks of the transaction through the loop. @@ -260,11 +246,11 @@ public static void main(String[] args) List>> consumers = null; AtomicReference atomicReference = null; try { - producers = buildProducers(client, arguments); - consumers = buildConsumer(client, arguments); - if (!arguments.isDisableTransaction) { + producers = buildProducers(client); + consumers = buildConsumer(client); + if (!this.isDisableTransaction) { atomicReference = new AtomicReference<>(client.newTransaction() - .withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS) + .withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS) .build() .get()); } else { @@ -278,11 +264,11 @@ public static void main(String[] args) //The while loop has no break, and finally ends the execution through the shutdownNow of //the executorService while (true) { - if (arguments.numTransactions > 0) { + if (this.numTransactions > 0) { if (totalNumTxnOpenTxnFail.sum() - + totalNumTxnOpenTxnSuccess.sum() >= arguments.numTransactions) { + + totalNumTxnOpenTxnSuccess.sum() >= this.numTransactions) { if (totalNumEndTxnOpFailed.sum() - + totalNumEndTxnOpSuccess.sum() < arguments.numTransactions) { + + totalNumEndTxnOpSuccess.sum() < this.numTransactions) { continue; } log.info("------------------- DONE -----------------------"); @@ -292,7 +278,7 @@ public static void main(String[] args) break; } } - if (arguments.testTime > 0) { + if (this.testTime > 0) { if (System.nanoTime() > testEndTime) { log.info("------------------- DONE -----------------------"); executing.compareAndSet(true, false); @@ -304,7 +290,7 @@ public static void main(String[] args) Transaction transaction = atomicReference.get(); for (List> subscriptions : consumers) { for (Consumer consumer : subscriptions) { - for (int j = 0; j < arguments.numMessagesReceivedPerTransaction; j++) { + for (int j = 0; j < this.numMessagesReceivedPerTransaction; j++) { Message message = null; try { message = consumer.receive(); @@ -314,7 +300,7 @@ public static void main(String[] args) PerfClientUtils.exit(1); } long receiveTime = System.nanoTime(); - if (!arguments.isDisableTransaction) { + if (!this.isDisableTransaction) { consumer.acknowledgeAsync(message.getMessageId(), transaction) .thenRun(() -> { long latencyMicros = NANOSECONDS.toMicros( @@ -355,9 +341,9 @@ public static void main(String[] args) } for (Producer producer : producers) { - for (int j = 0; j < arguments.numMessagesProducedPerTransaction; j++) { + for (int j = 0; j < this.numMessagesProducedPerTransaction; j++) { long sendTime = System.nanoTime(); - if (!arguments.isDisableTransaction) { + if (!this.isDisableTransaction) { producer.newMessage(transaction).value(payloadBytes) .sendAsync().thenRun(() -> { long latencyMicros = NANOSECONDS.toMicros( @@ -397,8 +383,8 @@ public static void main(String[] args) if (rateLimiter != null) { rateLimiter.tryAcquire(); } - if (!arguments.isDisableTransaction) { - if (!arguments.isAbortTransaction) { + if (!this.isDisableTransaction) { + if (!this.isAbortTransaction) { transaction.commit() .thenRun(() -> { numTxnOpSuccess.increment(); @@ -431,7 +417,7 @@ public static void main(String[] args) while (true) { try { Transaction newTransaction = client.newTransaction() - .withTransactionTimeout(arguments.transactionTimeout, TimeUnit.SECONDS) + .withTransactionTimeout(this.transactionTimeout, TimeUnit.SECONDS) .build() .get(); atomicReference.compareAndSet(transaction, newTransaction); @@ -483,7 +469,7 @@ public static void main(String[] args) double rate = numTxnOpSuccess.sumThenReset() / elapsed; reportSendHistogram = messageSendRecorder.getIntervalHistogram(reportSendHistogram); reportAckHistogram = messageAckRecorder.getIntervalHistogram(reportAckHistogram); - String txnOrTaskLog = !arguments.isDisableTransaction + String txnOrTaskLog = !this.isDisableTransaction ? "Throughput transaction: {} transaction executes --- {} transaction/s" : "Throughput task: {} task executes --- {} task/s"; log.info( @@ -609,24 +595,24 @@ private static void printAggregatedStats() { private static final Logger log = LoggerFactory.getLogger(PerformanceTransaction.class); - private static List>> buildConsumer(PulsarClient client, Arguments arguments) + private List>> buildConsumer(PulsarClient client) throws ExecutionException, InterruptedException { ConsumerBuilder consumerBuilder = client.newConsumer(Schema.BYTES) - .subscriptionType(arguments.subscriptionType) - .receiverQueueSize(arguments.receiverQueueSize) - .subscriptionInitialPosition(arguments.subscriptionInitialPosition) - .replicateSubscriptionState(arguments.replicatedSubscription); + .subscriptionType(this.subscriptionType) + .receiverQueueSize(this.receiverQueueSize) + .subscriptionInitialPosition(this.subscriptionInitialPosition) + .replicateSubscriptionState(this.replicatedSubscription); - Iterator consumerTopicsIterator = arguments.consumerTopic.iterator(); - List>> consumers = new ArrayList<>(arguments.consumerTopic.size()); + Iterator consumerTopicsIterator = this.consumerTopic.iterator(); + List>> consumers = new ArrayList<>(this.consumerTopic.size()); while (consumerTopicsIterator.hasNext()){ String topic = consumerTopicsIterator.next(); - final List> subscriptions = new ArrayList<>(arguments.numSubscriptions); + final List> subscriptions = new ArrayList<>(this.numSubscriptions); final List>> subscriptionFutures = - new ArrayList<>(arguments.numSubscriptions); + new ArrayList<>(this.numSubscriptions); log.info("Create subscriptions for topic {}", topic); - for (int j = 0; j < arguments.numSubscriptions; j++) { - String subscriberName = arguments.subscriptions.get(j); + for (int j = 0; j < this.numSubscriptions; j++) { + String subscriberName = this.subscriptions.get(j); subscriptionFutures .add(consumerBuilder.clone().topic(topic).subscriptionName(subscriberName) .subscribeAsync()); @@ -639,14 +625,14 @@ private static List>> buildConsumer(PulsarClient client, return consumers; } - private static List> buildProducers(PulsarClient client, Arguments arguments) + private List> buildProducers(PulsarClient client) throws ExecutionException, InterruptedException { ProducerBuilder producerBuilder = client.newProducer(Schema.BYTES) .sendTimeout(0, TimeUnit.SECONDS); final List>> producerFutures = new ArrayList<>(); - for (String topic : arguments.producerTopic) { + for (String topic : this.producerTopic) { log.info("Create producer for topic {}", topic); producerFutures.add(producerBuilder.clone().topic(topic).createAsync()); } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ProxyProtocolConverter.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ProxyProtocolConverter.java new file mode 100644 index 0000000000000..6cccc8ce480ae --- /dev/null +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ProxyProtocolConverter.java @@ -0,0 +1,35 @@ +/* + * 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.testclient; + +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.api.ProxyProtocol; +import picocli.CommandLine.ITypeConverter; + +public class ProxyProtocolConverter implements ITypeConverter { + + @Override + public ProxyProtocol convert(String value) throws Exception { + String proxyProtocolString = StringUtils.trimToNull(value); + if (proxyProtocolString != null) { + return ProxyProtocol.valueOf(proxyProtocolString.toUpperCase()); + } + return null; + } +} diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PulsarPerfTestTool.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PulsarPerfTestTool.java new file mode 100644 index 0000000000000..826060dc6b799 --- /dev/null +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PulsarPerfTestTool.java @@ -0,0 +1,133 @@ +/* + * 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.testclient; + +import static org.apache.commons.lang3.StringUtils.isBlank; +import static org.apache.commons.lang3.StringUtils.isNotBlank; +import java.io.FileInputStream; +import java.lang.reflect.Constructor; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import org.apache.pulsar.proxy.socket.client.PerformanceClient; +import picocli.CommandLine; + +@CommandLine.Command(name = "pulsar-perf", + scope = CommandLine.ScopeType.INHERIT, + mixinStandardHelpOptions = true, + showDefaultValues = true +) +public class PulsarPerfTestTool { + + protected Map> commandMap; + protected final CommandLine commander; + + public PulsarPerfTestTool() { + this.commander = new CommandLine(this); + commandMap = new HashMap<>(); + } + + private String[] initCommander(String[] args) throws Exception { + commandMap.put("produce", PerformanceProducer.class); + commandMap.put("consume", PerformanceConsumer.class); + commandMap.put("transaction", PerformanceTransaction.class); + commandMap.put("read", PerformanceReader.class); + commandMap.put("monitor-brokers", BrokerMonitor.class); + commandMap.put("simulation-client", LoadSimulationClient.class); + commandMap.put("simulation-controller", LoadSimulationController.class); + commandMap.put("websocket-producer", PerformanceClient.class); + commandMap.put("managed-ledger", ManagedLedgerWriter.class); + commandMap.put("gen-doc", CmdGenerateDocumentation.class); + if (args.length == 0) { + System.out.println("Usage: pulsar-perf CONF_FILE_PATH [options] [command] [command options]"); + PerfClientUtils.exit(0); + } + String configFile = args[0]; + Properties prop = new Properties(System.getProperties()); + if (configFile != null) { + try (FileInputStream fis = new FileInputStream(configFile)) { + prop.load(fis); + } + } + commander.setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + + for (Map.Entry> c : commandMap.entrySet()) { + Constructor constructor = c.getValue().getDeclaredConstructor(); + constructor.setAccessible(true); + addCommand(c.getKey(), constructor.newInstance()); + } + + // Remove the first argument, it's the config file path + return Arrays.copyOfRange(args, 1, args.length); + } + + private void addCommand(String name, Object o) { + if (o instanceof CmdBase) { + commander.addSubcommand(name, ((CmdBase) o).getCommander()); + } else { + commander.addSubcommand(o); + } + } + + public static void main(String[] args) throws Exception { + PulsarPerfTestTool tool = new PulsarPerfTestTool(); + args = tool.initCommander(args); + + if (tool.run(args)) { + PerfClientUtils.exit(0); + } else { + PerfClientUtils.exit(1); + } + } + + protected boolean run(String[] args) { + return commander.execute(args) == 0; + } + +} + +class PulsarPerfTestPropertiesProvider extends CommandLine.PropertiesDefaultProvider{ + private static final String brokerServiceUrlKey = "brokerServiceUrl"; + private static final String webServiceUrlKey = "webServiceUrl"; + private final Properties properties; + + public PulsarPerfTestPropertiesProvider(Properties properties) { + super(properties); + this.properties = properties; + } + + static PulsarPerfTestPropertiesProvider create(Properties properties) { + if (isBlank(properties.getProperty(brokerServiceUrlKey))) { + String webServiceUrl = properties.getProperty("webServiceUrl"); + if (isNotBlank(webServiceUrl)) { + properties.put(brokerServiceUrlKey, webServiceUrl); + } else if (isNotBlank(properties.getProperty("serviceUrl"))) { + properties.put(brokerServiceUrlKey, properties.getProperty("serviceUrl", "http://localhost:8080/")); + } + } + + // Used for produce and transaction to fill parameters. + if (isBlank(properties.getProperty(webServiceUrlKey))) { + properties.put(webServiceUrlKey, properties.getProperty("adminURL", "http://localhost:8080/")); + } + + return new PulsarPerfTestPropertiesProvider(properties); + } +} diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/proxy/socket/client/PerformanceClientTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/proxy/socket/client/PerformanceClientTest.java index f623662e0e946..d45c3e8f3a4e7 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/proxy/socket/client/PerformanceClientTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/proxy/socket/client/PerformanceClientTest.java @@ -31,29 +31,34 @@ public void testLoadArguments() throws Exception { PerformanceClient client = new PerformanceClient(); // "--proxy-url" has the highest priority - PerformanceClient.Arguments arguments = client.loadArguments( - getArgs("ws://broker0.pulsar.apache.org:8080/", "./src/test/resources/websocket_client1.conf")); - assertEquals(arguments.proxyURL, "ws://broker0.pulsar.apache.org:8080/"); + client.parse(getArgs("ws://broker0.pulsar.apache.org:8080/", "./src/test/resources/websocket_client1.conf")); + client.loadArguments(); + assertEquals(client.proxyURL, "ws://broker0.pulsar.apache.org:8080/"); // "webSocketServiceUrl" written in the conf file has the second priority - arguments = client.loadArguments(getArgs(null, "./src/test/resources/websocket_client1.conf")); - assertEquals(arguments.proxyURL, "ws://broker1.pulsar.apache.org:8080/"); + client.parse(getArgs(null, "./src/test/resources/websocket_client1.conf")); + client.loadArguments(); + assertEquals(client.proxyURL, "ws://broker1.pulsar.apache.org:8080/"); // "webServiceUrl" written in the conf file has the third priority - arguments = client.loadArguments(getArgs(null, "./src/test/resources/websocket_client2.conf")); - assertEquals(arguments.proxyURL, "ws://broker2.pulsar.apache.org:8080/"); + client.parse(getArgs(null, "./src/test/resources/websocket_client2.conf")); + client.loadArguments(); + assertEquals(client.proxyURL, "ws://broker2.pulsar.apache.org:8080/"); // "serviceUrl" written in the conf file has the fourth priority - arguments = client.loadArguments(getArgs(null, "./src/test/resources/websocket_client3.conf")); - assertEquals(arguments.proxyURL, "wss://broker3.pulsar.apache.org:8443/"); + client.parse(getArgs(null, "./src/test/resources/websocket_client3.conf")); + client.loadArguments(); + assertEquals(client.proxyURL, "wss://broker3.pulsar.apache.org:8443/"); // The default value is "ws://localhost:8080/" - arguments = client.loadArguments(getArgs(null, null)); - assertEquals(arguments.proxyURL, "ws://localhost:8080/"); + client.parse(getArgs(null, null)); + client.loadArguments(); + assertEquals(client.proxyURL, "ws://localhost:8080/"); // If the URL does not end with "/", it will be added - arguments = client.loadArguments(getArgs("ws://broker0.pulsar.apache.org:8080", null)); - assertEquals(arguments.proxyURL, "ws://broker0.pulsar.apache.org:8080/"); + client.parse(getArgs("ws://broker0.pulsar.apache.org:8080", null)); + client.loadArguments(); + assertEquals(client.proxyURL, "ws://broker0.pulsar.apache.org:8080/"); } private String[] getArgs(String proxyUrl, String confFile) { diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/GenerateDocumentionTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/GenerateDocumentionTest.java index 73d7751e33343..e76e0cca0cb76 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/GenerateDocumentionTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/GenerateDocumentionTest.java @@ -26,13 +26,13 @@ public class GenerateDocumentionTest { @Test public void testGenerateDocumention() throws Exception { - CmdGenerateDocumentation.main(new String[]{}); + new CmdGenerateDocumentation().run(new String[]{}); } @Test public void testSpecifyModuleName() throws Exception { String[] args = new String[]{"-n", "produce", "-n", "consume"}; - CmdGenerateDocumentation.main(args); + new CmdGenerateDocumentation().run(args); } private static final String DESC = "desc"; diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/Oauth2PerformanceTransactionTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/Oauth2PerformanceTransactionTest.java index f1be515e9c7f8..e8eeb3bf51993 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/Oauth2PerformanceTransactionTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/Oauth2PerformanceTransactionTest.java @@ -186,7 +186,7 @@ public void testTransactionPerf() throws Exception { Thread thread = new Thread(() -> { try { - PerformanceTransaction.main(args.split(" ")); + new PerformanceTransaction().run(args.split(" ")); } catch (Exception e) { e.printStackTrace(); } diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerfClientUtilsTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerfClientUtilsTest.java index a7aa3b5a976e3..ed0d055ce1188 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerfClientUtilsTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerfClientUtilsTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.testclient; +import java.io.FileInputStream; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; @@ -55,7 +56,7 @@ public void close() throws IOException { @Test public void testClientCreation() throws Exception { - final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(); + final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(""); args.tlsHostnameVerificationEnable = true; args.authPluginClassName = MyAuth.class.getName(); @@ -97,7 +98,7 @@ public void testClientCreation() throws Exception { @Test public void testClientCreationWithProxy() throws Exception { - final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(); + final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(""); args.serviceURL = "pulsar+ssl://my-pulsar:6651"; args.proxyServiceURL = "pulsar+ssl://my-proxy-pulsar:4443"; @@ -120,11 +121,13 @@ public void testClientCreationWithProxyDefinedInConfFile() throws Exception { + "proxyServiceUrl=pulsar+ssl://my-proxy-pulsar:4443\n" + "proxyProtocol=SNI"); - final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(); - - args.confFile = testConf.toString(); - args.fillArgumentsFromProperties(); - + final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(""); + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(testConf.toString())) { + prop.load(fis); + } + args.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + args.parse(new String[]{}); final ClientBuilderImpl builder = (ClientBuilderImpl) PerfClientUtils.createClientBuilderFromArguments(args); final ClientConfigurationData conf = builder.getClientConfigurationData(); @@ -145,16 +148,19 @@ public void testClientCreationWithEmptyProxyPropertyInConfFile() throws Exceptio + "proxyServiceUrl=\n" + "proxyProtocol="); - final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(); - - args.confFile = testConf.toString(); - args.fillArgumentsFromProperties(); + final PerformanceBaseArguments args = new PerformanceArgumentsTestDefault(""); + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(testConf.toString())) { + prop.load(fis); + } + args.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + args.parse(new String[]{}); final ClientBuilderImpl builder = (ClientBuilderImpl) PerfClientUtils.createClientBuilderFromArguments(args); final ClientConfigurationData conf = builder.getClientConfigurationData(); - Assert.assertNull(conf.getProxyServiceUrl()); + Assert.assertEquals(conf.getProxyServiceUrl(),""); Assert.assertNull(conf.getProxyProtocol()); } finally { Files.deleteIfExists(testConf); @@ -163,7 +169,13 @@ public void testClientCreationWithEmptyProxyPropertyInConfFile() throws Exceptio } class PerformanceArgumentsTestDefault extends PerformanceBaseArguments { + public PerformanceArgumentsTestDefault(String cmdName) { + super(cmdName); + } + + @Override - public void fillArgumentsFromProperties(Properties prop) { + public void run() throws Exception { + } } diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceBaseArgumentsTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceBaseArgumentsTest.java index 699f138bfdaa8..9b54fa510cee2 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceBaseArgumentsTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceBaseArgumentsTest.java @@ -22,6 +22,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.fail; import java.io.File; +import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; import java.util.List; @@ -31,23 +32,28 @@ import org.testng.Assert; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +import picocli.CommandLine; public class PerformanceBaseArgumentsTest { @Test - public void testReadFromConfigFile() { - - AtomicBoolean called = new AtomicBoolean(); - - final PerformanceBaseArguments args = new PerformanceBaseArguments() { + public void testReadFromConfigFile() throws Exception { + final PerformanceBaseArguments args = new PerformanceBaseArguments("") { @Override - public void fillArgumentsFromProperties(Properties prop) { - called.set(true); + public void run() throws Exception { + } }; - args.confFile = "./src/test/resources/perf_client1.conf"; - args.fillArgumentsFromProperties(); - Assert.assertTrue(called.get()); + + String confFile = "./src/test/resources/perf_client1.conf"; + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(confFile)) { + prop.load(fis); + } + args.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + args.parse(new String[]{}); + + Assert.assertEquals(args.serviceURL, "https://my-pulsar:8443/"); Assert.assertEquals(args.authPluginClassName, "org.apache.pulsar.testclient.PerfClientUtilsTest.MyAuth"); @@ -62,37 +68,41 @@ public void fillArgumentsFromProperties(Properties prop) { @Test public void testReadFromConfigFileWithoutProxyUrl() { - AtomicBoolean called = new AtomicBoolean(); - final PerformanceBaseArguments args = new PerformanceBaseArguments() { + final PerformanceBaseArguments args = new PerformanceBaseArguments("") { @Override - public void fillArgumentsFromProperties(Properties prop) { - called.set(true); + public void run() throws Exception { + } + }; + String confFile = "./src/test/resources/performance_client2.conf"; - File tempConfigFile = new File("./src/test/resources/performance_client2.conf"); + File tempConfigFile = new File(confFile); if (tempConfigFile.exists()) { tempConfigFile.delete(); } try { Properties props = new Properties(); - - Map configs = Map.of("brokerServiceUrl","https://my-pulsar:8443/", - "authPlugin","org.apache.pulsar.testclient.PerfClientUtilsTest.MyAuth", - "authParams", "myparams", - "tlsTrustCertsFilePath", "./path", - "tlsAllowInsecureConnection","true", - "tlsEnableHostnameVerification", "true" + + Map configs = Map.of("brokerServiceUrl", "https://my-pulsar:8443/", + "authPlugin", "org.apache.pulsar.testclient.PerfClientUtilsTest.MyAuth", + "authParams", "myparams", + "tlsTrustCertsFilePath", "./path", + "tlsAllowInsecureConnection", "true", + "tlsEnableHostnameVerification", "true" ); props.putAll(configs); FileOutputStream out = new FileOutputStream(tempConfigFile); props.store(out, "properties file"); out.close(); - args.confFile = "./src/test/resources/performance_client2.conf"; + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(confFile)) { + prop.load(fis); + } + args.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + args.parse(new String[]{}); - args.fillArgumentsFromProperties(); - Assert.assertTrue(called.get()); Assert.assertEquals(args.serviceURL, "https://my-pulsar:8443/"); Assert.assertEquals(args.authPluginClassName, "org.apache.pulsar.testclient.PerfClientUtilsTest.MyAuth"); @@ -100,7 +110,7 @@ public void fillArgumentsFromProperties(Properties prop) { Assert.assertEquals(args.tlsTrustCertsFilePath, "./path"); Assert.assertTrue(args.tlsAllowInsecureConnection); Assert.assertTrue(args.tlsHostnameVerificationEnable); - + } catch (IOException e) { e.printStackTrace(); fail("Error while updating/reading config file"); @@ -112,27 +122,27 @@ public void fillArgumentsFromProperties(Properties prop) { @Test public void testReadFromConfigFileProxyProtocolException() { - AtomicBoolean calledVar1 = new AtomicBoolean(); AtomicBoolean calledVar2 = new AtomicBoolean(); - final PerformanceBaseArguments args = new PerformanceBaseArguments() { + final PerformanceBaseArguments args = new PerformanceBaseArguments("") { @Override - public void fillArgumentsFromProperties(Properties prop) { - calledVar1.set(true); + public void run() throws Exception { + } }; - File tempConfigFile = new File("./src/test/resources/performance_client3.conf"); + String confFile = "./src/test/resources/performance_client3.conf"; + File tempConfigFile = new File(confFile); if (tempConfigFile.exists()) { tempConfigFile.delete(); } try { Properties props = new Properties(); - Map configs = Map.of("brokerServiceUrl","https://my-pulsar:8443/", - "authPlugin","org.apache.pulsar.testclient.PerfClientUtilsTest.MyAuth", + Map configs = Map.of("brokerServiceUrl", "https://my-pulsar:8443/", + "authPlugin", "org.apache.pulsar.testclient.PerfClientUtilsTest.MyAuth", "authParams", "myparams", "tlsTrustCertsFilePath", "./path", - "tlsAllowInsecureConnection","true", + "tlsAllowInsecureConnection", "true", "tlsEnableHostnameVerification", "true", "proxyServiceURL", "https://my-proxy-pulsar:4443/", "proxyProtocol", "TEST" @@ -141,15 +151,17 @@ public void fillArgumentsFromProperties(Properties prop) { FileOutputStream out = new FileOutputStream(tempConfigFile); props.store(out, "properties file"); out.close(); - args.confFile = "./src/test/resources/performance_client3.conf"; - PerfClientUtils.setExitProcedure(code -> { - calledVar2.set(true); - Assert.assertEquals(code, 1, "Incorrect exit code"); - }); - args.confFile = "./src/test/resources/performance_client3.conf"; - args.fillArgumentsFromProperties(); - Assert.assertTrue(calledVar1.get()); + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(confFile)) { + prop.load(fis); + } + args.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + try { + args.parse(new String[]{}); + }catch (CommandLine.ParameterException e){ + calledVar2.set(true); + } Assert.assertTrue(calledVar2.get()); } catch (IOException e) { e.printStackTrace(); @@ -161,15 +173,15 @@ public void fillArgumentsFromProperties(Properties prop) { @DataProvider(name = "memoryLimitCliArgumentProvider") public Object[][] memoryLimitCliArgumentProvider() { - return new Object[][] { - { new String[]{"-ml","1"}, 1L}, - { new String[]{"-ml","1K"}, 1024L}, - { new String[]{"--memory-limit", "1G"}, 1024 * 1024 * 1024} + return new Object[][]{ + {new String[]{"-ml", "1"}, 1L}, + {new String[]{"-ml", "1K"}, 1024L}, + {new String[]{"--memory-limit", "1G"}, 1024 * 1024 * 1024} }; } @Test(dataProvider = "memoryLimitCliArgumentProvider") - public void testMemoryLimitCliArgument(String[] cliArgs, long expectedMemoryLimit) { + public void testMemoryLimitCliArgument(String[] cliArgs, long expectedMemoryLimit) throws Exception { for (String cmd : List.of( "pulsar-perf read", "pulsar-perf produce", @@ -177,17 +189,24 @@ public void testMemoryLimitCliArgument(String[] cliArgs, long expectedMemoryLimi "pulsar-perf transaction" )) { // Arrange - AtomicBoolean called = new AtomicBoolean(); - final PerformanceBaseArguments baseArgument = new PerformanceBaseArguments() { + final PerformanceBaseArguments baseArgument = new PerformanceBaseArguments("") { @Override - public void fillArgumentsFromProperties(Properties prop) { - called.set(true); + public void run() throws Exception { + } + }; - baseArgument.confFile = "./src/test/resources/perf_client1.conf"; + String confFile = "./src/test/resources/perf_client1.conf"; + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(confFile)) { + prop.load(fis); + } + baseArgument.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + baseArgument.parse(new String[]{}); // Act - baseArgument.parseCLI(cmd, cliArgs); + baseArgument.parseCLI(); + baseArgument.getCommander().execute(cliArgs); // Assert assertEquals(baseArgument.memoryLimit, expectedMemoryLimit); @@ -196,15 +215,15 @@ public void fillArgumentsFromProperties(Properties prop) { @DataProvider(name = "invalidMemoryLimitCliArgumentProvider") public Object[][] invalidMemoryLimitCliArgumentProvider() { - return new Object[][] { - { new String[]{"-ml","-1"}}, - { new String[]{"-ml","1C"}}, - { new String[]{"--memory-limit", "1Q"}} + return new Object[][]{ + {new String[]{"-ml", "-1"}}, + {new String[]{"-ml", "1C"}}, + {new String[]{"--memory-limit", "1Q"}} }; } @Test - public void testMemoryLimitCliArgumentDefault() { + public void testMemoryLimitCliArgumentDefault() throws Exception { for (String cmd : List.of( "pulsar-perf read", "pulsar-perf produce", @@ -212,17 +231,23 @@ public void testMemoryLimitCliArgumentDefault() { "pulsar-perf transaction" )) { // Arrange - AtomicBoolean called = new AtomicBoolean(); - final PerformanceBaseArguments baseArgument = new PerformanceBaseArguments() { + final PerformanceBaseArguments baseArgument = new PerformanceBaseArguments("") { @Override - public void fillArgumentsFromProperties(Properties prop) { - called.set(true); + public void run() throws Exception { + } + }; - baseArgument.confFile = "./src/test/resources/perf_client1.conf"; + String confFile = "./src/test/resources/perf_client1.conf"; + Properties prop = new Properties(System.getProperties()); + try (FileInputStream fis = new FileInputStream(confFile)) { + prop.load(fis); + } + baseArgument.getCommander().setDefaultValueProvider(PulsarPerfTestPropertiesProvider.create(prop)); + baseArgument.parse(new String[]{}); // Act - baseArgument.parseCLI(cmd, new String[]{}); + baseArgument.parseCLI(); // Assert assertEquals(baseArgument.memoryLimit, 0L); diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java index 20679d8367677..d0b25c6971697 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java @@ -85,7 +85,8 @@ public void testMsgKey() throws Exception { String args = String.format(argString, topic, pulsar.getBrokerServiceUrl()); Thread thread = new Thread(() -> { try { - PerformanceProducer.main(args.split(" ")); + PerformanceProducer producer = new PerformanceProducer(); + producer.run(args.split(" ")); } catch (Exception e) { e.printStackTrace(); } @@ -131,7 +132,8 @@ public void testMsgKey() throws Exception { String newArgs = String.format(newArgString, topic2, pulsar.getBrokerServiceUrl()); Thread thread2 = new Thread(() -> { try { - PerformanceProducer.main(newArgs.split(" ")); + PerformanceProducer producer = new PerformanceProducer(); + producer.run(newArgs.split(" ")); } catch (Exception e) { e.printStackTrace(); } @@ -169,23 +171,23 @@ public void testMsgKey() throws Exception { @Test(timeOut = 20000) public void testBatchingDisabled() throws Exception { - PerformanceProducer.Arguments arguments = new PerformanceProducer.Arguments(); + PerformanceProducer producer = new PerformanceProducer(); int producerId = 0; String topic = testTopic + UUID.randomUUID(); - arguments.topics = List.of(topic); - arguments.msgRate = 10; - arguments.serviceURL = pulsar.getBrokerServiceUrl(); - arguments.numMessages = 500; - arguments.disableBatching = true; - - ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(arguments) - .enableTransaction(arguments.isEnableTransaction); + producer.topics = List.of(topic); + producer.msgRate = 10; + producer.serviceURL = pulsar.getBrokerServiceUrl(); + producer.numMessages = 500; + producer.disableBatching = true; + + ClientBuilder clientBuilder = PerfClientUtils.createClientBuilderFromArguments(producer) + .enableTransaction(producer.isEnableTransaction); @Cleanup PulsarClient client = clientBuilder.build(); - - ProducerBuilderImpl builder = (ProducerBuilderImpl) PerformanceProducer.createProducerBuilder(client, arguments, producerId); + ProducerBuilderImpl builder = (ProducerBuilderImpl) producer.createProducerBuilder(client, + producerId); Assert.assertFalse(builder.getConf().isBatchingEnabled()); } @@ -196,7 +198,8 @@ public void testCreatePartitions() throws Exception { String args = String.format(argString, topic, pulsar.getBrokerServiceUrl(), pulsar.getWebServiceAddress()); Thread thread = new Thread(() -> { try { - PerformanceProducer.main(args.split(" ")); + PerformanceProducer producer = new PerformanceProducer(); + producer.run(args.split(" ")); } catch (Exception e) { e.printStackTrace(); } @@ -227,7 +230,8 @@ public void testMaxOutstanding() throws Exception { .subscriptionType(SubscriptionType.Key_Shared).subscribe(); new Thread(() -> { try { - PerformanceProducer.main(args.split(" ")); + PerformanceProducer producer = new PerformanceProducer(); + producer.run(args.split(" ")); } catch (Exception e) { log.error("Failed to start perf producer"); } diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java index 12f457587f685..c8d71d98e701b 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceTransactionTest.java @@ -133,7 +133,7 @@ public void testTxnPerf() throws Exception { Thread thread = new Thread(() -> { try { - PerformanceTransaction.main(args.split(" ")); + new PerformanceTransaction().run(args.split(" ")); } catch (Exception e) { e.printStackTrace(); } @@ -184,7 +184,7 @@ public void testProduceTxnMessage() throws InterruptedException, PulsarClientExc .subscribe(); Thread thread = new Thread(() -> { try { - PerformanceProducer.main(args.split(" ")); + new PerformanceProducer().run(args.split(" ")); } catch (Exception e) { e.printStackTrace(); } @@ -237,7 +237,7 @@ public void testConsumeTxnMessage() throws Exception { Thread thread = new Thread(() -> { try { log.info(""); - PerformanceConsumer.main(args.split(" ")); + new PerformanceConsumer().run(args.split(" ")); } catch (Exception e) { e.printStackTrace(); } From bc44280e88e98fdf0a815fa384a0f52508ca4b8e Mon Sep 17 00:00:00 2001 From: Nikhil Erigila <60037808+nikhilerigila09@users.noreply.github.com> Date: Thu, 2 May 2024 18:58:02 +0530 Subject: [PATCH 138/580] [fix][sec] Upgrade aws-sdk.version to avoid CVE-2024-21634 (#22633) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 585347fb1f855..b72db9b3b7684 100644 --- a/pom.xml +++ b/pom.xml @@ -182,7 +182,7 @@ flexible messaging model and an intuitive client API. 4.5.0 3.4.0 5.18.0 - 1.12.262 + 1.12.638 1.11.3 2.10.10 2.6.0 From 41f633f81e53bfc35ad37bb13e62def5c0dcb11d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Fri, 3 May 2024 01:12:26 +0200 Subject: [PATCH 139/580] [fix][misc] NPE during standalone shutdown (#22636) --- .../org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java b/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java index 63d146a3a1521..e8a503c46e006 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java @@ -498,7 +498,9 @@ public void stop() throws Exception { LOG.debug("Local ZK/BK stopping ..."); for (LifecycleComponent bookie : bookieComponents) { try { - bookie.close(); + if (bookie != null) { + bookie.close(); + } } catch (Exception e) { LOG.warn("failed to shutdown bookie", e); } From 0219921b5b7cd157092ac8f2d86ab7e60787d36c Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 2 May 2024 16:57:49 -0700 Subject: [PATCH 140/580] [fix] [client] Fix Consumer should return configured batch receive max messages (#22619) --- .../client/api/ConsumerBatchReceiveTest.java | 8 ++--- .../api/SimpleProducerConsumerTest.java | 29 +++++++++++++++++++ .../client/impl/ConsumerBuilderImpl.java | 4 +++ 3 files changed, 37 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ConsumerBatchReceiveTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ConsumerBatchReceiveTest.java index d54b1c99e3e13..974d25aad64db 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ConsumerBatchReceiveTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ConsumerBatchReceiveTest.java @@ -112,7 +112,7 @@ public Object[][] batchReceivePolicyProvider() { // Number of message limitation exceed receiverQueue size { BatchReceivePolicy.builder() - .maxNumMessages(70) + .maxNumMessages(50) .build(), true, 50, false }, // Number of message limitation exceed receiverQueue size and timeout limitation @@ -147,7 +147,7 @@ public Object[][] batchReceivePolicyProvider() { // Number of message limitation exceed receiverQueue size { BatchReceivePolicy.builder() - .maxNumMessages(70) + .maxNumMessages(50) .build(), false, 50, false }, // Number of message limitation exceed receiverQueue size and timeout limitation @@ -248,7 +248,7 @@ public Object[][] batchReceivePolicyProvider() { // Number of message limitation exceed receiverQueue size { BatchReceivePolicy.builder() - .maxNumMessages(70) + .maxNumMessages(50) .build(), true, 50, true }, // Number of message limitation exceed receiverQueue size and timeout limitation @@ -283,7 +283,7 @@ public Object[][] batchReceivePolicyProvider() { // Number of message limitation exceed receiverQueue size { BatchReceivePolicy.builder() - .maxNumMessages(70) + .maxNumMessages(50) .build(), false, 50, true }, // Number of message limitation exceed receiverQueue size and timeout limitation diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index 70214fe6e3b87..d37bd484bfbfb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -4821,6 +4821,35 @@ public void onSendAcknowledgement(Producer producer, Message message, MessageId admin.topics().delete(topic, false); } + /** + * It verifies that consumer receives configured number of messages into the batch. + * @throws Exception + */ + @Test + public void testBatchReceiveWithMaxBatchSize() throws Exception { + int maxBatchSize = 100; + final int internalQueueSize = 10; + final int maxBytes = 2000000; + final int timeOutInSeconds = 900; + final String topic = "persistent://my-property/my-ns/testBatchReceive"; + BatchReceivePolicy batchReceivePolicy = BatchReceivePolicy.builder().maxNumBytes(maxBytes) + .maxNumMessages(maxBatchSize).timeout(timeOutInSeconds, TimeUnit.SECONDS).build(); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topic) + .subscriptionName("my-subscriber-name") + .receiverQueueSize(internalQueueSize) + .batchReceivePolicy(batchReceivePolicy).subscribe(); + @Cleanup + Producer producer = pulsarClient.newProducer().topic(topic).enableBatching(false).create(); + + final int numMessages = 100; + for (int i = 0; i < numMessages; i++) { + producer.newMessage().value(("value-" + i).getBytes(UTF_8)).eventTime((i + 1) * 100L).send(); + } + + assertEquals(consumer.batchReceive().size(), maxBatchSize); + } + private int compareMessageIds(MessageIdImpl messageId1, MessageIdImpl messageId2) { if (messageId2.getLedgerId() < messageId1.getLedgerId()) { return -1; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java index f644c6a18398f..7686d0072cffb 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java @@ -120,6 +120,10 @@ public CompletableFuture> subscribeAsync() { return FutureUtil.failedFuture( new InvalidConfigurationException("KeySharedPolicy must set with KeyShared subscription")); } + if (conf.getBatchReceivePolicy() != null) { + conf.setReceiverQueueSize( + Math.max(conf.getBatchReceivePolicy().getMaxNumMessages(), conf.getReceiverQueueSize())); + } CompletableFuture applyDLQConfig; if (conf.isRetryEnable() && conf.getTopicNames().size() > 0) { TopicName topicFirst = TopicName.get(conf.getTopicNames().iterator().next()); From d12f623739df80cbc57a22a2de3ddba343f26ee3 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 3 May 2024 19:55:31 +0300 Subject: [PATCH 141/580] [improve][build] Use slf4j-bom to align slf4j versions (#22646) --- buildtools/pom.xml | 8 +++++++- pom.xml | 16 +++------------- 2 files changed, 10 insertions(+), 14 deletions(-) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 58f99e9ea86b5..3b0b24d1d53a1 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -63,6 +63,13 @@ + + org.slf4j + slf4j-bom + ${slf4j.version} + pom + import + org.apache.logging.log4j log4j-bom @@ -122,7 +129,6 @@ org.slf4j jcl-over-slf4j - ${slf4j.version} org.apache.commons diff --git a/pom.xml b/pom.xml index b72db9b3b7684..048bc952466b3 100644 --- a/pom.xml +++ b/pom.xml @@ -763,20 +763,10 @@ flexible messaging model and an intuitive client API. org.slf4j - slf4j-api - ${slf4j.version} - - - - org.slf4j - slf4j-simple - ${slf4j.version} - - - - org.slf4j - jcl-over-slf4j + slf4j-bom ${slf4j.version} + pom + import From 7a8c4549639d67182049bca9f714c0f4b3061236 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 3 May 2024 20:16:08 +0300 Subject: [PATCH 142/580] [improve][broker] Add logging to leader election (#22645) --- .../apache/pulsar/broker/PulsarService.java | 6 ++--- .../coordination/impl/LeaderElectionImpl.java | 22 +++++++++++++++---- 2 files changed, 21 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 8c910fb91e109..559ca1e9e690b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -1181,7 +1181,7 @@ protected void startLeaderElectionService() { new LeaderElectionService(coordinationService, getBrokerId(), getSafeWebServiceAddress(), state -> { if (state == LeaderElectionState.Leading) { - LOG.info("This broker was elected leader"); + LOG.info("This broker {} was elected leader", getBrokerId()); if (getConfiguration().isLoadBalancerEnabled()) { long resourceQuotaUpdateInterval = TimeUnit.MINUTES .toMillis(getConfiguration().getLoadBalancerResourceQuotaUpdateIntervalMinutes()); @@ -1202,10 +1202,10 @@ protected void startLeaderElectionService() { if (leaderElectionService != null) { final Optional currentLeader = leaderElectionService.getCurrentLeader(); if (currentLeader.isPresent()) { - LOG.info("This broker is a follower. Current leader is {}", + LOG.info("This broker {} is a follower. Current leader is {}", getBrokerId(), currentLeader); } else { - LOG.info("This broker is a follower. No leader has been elected yet"); + LOG.info("This broker {} is a follower. No leader has been elected yet", getBrokerId()); } } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java index 9e6a9b94c42a3..aa606084173e5 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java @@ -129,19 +129,26 @@ private synchronized CompletableFuture handleExistingLeader return FutureUtils.exception(t); } - if (existingValue.equals(proposedValue.orElse(null))) { + T value = proposedValue.orElse(null); + if (existingValue.equals(value)) { // If the value is the same as our proposed value, it means this instance was the leader at some // point before. The existing value can either be for this same session or for a previous one. if (res.getStat().isCreatedBySelf()) { // The value is still valid because it was created in the same session changeState(LeaderElectionState.Leading); } else { + log.info("Conditionally deleting existing equals value {} for {} because it's not created in the " + + "current session. stat={}", existingValue, path, res.getStat()); // Since the value was created in a different session, it might be expiring. We need to delete it // and try the election again. return store.delete(path, Optional.of(res.getStat().getVersion())) .thenCompose(__ -> tryToBecomeLeader()); } } else if (res.getStat().isCreatedBySelf()) { + log.warn("Conditionally deleting existing value {} for {} because it's different from the proposed value " + + "({}). This is unexpected since it was created within the same session. " + + "In tests this could happen because of an invalid shared session id when using mocks.", + existingValue, path, value); // The existing value is different but was created from the same session return store.delete(path, Optional.of(res.getStat().getVersion())) .thenCompose(__ -> tryToBecomeLeader()); @@ -165,9 +172,10 @@ private synchronized void changeState(LeaderElectionState les) { } private synchronized CompletableFuture tryToBecomeLeader() { + T value = proposedValue.get(); byte[] payload; try { - payload = serde.serialize(path, proposedValue.get()); + payload = serde.serialize(path, value); } catch (Throwable t) { return FutureUtils.exception(t); } @@ -181,7 +189,7 @@ private synchronized CompletableFuture tryToBecomeLeader() cache.get(path) .thenRun(() -> { synchronized (LeaderElectionImpl.this) { - log.info("Acquired leadership on {}", path); + log.info("Acquired leadership on {} with {}", path, value); internalState = InternalState.LeaderIsPresent; if (leaderElectionState != LeaderElectionState.Leading) { leaderElectionState = LeaderElectionState.Leading; @@ -196,6 +204,8 @@ private synchronized CompletableFuture tryToBecomeLeader() }).exceptionally(ex -> { // We fail to do the get(), so clean up the leader election fail the whole // operation + log.warn("Failed to get the current state after acquiring leadership on {}. " + + " Conditionally deleting current entry.", path, ex); store.delete(path, Optional.of(stat.getVersion())) .thenRun(() -> result.completeExceptionally(ex)) .exceptionally(ex2 -> { @@ -205,6 +215,8 @@ private synchronized CompletableFuture tryToBecomeLeader() return null; }); } else { + log.info("Leadership on {} with value {} was lost. " + + "Conditionally deleting entry with stat={}.", path, value, stat); // LeaderElection was closed in between. Release the lock asynchronously store.delete(path, Optional.of(stat.getVersion())) .thenRun(() -> result.completeExceptionally( @@ -219,7 +231,9 @@ private synchronized CompletableFuture tryToBecomeLeader() if (ex.getCause() instanceof BadVersionException) { // There was a conflict between 2 participants trying to become leaders at same time. Retry // to fetch info on new leader. - + log.info("There was a conflict between 2 participants trying to become leaders at the same " + + "time on {}. Attempted with value {}. Retrying.", + path, value); elect() .thenAccept(lse -> result.complete(lse)) .exceptionally(ex2 -> { From 4a5953640fd93f8ecac39c7713851ac4c1ab902b Mon Sep 17 00:00:00 2001 From: Nikhil Erigila <60037808+nikhilerigila09@users.noreply.github.com> Date: Sat, 4 May 2024 02:00:28 +0530 Subject: [PATCH 143/580] [fix][sec] Upgrade Debezium oracle connector version to avoid CVE-2023-4586 (#22641) --- pom.xml | 1 + pulsar-io/debezium/oracle/pom.xml | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 048bc952466b3..4bfdc54e55d31 100644 --- a/pom.xml +++ b/pom.xml @@ -198,6 +198,7 @@ flexible messaging model and an intuitive client API. 1.2.4 8.5.2 1.9.7.Final + 2.2.0.Final 42.5.0 8.0.30 diff --git a/pulsar-io/debezium/oracle/pom.xml b/pulsar-io/debezium/oracle/pom.xml index c69640ecff72f..b22a5785dfbe6 100644 --- a/pulsar-io/debezium/oracle/pom.xml +++ b/pulsar-io/debezium/oracle/pom.xml @@ -48,7 +48,8 @@ io.debezium debezium-connector-oracle - ${debezium.version} + ${debezium.oracle.version} + runtime From 3ca4ddfc8fecab633b473e48e1a6f78adcfbd4df Mon Sep 17 00:00:00 2001 From: Nikhil Erigila <60037808+nikhilerigila09@users.noreply.github.com> Date: Sat, 4 May 2024 02:51:48 +0530 Subject: [PATCH 144/580] [fix][sec] Upgrade elasticsearch-java version to avoid CVE-2023-4043 (#22640) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4bfdc54e55d31..7507a5cb28b8e 100644 --- a/pom.xml +++ b/pom.xml @@ -196,7 +196,7 @@ flexible messaging model and an intuitive client API. 3.3.5 2.4.10 1.2.4 - 8.5.2 + 8.12.1 1.9.7.Final 2.2.0.Final 42.5.0 From 2821afad7a1fff056e4f04f71934dcd8c01fbcb1 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 4 May 2024 02:48:02 +0300 Subject: [PATCH 145/580] [improve][build] Upgrade OTel library versions (#22649) --- .../server/src/assemble/LICENSE.bin.txt | 51 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 6 +-- pom.xml | 20 ++++++-- pulsar-broker/pom.xml | 6 --- pulsar-client/pom.xml | 2 +- .../pulsar/client/impl/metrics/Counter.java | 2 +- .../client/impl/metrics/LatencyHistogram.java | 2 +- .../client/impl/metrics/UpDownCounter.java | 2 +- 8 files changed, 47 insertions(+), 44 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index c5c243796b6f3..aec4df2a93af9 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -338,12 +338,12 @@ The Apache Software License, Version 2.0 - io.prometheus-simpleclient_tracer_otel-0.16.0.jar - io.prometheus-simpleclient_tracer_otel_agent-0.16.0.jar * Prometheus exporter - - io.prometheus-prometheus-metrics-config-1.1.0.jar - - io.prometheus-prometheus-metrics-exporter-common-1.1.0.jar - - io.prometheus-prometheus-metrics-exporter-httpserver-1.1.0.jar - - io.prometheus-prometheus-metrics-exposition-formats-1.1.0.jar - - io.prometheus-prometheus-metrics-model-1.1.0.jar - - io.prometheus-prometheus-metrics-shaded-protobuf-1.1.0.jar + - io.prometheus-prometheus-metrics-config-1.2.1.jar + - io.prometheus-prometheus-metrics-exporter-common-1.2.1.jar + - io.prometheus-prometheus-metrics-exporter-httpserver-1.2.1.jar + - io.prometheus-prometheus-metrics-exposition-formats-1.2.1.jar + - io.prometheus-prometheus-metrics-model-1.2.1.jar + - io.prometheus-prometheus-metrics-shaded-protobuf-1.2.1.jar * Jakarta Bean Validation API - jakarta.validation-jakarta.validation-api-2.0.2.jar - javax.validation-validation-api-1.1.0.Final.jar @@ -524,26 +524,25 @@ The Apache Software License, Version 2.0 - org.roaringbitmap-RoaringBitmap-0.9.44.jar - org.roaringbitmap-shims-0.9.44.jar * OpenTelemetry - - io.opentelemetry-opentelemetry-api-1.34.1.jar - - io.opentelemetry-opentelemetry-api-events-1.34.1-alpha.jar - - io.opentelemetry-opentelemetry-context-1.34.1.jar - - io.opentelemetry-opentelemetry-exporter-common-1.34.1.jar - - io.opentelemetry-opentelemetry-exporter-otlp-1.34.1.jar - - io.opentelemetry-opentelemetry-exporter-otlp-common-1.34.1.jar - - io.opentelemetry-opentelemetry-exporter-prometheus-1.34.1-alpha.jar - - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.34.1.jar - - io.opentelemetry-opentelemetry-extension-incubator-1.34.1-alpha.jar - - io.opentelemetry-opentelemetry-sdk-1.34.1.jar - - io.opentelemetry-opentelemetry-sdk-common-1.34.1.jar - - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.34.1.jar - - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.34.1.jar - - io.opentelemetry-opentelemetry-sdk-logs-1.34.1.jar - - io.opentelemetry-opentelemetry-sdk-metrics-1.34.1.jar - - io.opentelemetry-opentelemetry-sdk-trace-1.34.1.jar - - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-1.32.1.jar - - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-semconv-1.32.1-alpha.jar - - io.opentelemetry.instrumentation-opentelemetry-resources-1.32.1-alpha.jar - - io.opentelemetry.semconv-opentelemetry-semconv-1.23.1-alpha.jar + - io.opentelemetry-opentelemetry-api-1.37.0.jar + - io.opentelemetry-opentelemetry-api-incubator-1.37.0-alpha.jar + - io.opentelemetry-opentelemetry-context-1.37.0.jar + - io.opentelemetry-opentelemetry-exporter-common-1.37.0.jar + - io.opentelemetry-opentelemetry-exporter-otlp-1.37.0.jar + - io.opentelemetry-opentelemetry-exporter-otlp-common-1.37.0.jar + - io.opentelemetry-opentelemetry-exporter-prometheus-1.37.0-alpha.jar + - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.37.0.jar + - io.opentelemetry-opentelemetry-sdk-1.37.0.jar + - io.opentelemetry-opentelemetry-sdk-common-1.37.0.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.37.0.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.37.0.jar + - io.opentelemetry-opentelemetry-sdk-logs-1.37.0.jar + - io.opentelemetry-opentelemetry-sdk-metrics-1.37.0.jar + - io.opentelemetry-opentelemetry-sdk-trace-1.37.0.jar + - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-1.33.2.jar + - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-semconv-1.33.2-alpha.jar + - io.opentelemetry.instrumentation-opentelemetry-resources-1.33.2-alpha.jar + - io.opentelemetry.semconv-opentelemetry-semconv-1.25.0-alpha.jar BSD 3-clause "New" or "Revised" License * Google auth library diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 41b38f17dce77..be1f7db63134c 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -389,9 +389,9 @@ The Apache Software License, Version 2.0 - log4j-slf4j2-impl-2.23.1.jar - log4j-web-2.23.1.jar * OpenTelemetry - - opentelemetry-api-1.34.1.jar - - opentelemetry-context-1.34.1.jar - - opentelemetry-extension-incubator-1.34.1-alpha.jar + - opentelemetry-api-1.37.0.jar + - opentelemetry-api-incubator-1.37.0-alpha.jar + - opentelemetry-context-1.37.0.jar * BookKeeper - bookkeeper-common-allocator-4.17.0.jar diff --git a/pom.xml b/pom.xml index 7507a5cb28b8e..8f7ae2ed1fc68 100644 --- a/pom.xml +++ b/pom.xml @@ -256,10 +256,11 @@ flexible messaging model and an intuitive client API. 3.4.3 1.5.2-3 2.0.6 - 1.34.1 - 1.34.1-alpha - 1.32.1-alpha - 1.23.1-alpha + 1.37.0 + ${opentelemetry.version}-alpha + 1.33.2 + ${opentelemetry.instrumentation.version}-alpha + 1.25.0-alpha 4.7.5 @@ -1497,8 +1498,17 @@ flexible messaging model and an intuitive client API.
io.opentelemetry.instrumentation - opentelemetry-resources + opentelemetry-instrumentation-bom ${opentelemetry.instrumentation.version} + pom + import + + + io.opentelemetry.instrumentation + opentelemetry-instrumentation-bom-alpha + ${opentelemetry.instrumentation.alpha.version} + pom + import io.opentelemetry.semconv diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 3548877912199..1fe67ca1e2d4f 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -496,12 +496,6 @@ pulsar-package-filesystem-storage ${project.version} - - - io.opentelemetry - opentelemetry-sdk-testing - test - diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index f79af79d57452..b2829fbd21cfc 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -59,7 +59,7 @@ io.opentelemetry - opentelemetry-extension-incubator + opentelemetry-api-incubator diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java index fffbab4217a86..4042ff8e5d66e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/Counter.java @@ -21,10 +21,10 @@ import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getDefaultAggregationLabels; import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getTopicAttributes; import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.incubator.metrics.ExtendedLongCounterBuilder; import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.LongCounterBuilder; import io.opentelemetry.api.metrics.Meter; -import io.opentelemetry.extension.incubator.metrics.ExtendedLongCounterBuilder; public class Counter { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java index ed04eff03b39d..fdae0a14d65fc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/LatencyHistogram.java @@ -23,10 +23,10 @@ import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getTopicAttributes; import com.google.common.collect.Lists; import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.incubator.metrics.ExtendedDoubleHistogramBuilder; import io.opentelemetry.api.metrics.DoubleHistogram; import io.opentelemetry.api.metrics.DoubleHistogramBuilder; import io.opentelemetry.api.metrics.Meter; -import io.opentelemetry.extension.incubator.metrics.ExtendedDoubleHistogramBuilder; import java.util.List; import java.util.concurrent.TimeUnit; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java index 3df0c2bb42302..dc2984268cdb6 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/metrics/UpDownCounter.java @@ -22,10 +22,10 @@ import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getDefaultAggregationLabels; import static org.apache.pulsar.client.impl.metrics.MetricsUtil.getTopicAttributes; import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.incubator.metrics.ExtendedLongUpDownCounterBuilder; import io.opentelemetry.api.metrics.LongUpDownCounter; import io.opentelemetry.api.metrics.LongUpDownCounterBuilder; import io.opentelemetry.api.metrics.Meter; -import io.opentelemetry.extension.incubator.metrics.ExtendedLongUpDownCounterBuilder; public class UpDownCounter { From eee3694f00e269eef0f75d791521d0d35d8ff411 Mon Sep 17 00:00:00 2001 From: Gvan Yao <50432408+gvanyao@users.noreply.github.com> Date: Sat, 4 May 2024 18:35:25 +0800 Subject: [PATCH 146/580] [fix][storage] ReadonlyManagedLedger initialization does not fill in the properties (#22630) --- .../impl/ReadOnlyManagedLedgerImpl.java | 8 ++ .../impl/ReadOnlyManagedLedgerImplTest.java | 103 ++++++++++++++++++ 2 files changed, 111 insertions(+) create mode 100644 managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImplTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java index 1fdf69395068f..707b71c9d9f09 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java @@ -32,6 +32,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.MetadataNotFoundException; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.pulsar.metadata.api.Stat; @@ -58,6 +59,13 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { ledgers.put(ls.getLedgerId(), ls); } + if (mlInfo.getPropertiesCount() > 0) { + for (int i = 0; i < mlInfo.getPropertiesCount(); i++) { + MLDataFormats.KeyValue property = mlInfo.getProperties(i); + propertiesMap.put(property.getKey(), property.getValue()); + } + } + // Last ledger stat may be zeroed, we must update it if (ledgers.size() > 0 && ledgers.lastEntry().getValue().getEntries() == 0) { long lastLedgerId = ledgers.lastKey(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImplTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImplTest.java new file mode 100644 index 0000000000000..028ecad407276 --- /dev/null +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImplTest.java @@ -0,0 +1,103 @@ +/* + * 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.bookkeeper.mledger.impl; + + +import static org.testng.Assert.assertEquals; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.test.MockedBookKeeperTestCase; +import org.testng.annotations.Test; + +public class ReadOnlyManagedLedgerImplTest extends MockedBookKeeperTestCase { + private static final String MANAGED_LEDGER_NAME_NON_PROPERTIES = "ml-non-properties"; + private static final String MANAGED_LEDGER_NAME_ATTACHED_PROPERTIES = "ml-attached-properties"; + + + @Test + public void testReadOnlyManagedLedgerImplAttachProperties() + throws ManagedLedgerException, InterruptedException, ExecutionException, TimeoutException { + final ManagedLedger ledger = factory.open(MANAGED_LEDGER_NAME_ATTACHED_PROPERTIES, + new ManagedLedgerConfig().setRetentionTime(1, TimeUnit.HOURS)); + final String propertiesKey = "test-key"; + final String propertiesValue = "test-value"; + + ledger.setConfig(new ManagedLedgerConfig()); + ledger.addEntry("entry-0".getBytes()); + Map properties = new HashMap<>(); + properties.put(propertiesKey, propertiesValue); + ledger.setProperties(Collections.unmodifiableMap(properties)); + CompletableFuture future = new CompletableFuture<>(); + factory.asyncOpenReadOnlyManagedLedger(MANAGED_LEDGER_NAME_ATTACHED_PROPERTIES, + new AsyncCallbacks.OpenReadOnlyManagedLedgerCallback() { + @Override + public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl managedLedger, + Object ctx) { + managedLedger.getProperties().forEach((key, value) -> { + assertEquals(key, propertiesKey); + assertEquals(value, propertiesValue); + }); + future.complete(null); + } + + @Override + public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + }, new ManagedLedgerConfig(), null); + + future.get(60, TimeUnit.SECONDS); + } + + @Test + public void testReadOnlyManagedLedgerImplNoProperties() + throws ManagedLedgerException, InterruptedException, ExecutionException, TimeoutException { + final ManagedLedger ledger = factory.open(MANAGED_LEDGER_NAME_NON_PROPERTIES, + new ManagedLedgerConfig().setRetentionTime(1, TimeUnit.HOURS)); + ledger.setConfig(new ManagedLedgerConfig()); + ledger.addEntry("entry-0".getBytes()); + CompletableFuture future = new CompletableFuture<>(); + factory.asyncOpenReadOnlyManagedLedger(MANAGED_LEDGER_NAME_NON_PROPERTIES, + new AsyncCallbacks.OpenReadOnlyManagedLedgerCallback() { + @Override + public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl managedLedger, + Object ctx) { + assertEquals(managedLedger.getProperties().size(), 0); + future.complete(null); + } + + @Override + public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + }, new ManagedLedgerConfig(), null); + + future.get(60, TimeUnit.SECONDS); + } + +} \ No newline at end of file From efcedf6b0d4217db7e47efef3420eb61da282c50 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 6 May 2024 20:35:51 +0800 Subject: [PATCH 147/580] [fix][io] Fix es index creation (#22654) Signed-off-by: Zixuan Liu --- .../client/elastic/ElasticSearchJavaRestClient.java | 4 ++-- .../io/elasticsearch/ElasticSearchSinkTests.java | 13 +++++++++++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticSearchJavaRestClient.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticSearchJavaRestClient.java index afda5ba0e7449..133daa8cd6a68 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticSearchJavaRestClient.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticSearchJavaRestClient.java @@ -144,7 +144,7 @@ public boolean deleteIndex(String index) throws IOException { public boolean deleteDocument(String index, String documentId) throws IOException { final DeleteRequest req = new DeleteRequest.Builder() - .index(config.getIndexName()) + .index(index) .id(documentId) .build(); @@ -156,7 +156,7 @@ public boolean deleteDocument(String index, String documentId) throws IOExceptio public boolean indexDocument(String index, String documentId, String documentSource) throws IOException { final Map mapped = objectMapper.readValue(documentSource, Map.class); final IndexRequest indexRequest = new IndexRequest.Builder<>() - .index(config.getIndexName()) + .index(index) .document(mapped) .id(documentId) .build(); diff --git a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchSinkTests.java b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchSinkTests.java index 9a2cb4ab5658a..f1da6fd0c7e15 100644 --- a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchSinkTests.java +++ b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchSinkTests.java @@ -28,6 +28,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import co.elastic.clients.transport.ElasticsearchTransport; import com.fasterxml.jackson.core.JsonParseException; @@ -43,6 +44,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.GenericObject; @@ -152,6 +154,7 @@ public Object getNativeObject() { }); when(mockRecord.getSchema()).thenAnswer((Answer>>) invocation -> kvSchema); + when(mockRecord.getEventTime()).thenAnswer(invocation -> Optional.of(System.currentTimeMillis())); } @AfterMethod(alwaysRun = true) @@ -209,6 +212,16 @@ public final void send100Test() throws Exception { verify(mockRecord, times(100)).ack(); } + @Test + public final void send1WithFormattedIndexTest() throws Exception { + map.put("indexName", "test-formatted-index-%{+yyyy-MM-dd}"); + sink.open(map, mockSinkContext); + send(1); + verify(mockRecord, times(1)).ack(); + String value = getHitIdAtIndex("test-formatted-index-*", 0); + assertTrue(StringUtils.isNotBlank(value)); + } + @Test public final void sendNoSchemaTest() throws Exception { From 025354ef4e733d62eee0d332edacb0b33b787da2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 6 May 2024 21:48:47 +0300 Subject: [PATCH 148/580] [fix][test] Clear MockedPulsarServiceBaseTest fields to prevent test runtime memory leak (#22659) --- .../pulsar/broker/auth/MockedPulsarServiceBaseTest.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index 10d56ce2245f9..eef4469aa95fa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -277,15 +277,22 @@ protected final void internalCleanup() throws Exception { } if (brokerGateway != null) { brokerGateway.close(); + brokerGateway = null; } if (pulsarTestContext != null) { pulsarTestContext.close(); pulsarTestContext = null; } + resetConfig(); callCloseables(closeables); closeables.clear(); onCleanup(); + + // clear fields to avoid test runtime memory leak, pulsarTestContext already handles closing of these instances + pulsar = null; + mockZooKeeper = null; + mockZooKeeperGlobal = null; } protected void closeAdmin() { From 1e1919000f19d642fed414e98e7607bbdac8b3eb Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 7 May 2024 07:07:45 +0300 Subject: [PATCH 149/580] [fix][broker] Fix thread safety of loadSheddingTask and loadResourceQuotaTask fields (#22660) --- .../apache/pulsar/broker/PulsarService.java | 52 ++++++++++--------- 1 file changed, 27 insertions(+), 25 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 559ca1e9e690b..58d7e71b65d84 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -517,9 +517,7 @@ public CompletableFuture closeAsync() { } // cancel loadShedding task and shutdown the loadManager executor before shutting down the broker - if (this.loadSheddingTask != null) { - this.loadSheddingTask.cancel(); - } + cancelLoadBalancerTasks(); executorServicesShutdown.shutdown(loadManagerExecutor); List> asyncCloseFutures = new ArrayList<>(); @@ -1183,20 +1181,7 @@ protected void startLeaderElectionService() { if (state == LeaderElectionState.Leading) { LOG.info("This broker {} was elected leader", getBrokerId()); if (getConfiguration().isLoadBalancerEnabled()) { - long resourceQuotaUpdateInterval = TimeUnit.MINUTES - .toMillis(getConfiguration().getLoadBalancerResourceQuotaUpdateIntervalMinutes()); - - if (loadSheddingTask != null) { - loadSheddingTask.cancel(); - } - if (loadResourceQuotaTask != null) { - loadResourceQuotaTask.cancel(false); - } - loadSheddingTask = new LoadSheddingTask(loadManager, loadManagerExecutor, config); - loadSheddingTask.start(); - loadResourceQuotaTask = loadManagerExecutor.scheduleAtFixedRate( - new LoadResourceQuotaUpdaterTask(loadManager), resourceQuotaUpdateInterval, - resourceQuotaUpdateInterval, TimeUnit.MILLISECONDS); + startLoadBalancerTasks(); } } else { if (leaderElectionService != null) { @@ -1209,20 +1194,37 @@ protected void startLeaderElectionService() { } } - if (loadSheddingTask != null) { - loadSheddingTask.cancel(); - loadSheddingTask = null; - } - if (loadResourceQuotaTask != null) { - loadResourceQuotaTask.cancel(false); - loadResourceQuotaTask = null; - } + cancelLoadBalancerTasks(); } }); leaderElectionService.start(); } + private synchronized void cancelLoadBalancerTasks() { + if (loadSheddingTask != null) { + loadSheddingTask.cancel(); + loadSheddingTask = null; + } + if (loadResourceQuotaTask != null) { + loadResourceQuotaTask.cancel(false); + loadResourceQuotaTask = null; + } + } + + private synchronized void startLoadBalancerTasks() { + cancelLoadBalancerTasks(); + if (isRunning()) { + long resourceQuotaUpdateInterval = TimeUnit.MINUTES + .toMillis(getConfiguration().getLoadBalancerResourceQuotaUpdateIntervalMinutes()); + loadSheddingTask = new LoadSheddingTask(loadManager, loadManagerExecutor, config); + loadSheddingTask.start(); + loadResourceQuotaTask = loadManagerExecutor.scheduleAtFixedRate( + new LoadResourceQuotaUpdaterTask(loadManager), resourceQuotaUpdateInterval, + resourceQuotaUpdateInterval, TimeUnit.MILLISECONDS); + } + } + protected void acquireSLANamespace() { try { // Namespace not created hence no need to unload it From c30765e789ba1ed28699edb9f159c7b71ca5b907 Mon Sep 17 00:00:00 2001 From: Masahiro Sakamoto Date: Tue, 7 May 2024 14:02:52 +0900 Subject: [PATCH 150/580] [improve][broker] Exclude producers for geo-replication from publishers field of topic stats (#22556) --- .../nonpersistent/NonPersistentTopic.java | 20 +++++---- .../service/persistent/PersistentTopic.java | 25 ++++++----- .../broker/service/OneWayReplicatorTest.java | 41 +++++++++++++++++-- 3 files changed, 66 insertions(+), 20 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 8cb8394440f33..d19aeaa4b0f82 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -39,6 +39,7 @@ import java.util.concurrent.atomic.AtomicLongFieldUpdater; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.broker.namespace.NamespaceService; @@ -746,8 +747,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats replicators.forEach((region, replicator) -> replicator.updateRates()); - nsStats.producerCount += producers.size(); - bundleStats.producerCount += producers.size(); + final MutableInt producerCount = new MutableInt(); topicStatsStream.startObject(topic); topicStatsStream.startList("publishers"); @@ -760,14 +760,19 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats if (producer.isRemote()) { topicStats.remotePublishersStats.put(producer.getRemoteCluster(), publisherStats); - } - - if (hydratePublishers) { - StreamingStats.writePublisherStats(topicStatsStream, publisherStats); + } else { + // Exclude producers for replication from "publishers" and "producerCount" + producerCount.increment(); + if (hydratePublishers) { + StreamingStats.writePublisherStats(topicStatsStream, publisherStats); + } } }); topicStatsStream.endList(); + nsStats.producerCount += producerCount.intValue(); + bundleStats.producerCount += producerCount.intValue(); + // Start replicator stats topicStatsStream.startObject("replication"); nsStats.replicatorCount += topicStats.remotePublishersStats.size(); @@ -856,7 +861,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats // Remaining dest stats. topicStats.averageMsgSize = topicStats.aggMsgRateIn == 0.0 ? 0.0 : (topicStats.aggMsgThroughputIn / topicStats.aggMsgRateIn); - topicStatsStream.writePair("producerCount", producers.size()); + topicStatsStream.writePair("producerCount", producerCount.intValue()); topicStatsStream.writePair("averageMsgSize", topicStats.averageMsgSize); topicStatsStream.writePair("msgRateIn", topicStats.aggMsgRateIn); topicStatsStream.writePair("msgRateOut", topicStats.aggMsgRateOut); @@ -930,6 +935,7 @@ public CompletableFuture asyncGetStats(GetStatsOptions if (producer.isRemote()) { remotePublishersStats.put(producer.getRemoteCluster(), publisherStats); } else if (!getStatsOptions.isExcludePublishers()) { + // Exclude producers for replication from "publishers" stats.addPublisher(publisherStats); } }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 22041326ba240..e9ed8aa6edf21 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -88,6 +88,7 @@ import org.apache.bookkeeper.net.BookieId; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; @@ -2257,8 +2258,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats replicators.forEach((region, replicator) -> replicator.updateRates()); - nsStats.producerCount += producers.size(); - bundleStats.producerCount += producers.size(); + final MutableInt producerCount = new MutableInt(); topicStatsStream.startObject(topic); // start publisher stats @@ -2272,14 +2272,19 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats if (producer.isRemote()) { topicStatsHelper.remotePublishersStats.put(producer.getRemoteCluster(), publisherStats); - } - - // Populate consumer specific stats here - if (hydratePublishers) { - StreamingStats.writePublisherStats(topicStatsStream, publisherStats); + } else { + // Exclude producers for replication from "publishers" and "producerCount" + producerCount.increment(); + if (hydratePublishers) { + StreamingStats.writePublisherStats(topicStatsStream, publisherStats); + } } }); topicStatsStream.endList(); + + nsStats.producerCount += producerCount.intValue(); + bundleStats.producerCount += producerCount.intValue(); + // if publish-rate increases (eg: 0 to 1K) then pick max publish-rate and if publish-rate decreases then keep // average rate. lastUpdatedAvgPublishRateInMsg = topicStatsHelper.aggMsgRateIn > lastUpdatedAvgPublishRateInMsg @@ -2447,7 +2452,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats // Remaining dest stats. topicStatsHelper.averageMsgSize = topicStatsHelper.aggMsgRateIn == 0.0 ? 0.0 : (topicStatsHelper.aggMsgThroughputIn / topicStatsHelper.aggMsgRateIn); - topicStatsStream.writePair("producerCount", producers.size()); + topicStatsStream.writePair("producerCount", producerCount.intValue()); topicStatsStream.writePair("averageMsgSize", topicStatsHelper.averageMsgSize); topicStatsStream.writePair("msgRateIn", topicStatsHelper.aggMsgRateIn); topicStatsStream.writePair("msgRateOut", topicStatsHelper.aggMsgRateOut); @@ -2535,8 +2540,8 @@ public CompletableFuture asyncGetStats(GetStatsOptions if (producer.isRemote()) { remotePublishersStats.put(producer.getRemoteCluster(), publisherStats); - } - if (!getStatsOptions.isExcludePublishers()){ + } else if (!getStatsOptions.isExcludePublishers()) { + // Exclude producers for replication from "publishers" stats.addPublisher(publisherStats); } }); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index eb31c13b0d528..99fd4d877c173 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -28,12 +28,15 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Sets; import io.netty.util.concurrent.FastThreadLocalThread; import java.lang.reflect.Field; import java.lang.reflect.Method; import java.time.Duration; import java.util.Arrays; +import java.util.Iterator; import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -138,17 +141,49 @@ public void testReplicatorProducerStatInTopic() throws Exception { // Verify replicator works. Producer producer1 = client1.newProducer().topic(topicName).create(); + Producer producer2 = client2.newProducer().topic(topicName).create(); // Do not publish messages Consumer consumer2 = client2.newConsumer().topic(topicName).subscriptionName(subscribeName).subscribe(); producer1.newMessage().value(msgValue).send(); pulsar1.getBrokerService().checkReplicationPolicies(); assertEquals(consumer2.receive(10, TimeUnit.SECONDS).getValue(), msgValue); - // Verify there has one item in the attribute "publishers" or "replications" + // Verify that the "publishers" field does not include the producer for replication TopicStats topicStats2 = admin2.topics().getStats(topicName); - assertTrue(topicStats2.getPublishers().size() + topicStats2.getReplication().size() > 0); + assertEquals(topicStats2.getPublishers().size(), 1); + assertFalse(topicStats2.getPublishers().get(0).getProducerName().startsWith(config1.getReplicatorPrefix())); + + // Update broker stats immediately (usually updated every minute) + pulsar2.getBrokerService().updateRates(); + String brokerStats2 = admin2.brokerStats().getTopics(); + + boolean found = false; + ObjectMapper mapper = new ObjectMapper(); + JsonNode rootNode = mapper.readTree(brokerStats2); + if (rootNode.hasNonNull(replicatedNamespace)) { + Iterator bundleNodes = rootNode.get(replicatedNamespace).elements(); + while (bundleNodes.hasNext()) { + JsonNode bundleNode = bundleNodes.next(); + if (bundleNode.hasNonNull("persistent") && bundleNode.get("persistent").hasNonNull(topicName)) { + found = true; + JsonNode topicNode = bundleNode.get("persistent").get(topicName); + // Verify that the "publishers" field does not include the producer for replication + assertEquals(topicNode.get("publishers").size(), 1); + assertEquals(topicNode.get("producerCount").intValue(), 1); + Iterator publisherNodes = topicNode.get("publishers").elements(); + while (publisherNodes.hasNext()) { + JsonNode publisherNode = publisherNodes.next(); + assertFalse(publisherNode.get("producerName").textValue() + .startsWith(config1.getReplicatorPrefix())); + } + break; + } + } + } + assertTrue(found); // cleanup. - consumer2.close(); + consumer2.unsubscribe(); + producer2.close(); producer1.close(); cleanupTopics(() -> { admin1.topics().delete(topicName); From a7e1fcd0c508d2a0ee1e6b0fbffa5ae397db5948 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 7 May 2024 14:54:31 +0800 Subject: [PATCH 151/580] [fix] [ml] Mark delete stuck due to switching cursor ledger fails (#22662) --- .../mledger/impl/ManagedCursorImpl.java | 19 ++++++--- .../mledger/impl/ManagedCursorTest.java | 42 +++++++++++++++++++ 2 files changed, 56 insertions(+), 5 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index c2f33639c3d26..3671385e60f75 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -2187,8 +2187,7 @@ public void operationFailed(ManagedLedgerException exception) { if (ledger.isNoMessagesAfterPos(mdEntry.newPosition)) { persistPositionToMetaStore(mdEntry, cb); } else { - mdEntry.callback.markDeleteFailed(new ManagedLedgerException("Create new cursor ledger failed"), - mdEntry.ctx); + cb.operationFailed(new ManagedLedgerException("Switch new cursor ledger failed")); } } else { persistPositionToLedger(cursorLedger, mdEntry, cb); @@ -2861,9 +2860,19 @@ public void operationFailed(ManagedLedgerException exception) { synchronized (pendingMarkDeleteOps) { // At this point we don't have a ledger ready STATE_UPDATER.set(ManagedCursorImpl.this, State.NoLedger); - // Note: if the stat is NoLedger, will persist the mark deleted position to metadata store. - // Before giving up, try to persist the position in the metadata store. - flushPendingMarkDeletes(); + // There are two case may cause switch ledger fails. + // 1. No enough BKs; BKs are in read-only mode... + // 2. Write ZK fails. + // Regarding the case "No enough BKs", try to persist the position in the metadata store before + // giving up. + if (!(exception instanceof MetaStoreException)) { + flushPendingMarkDeletes(); + } else { + while (!pendingMarkDeleteOps.isEmpty()) { + MarkDeleteEntry entry = pendingMarkDeleteOps.poll(); + entry.callback.markDeleteFailed(exception, entry.ctx); + } + } } } }); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 4e3f8b7908438..5c10533e2476b 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -270,6 +270,48 @@ void testPersistentMarkDeleteIfCreateCursorLedgerFailed() throws Exception { ml.delete(); } + @Test + void testSwitchLedgerFailed() throws Exception { + final String cursorName = "c1"; + final String mlName = UUID.randomUUID().toString().replaceAll("-", ""); + final ManagedLedgerConfig mlConfig = new ManagedLedgerConfig(); + mlConfig.setMaxEntriesPerLedger(1); + mlConfig.setMetadataMaxEntriesPerLedger(1); + mlConfig.setThrottleMarkDelete(Double.MAX_VALUE); + ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open(mlName, mlConfig); + ManagedCursor cursor = ml.openCursor(cursorName); + + List positionList = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + positionList.add(ml.addEntry(("entry-" + i).getBytes(Encoding))); + } + + // Inject an error when persistent at the third time. + AtomicInteger persistentCounter = new AtomicInteger(); + metadataStore.failConditional(new MetadataStoreException.BadVersionException("mock error"), (op, path) -> { + if (path.equals(String.format("/managed-ledgers/%s/%s", mlName, cursorName)) + && persistentCounter.incrementAndGet() == 3) { + log.info("Trigger an error"); + return true; + } + return false; + }); + + // Verify: the cursor can be recovered after it fails once. + int failedCount = 0; + for (Position position : positionList) { + try { + cursor.markDelete(position); + } catch (Exception ex) { + failedCount++; + } + } + assertEquals(failedCount, 1); + + // cleanup. + ml.delete(); + } + @Test void testPersistentMarkDeleteIfSwitchCursorLedgerFailed() throws Exception { final int entryCount = 10; From 7e88463d9a598f95725bee49fd7f713bce27cf28 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Tue, 7 May 2024 20:45:16 +0800 Subject: [PATCH 152/580] [fix] Fix Reader can be stuck from transaction aborted messages. (#22610) --- .../mledger/util/ManagedLedgerImplUtils.java | 17 ++--- .../service/persistent/PersistentTopic.java | 24 +++---- .../broker/transaction/TransactionTest.java | 69 +++++++++++++++++++ .../buffer/TopicTransactionBufferTest.java | 36 ++++++---- 4 files changed, 111 insertions(+), 35 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtils.java index cd8671b0e6289..01de115290ab9 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtils.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtils.java @@ -38,11 +38,7 @@ public static CompletableFuture asyncGetLastValidPosition(final Manage final Predicate predicate, final PositionImpl startPosition) { CompletableFuture future = new CompletableFuture<>(); - if (!ledger.isValidPosition(startPosition)) { - future.complete(startPosition); - } else { - internalAsyncReverseFindPositionOneByOne(ledger, predicate, startPosition, future); - } + internalAsyncReverseFindPositionOneByOne(ledger, predicate, startPosition, future); return future; } @@ -50,6 +46,10 @@ private static void internalAsyncReverseFindPositionOneByOne(final ManagedLedger final Predicate predicate, final PositionImpl position, final CompletableFuture future) { + if (!ledger.isValidPosition(position)) { + future.complete(position); + return; + } ledger.asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { @Override public void readEntryComplete(Entry entry, Object ctx) { @@ -60,12 +60,7 @@ public void readEntryComplete(Entry entry, Object ctx) { return; } PositionImpl previousPosition = ledger.getPreviousPosition((PositionImpl) position); - if (!ledger.isValidPosition(previousPosition)) { - future.complete(previousPosition); - } else { - internalAsyncReverseFindPositionOneByOne(ledger, predicate, - ledger.getPreviousPosition((PositionImpl) position), future); - } + internalAsyncReverseFindPositionOneByOne(ledger, predicate, previousPosition, future); } catch (Exception e) { future.completeExceptionally(e); } finally { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index e9ed8aa6edf21..58ea8088dba67 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -3768,18 +3768,18 @@ public Position getLastPosition() { @Override public CompletableFuture getLastDispatchablePosition() { - PositionImpl maxReadPosition = getMaxReadPosition(); - // If `maxReadPosition` is not equal to `LastPosition`. It means that there are uncommitted transactions. - // so return `maxRedPosition` directly. - if (maxReadPosition.compareTo((PositionImpl) getLastPosition()) != 0) { - return CompletableFuture.completedFuture(maxReadPosition); - } else { - return ManagedLedgerImplUtils.asyncGetLastValidPosition((ManagedLedgerImpl) ledger, entry -> { - MessageMetadata md = Commands.parseMessageMetadata(entry.getDataBuffer()); - // If a messages has marker will filter by AbstractBaseDispatcher.filterEntriesForConsumer - return !Markers.isServerOnlyMarker(md); - }, maxReadPosition); - } + return ManagedLedgerImplUtils.asyncGetLastValidPosition((ManagedLedgerImpl) ledger, entry -> { + MessageMetadata md = Commands.parseMessageMetadata(entry.getDataBuffer()); + // If a messages has marker will filter by AbstractBaseDispatcher.filterEntriesForConsumer + if (Markers.isServerOnlyMarker(md)) { + return false; + } else if (md.hasTxnidMostBits() && md.hasTxnidLeastBits()) { + // Filter-out transaction aborted messages. + TxnID txnID = new TxnID(md.getTxnidMostBits(), md.getTxnidLeastBits()); + return !isTxnAborted(txnID, (PositionImpl) entry.getPosition()); + } + return true; + }, getMaxReadPosition()); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index ed1b74c46e0f0..e8c15d193a22d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -1978,4 +1978,73 @@ public void testDelayedDeliveryExceedsMaxDelay() throws Exception { + maxDeliveryDelayInMillis + " milliseconds"); } } + + @Test + public void testPersistentTopicGetLastDispatchablePositionWithTxn() throws Exception { + String topic = "persistent://" + NAMESPACE1 + "/testPersistentTopicGetLastDispatchablePositionWithTxn"; + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .enableBatching(false) + .create(); + + BrokerService brokerService = pulsarTestContexts.get(0).getBrokerService(); + PersistentTopic persistentTopic = (PersistentTopic) brokerService.getTopicReference(topic).get(); + + + // send a normal message + String body = UUID.randomUUID().toString(); + MessageIdImpl msgId = (MessageIdImpl) producer.send(body); + + // send 3 txn messages + Transaction txn = pulsarClient.newTransaction().build().get(); + producer.newMessage(txn).value(UUID.randomUUID().toString()).send(); + producer.newMessage(txn).value(UUID.randomUUID().toString()).send(); + producer.newMessage(txn).value(UUID.randomUUID().toString()).send(); + + // get last dispatchable position + PositionImpl lastDispatchablePosition = (PositionImpl) persistentTopic.getLastDispatchablePosition().get(); + // the last dispatchable position should be the message id of the normal message + assertEquals(lastDispatchablePosition, PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId())); + + // abort the txn + txn.abort().get(5, TimeUnit.SECONDS); + + // get last dispatchable position + lastDispatchablePosition = (PositionImpl) persistentTopic.getLastDispatchablePosition().get(); + // the last dispatchable position should be the message id of the normal message + assertEquals(lastDispatchablePosition, PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId())); + + + @Cleanup + Reader reader = pulsarClient.newReader(Schema.STRING) + .topic(topic) + .startMessageId(MessageId.earliest) + .create(); + Transaction txn1 = pulsarClient.newTransaction().build().get(); + producer.newMessage(txn1).value(UUID.randomUUID().toString()).send(); + producer.newMessage(txn1).value(UUID.randomUUID().toString()).send(); + producer.newMessage(txn1).value(UUID.randomUUID().toString()).send(); + List> messages = new ArrayList<>(); + while (reader.hasMessageAvailable()) { + messages.add(reader.readNext()); + } + assertEquals(messages.size(), 1); + assertEquals(messages.get(0).getValue(), body); + + txn1.abort().get(5, TimeUnit.SECONDS); + + @Cleanup + Reader reader1 = pulsarClient.newReader(Schema.STRING) + .topic(topic) + .startMessageId(MessageId.earliest) + .create(); + List> messages1 = new ArrayList<>(); + while (reader1.hasMessageAvailable()) { + messages1.add(reader1.readNext()); + } + assertEquals(messages1.size(), 1); + assertEquals(messages1.get(0).getValue(), body); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java index b0903b00be380..f93cfbcdc50f0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java @@ -280,9 +280,9 @@ public void testGetLastMessageIdsWithOngoingTransactions() throws Exception { for (int i = 0; i < 3; i++) { expectedLastMessageID = (MessageIdImpl) producer.newMessage().send(); } - assertMessageId(consumer, expectedLastMessageID); + assertGetLastMessageId(consumer, expectedLastMessageID); // 2.2 Case2: send 2 ongoing transactional messages and 2 original messages. - // |1:0|1:1|1:2|txn1:start->1:3|1:4|txn2:start->1:5|1:6|. + // |1:0|1:1|1:2|txn1:start->1:3|1:4|txn2:start->1:5. Transaction txn1 = pulsarClient.newTransaction() .withTransactionTimeout(5, TimeUnit.HOURS) .build() @@ -291,25 +291,37 @@ public void testGetLastMessageIdsWithOngoingTransactions() throws Exception { .withTransactionTimeout(5, TimeUnit.HOURS) .build() .get(); + + // |1:0|1:1|1:2|txn1:1:3| producer.newMessage(txn1).send(); - // expectedLastMessageID1 == 1:4 + + // |1:0|1:1|1:2|txn1:1:3|1:4| MessageIdImpl expectedLastMessageID1 = (MessageIdImpl) producer.newMessage().send(); + + // |1:0|1:1|1:2|txn1:1:3|1:4|txn2:1:5| producer.newMessage(txn2).send(); - // expectedLastMessageID2 == 1:6 - MessageIdImpl expectedLastMessageID2 = (MessageIdImpl) producer.newMessage().send(); // 2.2.1 Last message ID will not change when txn1 and txn2 do not end. - assertMessageId(consumer, expectedLastMessageID); + assertGetLastMessageId(consumer, expectedLastMessageID); // 2.2.2 Last message ID will update to 1:4 when txn1 committed. - // |1:0|1:1|1:2|txn1:start->1:3|1:4|txn2:start->1:5|1:6|tx1:commit->1:7| + // |1:0|1:1|1:2|txn1:1:3|1:4|txn2:1:5|tx1:commit->1:6| txn1.commit().get(5, TimeUnit.SECONDS); - assertMessageId(consumer, expectedLastMessageID1); + assertGetLastMessageId(consumer, expectedLastMessageID1); - // 2.2.3 Last message ID will update to 1:6 when txn2 aborted. - // |1:0|1:1|1:2|txn1:start->1:3|1:4|txn2:start->1:5|1:6|tx1:commit->1:7|tx2:abort->1:8| + // 2.2.3 Last message ID will still to 1:4 when txn2 aborted. + // |1:0|1:1|1:2|txn1:1:3|1:4|txn2:1:5|tx1:commit->1:6|tx2:abort->1:7| txn2.abort().get(5, TimeUnit.SECONDS); - assertMessageId(consumer, expectedLastMessageID2); + assertGetLastMessageId(consumer, expectedLastMessageID1); + + // Handle the case of the maxReadPosition < lastPosition, but it's an aborted transactional message. + Transaction txn3 = pulsarClient.newTransaction() + .build() + .get(); + producer.newMessage(txn3).send(); + assertGetLastMessageId(consumer, expectedLastMessageID1); + txn3.abort().get(5, TimeUnit.SECONDS); + assertGetLastMessageId(consumer, expectedLastMessageID1); } /** @@ -368,7 +380,7 @@ private void triggerLedgerSwitch(String topicName) throws Exception{ }); } - private void assertMessageId(Consumer consumer, MessageIdImpl expected) throws Exception { + private void assertGetLastMessageId(Consumer consumer, MessageIdImpl expected) throws Exception { TopicMessageIdImpl actual = (TopicMessageIdImpl) consumer.getLastMessageIds().get(0); assertEquals(expected.getEntryId(), actual.getEntryId()); assertEquals(expected.getLedgerId(), actual.getLedgerId()); From 816755429a31439b8aea2ee06c1a877143156ca7 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 7 May 2024 15:53:13 +0300 Subject: [PATCH 153/580] [improve][test] Clear fields in AuthZTest classes at cleanup (#22661) --- .../apache/pulsar/broker/admin/AuthZTest.java | 15 +++++++++ .../broker/admin/NamespaceAuthZTest.java | 4 +++ .../pulsar/broker/admin/TopicAuthZTest.java | 32 ++++++++----------- .../broker/admin/TopicPoliciesAuthZTest.java | 2 ++ .../admin/TransactionAndSchemaAuthZTest.java | 14 +++----- .../security/MockedPulsarStandalone.java | 3 ++ 6 files changed, 41 insertions(+), 29 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java index a710a03970d06..3816b9a7a7ed0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AuthZTest.java @@ -47,6 +47,21 @@ public class AuthZTest extends MockedPulsarStandalone { protected static final String TENANT_ADMIN_TOKEN = Jwts.builder() .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); + @Override + public void close() throws Exception { + if (superUserAdmin != null) { + superUserAdmin.close(); + superUserAdmin = null; + } + if (tenantManagerAdmin != null) { + tenantManagerAdmin.close(); + tenantManagerAdmin = null; + } + authorizationService = null; + orignalAuthorizationService = null; + super.close(); + } + @BeforeMethod(alwaysRun = true) public void before() throws IllegalAccessException { orignalAuthorizationService = getPulsarService().getBrokerService().getAuthorizationService(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java index ec6a122f7df80..66e13ef59f0ef 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespaceAuthZTest.java @@ -129,10 +129,14 @@ public void setup() { public void cleanup() { if (superUserAdmin != null) { superUserAdmin.close(); + superUserAdmin = null; } if (tenantManagerAdmin != null) { tenantManagerAdmin.close(); + tenantManagerAdmin = null; } + pulsarClient = null; + authorizationService = null; close(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index ad47ac74a8980..2e05b28e747e4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -19,9 +19,19 @@ package org.apache.pulsar.broker.admin; +import static org.mockito.Mockito.doReturn; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import io.jsonwebtoken.Jwts; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import lombok.SneakyThrows; import org.apache.commons.lang3.reflect.FieldUtils; @@ -45,6 +55,7 @@ import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.EntryFilters; import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; +import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.OffloadPolicies; import org.apache.pulsar.common.policies.data.PersistencePolicies; import org.apache.pulsar.common.policies.data.PolicyName; @@ -53,24 +64,13 @@ import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TopicOperation; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.pulsar.common.policies.data.NamespaceOperation; -import org.apache.pulsar.common.policies.data.TopicOperation; -import org.mockito.Mockito; -import static org.mockito.Mockito.doReturn; @Test(groups = "broker-admin") public class TopicAuthZTest extends AuthZTest { @@ -98,12 +98,6 @@ public void setup() { @SneakyThrows @AfterClass(alwaysRun = true) public void cleanup() { - if (superUserAdmin != null) { - superUserAdmin.close(); - } - if (tenantManagerAdmin != null) { - tenantManagerAdmin.close(); - } close(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java index 1f02afd418326..002ba2cbfcf9d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java @@ -74,9 +74,11 @@ public void before() { public void after() { if (superUserAdmin != null) { superUserAdmin.close(); + superUserAdmin = null; } if (tenantManagerAdmin != null) { tenantManagerAdmin.close(); + tenantManagerAdmin = null; } close(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java index 1bca6f6e30835..f52d6dae9bb23 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TransactionAndSchemaAuthZTest.java @@ -19,6 +19,10 @@ package org.apache.pulsar.broker.admin; import io.jsonwebtoken.Jwts; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import lombok.Cleanup; import lombok.SneakyThrows; import org.apache.commons.lang3.reflect.FieldUtils; @@ -47,10 +51,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; @Test(groups = "broker-admin") public class TransactionAndSchemaAuthZTest extends AuthZTest { @@ -82,12 +82,6 @@ public void setup() { @SneakyThrows @AfterClass(alwaysRun = true) public void cleanup() { - if (superUserAdmin != null) { - superUserAdmin.close(); - } - if (tenantManagerAdmin != null) { - tenantManagerAdmin.close(); - } close(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java b/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java index b82f3b584065d..4a7d71c2b4f3e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java @@ -193,7 +193,10 @@ private void setupDefaultTenantAndNamespace() throws Exception { public void close() throws Exception { if (pulsarTestContext != null) { pulsarTestContext.close(); + pulsarTestContext = null; } + pulsarService = null; + serviceInternalAdmin = null; } // Utils From 09364a95f8429b12a5951d4d1ff45766b13e92cb Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 7 May 2024 07:13:45 -0700 Subject: [PATCH 154/580] [improve] Upgrade to Oxia client 0.2.0 (#22663) --- .../licenses/LICENSE-Reactive-gRPC.txt | 29 ---------- .../server/src/assemble/LICENSE.bin.txt | 9 +--- pom.xml | 2 +- .../metadata/impl/oxia/OxiaMetadataStore.java | 53 +++++++++++-------- 4 files changed, 33 insertions(+), 60 deletions(-) delete mode 100644 distribution/licenses/LICENSE-Reactive-gRPC.txt diff --git a/distribution/licenses/LICENSE-Reactive-gRPC.txt b/distribution/licenses/LICENSE-Reactive-gRPC.txt deleted file mode 100644 index bc589401e7bdf..0000000000000 --- a/distribution/licenses/LICENSE-Reactive-gRPC.txt +++ /dev/null @@ -1,29 +0,0 @@ -BSD 3-Clause License - -Copyright (c) 2019, Salesforce.com, Inc. -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - -* Redistributions of source code must retain the above copyright notice, this - list of conditions and the following disclaimer. - -* Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - -* Neither the name of the copyright holder nor the names of its - contributors may be used to endorse or promote products derived from - this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" -AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE -IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE -FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL -DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR -SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER -CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, -OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index aec4df2a93af9..818f389be88ef 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -481,12 +481,10 @@ The Apache Software License, Version 2.0 * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia - - io.streamnative.oxia-oxia-client-0.1.6.jar - - io.streamnative.oxia-oxia-client-metrics-api-0.1.6.jar + - io.streamnative.oxia-oxia-client-api-0.2.0.jar + - io.streamnative.oxia-oxia-client-0.2.0.jar * OpenHFT - net.openhft-zero-allocation-hashing-0.16.jar - * Project reactor - - io.projectreactor-reactor-core-3.5.2.jar * Java JSON WebTokens - io.jsonwebtoken-jjwt-api-0.11.1.jar - io.jsonwebtoken-jjwt-impl-0.11.1.jar @@ -552,9 +550,6 @@ BSD 3-clause "New" or "Revised" License * JSR305 -- com.google.code.findbugs-jsr305-3.0.2.jar -- ../licenses/LICENSE-JSR305.txt * JLine -- jline-jline-2.14.6.jar -- ../licenses/LICENSE-JLine.txt * JLine3 -- org.jline-jline-3.21.0.jar -- ../licenses/LICENSE-JLine.txt - * Reactive gRPC - - com.salesforce.servicelibs-reactive-grpc-common-1.2.4.jar -- ../licenses/LICENSE-Reactive-gRPC.txt - - com.salesforce.servicelibs-reactor-grpc-stub-1.2.4.jar -- ../licenses/LICENSE-Reactive-gRPC.txt BSD 2-Clause License * HdrHistogram -- org.hdrhistogram-HdrHistogram-2.1.9.jar -- ../licenses/LICENSE-HdrHistogram.txt diff --git a/pom.xml b/pom.xml index 8f7ae2ed1fc68..92e021d1eaa5f 100644 --- a/pom.xml +++ b/pom.xml @@ -249,7 +249,7 @@ flexible messaging model and an intuitive client API. 4.5.13 4.4.15 0.7.5 - 0.1.6 + 0.2.0 2.0 1.10.12 5.3.3 diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java index 2ab744e205320..728bc1175b9ba 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java @@ -18,20 +18,23 @@ */ package org.apache.pulsar.metadata.impl.oxia; -import io.streamnative.oxia.client.OxiaClientBuilder; import io.streamnative.oxia.client.api.AsyncOxiaClient; import io.streamnative.oxia.client.api.DeleteOption; -import io.streamnative.oxia.client.api.KeyAlreadyExistsException; import io.streamnative.oxia.client.api.Notification; +import io.streamnative.oxia.client.api.OxiaClientBuilder; import io.streamnative.oxia.client.api.PutOption; import io.streamnative.oxia.client.api.PutResult; -import io.streamnative.oxia.client.api.UnexpectedVersionIdException; import io.streamnative.oxia.client.api.Version; +import io.streamnative.oxia.client.api.exceptions.KeyAlreadyExistsException; +import io.streamnative.oxia.client.api.exceptions.UnexpectedVersionIdException; import java.time.Duration; +import java.util.Collections; import java.util.EnumSet; +import java.util.HashSet; import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; @@ -69,7 +72,7 @@ public class OxiaMetadataStore extends AbstractMetadataStore { this.synchronizer = Optional.ofNullable(metadataStoreConfig.getSynchronizer()); identity = UUID.randomUUID().toString(); client = - new OxiaClientBuilder(serviceAddress) + OxiaClientBuilder.create(serviceAddress) .clientIdentifier(identity) .namespace(namespace) .sessionTimeout(Duration.ofMillis(metadataStoreConfig.getSessionTimeoutMillis())) @@ -153,14 +156,14 @@ protected CompletableFuture storeDelete(String path, Optional expect return getChildrenFromStore(path) .thenCompose( children -> { - if (children.size() > 0) { + if (!children.isEmpty()) { return CompletableFuture.failedFuture( new MetadataStoreException("Key '" + path + "' has children")); } else { - var delOption = + Set delOption = expectedVersion - .map(DeleteOption::ifVersionIdEquals) - .orElse(DeleteOption.Unconditionally); + .map(v -> Collections.singleton(DeleteOption.IfVersionIdEquals(v))) + .orElse(Collections.emptySet()); CompletableFuture result = client.delete(path, delOption); return result .thenCompose( @@ -205,20 +208,20 @@ protected CompletableFuture storePut( } else { actualPath = CompletableFuture.completedFuture(path); } - var versionCondition = - expectedVersion - .map( - ver -> { - if (ver == -1) { - return PutOption.IfRecordDoesNotExist; - } - return PutOption.ifVersionIdEquals(ver); - }) - .orElse(PutOption.Unconditionally); - var putOptions = - options.contains(CreateOption.Ephemeral) - ? new PutOption[] {PutOption.AsEphemeralRecord, versionCondition} - : new PutOption[] {versionCondition}; + Set putOptions = new HashSet<>(); + expectedVersion + .map( + ver -> { + if (ver == -1) { + return PutOption.IfRecordDoesNotExist; + } + return PutOption.IfVersionIdEquals(ver); + }) + .ifPresent(putOptions::add); + + if (options.contains(CreateOption.Ephemeral)) { + putOptions.add(PutOption.AsEphemeralRecord); + } return actualPath .thenCompose( aPath -> @@ -242,6 +245,10 @@ private CompletionStage convertException(Throwable ex) { } } + private static final byte[] EMPTY_VALUE = new byte[0]; + private static final Set IF_RECORD_DOES_NOT_EXIST = + Collections.singleton(PutOption.IfRecordDoesNotExist); + private CompletableFuture createParents(String path) { var parent = parent(path); if (parent == null || parent.isEmpty()) { @@ -254,7 +261,7 @@ private CompletableFuture createParents(String path) { return CompletableFuture.completedFuture(null); } else { return client - .put(parent, new byte[] {}, PutOption.IfRecordDoesNotExist) + .put(parent, EMPTY_VALUE, IF_RECORD_DOES_NOT_EXIST) .thenCompose(__ -> createParents(parent)); } }) From 788b5ae9bde0b6b4732eb53237288924c711b8b7 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 7 May 2024 09:46:18 -0700 Subject: [PATCH 155/580] [feat][broker] PIP-264: Add Java runtime metrics (#22616) Co-authored-by: Matteo Merli Co-authored-by: Lari Hotari --- build/run_unit_group.sh | 2 + conf/pulsar_env.sh | 4 ++ .../server/src/assemble/LICENSE.bin.txt | 2 + pom.xml | 3 +- .../prometheus/PrometheusMetricsClient.java | 2 +- .../AdminApiTransactionMultiBrokerTest.java | 2 +- pulsar-opentelemetry/pom.xml | 14 ++++++ .../opentelemetry/OpenTelemetryService.java | 25 ++++++++-- .../OpenTelemetryServiceTest.java | 48 +++++++++++++++++++ .../latest-version-image/conf/bookie.conf | 2 +- .../latest-version-image/conf/broker.conf | 2 +- .../conf/functions_worker.conf | 2 +- .../latest-version-image/conf/global-zk.conf | 2 +- .../latest-version-image/conf/local-zk.conf | 2 +- .../latest-version-image/conf/proxy.conf | 2 +- .../latest-version-image/conf/websocket.conf | 2 +- 16 files changed, 102 insertions(+), 14 deletions(-) diff --git a/build/run_unit_group.sh b/build/run_unit_group.sh index 351477aed1c92..2694505e0e098 100755 --- a/build/run_unit_group.sh +++ b/build/run_unit_group.sh @@ -85,6 +85,8 @@ function test_group_broker_group_2() { function test_group_broker_group_3() { mvn_test -pl pulsar-broker -Dgroups='broker-admin' + # run AdminApiTransactionMultiBrokerTest independently with a larger heap size + mvn_test -pl pulsar-broker -DtestMaxHeapSize=1500M -Dtest=org.apache.pulsar.broker.admin.v3.AdminApiTransactionMultiBrokerTest -DtestForkCount=1 -DtestReuseFork=false } function test_group_broker_group_4() { diff --git a/conf/pulsar_env.sh b/conf/pulsar_env.sh index c7bba23c234d9..3a069e31fdc90 100755 --- a/conf/pulsar_env.sh +++ b/conf/pulsar_env.sh @@ -94,3 +94,7 @@ PULSAR_EXTRA_OPTS="${PULSAR_EXTRA_OPTS:-" -Dpulsar.allocator.exit_on_oom=true -D #Wait time before forcefully kill the pulsar server instance, if the stop is not successful #PULSAR_STOP_TIMEOUT= +# Enable semantically stable telemetry for JVM metrics, unless otherwise overridden by the user. +if [ -z "$OTEL_SEMCONV_STABILITY_OPT_IN" ]; then + export OTEL_SEMCONV_STABILITY_OPT_IN=jvm +fi diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 818f389be88ef..84b93647d0ec4 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -540,6 +540,8 @@ The Apache Software License, Version 2.0 - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-1.33.2.jar - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-semconv-1.33.2-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-resources-1.33.2-alpha.jar + - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-1.33.2-alpha.jar + - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-1.33.2-alpha.jar - io.opentelemetry.semconv-opentelemetry-semconv-1.25.0-alpha.jar BSD 3-clause "New" or "Revised" License diff --git a/pom.xml b/pom.xml index 92e021d1eaa5f..cec3b3c60db9e 100644 --- a/pom.xml +++ b/pom.xml @@ -115,6 +115,7 @@ flexible messaging model and an intuitive client API. --add-opens jdk.management/com.sun.management.internal=ALL-UNNAMED --add-opens java.base/jdk.internal.platform=ALL-UNNAMED + 1300M true 4 false @@ -1652,7 +1653,7 @@ flexible messaging model and an intuitive client API. org.apache.maven.plugins maven-surefire-plugin - ${testJacocoAgentArgument} -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=${testHeapDumpPath} -XX:+ExitOnOutOfMemoryError -Xmx1G -XX:+UseZGC + ${testJacocoAgentArgument} -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=${testHeapDumpPath} -XX:+ExitOnOutOfMemoryError -Xmx${testMaxHeapSize} -XX:+UseZGC -Dpulsar.allocator.pooled=true -Dpulsar.allocator.leak_detection=Advanced -Dpulsar.allocator.exit_on_oom=false 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 6fd509690278d..6d724c289b52c 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 @@ -59,7 +59,7 @@ public static Multimap parseMetrics(String metrics) { // or // pulsar_subscriptions_count{cluster="standalone", namespace="public/default", // topic="persistent://public/default/test-2"} 0.0 - Pattern pattern = Pattern.compile("^(\\w+)\\{([^}]+)}\\s([+-]?[\\d\\w.-]+)$"); + Pattern pattern = Pattern.compile("^(\\w+)\\{([^}]+)}\\s([+-]?[\\d\\w.+-]+)$"); Pattern tagsPattern = Pattern.compile("(\\w+)=\"([^\"]+)\"(,\\s?)?"); Splitter.on("\n").split(metrics).forEach(line -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionMultiBrokerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionMultiBrokerTest.java index e2f4a5abdb9e0..113937c2558d9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionMultiBrokerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionMultiBrokerTest.java @@ -40,7 +40,7 @@ import org.testng.annotations.Test; @Slf4j -@Test(groups = "broker-admin") +@Test(groups = "broker-admin-isolated") public class AdminApiTransactionMultiBrokerTest extends TransactionTestBase { private static final int NUM_BROKERS = 16; diff --git a/pulsar-opentelemetry/pom.xml b/pulsar-opentelemetry/pom.xml index 82a9658cc9d31..e32f1b81ff964 100644 --- a/pulsar-opentelemetry/pom.xml +++ b/pulsar-opentelemetry/pom.xml @@ -58,6 +58,10 @@ io.opentelemetry.semconv opentelemetry-semconv + + io.opentelemetry.instrumentation + opentelemetry-runtime-telemetry-java17 + com.google.guava @@ -130,6 +134,16 @@ + + + org.apache.maven.plugins + maven-surefire-plugin + + + jvm + + + diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java index 16c4264be6d12..4560d3813d6dd 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java @@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.common.annotations.VisibleForTesting; import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.instrumentation.runtimemetrics.java17.RuntimeMetrics; import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; @@ -29,6 +30,7 @@ import java.io.Closeable; import java.util.Map; import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import lombok.Builder; import org.apache.commons.lang3.StringUtils; @@ -42,7 +44,9 @@ public class OpenTelemetryService implements Closeable { public static final String OTEL_SDK_DISABLED_KEY = "otel.sdk.disabled"; static final int MAX_CARDINALITY_LIMIT = 10000; - private final OpenTelemetrySdk openTelemetrySdk; + private final AtomicReference openTelemetrySdkReference = new AtomicReference<>(); + + private final AtomicReference runtimeMetricsReference = new AtomicReference<>(); /** * Instantiates the OpenTelemetry SDK. All attributes are overridden by system properties or environment @@ -94,15 +98,28 @@ public OpenTelemetryService(String clusterName, builderCustomizer.accept(sdkBuilder); } - openTelemetrySdk = sdkBuilder.build().getOpenTelemetrySdk(); + openTelemetrySdkReference.set(sdkBuilder.build().getOpenTelemetrySdk()); + + // For a list of exposed metrics, see https://opentelemetry.io/docs/specs/semconv/runtime/jvm-metrics/ + runtimeMetricsReference.set(RuntimeMetrics.builder(openTelemetrySdkReference.get()) + .enableAllFeatures() + .enableExperimentalJmxTelemetry() + .build()); } public OpenTelemetry getOpenTelemetry() { - return openTelemetrySdk; + return openTelemetrySdkReference.get(); } @Override public void close() { - openTelemetrySdk.close(); + RuntimeMetrics runtimeMetrics = runtimeMetricsReference.getAndSet(null); + if (runtimeMetrics != null) { + runtimeMetrics.close(); + } + OpenTelemetrySdk openTelemetrySdk = openTelemetrySdkReference.getAndSet(null); + if (openTelemetrySdk != null) { + openTelemetrySdk.close(); + } } } diff --git a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java index bf404496a2eca..31a6c60f83afe 100644 --- a/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java +++ b/pulsar-opentelemetry/src/test/java/org/apache/pulsar/opentelemetry/OpenTelemetryServiceTest.java @@ -198,4 +198,52 @@ public void testServiceIsDisabledByDefault() throws Exception { // Validate that the callback has not being called. assertThat(callback).isFalse(); } + + @Test + public void testJvmRuntimeMetrics() { + // Attempt collection of GC metrics. The metrics should be populated regardless if GC is triggered or not. + Runtime.getRuntime().gc(); + + var metrics = reader.collectAllMetrics(); + + // Process Metrics + // Replaces process_cpu_seconds_total + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.cpu.time")); + + // Memory Metrics + // Replaces jvm_memory_bytes_used + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.memory.used")); + // Replaces jvm_memory_bytes_committed + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.memory.committed")); + // Replaces jvm_memory_bytes_max + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.memory.limit")); + // Replaces jvm_memory_bytes_init + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.memory.init")); + // Replaces jvm_memory_pool_allocated_bytes_total + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.memory.used_after_last_gc")); + + // Buffer Pool Metrics + // Replaces jvm_buffer_pool_used_bytes + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.buffer.memory.usage")); + // Replaces jvm_buffer_pool_capacity_bytes + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.buffer.memory.limit")); + // Replaces jvm_buffer_pool_used_buffers + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.buffer.count")); + + // Garbage Collector Metrics + // Replaces jvm_gc_collection_seconds + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.gc.duration")); + + // Thread Metrics + // Replaces jvm_threads_state, jvm_threads_current and jvm_threads_daemon + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.thread.count")); + + // Class Loading Metrics + // Replaces jvm_classes_currently_loaded + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.class.count")); + // Replaces jvm_classes_loaded_total + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.class.loaded")); + // Replaces jvm_classes_unloaded_total + assertThat(metrics).anySatisfy(metric -> assertThat(metric).hasName("jvm.class.unloaded")); + } } diff --git a/tests/docker-images/latest-version-image/conf/bookie.conf b/tests/docker-images/latest-version-image/conf/bookie.conf index 07547bcaef6d3..df7501057a58f 100644 --- a/tests/docker-images/latest-version-image/conf/bookie.conf +++ b/tests/docker-images/latest-version-image/conf/bookie.conf @@ -22,7 +22,7 @@ autostart=false redirect_stderr=true stdout_logfile=/var/log/pulsar/bookie.log directory=/pulsar -environment=PULSAR_MEM="-Xmx128M -XX:MaxDirectMemorySize=512M",PULSAR_GC="-XX:+UseZGC" +environment=PULSAR_MEM="-Xmx128M -XX:MaxDirectMemorySize=512M -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/var/log/pulsar -XX:+ExitOnOutOfMemoryError",PULSAR_GC="-XX:+UseZGC" command=/pulsar/bin/pulsar bookie user=pulsar stopwaitsecs=15 diff --git a/tests/docker-images/latest-version-image/conf/broker.conf b/tests/docker-images/latest-version-image/conf/broker.conf index 63be36437741b..790dace8d6d85 100644 --- a/tests/docker-images/latest-version-image/conf/broker.conf +++ b/tests/docker-images/latest-version-image/conf/broker.conf @@ -22,7 +22,7 @@ autostart=false redirect_stderr=true stdout_logfile=/var/log/pulsar/broker.log directory=/pulsar -environment=PULSAR_MEM="-Xmx128M",PULSAR_GC="-XX:+UseZGC" +environment=PULSAR_MEM="-Xmx150M -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/var/log/pulsar -XX:+ExitOnOutOfMemoryError",PULSAR_GC="-XX:+UseZGC" command=/pulsar/bin/pulsar broker user=pulsar stopwaitsecs=15 diff --git a/tests/docker-images/latest-version-image/conf/functions_worker.conf b/tests/docker-images/latest-version-image/conf/functions_worker.conf index 6feb660231cec..b5d151ce3f9be 100644 --- a/tests/docker-images/latest-version-image/conf/functions_worker.conf +++ b/tests/docker-images/latest-version-image/conf/functions_worker.conf @@ -22,7 +22,7 @@ autostart=false redirect_stderr=true stdout_logfile=/var/log/pulsar/functions_worker.log directory=/pulsar -environment=PULSAR_MEM="-Xmx128M -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/pulsar/logs/functions",PULSAR_GC="-XX:+UseZGC" +environment=PULSAR_MEM="-Xmx150M -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/var/log/pulsar -XX:+ExitOnOutOfMemoryError",PULSAR_GC="-XX:+UseZGC" command=/pulsar/bin/pulsar functions-worker user=pulsar stopwaitsecs=15 \ No newline at end of file diff --git a/tests/docker-images/latest-version-image/conf/global-zk.conf b/tests/docker-images/latest-version-image/conf/global-zk.conf index e5ffd2eb9e769..ef521506846c8 100644 --- a/tests/docker-images/latest-version-image/conf/global-zk.conf +++ b/tests/docker-images/latest-version-image/conf/global-zk.conf @@ -22,7 +22,7 @@ autostart=false redirect_stderr=true stdout_logfile=/var/log/pulsar/global-zk.log directory=/pulsar -environment=PULSAR_MEM="-Xmx128M",PULSAR_GC="-XX:+UseZGC" +environment=PULSAR_MEM="-Xmx128M -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/var/log/pulsar -XX:+ExitOnOutOfMemoryError",PULSAR_GC="-XX:+UseZGC" command=/pulsar/bin/pulsar configuration-store user=pulsar stopwaitsecs=15 \ No newline at end of file diff --git a/tests/docker-images/latest-version-image/conf/local-zk.conf b/tests/docker-images/latest-version-image/conf/local-zk.conf index c96543db8a865..d6bfdcb621b43 100644 --- a/tests/docker-images/latest-version-image/conf/local-zk.conf +++ b/tests/docker-images/latest-version-image/conf/local-zk.conf @@ -22,7 +22,7 @@ autostart=false redirect_stderr=true stdout_logfile=/var/log/pulsar/local-zk.log directory=/pulsar -environment=PULSAR_MEM="-Xmx128M",PULSAR_GC="-XX:+UseZGC" +environment=PULSAR_MEM="-Xmx128M -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/var/log/pulsar -XX:+ExitOnOutOfMemoryError",PULSAR_GC="-XX:+UseZGC" command=/pulsar/bin/pulsar zookeeper user=pulsar stopwaitsecs=15 \ No newline at end of file diff --git a/tests/docker-images/latest-version-image/conf/proxy.conf b/tests/docker-images/latest-version-image/conf/proxy.conf index 343a0f9614e30..17a0a658b4226 100644 --- a/tests/docker-images/latest-version-image/conf/proxy.conf +++ b/tests/docker-images/latest-version-image/conf/proxy.conf @@ -22,7 +22,7 @@ autostart=false redirect_stderr=true stdout_logfile=/var/log/pulsar/proxy.log directory=/pulsar -environment=PULSAR_MEM="-Xmx128M",PULSAR_GC="-XX:+UseZGC" +environment=PULSAR_MEM="-Xmx150M -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/var/log/pulsar -XX:+ExitOnOutOfMemoryError",PULSAR_GC="-XX:+UseZGC" command=/pulsar/bin/pulsar proxy user=pulsar stopwaitsecs=15 \ No newline at end of file diff --git a/tests/docker-images/latest-version-image/conf/websocket.conf b/tests/docker-images/latest-version-image/conf/websocket.conf index 0418c4cbc26a3..7625dba3e030d 100644 --- a/tests/docker-images/latest-version-image/conf/websocket.conf +++ b/tests/docker-images/latest-version-image/conf/websocket.conf @@ -22,7 +22,7 @@ autostart=false redirect_stderr=true stdout_logfile=/var/log/pulsar/pulsar-websocket.log directory=/pulsar -environment=PULSAR_MEM="-Xmx128M",PULSAR_GC="-XX:+UseZGC" +environment=PULSAR_MEM="-Xmx150M -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/var/log/pulsar -XX:+ExitOnOutOfMemoryError",PULSAR_GC="-XX:+UseZGC" command=/pulsar/bin/pulsar websocket user=pulsar stopwaitsecs=15 \ No newline at end of file From 83b86abcb74595d7e8aa31b238a7dbb19a04dde2 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 7 May 2024 10:35:23 -0700 Subject: [PATCH 156/580] [improve] Retry re-validating ResourceLock with backoff after errors (#22617) --- .../broker/service/AbstractReplicator.java | 2 +- .../PulsarMetadataEventSynchronizer.java | 2 +- .../broker/service/TopicPoliciesService.java | 4 +- ...PersistentDispatcherMultipleConsumers.java | 2 +- ...sistentDispatcherSingleActiveConsumer.java | 2 +- .../persistent/PersistentReplicator.java | 2 +- .../pendingack/impl/PendingAckHandleImpl.java | 2 +- .../common/naming/NamespaceBundleFactory.java | 2 +- ...temTopicBasedTopicPoliciesServiceTest.java | 4 +- .../client/impl/ConnectionHandlerTest.java | 2 + .../pulsar/client/impl/RetryUtilTest.java | 2 + .../client/impl/BinaryProtoLookupService.java | 2 + .../pulsar/client/impl/ConnectionHandler.java | 1 + .../pulsar/client/impl/ConsumerImpl.java | 2 + .../impl/PatternMultiTopicsConsumerImpl.java | 2 + .../pulsar/client/impl/ProducerImpl.java | 1 + .../pulsar/client/impl/PulsarClientImpl.java | 2 + .../pulsar/client/impl/TopicListWatcher.java | 1 + .../impl/TransactionMetaStoreHandler.java | 2 + .../apache/pulsar/client/util/RetryUtil.java | 2 +- .../pulsar/client/impl/ConsumerImplTest.java | 1 + .../apache/pulsar/common/util}/Backoff.java | 2 +- .../pulsar/common/util}/BackoffBuilder.java | 5 ++- .../pulsar/common/util}/BackoffTest.java | 2 +- .../coordination/impl/LockManagerImpl.java | 10 ++--- .../coordination/impl/ResourceLockImpl.java | 37 +++++++++++++++++-- .../pulsar/metadata/LockManagerTest.java | 31 ++++++++++++++++ 27 files changed, 105 insertions(+), 24 deletions(-) rename {pulsar-client/src/main/java/org/apache/pulsar/client/impl => pulsar-common/src/main/java/org/apache/pulsar/common/util}/Backoff.java (99%) rename {pulsar-client/src/main/java/org/apache/pulsar/client/impl => pulsar-common/src/main/java/org/apache/pulsar/common/util}/BackoffBuilder.java (91%) rename {pulsar-client/src/test/java/org/apache/pulsar/client/impl => pulsar-common/src/test/java/org/apache/pulsar/common/util}/BackoffTest.java (99%) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index 394fad21ae6dc..869a4bc81d310 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -36,10 +36,10 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.StringInterner; import org.slf4j.Logger; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarMetadataEventSynchronizer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarMetadataEventSynchronizer.java index 80743e44ab7d2..0383a0b755245 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarMetadataEventSynchronizer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarMetadataEventSynchronizer.java @@ -33,8 +33,8 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; -import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.MetadataEvent; import org.apache.pulsar.metadata.api.MetadataEventSynchronizer; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java index 41fecb3b87ed4..eca31ec230a8e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java @@ -24,13 +24,13 @@ import java.util.concurrent.TimeUnit; import javax.annotation.Nonnull; import org.apache.pulsar.broker.service.BrokerServiceException.TopicPoliciesCacheNotInitException; -import org.apache.pulsar.client.impl.Backoff; -import org.apache.pulsar.client.impl.BackoffBuilder; import org.apache.pulsar.client.util.RetryUtil; import org.apache.pulsar.common.classification.InterfaceStability; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicPolicies; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.FutureUtil; import org.jetbrains.annotations.NotNull; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index b441400dae11f..49a19c0fe3138 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -69,11 +69,11 @@ import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; import org.apache.pulsar.broker.transaction.exception.buffer.TransactionBufferException; -import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index a414848e105cc..adaa5a66a0cbe 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -50,8 +50,8 @@ import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; import org.apache.pulsar.broker.transaction.exception.buffer.TransactionBufferException; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; +import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.compaction.CompactedTopicUtils; import org.apache.pulsar.compaction.Compactor; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index 367d19652072d..c3a27a15e9d90 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -56,7 +56,6 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; @@ -65,6 +64,7 @@ import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.stats.Rate; +import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.common.util.Codec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java index 5ed271c6fd414..9d07af4d26c44 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java @@ -59,11 +59,11 @@ import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.transaction.TxnID; -import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.policies.data.TransactionInPendingAckStats; import org.apache.pulsar.common.policies.data.TransactionPendingAckStats; import org.apache.pulsar.common.stats.PositionInPendingAckStats; +import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.RecoverTimeRecord; import org.apache.pulsar.common.util.collections.BitSetRecyclable; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundleFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundleFactory.java index c136ed42f8119..2b285cbb0e2ab 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundleFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/common/naming/NamespaceBundleFactory.java @@ -51,10 +51,10 @@ import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; import org.apache.pulsar.broker.resources.LocalPoliciesResources; import org.apache.pulsar.broker.resources.PulsarResources; -import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.LocalPolicies; import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.policies.data.loadbalancer.BundleData; import org.apache.pulsar.stats.CacheMetricsCollector; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java index 9a5ac50e5a730..9caee00cb6134 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java @@ -45,8 +45,8 @@ import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.impl.Backoff; -import org.apache.pulsar.client.impl.BackoffBuilder; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.events.PulsarEvent; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionHandlerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionHandlerTest.java index d61dc3442dcdc..4bc5707946957 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionHandlerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionHandlerTest.java @@ -32,6 +32,8 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.FutureUtil; import org.awaitility.Awaitility; import org.awaitility.core.ConditionTimeoutException; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RetryUtilTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RetryUtilTest.java index f7a0485a512c7..603378c271f3e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RetryUtilTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RetryUtilTest.java @@ -20,6 +20,8 @@ import lombok.Cleanup; import org.apache.pulsar.client.util.RetryUtil; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.FutureUtil; import org.testng.annotations.Test; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 81c196c731f70..8eedb3250cdf5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -46,6 +46,8 @@ import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.schema.BytesSchemaVersion; import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java index f0f78420115a9..934985949197c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.HandlerState.State; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.util.Backoff; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index f1e259086ec8a..c8f4b0acec36d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -123,6 +123,8 @@ import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.CompletableFutureCancellationHandler; import org.apache.pulsar.common.util.ExceptionHandler; import org.apache.pulsar.common.util.FutureUtil; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index 4d179f7d914c2..ec7ff7930c0ac 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -43,6 +43,8 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.topics.TopicList; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index b8def7e3042bd..6d5a81454631f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -98,6 +98,7 @@ import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.RelativeTimeUtil; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index a919eb19a7ff8..bd1b9564f932c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -87,6 +87,8 @@ import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.topics.TopicList; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.slf4j.Logger; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java index 86adf69f06e0f..4e635e0d2e8d2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java @@ -31,6 +31,7 @@ import org.apache.pulsar.common.api.proto.CommandWatchTopicUpdate; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.util.BackoffBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java index 0b5174a015118..2a43ca20beb38 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java @@ -46,6 +46,8 @@ import org.apache.pulsar.common.api.proto.Subscription; import org.apache.pulsar.common.api.proto.TxnAction; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.collections.ConcurrentLongHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/util/RetryUtil.java b/pulsar-client/src/main/java/org/apache/pulsar/client/util/RetryUtil.java index 93501d7b6c18b..912cb7d7c5832 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/util/RetryUtil.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/util/RetryUtil.java @@ -22,7 +22,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; -import org.apache.pulsar.client.impl.Backoff; +import org.apache.pulsar.common.util.Backoff; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java index 9995246c175e1..0c47d17098eb9 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java @@ -49,6 +49,7 @@ import org.apache.pulsar.client.impl.conf.TopicConsumerConfigurationData; import org.apache.pulsar.client.util.ExecutorProvider; import org.apache.pulsar.client.util.ScheduledExecutorProvider; +import org.apache.pulsar.common.util.Backoff; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/Backoff.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/Backoff.java similarity index 99% rename from pulsar-client/src/main/java/org/apache/pulsar/client/impl/Backoff.java rename to pulsar-common/src/main/java/org/apache/pulsar/common/util/Backoff.java index daaf349940035..4eab85f3c41be 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/Backoff.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/Backoff.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.client.impl; +package org.apache.pulsar.common.util; import com.google.common.annotations.VisibleForTesting; import java.time.Clock; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BackoffBuilder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/BackoffBuilder.java similarity index 91% rename from pulsar-client/src/main/java/org/apache/pulsar/client/impl/BackoffBuilder.java rename to pulsar-common/src/main/java/org/apache/pulsar/common/util/BackoffBuilder.java index 9913393fa9aa9..69b390300815b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BackoffBuilder.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/BackoffBuilder.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.client.impl; +package org.apache.pulsar.common.util; import java.time.Clock; import java.util.concurrent.TimeUnit; @@ -32,8 +32,11 @@ public class BackoffBuilder { public BackoffBuilder() { this.initial = 0; + this.unitInitial = TimeUnit.MILLISECONDS; this.max = 0; + this.unitMax = TimeUnit.MILLISECONDS; this.mandatoryStop = 0; + this.unitMandatoryStop = TimeUnit.MILLISECONDS; this.clock = Clock.systemDefaultZone(); } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BackoffTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/BackoffTest.java similarity index 99% rename from pulsar-client/src/test/java/org/apache/pulsar/client/impl/BackoffTest.java rename to pulsar-common/src/test/java/org/apache/pulsar/common/util/BackoffTest.java index 7f13acb769492..b3786236a70ef 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BackoffTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/BackoffTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.client.impl; +package org.apache.pulsar.common.util; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LockManagerImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LockManagerImpl.java index 4da6b7998a0c4..b6b5c57ccea39 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LockManagerImpl.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LockManagerImpl.java @@ -27,7 +27,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.util.FutureUtil; @@ -52,7 +52,7 @@ class LockManagerImpl implements LockManager { private final MetadataCache cache; private final MetadataSerde serde; private final FutureUtil.Sequencer sequencer; - private final ExecutorService executor; + private final ScheduledExecutorService executor; private enum State { Ready, Closed @@ -60,13 +60,13 @@ private enum State { private State state = State.Ready; - LockManagerImpl(MetadataStoreExtended store, Class clazz, ExecutorService executor) { + LockManagerImpl(MetadataStoreExtended store, Class clazz, ScheduledExecutorService executor) { this(store, new JSONMetadataSerdeSimpleType<>( TypeFactory.defaultInstance().constructSimpleType(clazz, null)), executor); } - LockManagerImpl(MetadataStoreExtended store, MetadataSerde serde, ExecutorService executor) { + LockManagerImpl(MetadataStoreExtended store, MetadataSerde serde, ScheduledExecutorService executor) { this.store = store; this.cache = store.getMetadataCache(serde); this.serde = serde; @@ -83,7 +83,7 @@ public CompletableFuture> readLock(String path) { @Override public CompletableFuture> acquireLock(String path, T value) { - ResourceLockImpl lock = new ResourceLockImpl<>(store, serde, path); + ResourceLockImpl lock = new ResourceLockImpl<>(store, serde, path, executor); CompletableFuture> result = new CompletableFuture<>(); lock.acquire(value).thenRun(() -> { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/ResourceLockImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/ResourceLockImpl.java index 93c994b2436b9..692f224594cae 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/ResourceLockImpl.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/ResourceLockImpl.java @@ -21,8 +21,13 @@ import java.util.EnumSet; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataSerde; @@ -44,7 +49,10 @@ public class ResourceLockImpl implements ResourceLock { private long version; private final CompletableFuture expiredFuture; private boolean revalidateAfterReconnection = false; + private final Backoff backoff; private final FutureUtil.Sequencer sequencer; + private final ScheduledExecutorService executor; + private ScheduledFuture revalidateTask; private enum State { Init, @@ -55,7 +63,8 @@ private enum State { private State state; - public ResourceLockImpl(MetadataStoreExtended store, MetadataSerde serde, String path) { + ResourceLockImpl(MetadataStoreExtended store, MetadataSerde serde, String path, + ScheduledExecutorService executor) { this.store = store; this.serde = serde; this.path = path; @@ -63,6 +72,11 @@ public ResourceLockImpl(MetadataStoreExtended store, MetadataSerde serde, Str this.expiredFuture = new CompletableFuture<>(); this.sequencer = FutureUtil.Sequencer.create(); this.state = State.Init; + this.executor = executor; + this.backoff = new BackoffBuilder() + .setInitialTime(100, TimeUnit.MILLISECONDS) + .setMax(60, TimeUnit.SECONDS) + .create(); } @Override @@ -93,6 +107,10 @@ public synchronized CompletableFuture release() { } state = State.Releasing; + if (revalidateTask != null) { + revalidateTask.cancel(true); + } + CompletableFuture result = new CompletableFuture<>(); store.delete(path, Optional.of(version)) @@ -210,8 +228,15 @@ synchronized CompletableFuture revalidateIfNeededAfterReconnection() { * This method is thread-safe and it will perform multiple re-validation operations in turn. */ synchronized CompletableFuture silentRevalidateOnce() { + if (state != State.Valid) { + return CompletableFuture.completedFuture(null); + } + return sequencer.sequential(() -> revalidate(value)) - .thenRun(() -> log.info("Successfully revalidated the lock on {}", path)) + .thenRun(() -> { + log.info("Successfully revalidated the lock on {}", path); + backoff.reset(); + }) .exceptionally(ex -> { synchronized (ResourceLockImpl.this) { Throwable realCause = FutureUtil.unwrapCompletionException(ex); @@ -225,8 +250,12 @@ synchronized CompletableFuture silentRevalidateOnce() { // Continue assuming we hold the lock, until we can revalidate it, either // on Reconnected or SessionReestablished events. revalidateAfterReconnection = true; - log.warn("Failed to revalidate the lock at {}. Retrying later on reconnection {}", path, - realCause.getMessage()); + + long delayMillis = backoff.next(); + log.warn("Failed to revalidate the lock at {}: {} - Retrying in {} seconds", path, + realCause.getMessage(), delayMillis / 1000.0); + revalidateTask = + executor.schedule(this::silentRevalidateOnce, delayMillis, TimeUnit.MILLISECONDS); } } return null; diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/LockManagerTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/LockManagerTest.java index 05e6d4a3845e2..ebd60bad5507d 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/LockManagerTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/LockManagerTest.java @@ -35,6 +35,7 @@ import java.util.function.Supplier; import lombok.Cleanup; import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.MetadataStoreException.LockBusyException; @@ -352,4 +353,34 @@ public void testCleanUpStateWhenRevalidationGotLockBusy(String provider, Supplie } }); } + + @Test(dataProvider = "impl") + public void lockDeletedAndReacquired(String provider, Supplier urlSupplier) throws Exception { + @Cleanup + MetadataStoreExtended store = MetadataStoreExtended.create(urlSupplier.get(), + MetadataStoreConfig.builder().fsyncEnable(false).build()); + + MetadataCache cache = store.getMetadataCache(String.class); + + @Cleanup + CoordinationService coordinationService = new CoordinationServiceImpl(store); + + @Cleanup + LockManager lockManager = coordinationService.getLockManager(String.class); + + String key = newKey(); + ResourceLock lock = lockManager.acquireLock(key, "lock").join(); + assertEquals(lock.getValue(), "lock"); + var res = cache.get(key).join(); + assertTrue(res.isPresent()); + assertEquals(res.get(), "lock"); + + store.delete(key, Optional.empty()).join(); + + Awaitility.await().untilAsserted(() -> { + Optional val = store.get(key).join(); + assertTrue(val.isPresent()); + assertFalse(lock.getLockExpiredFuture().isDone()); + }); + } } From 519d8e2c3a07b75e4a8c656287e412ca860aecf6 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 8 May 2024 03:56:29 +0800 Subject: [PATCH 157/580] [fix][broker] Add the missed opentelemetry-sdk-testing dependency to tests of pulsar-broker-auth-sasl (#22665) --- pulsar-broker-auth-sasl/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pulsar-broker-auth-sasl/pom.xml b/pulsar-broker-auth-sasl/pom.xml index 7582f694bca44..78404156dfce4 100644 --- a/pulsar-broker-auth-sasl/pom.xml +++ b/pulsar-broker-auth-sasl/pom.xml @@ -41,6 +41,12 @@ ${project.version} + + io.opentelemetry + opentelemetry-sdk-testing + test + + org.apache.kerby kerby-config From 5ff0fb9604e5e74eddaf35bb072541923c03f373 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Wed, 8 May 2024 10:53:53 +0800 Subject: [PATCH 158/580] [fix][broker] Disable system topic message deduplication (#22582) --- .../apache/pulsar/broker/service/Topic.java | 10 ++++++ .../persistent/MessageDeduplication.java | 6 +--- .../service/persistent/PersistentTopic.java | 9 +++--- .../service/persistent/SystemTopic.java | 16 ++++++++++ .../persistent/MessageDuplicationTest.java | 32 +++++++++++++++++++ 5 files changed, 64 insertions(+), 9 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java index 37696d7a7c53c..e902de8a45a10 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java @@ -213,6 +213,16 @@ CompletableFuture close( void checkCursorsToCacheEntries(); + /** + * Indicate if the current topic enabled server side deduplication. + * This is a dynamic configuration, user may update it by namespace/topic policies. + * + * @return whether enabled server side deduplication + */ + default boolean isDeduplicationEnabled() { + return false; + } + void checkDeduplicationSnapshot(); void checkMessageExpiry(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java index e508661364d74..ab3b799093be6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java @@ -217,7 +217,7 @@ public Status getStatus() { * returning a future to track the completion of the task */ public CompletableFuture checkStatus() { - boolean shouldBeEnabled = isDeduplicationEnabled(); + boolean shouldBeEnabled = topic.isDeduplicationEnabled(); synchronized (this) { if (status == Status.Recovering || status == Status.Removing) { // If there's already a transition happening, check later for status @@ -472,10 +472,6 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { }, null); } - private boolean isDeduplicationEnabled() { - return topic.getHierarchyTopicPolicies().getDeduplicationEnabled().get(); - } - /** * Topic will call this method whenever a producer connects. */ diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 58ea8088dba67..7228bdeb2d334 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -2191,10 +2191,6 @@ public void deleteCursorFailed(ManagedLedgerException exception, Object ctx) { return future; } - public boolean isDeduplicationEnabled() { - return messageDeduplication.isEnabled(); - } - @Override public int getNumberOfConsumers() { int count = 0; @@ -4298,6 +4294,10 @@ public boolean isMigrated() { return ledger.isMigrated(); } + public boolean isDeduplicationEnabled() { + return getHierarchyTopicPolicies().getDeduplicationEnabled().get(); + } + public TransactionInPendingAckStats getTransactionInPendingAckStats(TxnID txnID, String subName) { return this.subscriptions.get(subName).getTransactionInPendingAckStats(txnID); } @@ -4332,4 +4332,5 @@ protected boolean isExceedMaximumDeliveryDelay(ByteBuf headersAndPayload) { } return false; } + } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java index 720ae3c51891e..f2cec2138a3a0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/SystemTopic.java @@ -80,6 +80,22 @@ public boolean isCompactionEnabled() { return !NamespaceService.isHeartbeatNamespace(TopicName.get(topic)); } + @Override + public boolean isDeduplicationEnabled() { + /* + Disable deduplication on system topic to avoid recovering deduplication WAL + (especially from offloaded topic). + Because the system topic usually is a precondition of other topics. therefore, + we should pay attention on topic loading time. + + Note: If the system topic loading timeout may cause dependent topics to fail to run. + + Dependency diagram: normal topic --rely on--> system topic --rely on--> deduplication recover + --may rely on--> (tiered storage) + */ + return false; + } + @Override public boolean isEncryptionRequired() { // System topics are only written by the broker that can't know the encryption context. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java index 402b5c4972ce2..f034717ccf2e3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java @@ -37,6 +37,8 @@ import io.netty.channel.EventLoopGroup; import java.lang.reflect.Field; import java.util.Map; +import java.util.Optional; +import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -50,9 +52,11 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.broker.qos.AsyncTokenBucket; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; @@ -490,4 +494,32 @@ public void testMessageDeduplication() throws Exception { messageDeduplication.purgeInactiveProducers(); assertTrue(messageDeduplication.getInactiveProducers().isEmpty()); } + + + @Test + public void testMessageDeduplicationShouldNotWorkForSystemTopic() throws PulsarAdminException { + final String localName = UUID.randomUUID().toString(); + final String namespace = "prop/ns-abc"; + final String prefix = "persistent://%s/".formatted(namespace); + final String topic = prefix + localName; + admin.topics().createNonPartitionedTopic(topic); + + // broker level policies + final String eventSystemTopic = prefix + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME; + final Optional optionalTopic = pulsar.getBrokerService().getTopic(eventSystemTopic, true).join(); + assertTrue(optionalTopic.isPresent()); + final Topic ptRef = optionalTopic.get(); + assertTrue(ptRef.isSystemTopic()); + assertFalse(ptRef.isDeduplicationEnabled()); + + // namespace level policies + admin.namespaces().setDeduplicationStatus(namespace, true); + assertTrue(ptRef.isSystemTopic()); + assertFalse(ptRef.isDeduplicationEnabled()); + + // topic level policies + admin.topicPolicies().setDeduplicationStatus(eventSystemTopic, true); + assertTrue(ptRef.isSystemTopic()); + assertFalse(ptRef.isDeduplicationEnabled()); + } } From e2feec827fe8c0894dfa59b23064ec552a2abc5c Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 7 May 2024 20:03:47 -0700 Subject: [PATCH 159/580] [improve][broker] Disable JFR metric reporting in OpenTelemetry (#22669) Co-authored-by: Matteo Merli Co-authored-by: Lari Hotari Co-authored-by: Lari Hotari --- .../org/apache/pulsar/opentelemetry/OpenTelemetryService.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java index 4560d3813d6dd..eb09e64fe731f 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java @@ -102,7 +102,6 @@ public OpenTelemetryService(String clusterName, // For a list of exposed metrics, see https://opentelemetry.io/docs/specs/semconv/runtime/jvm-metrics/ runtimeMetricsReference.set(RuntimeMetrics.builder(openTelemetrySdkReference.get()) - .enableAllFeatures() .enableExperimentalJmxTelemetry() .build()); } From 80d46758e89b088688d521aa8ae401bfb00c98b2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 8 May 2024 06:56:35 +0300 Subject: [PATCH 160/580] [improve][ws] Add memory limit configuration for Pulsar client used in Websocket proxy (#22666) --- conf/broker.conf | 3 +++ conf/standalone.conf | 3 +++ conf/websocket.conf | 3 +++ .../org/apache/pulsar/broker/ServiceConfiguration.java | 7 +++++++ .../java/org/apache/pulsar/websocket/WebSocketService.java | 3 ++- .../websocket/service/WebSocketProxyConfiguration.java | 3 +++ 6 files changed, 21 insertions(+), 1 deletion(-) diff --git a/conf/broker.conf b/conf/broker.conf index d97e3a5ef89ad..1b51ff4755173 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1544,6 +1544,9 @@ webSocketNumServiceThreads= # Number of connections per Broker in Pulsar Client used in WebSocket proxy webSocketConnectionsPerBroker= +# Memory limit in MBs for direct memory in Pulsar Client used in WebSocket proxy +webSocketPulsarClientMemoryLimitInMB=0 + # Time in milliseconds that idle WebSocket session times out webSocketSessionIdleTimeoutMillis=300000 diff --git a/conf/standalone.conf b/conf/standalone.conf index b04e5ccefa640..51035235d4d30 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -967,6 +967,9 @@ webSocketNumIoThreads=8 # Number of connections per Broker in Pulsar Client used in WebSocket proxy webSocketConnectionsPerBroker=8 +# Memory limit in MBs for direct memory in Pulsar Client used in WebSocket proxy +webSocketPulsarClientMemoryLimitInMB=0 + # Time in milliseconds that idle WebSocket session times out webSocketSessionIdleTimeoutMillis=300000 diff --git a/conf/websocket.conf b/conf/websocket.conf index 9051f3b590c8e..91f7f7d4c23bb 100644 --- a/conf/websocket.conf +++ b/conf/websocket.conf @@ -71,6 +71,9 @@ numHttpServerThreads= # Number of connections per Broker in Pulsar Client used in WebSocket proxy webSocketConnectionsPerBroker= +# Memory limit in MBs for direct memory in Pulsar Client used in WebSocket proxy +webSocketPulsarClientMemoryLimitInMB=0 + # Time in milliseconds that idle WebSocket session times out webSocketSessionIdleTimeoutMillis=300000 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 156c83bd6960c..a9d170ea5de87 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 @@ -2912,6 +2912,13 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "Number of connections per Broker in Pulsar Client used in WebSocket proxy" ) private int webSocketConnectionsPerBroker = Runtime.getRuntime().availableProcessors(); + + @FieldContext( + category = CATEGORY_WEBSOCKET, + doc = "Memory limit in MBs for direct memory in Pulsar Client used in WebSocket proxy" + ) + private int webSocketPulsarClientMemoryLimitInMB = 0; + @FieldContext( category = CATEGORY_WEBSOCKET, doc = "Time in milliseconds that idle WebSocket session times out" diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java index 66b2a0075ec2d..889f4431cc35b 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java @@ -195,7 +195,8 @@ public synchronized void setLocalCluster(ClusterData clusterData) { private PulsarClient createClientInstance(ClusterData clusterData) throws IOException { ClientBuilder clientBuilder = PulsarClient.builder() // - .memoryLimit(0, SizeUnit.BYTES) + .memoryLimit(SizeUnit.MEGA_BYTES.toBytes(config.getWebSocketPulsarClientMemoryLimitInMB()), + SizeUnit.BYTES) .statsInterval(0, TimeUnit.SECONDS) // .enableTls(config.isTlsEnabled()) // .allowTlsInsecureConnection(config.isTlsAllowInsecureConnection()) // diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java index 3fcbcf4b21567..31a1adc291553 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java @@ -176,6 +176,9 @@ public class WebSocketProxyConfiguration implements PulsarConfiguration { @FieldContext(doc = "Number of connections per broker in Pulsar client used in WebSocket proxy") private int webSocketConnectionsPerBroker = Runtime.getRuntime().availableProcessors(); + @FieldContext(doc = "Memory limit in MBs for direct memory in Pulsar Client used in WebSocket proxy") + private int webSocketPulsarClientMemoryLimitInMB = 0; + @FieldContext(doc = "Timeout of idling WebSocket session (in milliseconds)") private int webSocketSessionIdleTimeoutMillis = 300000; From 3114199c185cb03a7fdb1b8af2bbc356162cf42d Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Wed, 8 May 2024 13:10:49 +0800 Subject: [PATCH 161/580] [fix][broker] avoid offload system topic (#22497) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: 道君 --- .../pulsar/broker/service/BrokerService.java | 8 +- .../broker/service/BrokerServiceTest.java | 94 +++++++++++++++++++ 2 files changed, 101 insertions(+), 1 deletion(-) 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 dff6c40054060..6e23deaa6fa50 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 @@ -1963,7 +1963,13 @@ private CompletableFuture getManagedLedgerConfig(@Nonnull T topicLevelOffloadPolicies, OffloadPoliciesImpl.oldPoliciesCompatible(nsLevelOffloadPolicies, policies.orElse(null)), getPulsar().getConfig().getProperties()); - if (NamespaceService.isSystemServiceNamespace(namespace.toString())) { + if (NamespaceService.isSystemServiceNamespace(namespace.toString()) + || SystemTopicNames.isSystemTopic(topicName)) { + /* + Avoid setting broker internal system topics using off-loader because some of them are the + preconditions of other topics. The slow replying log speed will cause a delay in all the topic + loading.(timeout) + */ managedLedgerConfig.setLedgerOffloader(NullLedgerOffloader.INSTANCE); } else { if (topicLevelOffloadPolicies != null) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 5fbe147638026..1818163cd340e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -67,11 +67,14 @@ import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.mledger.LedgerOffloader; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; import org.apache.http.HttpResponse; import org.apache.http.client.methods.HttpGet; import org.apache.http.impl.client.CloseableHttpClient; @@ -112,6 +115,9 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.LocalPolicies; +import org.apache.pulsar.common.policies.data.OffloadPolicies; +import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; +import org.apache.pulsar.common.policies.data.OffloadedReadPriority; import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.protocol.Commands; @@ -1745,4 +1751,92 @@ public void testUnsubscribeNonDurableSub() throws Exception { fail("Unsubscribe failed"); } } + + + @Test + public void testOffloadConfShouldNotAppliedForSystemTopic() throws PulsarAdminException { + final String driver = "aws-s3"; + final String region = "test-region"; + final String bucket = "test-bucket"; + final String role = "test-role"; + final String roleSessionName = "test-role-session-name"; + final String credentialId = "test-credential-id"; + final String credentialSecret = "test-credential-secret"; + final String endPoint = "test-endpoint"; + final Integer maxBlockSizeInBytes = 5; + final Integer readBufferSizeInBytes = 2; + final Long offloadThresholdInBytes = 10L; + final Long offloadThresholdInSeconds = 1000L; + final Long offloadDeletionLagInMillis = 5L; + + final OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.create( + driver, + region, + bucket, + endPoint, + role, + roleSessionName, + credentialId, + credentialSecret, + maxBlockSizeInBytes, + readBufferSizeInBytes, + offloadThresholdInBytes, + offloadThresholdInSeconds, + offloadDeletionLagInMillis, + OffloadedReadPriority.TIERED_STORAGE_FIRST + ); + + var fakeOffloader = new LedgerOffloader() { + @Override + public String getOffloadDriverName() { + return driver; + } + + @Override + public CompletableFuture offload(ReadHandle ledger, UUID uid, Map extraMetadata) { + return CompletableFuture.completedFuture(null); + } + + @Override + public CompletableFuture readOffloaded(long ledgerId, UUID uid, Map offloadDriverMetadata) { + return CompletableFuture.completedFuture(null); + } + + @Override + public CompletableFuture deleteOffloaded(long ledgerId, UUID uid, Map offloadDriverMetadata) { + return CompletableFuture.completedFuture(null); + } + + @Override + public OffloadPolicies getOffloadPolicies() { + return offloadPolicies; + } + + @Override + public void close() { + } + }; + + final BrokerService brokerService = pulsar.getBrokerService(); + final String namespace = "prop/" + UUID.randomUUID(); + admin.namespaces().createNamespace(namespace); + admin.namespaces().setOffloadPolicies(namespace, offloadPolicies); + + // Inject the cache to avoid real load off-loader jar + final Map ledgerOffloaderMap = pulsar.getLedgerOffloaderMap(); + ledgerOffloaderMap.put(NamespaceName.get(namespace), fakeOffloader); + + // (1) test normal topic + final String normalTopic = "persistent://" + namespace + "/" + UUID.randomUUID(); + var managedLedgerConfig = brokerService.getManagedLedgerConfig(TopicName.get(normalTopic)).join(); + + Assert.assertEquals(managedLedgerConfig.getLedgerOffloader(), fakeOffloader); + + // (2) test system topic + for (String eventTopicName : SystemTopicNames.EVENTS_TOPIC_NAMES) { + managedLedgerConfig = brokerService.getManagedLedgerConfig(TopicName.get(eventTopicName)).join(); + Assert.assertEquals(managedLedgerConfig.getLedgerOffloader(), NullLedgerOffloader.INSTANCE); + } + } } + From 188355b2df08cafd9402e75baf1164ba4b44a052 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 8 May 2024 13:36:08 +0800 Subject: [PATCH 162/580] [fix][admin] Fix deprecated check (#22653) Signed-off-by: Zixuan Liu --- .../org/apache/pulsar/admin/cli/CmdSinks.java | 14 ++++++++++---- .../org/apache/pulsar/admin/cli/CmdSources.java | 15 ++++++++++----- .../org/apache/pulsar/admin/cli/TestCmdSinks.java | 12 ++++++++++++ .../apache/pulsar/admin/cli/TestCmdSources.java | 13 +++++++++++++ 4 files changed, 45 insertions(+), 9 deletions(-) diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java index be1cd0af96085..a4fb047550dcb 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSinks.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.gson.Gson; import com.google.gson.GsonBuilder; import com.google.gson.reflect.TypeToken; @@ -197,8 +198,8 @@ private void mergeArgs() { } } - @Override - public void runCmd() throws Exception { + @VisibleForTesting + List getLocalRunArgs() throws Exception { // merge deprecated args with new args mergeArgs(); List localRunArgs = new LinkedList<>(); @@ -206,7 +207,7 @@ public void runCmd() throws Exception { localRunArgs.add("--sinkConfig"); localRunArgs.add(new Gson().toJson(sinkConfig)); for (Field field : this.getClass().getDeclaredFields()) { - if (field.getName().startsWith("DEPRECATED")) { + if (field.getName().toUpperCase().startsWith("DEPRECATED")) { continue; } if (field.getName().contains("$")) { @@ -218,7 +219,12 @@ public void runCmd() throws Exception { localRunArgs.add(value.toString()); } } - ProcessBuilder processBuilder = new ProcessBuilder(localRunArgs).inheritIO(); + return localRunArgs; + } + + @Override + public void runCmd() throws Exception { + ProcessBuilder processBuilder = new ProcessBuilder(getLocalRunArgs()).inheritIO(); Process process = processBuilder.start(); process.waitFor(); } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSources.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSources.java index e691d7c126778..c8af7ddd954b1 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSources.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSources.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.gson.Gson; import com.google.gson.GsonBuilder; import com.google.gson.reflect.TypeToken; @@ -198,17 +199,16 @@ private void mergeArgs() { } } - @Override - public void runCmd() throws Exception { + @VisibleForTesting + List getLocalRunArgs() throws Exception { // merge deprecated args with new args mergeArgs(); - List localRunArgs = new LinkedList<>(); localRunArgs.add(System.getenv("PULSAR_HOME") + "/bin/function-localrunner"); localRunArgs.add("--sourceConfig"); localRunArgs.add(new Gson().toJson(sourceConfig)); for (Field field : this.getClass().getDeclaredFields()) { - if (field.getName().startsWith("DEPRECATED")) { + if (field.getName().toUpperCase().startsWith("DEPRECATED")) { continue; } if (field.getName().contains("$")) { @@ -220,7 +220,12 @@ public void runCmd() throws Exception { localRunArgs.add(value.toString()); } } - ProcessBuilder processBuilder = new ProcessBuilder(localRunArgs).inheritIO(); + return localRunArgs; + } + + @Override + public void runCmd() throws Exception { + ProcessBuilder processBuilder = new ProcessBuilder(getLocalRunArgs()).inheritIO(); Process process = processBuilder.start(); process.waitFor(); } diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSinks.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSinks.java index 6fbe3bc5da26d..5885b60aef24a 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSinks.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSinks.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.testng.Assert.assertFalse; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.dataformat.yaml.YAMLMapper; import java.io.Closeable; @@ -37,6 +38,7 @@ import java.util.List; import java.util.Map; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.admin.cli.CmdSinks.LocalSinkRunner; import org.apache.pulsar.admin.cli.utils.CmdUtils; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.Sinks; @@ -808,4 +810,14 @@ public void testParseConfigs() throws Exception { Assert.assertEquals(config.get("float_string"), "1000.0"); Assert.assertEquals(config.get("created_at"), "Mon Jul 02 00:33:15 +0000 2018"); } + + @Test + public void testExcludeDeprecatedOptions() throws Exception { + SinkConfig testSinkConfig = getSinkConfig(); + LocalSinkRunner localSinkRunner = spy(new CmdSinks(() -> pulsarAdmin)).getLocalSinkRunner(); + localSinkRunner.sinkConfig = testSinkConfig; + localSinkRunner.deprecatedBrokerServiceUrl = "pulsar://localhost:6650"; + List localRunArgs = localSinkRunner.getLocalRunArgs(); + assertFalse(String.join(",", localRunArgs).contains("--deprecated")); + } } diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSources.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSources.java index d96b0933d3f84..576e63310c1fa 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSources.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSources.java @@ -26,6 +26,7 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.dataformat.yaml.YAMLMapper; @@ -33,8 +34,10 @@ import java.io.File; import java.io.IOException; import java.nio.file.Files; +import java.util.List; import java.util.Map; import java.util.UUID; +import org.apache.pulsar.admin.cli.CmdSources.LocalSourceRunner; import org.apache.pulsar.admin.cli.utils.CmdUtils; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.Sources; @@ -680,4 +683,14 @@ public void testParseConfigs() throws Exception { Assert.assertEquals(config.get("float_string"), "1000.0"); Assert.assertEquals(config.get("created_at"), "Mon Jul 02 00:33:15 +0000 2018"); } + + @Test + public void testExcludeDeprecatedOptions() throws Exception { + SourceConfig testSinkConfig = getSourceConfig(); + LocalSourceRunner localSourceRunner = spy(new CmdSources(() -> pulsarAdmin)).getLocalSourceRunner(); + localSourceRunner.sourceConfig = testSinkConfig; + localSourceRunner.deprecatedBrokerServiceUrl = "pulsar://localhost:6650"; + List localRunArgs = localSourceRunner.getLocalRunArgs(); + assertFalse(String.join(",", localRunArgs).contains("--deprecated")); + } } From ca44b9bc7c48eca59692744399872e1f14f4fe6f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 8 May 2024 13:43:24 +0300 Subject: [PATCH 163/580] =?UTF-8?q?Revert=20"[fix][sec]=20Upgrade=20Debezi?= =?UTF-8?q?um=20oracle=20connector=20version=20to=20avoid=E2=80=A6=20(#226?= =?UTF-8?q?68)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- pom.xml | 1 - pulsar-io/debezium/oracle/pom.xml | 3 +-- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index cec3b3c60db9e..c2f563eb60edc 100644 --- a/pom.xml +++ b/pom.xml @@ -199,7 +199,6 @@ flexible messaging model and an intuitive client API. 1.2.4 8.12.1 1.9.7.Final - 2.2.0.Final 42.5.0 8.0.30 diff --git a/pulsar-io/debezium/oracle/pom.xml b/pulsar-io/debezium/oracle/pom.xml index b22a5785dfbe6..c69640ecff72f 100644 --- a/pulsar-io/debezium/oracle/pom.xml +++ b/pulsar-io/debezium/oracle/pom.xml @@ -48,8 +48,7 @@ io.debezium debezium-connector-oracle - ${debezium.oracle.version} - runtime + ${debezium.version} From 5ab05129514c1e71a09ec3f28b2b2dda9ce3e47f Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Wed, 8 May 2024 19:34:00 +0800 Subject: [PATCH 164/580] [fix] [broker] rename to changeMaxReadPositionCount (#22656) --- .../buffer/impl/TopicTransactionBuffer.java | 16 ++++++++-------- .../broker/transaction/TransactionTest.java | 12 ++++++------ 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index a36216bd6258b..81c9ecfc728e9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -76,8 +76,8 @@ public class TopicTransactionBuffer extends TopicTransactionBufferState implemen */ private final LinkedMap ongoingTxns = new LinkedMap<>(); - // when add abort or change max read position, the count will +1. Take snapshot will set 0 into it. - private final AtomicLong changeMaxReadPositionAndAddAbortTimes = new AtomicLong(); + // when change max read position, the count will +1. Take snapshot will reset the count. + private final AtomicLong changeMaxReadPositionCount = new AtomicLong(); private final LongAdder txnCommittedCounter = new LongAdder(); @@ -429,15 +429,15 @@ private void handleLowWaterMark(TxnID txnID, long lowWaterMark) { } private void takeSnapshotByChangeTimes() { - if (changeMaxReadPositionAndAddAbortTimes.get() >= takeSnapshotIntervalNumber) { - this.changeMaxReadPositionAndAddAbortTimes.set(0); + if (changeMaxReadPositionCount.get() >= takeSnapshotIntervalNumber) { + this.changeMaxReadPositionCount.set(0); this.snapshotAbortedTxnProcessor.takeAbortedTxnsSnapshot(this.maxReadPosition); } } private void takeSnapshotByTimeout() { - if (changeMaxReadPositionAndAddAbortTimes.get() > 0) { - this.changeMaxReadPositionAndAddAbortTimes.set(0); + if (changeMaxReadPositionCount.get() > 0) { + this.changeMaxReadPositionCount.set(0); this.snapshotAbortedTxnProcessor.takeAbortedTxnsSnapshot(this.maxReadPosition); } this.timer.newTimeout(TopicTransactionBuffer.this, @@ -454,7 +454,7 @@ void updateMaxReadPosition(TxnID txnID) { maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(); } if (preMaxReadPosition.compareTo(this.maxReadPosition) != 0) { - this.changeMaxReadPositionAndAddAbortTimes.getAndIncrement(); + this.changeMaxReadPositionCount.getAndIncrement(); } } @@ -489,7 +489,7 @@ public void syncMaxReadPositionForNormalPublish(PositionImpl position) { } else if (checkIfReady()) { if (ongoingTxns.isEmpty()) { maxReadPosition = position; - changeMaxReadPositionAndAddAbortTimes.incrementAndGet(); + changeMaxReadPositionCount.incrementAndGet(); } } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index e8c15d193a22d..5e806bb9ceee2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -1095,10 +1095,10 @@ public void testCancelTxnTimeout() throws Exception{ } @Test - public void testNotChangeMaxReadPositionAndAddAbortTimesWhenCheckIfNoSnapshot() throws Exception { + public void testNotChangeMaxReadPositionCountWhenCheckIfNoSnapshot() throws Exception { PersistentTopic persistentTopic = (PersistentTopic) getPulsarServiceList().get(0) .getBrokerService() - .getTopic(NAMESPACE1 + "/changeMaxReadPositionAndAddAbortTimes" + UUID.randomUUID(), true) + .getTopic(NAMESPACE1 + "/changeMaxReadPositionCount" + UUID.randomUUID(), true) .get().get(); TransactionBuffer buffer = persistentTopic.getTransactionBuffer(); Field processorField = TopicTransactionBuffer.class.getDeclaredField("snapshotAbortedTxnProcessor"); @@ -1106,9 +1106,9 @@ public void testNotChangeMaxReadPositionAndAddAbortTimesWhenCheckIfNoSnapshot() AbortedTxnProcessor abortedTxnProcessor = (AbortedTxnProcessor) processorField.get(buffer); Field changeTimeField = TopicTransactionBuffer - .class.getDeclaredField("changeMaxReadPositionAndAddAbortTimes"); + .class.getDeclaredField("changeMaxReadPositionCount"); changeTimeField.setAccessible(true); - AtomicLong changeMaxReadPositionAndAddAbortTimes = (AtomicLong) changeTimeField.get(buffer); + AtomicLong changeMaxReadPositionCount = (AtomicLong) changeTimeField.get(buffer); Field field1 = TopicTransactionBufferState.class.getDeclaredField("state"); field1.setAccessible(true); @@ -1117,10 +1117,10 @@ public void testNotChangeMaxReadPositionAndAddAbortTimesWhenCheckIfNoSnapshot() TopicTransactionBufferState.State state = (TopicTransactionBufferState.State) field1.get(buffer); Assert.assertEquals(state, TopicTransactionBufferState.State.NoSnapshot); }); - Assert.assertEquals(changeMaxReadPositionAndAddAbortTimes.get(), 0L); + Assert.assertEquals(changeMaxReadPositionCount.get(), 0L); buffer.syncMaxReadPositionForNormalPublish(new PositionImpl(1, 1)); - Assert.assertEquals(changeMaxReadPositionAndAddAbortTimes.get(), 0L); + Assert.assertEquals(changeMaxReadPositionCount.get(), 0L); } From ada31a96db9aabbb071f65229be746e61f954696 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 8 May 2024 21:41:22 +0800 Subject: [PATCH 165/580] [fix] [broker] Fix nothing changed after removing dynamic configs (#22673) --- .../pulsar/broker/service/BrokerService.java | 137 ++++++++++++------ .../AdminApiDynamicConfigurationsTest.java | 68 +++++++++ 2 files changed, 160 insertions(+), 45 deletions(-) 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 6e23deaa6fa50..c1b2b9e1da974 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 @@ -229,8 +229,7 @@ public class BrokerService implements Closeable { private final OrderedExecutor topicOrderedExecutor; // offline topic backlog cache private final ConcurrentOpenHashMap offlineTopicStatCache; - private final ConcurrentOpenHashMap dynamicConfigurationMap = - prepareDynamicConfigurationMap(); + private final ConcurrentOpenHashMap dynamicConfigurationMap; private final ConcurrentOpenHashMap> configRegisteredListeners; private final ConcurrentLinkedQueue pendingTopicLoadingQueue; @@ -313,6 +312,7 @@ public class BrokerService implements Closeable { public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws Exception { this.pulsar = pulsar; + this.dynamicConfigurationMap = prepareDynamicConfigurationMap(); this.brokerPublishRateLimiter = new PublishRateLimiterImpl(pulsar.getMonotonicSnapshotClock()); this.preciseTopicPublishRateLimitingEnable = pulsar.getConfiguration().isPreciseTopicPublishRateLimiterEnable(); @@ -2496,40 +2496,71 @@ private void handleDynamicConfigurationUpdates() { if (dynamicConfigResources != null) { dynamicConfigResources.getDynamicConfigurationAsync() - .thenAccept(optMap -> { - if (!optMap.isPresent()) { - return; + .thenAccept(optMap -> { + // Case some dynamic configs have been removed. + dynamicConfigurationMap.forEach((configKey, fieldWrapper) -> { + boolean configRemoved = optMap.isEmpty() || !optMap.get().containsKey(configKey); + if (fieldWrapper.lastDynamicValue != null && configRemoved) { + configValueChanged(configKey, null); } - Map data = optMap.get(); - data.forEach((configKey, value) -> { - ConfigField configFieldWrapper = dynamicConfigurationMap.get(configKey); - if (configFieldWrapper == null) { - log.warn("{} does not exist in dynamicConfigurationMap, skip this config.", configKey); - return; - } - Field configField = configFieldWrapper.field; - Consumer listener = configRegisteredListeners.get(configKey); - try { - final Object existingValue; - final Object newValue; - if (configField != null) { - newValue = FieldParser.value(data.get(configKey), configField); - existingValue = configField.get(pulsar.getConfiguration()); - configField.set(pulsar.getConfiguration(), newValue); - } else { - newValue = value; - existingValue = configFieldWrapper.customValue; - configFieldWrapper.customValue = newValue == null ? null : String.valueOf(newValue); - } - log.info("Successfully updated configuration {}/{}", configKey, data.get(configKey)); - if (listener != null && !Objects.equals(existingValue, newValue)) { - listener.accept(newValue); - } - } catch (Exception e) { - log.error("Failed to update config {}", configKey, e); - } - }); }); + // Some configs have been changed. + if (!optMap.isPresent()) { + return; + } + Map data = optMap.get(); + data.forEach((configKey, value) -> { + configValueChanged(configKey, value); + }); + }); + } + } + + private void configValueChanged(String configKey, String newValueStr) { + ConfigField configFieldWrapper = dynamicConfigurationMap.get(configKey); + if (configFieldWrapper == null) { + log.warn("{} does not exist in dynamicConfigurationMap, skip this config.", configKey); + return; + } + Consumer listener = configRegisteredListeners.get(configKey); + try { + // Convert existingValue and newValue. + final Object existingValue; + final Object newValue; + if (configFieldWrapper.field != null) { + if (StringUtils.isBlank(newValueStr)) { + newValue = configFieldWrapper.defaultValue; + } else { + newValue = FieldParser.value(newValueStr, configFieldWrapper.field); + } + existingValue = configFieldWrapper.field.get(pulsar.getConfiguration()); + configFieldWrapper.field.set(pulsar.getConfiguration(), newValue); + } else { + // This case only occurs when it is a customized item. + // See: https://github.com/apache/pulsar/blob/master/pip/pip-300.md. + log.info("Skip update customized dynamic configuration {}/{} in memory, only trigger an event" + + " listeners.", configKey, newValueStr); + existingValue = configFieldWrapper.lastDynamicValue; + newValue = newValueStr == null ? configFieldWrapper.defaultValue : newValueStr; + } + // Record the latest dynamic config. + configFieldWrapper.lastDynamicValue = newValueStr; + + if (newValueStr == null) { + log.info("Successfully remove the dynamic configuration {}, and revert to the default value", + configKey); + } else { + log.info("Successfully updated configuration {}/{}", configKey, newValueStr); + } + + if (listener != null && !Objects.equals(existingValue, newValue)) { + // So far, all config items that related to configuration listeners, their default value is not null. + // And the customized config can be null before. + // So call "listener.accept(null)" is okay. + listener.accept(newValue); + } + } catch (Exception e) { + log.error("Failed to update config {}", configKey, e); } } @@ -2936,6 +2967,9 @@ private void updateManagedLedgerConfig() { * On notification, listener should first check if config value has been changed and after taking appropriate * action, listener should update config value with new value if it has been changed (so, next time listener can * compare values on configMap change). + * + * Note: The new value that the {@param listener} may accept could be a null value. + * * @param * * @param configKey @@ -3057,16 +3091,23 @@ public boolean validateDynamicConfiguration(String key, String value) { return true; } - private static ConcurrentOpenHashMap prepareDynamicConfigurationMap() { + private ConcurrentOpenHashMap prepareDynamicConfigurationMap() { ConcurrentOpenHashMap dynamicConfigurationMap = ConcurrentOpenHashMap.newBuilder().build(); - for (Field field : ServiceConfiguration.class.getDeclaredFields()) { - if (field != null && field.isAnnotationPresent(FieldContext.class)) { - field.setAccessible(true); - if (field.getAnnotation(FieldContext.class).dynamic()) { - dynamicConfigurationMap.put(field.getName(), new ConfigField(field)); + try { + for (Field field : ServiceConfiguration.class.getDeclaredFields()) { + if (field != null && field.isAnnotationPresent(FieldContext.class)) { + field.setAccessible(true); + if (field.getAnnotation(FieldContext.class).dynamic()) { + Object defaultValue = field.get(pulsar.getConfiguration()); + dynamicConfigurationMap.put(field.getName(), new ConfigField(field, defaultValue)); + } } } + } catch (IllegalArgumentException | IllegalAccessException ex) { + // This error never occurs. + log.error("Failed to initialize dynamic configuration map", ex); + throw new RuntimeException(ex); } return dynamicConfigurationMap; } @@ -3348,19 +3389,25 @@ private static class ConfigField { // field holds the pulsar dynamic configuration. final Field field; - // customValue holds the external dynamic configuration. - volatile String customValue; + // It is the dynamic config value if set. + // It is null if has does not set a dynamic config, even if the value of "pulsar.config" is present. + volatile String lastDynamicValue; + + // The default value of "pulsar.config", which is initialized when the broker is starting. + // After the dynamic config has been removed, revert the config to this default value. + final Object defaultValue; Predicate validator; - public ConfigField(Field field) { + public ConfigField(Field field, Object defaultValue) { super(); this.field = field; + this.defaultValue = defaultValue; } public static ConfigField newCustomConfigField(String customValue) { - ConfigField configField = new ConfigField(null); - configField.customValue = customValue; + ConfigField configField = new ConfigField(null, null); + configField.lastDynamicValue = customValue; return configField; } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiDynamicConfigurationsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiDynamicConfigurationsTest.java index 12f231a4d2ce3..aa7c2d720e353 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiDynamicConfigurationsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiDynamicConfigurationsTest.java @@ -18,15 +18,18 @@ */ package org.apache.pulsar.broker.admin; +import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; import static org.testng.Assert.assertThrows; import static org.testng.Assert.fail; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import javax.ws.rs.core.Response; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.admin.PulsarAdminException; import org.awaitility.Awaitility; @@ -107,4 +110,69 @@ public void testRegisterCustomDynamicConfiguration() throws PulsarAdminException allDynamicConfigurations = admin.brokers().getAllDynamicConfigurations(); assertThat(allDynamicConfigurations).doesNotContainKey(key); } + + @Test + public void testDeleteStringDynamicConfig() throws PulsarAdminException { + String syncEventTopic = BrokerTestUtil.newUniqueName(SYSTEM_NAMESPACE + "/tp"); + // The default value is null; + Awaitility.await().untilAsserted(() -> { + assertNull(pulsar.getConfig().getConfigurationMetadataSyncEventTopic()); + }); + // Set dynamic config. + admin.brokers().updateDynamicConfiguration("configurationMetadataSyncEventTopic", syncEventTopic); + Awaitility.await().untilAsserted(() -> { + assertEquals(pulsar.getConfig().getConfigurationMetadataSyncEventTopic(), syncEventTopic); + }); + // Remove dynamic config. + admin.brokers().deleteDynamicConfiguration("configurationMetadataSyncEventTopic"); + Awaitility.await().untilAsserted(() -> { + assertNull(pulsar.getConfig().getConfigurationMetadataSyncEventTopic()); + }); + } + + @Test + public void testDeleteIntDynamicConfig() throws PulsarAdminException { + // Record the default value; + int defaultValue = pulsar.getConfig().getMaxConcurrentTopicLoadRequest(); + // Set dynamic config. + int newValue = defaultValue + 1000; + admin.brokers().updateDynamicConfiguration("maxConcurrentTopicLoadRequest", newValue + ""); + Awaitility.await().untilAsserted(() -> { + assertEquals(pulsar.getConfig().getMaxConcurrentTopicLoadRequest(), newValue); + }); + // Verify: it has been reverted to the default value. + admin.brokers().deleteDynamicConfiguration("maxConcurrentTopicLoadRequest"); + Awaitility.await().untilAsserted(() -> { + assertEquals(pulsar.getConfig().getMaxConcurrentTopicLoadRequest(), defaultValue); + }); + } + + @Test + public void testDeleteCustomizedDynamicConfig() throws PulsarAdminException { + // Record the default value; + String customizedConfigName = "a123"; + pulsar.getBrokerService().registerCustomDynamicConfiguration(customizedConfigName, v -> true); + + AtomicReference currentValue = new AtomicReference<>(); + pulsar.getBrokerService().registerConfigurationListener(customizedConfigName, v -> { + currentValue.set(v); + }); + + // The default value is null; + Awaitility.await().untilAsserted(() -> { + assertNull(currentValue.get()); + }); + + // Set dynamic config. + admin.brokers().updateDynamicConfiguration(customizedConfigName, "xxx"); + Awaitility.await().untilAsserted(() -> { + assertEquals(currentValue.get(), "xxx"); + }); + + // Remove dynamic config. + admin.brokers().deleteDynamicConfiguration(customizedConfigName); + Awaitility.await().untilAsserted(() -> { + assertNull(currentValue.get()); + }); + } } From ad75e3f0921bb735766d5e699baea0fc39ac4d41 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 8 May 2024 13:54:16 -0700 Subject: [PATCH 166/580] [improve][broker] Remove unused method CompactionRecord.reset (#22670) --- .../org/apache/pulsar/compaction/CompactionRecord.java | 8 -------- .../org/apache/pulsar/compaction/CompactorMXBeanImpl.java | 4 ---- .../apache/pulsar/compaction/CompactorMXBeanImplTest.java | 5 ----- 3 files changed, 17 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactionRecord.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactionRecord.java index 1d2af6638c33a..cea005d51b82c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactionRecord.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactionRecord.java @@ -51,14 +51,6 @@ public class CompactionRecord { public final Rate writeRate = new Rate(); public final Rate readRate = new Rate(); - public void reset() { - compactionRemovedEventCount.reset(); - compactionSucceedCount.reset(); - compactionFailedCount.reset(); - compactionDurationTimeInMills.reset(); - writeLatencyStats.reset(); - } - public void addCompactionRemovedEvent() { lastCompactionRemovedEventCountOp.increment(); compactionRemovedEventCount.increment(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorMXBeanImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorMXBeanImpl.java index 64b91d17d2508..8a9d266b56e26 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorMXBeanImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorMXBeanImpl.java @@ -53,10 +53,6 @@ public Set getTopics() { return compactionRecordOps.keySet(); } - public void reset() { - compactionRecordOps.values().forEach(CompactionRecord::reset); - } - public void addCompactionReadOp(String topic, long readableBytes) { compactionRecordOps.computeIfAbsent(topic, k -> new CompactionRecord()).addCompactionReadOp(readableBytes); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorMXBeanImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorMXBeanImplTest.java index bbde59d7da8bd..73e7430bd2d08 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorMXBeanImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorMXBeanImplTest.java @@ -59,11 +59,6 @@ public void testSimple() throws Exception { assertTrue(compaction.getCompactionWriteThroughput() > 0L); mxBean.addCompactionLatencyOp(topic, 10, TimeUnit.NANOSECONDS); assertTrue(compaction.getCompactionLatencyBuckets()[0] > 0L); - mxBean.reset(); - assertEquals(compaction.getCompactionRemovedEventCount(), 0, 0); - assertEquals(compaction.getCompactionSucceedCount(), 0, 0); - assertEquals(compaction.getCompactionFailedCount(), 0, 0); - assertEquals(compaction.getCompactionDurationTimeInMills(), 0, 0); } } From 88feb874bb3ad58a74b3d40d931b2aa7380dc7e1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Thu, 9 May 2024 08:53:59 +0800 Subject: [PATCH 167/580] [fix][ml] Remove duplicated field initialization of ML (#22676) --- .../org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index e5e163127f7b6..b12346cadc96a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -365,9 +365,6 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper this.mlOwnershipChecker = mlOwnershipChecker; this.propertiesMap = new ConcurrentHashMap<>(); this.inactiveLedgerRollOverTimeMs = config.getInactiveLedgerRollOverTimeMs(); - if (config.getManagedLedgerInterceptor() != null) { - this.managedLedgerInterceptor = config.getManagedLedgerInterceptor(); - } this.minBacklogCursorsForCaching = config.getMinimumBacklogCursorsForCaching(); this.minBacklogEntriesForCaching = config.getMinimumBacklogEntriesForCaching(); this.maxBacklogBetweenCursorsForCaching = config.getMaxBacklogBetweenCursorsForCaching(); From 8f015d89e5d246325ae5cada02c4af3017a97ed9 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Thu, 9 May 2024 09:42:17 +0800 Subject: [PATCH 168/580] [fix][broker] usedLocallySinceLastReport should always be reset (#22672) Signed-off-by: Zixuan Liu --- .../broker/resourcegroup/ResourceGroup.java | 3 +- .../ResourceGroupReportLocalUsageTest.java | 50 ++++++++++++------- 2 files changed, 34 insertions(+), 19 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java index f8ec52bfe3c5a..541a645f18bf3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java @@ -458,14 +458,13 @@ protected boolean setUsageInMonitoredEntity(ResourceGroupMonitoringClass monClas bytesUsed = monEntity.usedLocallySinceLastReport.bytes; messagesUsed = monEntity.usedLocallySinceLastReport.messages; - + monEntity.usedLocallySinceLastReport.bytes = monEntity.usedLocallySinceLastReport.messages = 0; if (sendReport) { p.setBytesPerPeriod(bytesUsed); p.setMessagesPerPeriod(messagesUsed); monEntity.lastReportedValues.bytes = bytesUsed; monEntity.lastReportedValues.messages = messagesUsed; monEntity.numSuppressedUsageReports = 0; - monEntity.usedLocallySinceLastReport.bytes = monEntity.usedLocallySinceLastReport.messages = 0; monEntity.totalUsedLocally.bytes += bytesUsed; monEntity.totalUsedLocally.messages += messagesUsed; monEntity.lastResourceUsageFillTimeMSecsSinceEpoch = System.currentTimeMillis(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupReportLocalUsageTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupReportLocalUsageTest.java index 658b7c94165d9..139d19886c7d1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupReportLocalUsageTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupReportLocalUsageTest.java @@ -72,34 +72,50 @@ public long computeLocalQuota(long confUsage, long myUsage, long[] allUsages) { rgConfig.setPublishRateInMsgs(2000); service.resourceGroupCreate(rgName, rgConfig); - org.apache.pulsar.broker.resourcegroup.ResourceGroup resourceGroup = service.resourceGroupGet(rgName); BytesAndMessagesCount bytesAndMessagesCount = new BytesAndMessagesCount(); bytesAndMessagesCount.bytes = 20; bytesAndMessagesCount.messages = 10; - resourceGroup.incrementLocalUsageStats(ResourceGroupMonitoringClass.Publish, bytesAndMessagesCount); + + org.apache.pulsar.broker.resourcegroup.ResourceGroup resourceGroup = service.resourceGroupGet(rgName); + for (ResourceGroupMonitoringClass value : ResourceGroupMonitoringClass.values()) { + resourceGroup.incrementLocalUsageStats(value, bytesAndMessagesCount); + } + + // Case1: Suppress report ResourceUsage. + needReport.set(false); ResourceUsage resourceUsage = new ResourceUsage(); resourceGroup.rgFillResourceUsage(resourceUsage); assertFalse(resourceUsage.hasDispatch()); assertFalse(resourceUsage.hasPublish()); + for (ResourceGroupMonitoringClass value : ResourceGroupMonitoringClass.values()) { + PerMonitoringClassFields monitoredEntity = + resourceGroup.getMonitoredEntity(value); + assertEquals(monitoredEntity.usedLocallySinceLastReport.messages, 0); + assertEquals(monitoredEntity.usedLocallySinceLastReport.bytes, 0); + assertEquals(monitoredEntity.totalUsedLocally.messages, 0); + assertEquals(monitoredEntity.totalUsedLocally.bytes, 0); + assertEquals(monitoredEntity.lastReportedValues.messages, 0); + assertEquals(monitoredEntity.lastReportedValues.bytes, 0); + } - PerMonitoringClassFields publishMonitoredEntity = - resourceGroup.getMonitoredEntity(ResourceGroupMonitoringClass.Publish); - assertEquals(publishMonitoredEntity.usedLocallySinceLastReport.messages, bytesAndMessagesCount.messages); - assertEquals(publishMonitoredEntity.usedLocallySinceLastReport.bytes, bytesAndMessagesCount.bytes); - assertEquals(publishMonitoredEntity.totalUsedLocally.messages, 0); - assertEquals(publishMonitoredEntity.totalUsedLocally.bytes, 0); - assertEquals(publishMonitoredEntity.lastReportedValues.messages, 0); - assertEquals(publishMonitoredEntity.lastReportedValues.bytes, 0); - + // Case2: Report ResourceUsage. + for (ResourceGroupMonitoringClass value : ResourceGroupMonitoringClass.values()) { + resourceGroup.incrementLocalUsageStats(value, bytesAndMessagesCount); + } needReport.set(true); + resourceUsage = new ResourceUsage(); resourceGroup.rgFillResourceUsage(resourceUsage); assertTrue(resourceUsage.hasDispatch()); assertTrue(resourceUsage.hasPublish()); - assertEquals(publishMonitoredEntity.usedLocallySinceLastReport.messages, 0); - assertEquals(publishMonitoredEntity.usedLocallySinceLastReport.bytes, 0); - assertEquals(publishMonitoredEntity.totalUsedLocally.messages, bytesAndMessagesCount.messages); - assertEquals(publishMonitoredEntity.totalUsedLocally.bytes, bytesAndMessagesCount.bytes); - assertEquals(publishMonitoredEntity.lastReportedValues.messages, bytesAndMessagesCount.messages); - assertEquals(publishMonitoredEntity.lastReportedValues.bytes, bytesAndMessagesCount.bytes); + for (ResourceGroupMonitoringClass value : ResourceGroupMonitoringClass.values()) { + PerMonitoringClassFields monitoredEntity = + resourceGroup.getMonitoredEntity(value); + assertEquals(monitoredEntity.usedLocallySinceLastReport.messages, 0); + assertEquals(monitoredEntity.usedLocallySinceLastReport.bytes, 0); + assertEquals(monitoredEntity.totalUsedLocally.messages, bytesAndMessagesCount.messages); + assertEquals(monitoredEntity.totalUsedLocally.bytes, bytesAndMessagesCount.bytes); + assertEquals(monitoredEntity.lastReportedValues.messages, bytesAndMessagesCount.messages); + assertEquals(monitoredEntity.lastReportedValues.bytes, bytesAndMessagesCount.bytes); + } } } \ No newline at end of file From bd4c57d27c4acd37206a4f5ffdad3705cdc96c8c Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Thu, 9 May 2024 13:53:02 +0800 Subject: [PATCH 169/580] [fix][broker] Fix geo-replication admin client url (#22584) --- .../pulsar/broker/service/BrokerService.java | 8 +++----- .../pulsar/broker/service/ReplicatorTestBase.java | 14 ++++++++++++-- 2 files changed, 15 insertions(+), 7 deletions(-) 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 c1b2b9e1da974..b61bc58e3b592 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 @@ -1468,13 +1468,11 @@ public PulsarAdmin getClusterPulsarAdmin(String cluster, Optional c } boolean isTlsEnabled = data.isBrokerClientTlsEnabled() || conf.isBrokerClientTlsEnabled(); - if (isTlsEnabled && StringUtils.isEmpty(data.getServiceUrlTls())) { - throw new IllegalArgumentException("serviceUrlTls is empty, brokerClientTlsEnabled: " + final String adminApiUrl = isTlsEnabled ? data.getServiceUrlTls() : data.getServiceUrl(); + if (StringUtils.isEmpty(adminApiUrl)) { + throw new IllegalArgumentException("The adminApiUrl is empty, brokerClientTlsEnabled: " + isTlsEnabled); - } else if (StringUtils.isEmpty(data.getServiceUrl())) { - throw new IllegalArgumentException("serviceUrl is empty, brokerClientTlsEnabled: " + isTlsEnabled); } - String adminApiUrl = isTlsEnabled ? data.getServiceUrlTls() : data.getServiceUrl(); builder.serviceHttpUrl(adminApiUrl); if (data.isBrokerClientTlsEnabled()) { configAdminTlsSettings(builder, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java index d87f896e31a1c..838632febd889 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java @@ -20,6 +20,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; import com.google.common.io.Resources; import com.google.common.collect.Sets; @@ -259,9 +260,7 @@ protected void setup() throws Exception { .brokerClientTlsTrustStoreType(keyStoreType) .build()); admin4.clusters().createCluster(cluster4, ClusterData.builder() - .serviceUrl(url4.toString()) .serviceUrlTls(urlTls4.toString()) - .brokerServiceUrl(pulsar4.getBrokerServiceUrl()) .brokerServiceUrlTls(pulsar4.getBrokerServiceUrlTls()) .brokerClientTlsEnabled(true) .brokerClientCertificateFilePath(clientCertFilePath) @@ -285,9 +284,20 @@ protected void setup() throws Exception { assertEquals(admin2.clusters().getCluster(cluster1).getServiceUrl(), url1.toString()); assertEquals(admin2.clusters().getCluster(cluster2).getServiceUrl(), url2.toString()); assertEquals(admin2.clusters().getCluster(cluster3).getServiceUrl(), url3.toString()); + assertNull(admin2.clusters().getCluster(cluster4).getServiceUrl()); assertEquals(admin2.clusters().getCluster(cluster1).getBrokerServiceUrl(), pulsar1.getBrokerServiceUrl()); assertEquals(admin2.clusters().getCluster(cluster2).getBrokerServiceUrl(), pulsar2.getBrokerServiceUrl()); assertEquals(admin2.clusters().getCluster(cluster3).getBrokerServiceUrl(), pulsar3.getBrokerServiceUrl()); + assertNull(admin2.clusters().getCluster(cluster4).getBrokerServiceUrl()); + + assertEquals(admin2.clusters().getCluster(cluster1).getServiceUrlTls(), urlTls1.toString()); + assertEquals(admin2.clusters().getCluster(cluster2).getServiceUrlTls(), urlTls2.toString()); + assertEquals(admin2.clusters().getCluster(cluster3).getServiceUrlTls(), urlTls3.toString()); + assertEquals(admin2.clusters().getCluster(cluster4).getServiceUrlTls(), urlTls4.toString()); + assertEquals(admin2.clusters().getCluster(cluster1).getBrokerServiceUrlTls(), pulsar1.getBrokerServiceUrlTls()); + assertEquals(admin2.clusters().getCluster(cluster2).getBrokerServiceUrlTls(), pulsar2.getBrokerServiceUrlTls()); + assertEquals(admin2.clusters().getCluster(cluster3).getBrokerServiceUrlTls(), pulsar3.getBrokerServiceUrlTls()); + assertEquals(admin2.clusters().getCluster(cluster4).getBrokerServiceUrlTls(), pulsar4.getBrokerServiceUrlTls()); // Also create V1 namespace for compatibility check admin1.clusters().createCluster("global", ClusterData.builder() From 566330ca8d0b3419853e0252276ef42c643d3465 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 9 May 2024 10:25:13 +0300 Subject: [PATCH 170/580] [fix][offload] Fix OOM in tiered storage, caused by unbounded offsets cache (#22679) Co-authored-by: Jiwe Guo --- .../mledger/LedgerOffloaderFactory.java | 7 +- .../mledger/offload/Offloaders.java | 6 ++ .../jcloud/JCloudLedgerOffloaderFactory.java | 16 ++-- .../impl/BlobStoreBackedReadHandleImpl.java | 24 ++---- .../impl/BlobStoreManagedLedgerOffloader.java | 14 ++- .../offload/jcloud/impl/OffsetsCache.java | 85 +++++++++++++++++++ .../BlobStoreManagedLedgerOffloaderBase.java | 9 ++ ...reManagedLedgerOffloaderStreamingTest.java | 4 +- .../BlobStoreManagedLedgerOffloaderTest.java | 6 +- .../offload/jcloud/impl/OffsetsCacheTest.java | 45 ++++++++++ 10 files changed, 185 insertions(+), 31 deletions(-) create mode 100644 tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffsetsCache.java create mode 100644 tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffsetsCacheTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloaderFactory.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloaderFactory.java index 7ecb8f08d573d..9fbf9b73c057e 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloaderFactory.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloaderFactory.java @@ -31,7 +31,7 @@ */ @LimitedPrivate @Evolving -public interface LedgerOffloaderFactory { +public interface LedgerOffloaderFactory extends AutoCloseable { /** * Check whether the provided driver driverName is supported. @@ -111,4 +111,9 @@ default T create(OffloadPoliciesImpl offloadPolicies, throws IOException { return create(offloadPolicies, userMetadata, scheduler, offloaderStats); } + + @Override + default void close() throws Exception { + // no-op + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/Offloaders.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/Offloaders.java index 6910439e09131..cec15599242ae 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/Offloaders.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/Offloaders.java @@ -46,6 +46,12 @@ public LedgerOffloaderFactory getOffloaderFactory(String driverName) throws IOEx @Override public void close() throws Exception { offloaders.forEach(offloader -> { + try { + offloader.getRight().close(); + } catch (Exception e) { + log.warn("Failed to close offloader '{}': {}", + offloader.getRight().getClass(), e.getMessage()); + } try { offloader.getLeft().close(); } catch (IOException e) { diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/JCloudLedgerOffloaderFactory.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/JCloudLedgerOffloaderFactory.java index 2c9165674444d..60363cf8406db 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/JCloudLedgerOffloaderFactory.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/JCloudLedgerOffloaderFactory.java @@ -25,6 +25,7 @@ import org.apache.bookkeeper.mledger.LedgerOffloaderStats; import org.apache.bookkeeper.mledger.LedgerOffloaderStatsDisable; import org.apache.bookkeeper.mledger.offload.jcloud.impl.BlobStoreManagedLedgerOffloader; +import org.apache.bookkeeper.mledger.offload.jcloud.impl.OffsetsCache; import org.apache.bookkeeper.mledger.offload.jcloud.provider.JCloudBlobStoreProvider; import org.apache.bookkeeper.mledger.offload.jcloud.provider.TieredStorageConfiguration; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; @@ -33,12 +34,7 @@ * A jcloud based offloader factory. */ public class JCloudLedgerOffloaderFactory implements LedgerOffloaderFactory { - - public static JCloudLedgerOffloaderFactory of() { - return INSTANCE; - } - - private static final JCloudLedgerOffloaderFactory INSTANCE = new JCloudLedgerOffloaderFactory(); + private final OffsetsCache entryOffsetsCache = new OffsetsCache(); @Override public boolean isDriverSupported(String driverName) { @@ -58,6 +54,12 @@ public BlobStoreManagedLedgerOffloader create(OffloadPoliciesImpl offloadPolicie TieredStorageConfiguration config = TieredStorageConfiguration.create(offloadPolicies.toProperties()); - return BlobStoreManagedLedgerOffloader.create(config, userMetadata, scheduler, offloaderStats); + return BlobStoreManagedLedgerOffloader.create(config, userMetadata, scheduler, offloaderStats, + entryOffsetsCache); + } + + @Override + public void close() throws Exception { + entryOffsetsCache.close(); } } diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java index 4f68f90370e6f..e050d74a332bc 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java @@ -19,8 +19,6 @@ package org.apache.bookkeeper.mledger.offload.jcloud.impl; import com.google.common.annotations.VisibleForTesting; -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; import io.netty.buffer.ByteBuf; import java.io.DataInputStream; import java.io.IOException; @@ -56,19 +54,13 @@ public class BlobStoreBackedReadHandleImpl implements ReadHandle { private static final Logger log = LoggerFactory.getLogger(BlobStoreBackedReadHandleImpl.class); - private static final int CACHE_TTL_SECONDS = - Integer.getInteger("pulsar.jclouds.readhandleimpl.offsetsscache.ttl.seconds", 30 * 60); private final long ledgerId; private final OffloadIndexBlock index; private final BackedInputStream inputStream; private final DataInputStream dataStream; private final ExecutorService executor; - // this Cache is accessed only by one thread - private final Cache entryOffsets = CacheBuilder - .newBuilder() - .expireAfterAccess(CACHE_TTL_SECONDS, TimeUnit.SECONDS) - .build(); + private final OffsetsCache entryOffsetsCache; private final AtomicReference> closeFuture = new AtomicReference<>(); enum State { @@ -79,12 +71,14 @@ enum State { private volatile State state = null; private BlobStoreBackedReadHandleImpl(long ledgerId, OffloadIndexBlock index, - BackedInputStream inputStream, ExecutorService executor) { + BackedInputStream inputStream, ExecutorService executor, + OffsetsCache entryOffsetsCache) { this.ledgerId = ledgerId; this.index = index; this.inputStream = inputStream; this.dataStream = new DataInputStream(inputStream); this.executor = executor; + this.entryOffsetsCache = entryOffsetsCache; state = State.Opened; } @@ -109,7 +103,6 @@ public CompletableFuture closeAsync() { try { index.close(); inputStream.close(); - entryOffsets.invalidateAll(); state = State.Closed; promise.complete(null); } catch (IOException t) { @@ -164,7 +157,7 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr long entryId = dataStream.readLong(); if (entryId == nextExpectedId) { - entryOffsets.put(entryId, currentPosition); + entryOffsetsCache.put(ledgerId, entryId, currentPosition); ByteBuf buf = PulsarByteBufAllocator.DEFAULT.buffer(length, length); entries.add(LedgerEntryImpl.create(ledgerId, entryId, length, buf)); int toWrite = length; @@ -215,7 +208,7 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr } private void seekToEntry(long nextExpectedId) throws IOException { - Long knownOffset = entryOffsets.getIfPresent(nextExpectedId); + Long knownOffset = entryOffsetsCache.getIfPresent(ledgerId, nextExpectedId); if (knownOffset != null) { inputStream.seek(knownOffset); } else { @@ -269,7 +262,8 @@ public static ReadHandle open(ScheduledExecutorService executor, BlobStore blobStore, String bucket, String key, String indexKey, VersionCheck versionCheck, long ledgerId, int readBufferSize, - LedgerOffloaderStats offloaderStats, String managedLedgerName) + LedgerOffloaderStats offloaderStats, String managedLedgerName, + OffsetsCache entryOffsetsCache) throws IOException, BKException.BKNoSuchLedgerExistsException { int retryCount = 3; OffloadIndexBlock index = null; @@ -310,7 +304,7 @@ public static ReadHandle open(ScheduledExecutorService executor, BackedInputStream inputStream = new BlobStoreBackedInputStreamImpl(blobStore, bucket, key, versionCheck, index.getDataObjectLength(), readBufferSize, offloaderStats, managedLedgerName); - return new BlobStoreBackedReadHandleImpl(ledgerId, index, inputStream, executor); + return new BlobStoreBackedReadHandleImpl(ledgerId, index, inputStream, executor, entryOffsetsCache); } // for testing diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java index 1b6062ffa0358..9f89bd52a8626 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java @@ -108,6 +108,7 @@ public class BlobStoreManagedLedgerOffloader implements LedgerOffloader { private AtomicLong bufferLength = new AtomicLong(0); private AtomicLong segmentLength = new AtomicLong(0); private final long maxBufferLength; + private final OffsetsCache entryOffsetsCache; private final ConcurrentLinkedQueue offloadBuffer = new ConcurrentLinkedQueue<>(); private CompletableFuture offloadResult; private volatile PositionImpl lastOfferedPosition = PositionImpl.LATEST; @@ -123,13 +124,16 @@ public class BlobStoreManagedLedgerOffloader implements LedgerOffloader { public static BlobStoreManagedLedgerOffloader create(TieredStorageConfiguration config, Map userMetadata, OrderedScheduler scheduler, - LedgerOffloaderStats offloaderStats) throws IOException { + LedgerOffloaderStats offloaderStats, + OffsetsCache entryOffsetsCache) + throws IOException { - return new BlobStoreManagedLedgerOffloader(config, scheduler, userMetadata, offloaderStats); + return new BlobStoreManagedLedgerOffloader(config, scheduler, userMetadata, offloaderStats, entryOffsetsCache); } BlobStoreManagedLedgerOffloader(TieredStorageConfiguration config, OrderedScheduler scheduler, - Map userMetadata, LedgerOffloaderStats offloaderStats) { + Map userMetadata, LedgerOffloaderStats offloaderStats, + OffsetsCache entryOffsetsCache) { this.scheduler = scheduler; this.userMetadata = userMetadata; @@ -140,6 +144,7 @@ public static BlobStoreManagedLedgerOffloader create(TieredStorageConfiguration this.minSegmentCloseTimeMillis = Duration.ofSeconds(config.getMinSegmentTimeInSecond()).toMillis(); //ensure buffer can have enough content to fill a block this.maxBufferLength = Math.max(config.getWriteBufferSizeInBytes(), config.getMinBlockSizeInBytes()); + this.entryOffsetsCache = entryOffsetsCache; this.segmentBeginTimeMillis = System.currentTimeMillis(); if (!Strings.isNullOrEmpty(config.getRegion())) { this.writeLocation = new LocationBuilder() @@ -555,7 +560,8 @@ public CompletableFuture readOffloaded(long ledgerId, UUID uid, readBucket, key, indexKey, DataBlockUtils.VERSION_CHECK, ledgerId, config.getReadBufferSizeInBytes(), - this.offloaderStats, offloadDriverMetadata.get(MANAGED_LEDGER_NAME))); + this.offloaderStats, offloadDriverMetadata.get(MANAGED_LEDGER_NAME), + this.entryOffsetsCache)); } catch (Throwable t) { log.error("Failed readOffloaded: ", t); promise.completeExceptionally(t); diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffsetsCache.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffsetsCache.java new file mode 100644 index 0000000000000..fa13afa8ff0e7 --- /dev/null +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffsetsCache.java @@ -0,0 +1,85 @@ +/* + * 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.bookkeeper.mledger.offload.jcloud.impl; + +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import io.grpc.netty.shaded.io.netty.util.concurrent.DefaultThreadFactory; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class OffsetsCache implements AutoCloseable { + private static final int CACHE_TTL_SECONDS = + Integer.getInteger("pulsar.jclouds.readhandleimpl.offsetsscache.ttl.seconds", 5 * 60); + // limit the cache size to avoid OOM + // 1 million entries consumes about 60MB of heap space + private static final int CACHE_MAX_SIZE = + Integer.getInteger("pulsar.jclouds.readhandleimpl.offsetsscache.max.size", 1_000_000); + private final ScheduledExecutorService cacheEvictionExecutor; + + record Key(long ledgerId, long entryId) { + + } + + private final Cache entryOffsetsCache; + + public OffsetsCache() { + if (CACHE_MAX_SIZE > 0) { + entryOffsetsCache = CacheBuilder + .newBuilder() + .expireAfterAccess(CACHE_TTL_SECONDS, TimeUnit.SECONDS) + .maximumSize(CACHE_MAX_SIZE) + .build(); + cacheEvictionExecutor = + Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("jcloud-offsets-cache-eviction")); + int period = Math.max(CACHE_TTL_SECONDS / 2, 1); + cacheEvictionExecutor.scheduleAtFixedRate(() -> { + entryOffsetsCache.cleanUp(); + }, period, period, TimeUnit.SECONDS); + } else { + cacheEvictionExecutor = null; + entryOffsetsCache = null; + } + } + + public void put(long ledgerId, long entryId, long currentPosition) { + if (entryOffsetsCache != null) { + entryOffsetsCache.put(new Key(ledgerId, entryId), currentPosition); + } + } + + public Long getIfPresent(long ledgerId, long entryId) { + return entryOffsetsCache != null ? entryOffsetsCache.getIfPresent(new Key(ledgerId, entryId)) : null; + } + + public void clear() { + if (entryOffsetsCache != null) { + entryOffsetsCache.invalidateAll(); + } + } + + @Override + public void close() { + if (cacheEvictionExecutor != null) { + cacheEvictionExecutor.shutdownNow(); + } + } +} diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderBase.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderBase.java index 89d9021d36d7d..75faf098b409b 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderBase.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderBase.java @@ -33,6 +33,7 @@ import org.jclouds.blobstore.BlobStore; import org.jclouds.domain.Credentials; import org.testng.Assert; +import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; public abstract class BlobStoreManagedLedgerOffloaderBase { @@ -46,6 +47,7 @@ public abstract class BlobStoreManagedLedgerOffloaderBase { protected final JCloudBlobStoreProvider provider; protected TieredStorageConfiguration config; protected BlobStore blobStore = null; + protected final OffsetsCache entryOffsetsCache = new OffsetsCache(); protected BlobStoreManagedLedgerOffloaderBase() throws Exception { scheduler = OrderedScheduler.newSchedulerBuilder().numThreads(5).name("offloader").build(); @@ -56,6 +58,13 @@ protected BlobStoreManagedLedgerOffloaderBase() throws Exception { @AfterMethod(alwaysRun = true) public void cleanupMockBookKeeper() { bk.getLedgerMap().clear(); + entryOffsetsCache.clear(); + } + + @AfterClass(alwaysRun = true) + public void cleanup() throws Exception { + entryOffsetsCache.close(); + scheduler.shutdownNow(); } protected static MockManagedLedger createMockManagedLedger() { diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java index ad1529072f813..e706e4254cb11 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java @@ -82,7 +82,7 @@ private BlobStoreManagedLedgerOffloader getOffloader(String bucket, Map(), scheduler, this.offloaderStats); + .create(mockedConfig, new HashMap(), scheduler, this.offloaderStats, entryOffsetsCache); return offloader; } @@ -91,7 +91,7 @@ private BlobStoreManagedLedgerOffloader getOffloader(String bucket, BlobStore mo mockedConfig = mock(TieredStorageConfiguration.class, delegatesTo(getConfiguration(bucket, additionalConfig))); Mockito.doReturn(mockedBlobStore).when(mockedConfig).getBlobStore(); BlobStoreManagedLedgerOffloader offloader = BlobStoreManagedLedgerOffloader - .create(mockedConfig, new HashMap(), scheduler, this.offloaderStats); + .create(mockedConfig, new HashMap(), scheduler, this.offloaderStats, entryOffsetsCache); return offloader; } diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java index 4419210c251f1..bf6ede896ab28 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java @@ -98,14 +98,16 @@ private BlobStoreManagedLedgerOffloader getOffloader(BlobStore mockedBlobStore) private BlobStoreManagedLedgerOffloader getOffloader(String bucket) throws IOException { mockedConfig = mock(TieredStorageConfiguration.class, delegatesTo(getConfiguration(bucket))); Mockito.doReturn(blobStore).when(mockedConfig).getBlobStore(); // Use the REAL blobStore - BlobStoreManagedLedgerOffloader offloader = BlobStoreManagedLedgerOffloader.create(mockedConfig, new HashMap(), scheduler, this.offloaderStats); + BlobStoreManagedLedgerOffloader offloader = BlobStoreManagedLedgerOffloader.create(mockedConfig, new HashMap(), scheduler, this.offloaderStats, + entryOffsetsCache); return offloader; } private BlobStoreManagedLedgerOffloader getOffloader(String bucket, BlobStore mockedBlobStore) throws IOException { mockedConfig = mock(TieredStorageConfiguration.class, delegatesTo(getConfiguration(bucket))); Mockito.doReturn(mockedBlobStore).when(mockedConfig).getBlobStore(); - BlobStoreManagedLedgerOffloader offloader = BlobStoreManagedLedgerOffloader.create(mockedConfig, new HashMap(), scheduler, this.offloaderStats); + BlobStoreManagedLedgerOffloader offloader = BlobStoreManagedLedgerOffloader.create(mockedConfig, new HashMap(), scheduler, this.offloaderStats, + entryOffsetsCache); return offloader; } diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffsetsCacheTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffsetsCacheTest.java new file mode 100644 index 0000000000000..86a72c7b5547e --- /dev/null +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffsetsCacheTest.java @@ -0,0 +1,45 @@ +/* + * 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.bookkeeper.mledger.offload.jcloud.impl; + +import lombok.extern.slf4j.Slf4j; +import org.testng.annotations.Test; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; + +@Slf4j +public class OffsetsCacheTest { + + @Test + public void testCache() throws Exception { + System.setProperty("pulsar.jclouds.readhandleimpl.offsetsscache.ttl.seconds", "1"); + OffsetsCache offsetsCache = new OffsetsCache(); + assertNull(offsetsCache.getIfPresent(1, 2)); + offsetsCache.put(1, 1, 1); + assertEquals(offsetsCache.getIfPresent(1, 1), 1); + offsetsCache.clear(); + assertNull(offsetsCache.getIfPresent(1, 1)); + // test ttl + offsetsCache.put(1, 2, 2); + assertEquals(offsetsCache.getIfPresent(1, 2), 2); + Thread.sleep(1500); + assertNull(offsetsCache.getIfPresent(1, 2)); + offsetsCache.close(); + } +} From 03a4995a161face8998eadbbf9baaf2d4f55e63a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 9 May 2024 13:04:12 +0300 Subject: [PATCH 171/580] [improve][offload] Replace usage of shaded class in OffsetsCache (#22683) --- .../bookkeeper/mledger/offload/jcloud/impl/OffsetsCache.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffsetsCache.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffsetsCache.java index fa13afa8ff0e7..6651b199e4e60 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffsetsCache.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffsetsCache.java @@ -20,7 +20,7 @@ import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; -import io.grpc.netty.shaded.io.netty.util.concurrent.DefaultThreadFactory; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -49,7 +49,7 @@ public OffsetsCache() { .build(); cacheEvictionExecutor = Executors.newSingleThreadScheduledExecutor( - new DefaultThreadFactory("jcloud-offsets-cache-eviction")); + new ThreadFactoryBuilder().setNameFormat("jcloud-offsets-cache-eviction").build()); int period = Math.max(CACHE_TTL_SECONDS / 2, 1); cacheEvictionExecutor.scheduleAtFixedRate(() -> { entryOffsetsCache.cleanUp(); From 0fd223d23d5b4226f2afa3e67f4ecfd6e665c470 Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Thu, 9 May 2024 18:07:13 +0800 Subject: [PATCH 172/580] [improve] [pip] PIP-349: Add additionalSystemCursorNames ignore list for TTL check (#22651) Co-authored-by: Jiwe Guo --- pip/pip-349.md | 33 +++++++++++++++++++++++++++++++++ 1 file changed, 33 insertions(+) create mode 100644 pip/pip-349.md diff --git a/pip/pip-349.md b/pip/pip-349.md new file mode 100644 index 0000000000000..b676b09aa2ee4 --- /dev/null +++ b/pip/pip-349.md @@ -0,0 +1,33 @@ +# PIP-349: Add additionalSystemCursorNames ignore list for ttl check + +# Background knowledge + +In Pulsar topic, we have [retention policy](https://pulsar.apache.org/docs/3.2.x/cookbooks-retention-expiry/#retention-policies) to control the acknowledged message lifetime. For the unacknowledged messages, we have a separate mechanism to control the message lifetime, which is called [`TTL`](https://pulsar.apache.org/docs/3.2.x/cookbooks-retention-expiry/#time-to-live-ttl). The `TTL` is a time-to-live value for the message, which is controlled by `ttlDurationDefaultInSeconds`. The message will be automatically acknowledged if it is not consumed within the `TTL` value. + +# Motivation + +In Pulsar, we have two kinds of topics, system topic and normal topic. The system topics are used for internal purposes, such as transaction internal topics. The system topics are not supposed to be consumed by the users. However, the system topics are still subject to the `TTL` check. If the system topics are not consumed within the `TTL` value, the messages in the system topics will be automatically acknowledged. This is not the expected behavior for the system topics and may lead to data loss. +For normal topics, we also has two kinds of subscriptions, system subscription and normal subscription. The system subscription is used for internal purposes, such as compaction service or third-party plugins. The system subscription is not supposed to be used by the users. However, the system subscription is still subject to the `TTL` check. If the system subscription is not consumed within the `TTL` value, the messages in the system subscription will be automatically acknowledged. This is not the expected behavior for the system subscription. + +We had one PR [#21865](https://github.com/apache/pulsar/pull/21865) to filter the compaction service cursors for TTL check, but it doesn't cover other system cursors. To provide a general solution and support third-party plugin cursors not impacted by TTL, I proposed to add an additionalSystemCursorNames ignore list to filter the TTL check. + +# Goals + +## In Scope + +Add an additionalSystemCursorNames ignore list to filter the TTL check for additional system subscriptions except for compaction service subscription. The additionalSystemCursorNames ignore list is an optional configuration, and the default value is empty. Pulsar broker will filter the TTL check for the additionalSystemCursorNames subscriptions. +The compaction service subscription is a system subscription and should not be impacted by TTL. To reduce the risk of data loss after enabled compaction service, we will add the compaction service subscription to the TTL ignore list by default and can't be removed. + +# Detailed Design + +## Design & Implementation Details + +Add a additionalSystemCursorNames ignore list to filter the TTL check for system subscriptions. The additionalSystemCursorNames ignore list is an optional configuration, and the default value is empty. Pulsar broker will filter the TTL check for the additionalSystemCursorNames subscriptions. + +# Backward & Forward Compatibility + +This change is fully compatible. + +# Links +* Mailing List discussion thread: https://lists.apache.org/thread/xgcworz4j8rjlqwr476s7sqn9do43f1t +* Mailing List voting thread: https://lists.apache.org/thread/xs3g2y6fgjpfjr8fhf1qghcxkrt3yby7 From bed032e714aff9f5d2594bdc80a3e7888e53b1bf Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Thu, 9 May 2024 20:45:56 +0800 Subject: [PATCH 173/580] [improve] [broker] Add additionalSystemCursorNames ignore list for TTL check (#22614) --- conf/broker.conf | 4 + conf/standalone.conf | 4 + .../pulsar/broker/ServiceConfiguration.java | 7 ++ .../service/persistent/PersistentTopic.java | 7 +- .../pulsar/broker/service/MessageTTLTest.java | 96 +++++++++++++++++++ 5 files changed, 117 insertions(+), 1 deletion(-) diff --git a/conf/broker.conf b/conf/broker.conf index 1b51ff4755173..1ef68a0395cef 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -180,6 +180,10 @@ backlogQuotaDefaultRetentionPolicy=producer_request_hold # Default ttl for namespaces if ttl is not already configured at namespace policies. (disable default-ttl with value 0) ttlDurationDefaultInSeconds=0 +# Additional system subscriptions that will be ignored by ttl check. The cursor names are comma separated. +# Default is empty. +# additionalSystemCursorNames= + # Enable topic auto creation if new producer or consumer connected (disable auto creation with value false) allowAutoTopicCreation=true diff --git a/conf/standalone.conf b/conf/standalone.conf index 51035235d4d30..a8615b70293d6 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -121,6 +121,10 @@ backlogQuotaDefaultLimitSecond=-1 # Default ttl for namespaces if ttl is not already configured at namespace policies. (disable default-ttl with value 0) ttlDurationDefaultInSeconds=0 +# Additional system subscriptions that will be ignored by ttl check. The cursor names are comma separated. +# Default is empty. +# additionalSystemCursorNames= + # Enable the deletion of inactive topics. This parameter need to cooperate with the allowAutoTopicCreation parameter. # If brokerDeleteInactiveTopicsEnabled is set to true, we should ensure that allowAutoTopicCreation is also set to true. brokerDeleteInactiveTopicsEnabled=true 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 a9d170ea5de87..9efe185650969 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 @@ -652,6 +652,13 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece ) private int ttlDurationDefaultInSeconds = 0; + @FieldContext( + category = CATEGORY_POLICIES, + doc = "Additional system subscriptions that will be ignored by ttl check. " + + "The cursor names are comma separated. Default is empty." + ) + private Set additionalSystemCursorNames = new TreeSet<>(); + @FieldContext( category = CATEGORY_POLICIES, dynamic = true, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 7228bdeb2d334..28bc27f796157 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -40,6 +40,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -279,6 +280,7 @@ protected TopicStatsHelper initialValue() { private final ExecutorService orderedExecutor; private volatile CloseFutures closeFutures; + private Set additionalSystemCursorNames = new TreeSet<>(); @Getter private final PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); @@ -414,6 +416,7 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS } else { shadowSourceTopic = null; } + additionalSystemCursorNames = brokerService.pulsar().getConfiguration().getAdditionalSystemCursorNames(); } @Override @@ -1934,7 +1937,9 @@ public void checkMessageExpiry() { int messageTtlInSeconds = topicPolicies.getMessageTTLInSeconds().get(); if (messageTtlInSeconds != 0) { subscriptions.forEach((__, sub) -> { - if (!isCompactionSubscription(sub.getName())) { + if (!isCompactionSubscription(sub.getName()) + && (additionalSystemCursorNames.isEmpty() + || !additionalSystemCursorNames.contains(sub.getName()))) { sub.expireMessages(messageTtlInSeconds); } }); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageTTLTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageTTLTest.java index 68a9a769ac1fe..2f5ad215a1b6e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageTTLTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageTTLTest.java @@ -23,15 +23,23 @@ import static org.mockito.Mockito.verify; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; + import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats.CursorStats; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; @@ -138,4 +146,92 @@ public void testTTLPoliciesUpdate() throws Exception { topicRefMock.onUpdate(topicPolicies); verify(topicRefMock, times(2)).checkMessageExpiry(); } + + @Test + public void testTtlFilteredByIgnoreSubscriptions() throws Exception { + String topicName = "persistent://prop/ns-abc/testTTLFilteredByIgnoreSubscriptions"; + String subName = "__SUB_FILTER"; + cleanup(); + Set ignoredSubscriptions = new HashSet<>(); + ignoredSubscriptions.add(subName); + int defaultTtl = 5; + conf.setAdditionalSystemCursorNames(ignoredSubscriptions); + conf.setTtlDurationDefaultInSeconds(defaultTtl); + super.baseSetup(); + + pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subName) + .subscribe().close(); + + @Cleanup + org.apache.pulsar.client.api.Producer producer = pulsarClient.newProducer(Schema.STRING) + .enableBatching(false).topic(topicName).create(); + + final int messages = 10; + + for (int i = 0; i < messages; i++) { + String message = "my-message-" + i; + producer.send(message); + } + producer.close(); + + Optional topic = pulsar.getBrokerService().getTopicReference(topicName); + assertTrue(topic.isPresent()); + PersistentSubscription subscription = (PersistentSubscription) topic.get().getSubscription(subName); + + Thread.sleep((defaultTtl - 1) * 1000); + topic.get().checkMessageExpiry(); + // Wait the message expire task done and make sure the message does not expire early. + Thread.sleep(1000); + assertEquals(subscription.getNumberOfEntriesInBacklog(false), 10); + Thread.sleep(2000); + topic.get().checkMessageExpiry(); + // Wait the message expire task done. + retryStrategically((test) -> subscription.getNumberOfEntriesInBacklog(false) == 0, 5, 200); + // The message should not expire because the subscription is ignored. + assertEquals(subscription.getNumberOfEntriesInBacklog(false), 10); + + conf.setAdditionalSystemCursorNames(new TreeSet<>()); + } + + @Test + public void testTtlWithoutIgnoreSubscriptions() throws Exception { + String topicName = "persistent://prop/ns-abc/testTTLWithoutIgnoreSubscriptions"; + String subName = "__SUB_FILTER"; + cleanup(); + int defaultTtl = 5; + conf.setTtlDurationDefaultInSeconds(defaultTtl); + conf.setBrokerDeleteInactiveTopicsEnabled(false); + super.baseSetup(); + + pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subName) + .subscribe().close(); + + @Cleanup + org.apache.pulsar.client.api.Producer producer = pulsarClient.newProducer(Schema.STRING) + .enableBatching(false).topic(topicName).create(); + + final int messages = 10; + + for (int i = 0; i < messages; i++) { + String message = "my-message-" + i; + producer.send(message); + } + producer.close(); + + Optional topic = pulsar.getBrokerService().getTopicReference(topicName); + assertTrue(topic.isPresent()); + PersistentSubscription subscription = (PersistentSubscription) topic.get().getSubscription(subName); + + Thread.sleep((defaultTtl - 1) * 1000); + topic.get().checkMessageExpiry(); + // Wait the message expire task done and make sure the message does not expire early. + Thread.sleep(1000); + assertEquals(subscription.getNumberOfEntriesInBacklog(false), 10); + Thread.sleep(2000); + topic.get().checkMessageExpiry(); + // Wait the message expire task done and make sure the message expired. + retryStrategically((test) -> subscription.getNumberOfEntriesInBacklog(false) == 0, 5, 200); + assertEquals(subscription.getNumberOfEntriesInBacklog(false), 0); + } + } From 253e6506ea2c5ccc6afe1117e311cf24685ce4e9 Mon Sep 17 00:00:00 2001 From: hrzzzz <64506104+hrzzzz@users.noreply.github.com> Date: Thu, 9 May 2024 21:49:27 +0800 Subject: [PATCH 174/580] [fix][broker] Fix ProducerBusy issue due to incorrect userCreatedProducerCount on non-persistent topic (#22685) Co-authored-by: ruihongzhou --- .../nonpersistent/NonPersistentTopic.java | 10 --------- .../nonpersistent/NonPersistentTopicTest.java | 22 +++++++++++++++++++ 2 files changed, 22 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index d19aeaa4b0f82..86eab3d38b0aa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service.nonpersistent; -import static com.google.common.base.Preconditions.checkArgument; import static org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheManagerImpl.create; import static org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; @@ -58,7 +57,6 @@ import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.GetStatsOptions; -import org.apache.pulsar.broker.service.Producer; import org.apache.pulsar.broker.service.Replicator; import org.apache.pulsar.broker.service.StreamingStats; import org.apache.pulsar.broker.service.Subscription; @@ -249,14 +247,6 @@ public boolean isReplicationBacklogExist() { return false; } - @Override - public void removeProducer(Producer producer) { - checkArgument(producer.getTopic() == this); - if (producers.remove(producer.getProducerName(), producer)) { - handleProducerRemoved(producer); - } - } - @Override public CompletableFuture checkIfTransactionBufferRecoverCompletely(boolean isTxnEnabled) { return CompletableFuture.completedFuture(null); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java index b33381126e5c2..e2aec70fb114e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java @@ -18,11 +18,13 @@ */ package org.apache.pulsar.broker.service.nonpersistent; +import java.lang.reflect.Field; import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.SubscriptionOption; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -250,4 +252,24 @@ public void testSubscriptionsOnNonPersistentTopic() throws Exception { Awaitility.waitAtMost(10, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS) .until(() -> subscriptionMap.get(keySharedSubName) == null); } + + + @Test + public void testRemoveProducerOnNonPersistentTopic() throws Exception { + final String topicName = "non-persistent://prop/ns-abc/topic_" + UUID.randomUUID(); + + Producer producer = pulsarClient.newProducer() + .topic(topicName) + .create(); + + NonPersistentTopic topic = (NonPersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); + Field field = AbstractTopic.class.getDeclaredField("userCreatedProducerCount"); + field.setAccessible(true); + int userCreatedProducerCount = (int) field.get(topic); + assertEquals(userCreatedProducerCount, 1); + + producer.close(); + userCreatedProducerCount = (int) field.get(topic); + assertEquals(userCreatedProducerCount, 0); + } } From ff4853e06259d2c278d76d393dd9b650ad3edf4a Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 10 May 2024 08:38:49 +0800 Subject: [PATCH 175/580] [fix] [broker] Fix configurationMetadataSyncEventTopic is marked supporting dynamic setting, but not implemented (#22684) --- .../apache/pulsar/broker/PulsarService.java | 80 +++++- .../pulsar/broker/service/BrokerService.java | 5 + .../PulsarMetadataEventSynchronizer.java | 237 +++++++++++++----- ...licationWithConfigurationSyncTestBase.java | 234 +++++++++++++++++ .../broker/service/SyncConfigStoreTest.java | 116 +++++++++ .../api/MetadataEventSynchronizer.java | 2 +- .../api/extended/MetadataStoreExtended.java | 2 + .../impl/LocalMemoryMetadataStore.java | 9 +- .../metadata/impl/RocksdbMetadataStore.java | 9 +- .../AbstractBatchedMetadataStore.java | 11 +- .../metadata/impl/oxia/OxiaMetadataStore.java | 10 +- .../impl/LocalMemoryMetadataStoreTest.java | 4 +- 12 files changed, 641 insertions(+), 78 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/GeoReplicationWithConfigurationSyncTestBase.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SyncConfigStoreTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 58d7e71b65d84..ac37aca531af9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -607,13 +607,12 @@ public CompletableFuture closeAsync() { } } - closeLocalMetadataStore(); + asyncCloseFutures.add(closeLocalMetadataStore()); + if (configMetadataSynchronizer != null) { + asyncCloseFutures.add(configMetadataSynchronizer.closeAsync()); + } if (configurationMetadataStore != null && shouldShutdownConfigurationMetadataStore) { configurationMetadataStore.close(); - if (configMetadataSynchronizer != null) { - configMetadataSynchronizer.close(); - configMetadataSynchronizer = null; - } } if (transactionExecutorProvider != null) { @@ -1160,14 +1159,16 @@ public MetadataStoreExtended createLocalMetadataStore(PulsarMetadataEventSynchro .build()); } - protected void closeLocalMetadataStore() throws Exception { + protected CompletableFuture closeLocalMetadataStore() throws Exception { if (localMetadataStore != null) { localMetadataStore.close(); } if (localMetadataSynchronizer != null) { - localMetadataSynchronizer.close(); + CompletableFuture closeSynchronizer = localMetadataSynchronizer.closeAsync(); localMetadataSynchronizer = null; + return closeSynchronizer; } + return CompletableFuture.completedFuture(null); } protected void startLeaderElectionService() { @@ -1928,4 +1929,69 @@ public CompletableFuture newTopicCompactionService(Strin return CompletableFuture.failedFuture(e); } } + + public void initConfigMetadataSynchronizerIfNeeded() { + mutex.lock(); + try { + final String newTopic = config.getConfigurationMetadataSyncEventTopic(); + final PulsarMetadataEventSynchronizer oldSynchronizer = configMetadataSynchronizer; + // Skip if not support. + if (!(configurationMetadataStore instanceof MetadataStoreExtended)) { + LOG.info( + "Skip to update Metadata Synchronizer because of the Configuration Metadata Store using[{}]" + + " does not support.", configurationMetadataStore.getClass().getName()); + return; + } + // Skip if no changes. + // case-1: both null. + // case-2: both topics are the same. + if ((oldSynchronizer == null && StringUtils.isBlank(newTopic))) { + LOG.info("Skip to update Metadata Synchronizer because the topic[null] does not changed."); + } + if (StringUtils.isNotBlank(newTopic) && oldSynchronizer != null) { + TopicName newTopicName = TopicName.get(newTopic); + TopicName oldTopicName = TopicName.get(oldSynchronizer.getTopicName()); + if (newTopicName.equals(oldTopicName)) { + LOG.info("Skip to update Metadata Synchronizer because the topic[{}] does not changed.", + oldTopicName); + } + } + // Update(null or not null). + // 1.set the new one. + // 2.close the old one. + // 3.async start the new one. + if (StringUtils.isBlank(newTopic)) { + configMetadataSynchronizer = null; + } else { + configMetadataSynchronizer = new PulsarMetadataEventSynchronizer(this, newTopic); + } + // close the old one and start the new one. + PulsarMetadataEventSynchronizer newSynchronizer = configMetadataSynchronizer; + MetadataStoreExtended metadataStoreExtended = (MetadataStoreExtended) configurationMetadataStore; + metadataStoreExtended.updateMetadataEventSynchronizer(newSynchronizer); + Runnable startNewSynchronizer = () -> { + if (newSynchronizer == null) { + return; + } + try { + newSynchronizer.start(); + } catch (Exception e) { + // It only occurs when get internal client fails. + LOG.error("Start Metadata Synchronizer with topic {} failed.", + newTopic, e); + } + }; + executor.submit(() -> { + if (oldSynchronizer != null) { + oldSynchronizer.closeAsync().whenComplete((ignore, ex) -> { + startNewSynchronizer.run(); + }); + } else { + startNewSynchronizer.run(); + } + }); + } finally { + mutex.unlock(); + } + } } 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 b61bc58e3b592..566ad1ff377e1 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 @@ -2804,6 +2804,11 @@ private void updateConfigurationAndRegisterListeners() { pulsar.getWebService().updateHttpRequestsFailOnUnknownPropertiesEnabled((boolean) enabled); }); + // add listener to notify web service httpRequestsFailOnUnknownPropertiesEnabled changed. + registerConfigurationListener("configurationMetadataSyncEventTopic", enabled -> { + pulsar.initConfigMetadataSynchronizerIfNeeded(); + }); + // add more listeners here // (3) create dynamic-config if not exist. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarMetadataEventSynchronizer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarMetadataEventSynchronizer.java index 0383a0b755245..8b2ebf200537e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarMetadataEventSynchronizer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarMetadataEventSynchronizer.java @@ -19,11 +19,15 @@ package org.apache.pulsar.broker.service; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; +import java.util.Arrays; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; +import lombok.Getter; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -46,6 +50,7 @@ public class PulsarMetadataEventSynchronizer implements MetadataEventSynchronize private static final Logger log = LoggerFactory.getLogger(PulsarMetadataEventSynchronizer.class); protected PulsarService pulsar; protected BrokerService brokerService; + @Getter protected String topicName; protected PulsarClientImpl client; protected volatile Producer producer; @@ -53,19 +58,32 @@ public class PulsarMetadataEventSynchronizer implements MetadataEventSynchronize private final CopyOnWriteArrayList>> listeners = new CopyOnWriteArrayList<>(); - private volatile boolean started = false; + static final AtomicReferenceFieldUpdater STATE_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(PulsarMetadataEventSynchronizer.class, State.class, "state"); + @Getter + private volatile State state; public static final String SUBSCRIPTION_NAME = "metadata-syncer"; private static final int MAX_PRODUCER_PENDING_SIZE = 1000; protected final Backoff backOff = new Backoff(100, TimeUnit.MILLISECONDS, 1, TimeUnit.MINUTES, 0, TimeUnit.MILLISECONDS); + private volatile CompletableFuture closeFuture; - public PulsarMetadataEventSynchronizer(PulsarService pulsar, String topicName) throws PulsarServerException { + public enum State { + Init, + Starting_Producer, + Starting_Consumer, + Started, + Closing, + Closed; + } + + public PulsarMetadataEventSynchronizer(PulsarService pulsar, String topicName) { this.pulsar = pulsar; this.brokerService = pulsar.getBrokerService(); this.topicName = topicName; + this.state = State.Init; if (!StringUtils.isNotBlank(topicName)) { log.info("Metadata synchronizer is disabled"); - return; } } @@ -74,10 +92,11 @@ public void start() throws PulsarServerException { log.info("metadata topic doesn't exist.. skipping metadata synchronizer init.."); return; } + log.info("Metadata event synchronizer is starting on topic {}", topicName); this.client = (PulsarClientImpl) pulsar.getClient(); - startProducer(); - startConsumer(); - log.info("Metadata event synchronizer started on topic {}", topicName); + if (STATE_UPDATER.compareAndSet(this, State.Init, State.Starting_Producer)) { + startProducer(); + } } @Override @@ -98,7 +117,7 @@ public String getClusterName() { } private void publishAsync(MetadataEvent event, CompletableFuture future) { - if (!started) { + if (!isProducerStarted()) { log.info("Producer is not started on {}, failed to publish {}", topicName, event); future.completeExceptionally(new IllegalStateException("producer is not started yet")); } @@ -114,62 +133,100 @@ private void publishAsync(MetadataEvent event, CompletableFuture future) { } private void startProducer() { + if (isClosingOrClosed()) { + log.info("[{}] Skip to start new producer because the synchronizer is closed", topicName); + } + if (producer != null) { + log.error("[{}] Failed to start the producer because the producer has been set, state: {}", + topicName, state); + return; + } log.info("[{}] Starting producer", topicName); client.newProducer(Schema.AVRO(MetadataEvent.class)).topic(topicName) - .messageRoutingMode(MessageRoutingMode.SinglePartition).enableBatching(false).enableBatching(false) - .sendTimeout(0, TimeUnit.SECONDS) // - .maxPendingMessages(MAX_PRODUCER_PENDING_SIZE).createAsync().thenAccept(prod -> { + .messageRoutingMode(MessageRoutingMode.SinglePartition).enableBatching(false).enableBatching(false) + .sendTimeout(0, TimeUnit.SECONDS) // + .maxPendingMessages(MAX_PRODUCER_PENDING_SIZE).createAsync().thenAccept(prod -> { + backOff.reset(); + if (STATE_UPDATER.compareAndSet(this, State.Starting_Producer, State.Starting_Consumer)) { producer = prod; - started = true; log.info("producer is created successfully {}", topicName); - }).exceptionally(ex -> { - long waitTimeMs = backOff.next(); - log.warn("[{}] Failed to create producer ({}), retrying in {} s", topicName, ex.getMessage(), - waitTimeMs / 1000.0); - // BackOff before retrying - brokerService.executor().schedule(this::startProducer, waitTimeMs, TimeUnit.MILLISECONDS); - return null; - }); + PulsarMetadataEventSynchronizer.this.startConsumer(); + } else { + State stateTransient = state; + log.info("[{}] Closing the new producer because the synchronizer state is {}", prod, + stateTransient); + CompletableFuture closeProducer = new CompletableFuture<>(); + closeResource(() -> prod.closeAsync(), closeProducer); + closeProducer.thenRun(() -> { + log.info("[{}] Closed the new producer because the synchronizer state is {}", prod, + stateTransient); + }); + } + }).exceptionally(ex -> { + long waitTimeMs = backOff.next(); + log.warn("[{}] Failed to create producer ({}), retrying in {} s", topicName, ex.getMessage(), + waitTimeMs / 1000.0); + // BackOff before retrying + brokerService.executor().schedule(this::startProducer, waitTimeMs, TimeUnit.MILLISECONDS); + return null; + }); } private void startConsumer() { + if (isClosingOrClosed()) { + log.info("[{}] Skip to start new consumer because the synchronizer is closed", topicName); + } if (consumer != null) { + log.error("[{}] Failed to start the consumer because the consumer has been set, state: {}", + topicName, state); return; } + log.info("[{}] Starting consumer", topicName); ConsumerBuilder consumerBuilder = client.newConsumer(Schema.AVRO(MetadataEvent.class)) - .topic(topicName).subscriptionName(SUBSCRIPTION_NAME).ackTimeout(60, TimeUnit.SECONDS) - .subscriptionType(SubscriptionType.Failover).messageListener((c, msg) -> { - log.info("Processing metadata event for {} with listeners {}", msg.getValue().getPath(), - listeners.size()); - try { - if (listeners.size() == 0) { - c.acknowledgeAsync(msg); - return; - - } - if (listeners.size() == 1) { - listeners.get(0).apply(msg.getValue()).thenApply(__ -> c.acknowledgeAsync(msg)) - .exceptionally(ex -> { - log.warn("Failed to synchronize {} for {}", msg.getMessageId(), topicName, - ex.getCause()); - return null; - }); - } else { - FutureUtil - .waitForAll(listeners.stream().map(listener -> listener.apply(msg.getValue())) - .collect(Collectors.toList())) - .thenApply(__ -> c.acknowledgeAsync(msg)).exceptionally(ex -> { - log.warn("Failed to synchronize {} for {}", msg.getMessageId(), topicName); - return null; - }); - } - } catch (Exception e) { - log.warn("Failed to synchronize {} for {}", msg.getMessageId(), topicName); + .topic(topicName).subscriptionName(SUBSCRIPTION_NAME).ackTimeout(60, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Failover).messageListener((c, msg) -> { + log.info("Processing metadata event for {} with listeners {}", msg.getValue().getPath(), + listeners.size()); + try { + if (listeners.size() == 0) { + c.acknowledgeAsync(msg); + return; + } - }); + if (listeners.size() == 1) { + listeners.get(0).apply(msg.getValue()).thenApply(__ -> c.acknowledgeAsync(msg)) + .exceptionally(ex -> { + log.warn("Failed to synchronize {} for {}", msg.getMessageId(), topicName, + ex.getCause()); + return null; + }); + } else { + FutureUtil + .waitForAll(listeners.stream().map(listener -> listener.apply(msg.getValue())) + .collect(Collectors.toList())) + .thenApply(__ -> c.acknowledgeAsync(msg)).exceptionally(ex -> { + log.warn("Failed to synchronize {} for {}", msg.getMessageId(), topicName); + return null; + }); + } + } catch (Exception e) { + log.warn("Failed to synchronize {} for {}", msg.getMessageId(), topicName); + } + }); consumerBuilder.subscribeAsync().thenAccept(consumer -> { - log.info("successfully created consumer {}", topicName); - this.consumer = consumer; + backOff.reset(); + if (STATE_UPDATER.compareAndSet(this, State.Starting_Consumer, State.Started)) { + this.consumer = consumer; + log.info("successfully created consumer {}", topicName); + } else { + State stateTransient = state; + log.info("[{}] Closing the new consumer because the synchronizer state is {}", stateTransient); + CompletableFuture closeConsumer = new CompletableFuture<>(); + closeResource(() -> consumer.closeAsync(), closeConsumer); + closeConsumer.thenRun(() -> { + log.info("[{}] Closed the new consumer because the synchronizer state is {}", stateTransient); + }); + } }).exceptionally(ex -> { long waitTimeMs = backOff.next(); log.warn("[{}] Failed to create consumer ({}), retrying in {} s", topicName, ex.getMessage(), @@ -181,19 +238,81 @@ private void startConsumer() { } public boolean isStarted() { - return started; + return this.state == State.Started; + } + + public boolean isProducerStarted() { + return this.state.ordinal() > State.Starting_Producer.ordinal() + && this.state.ordinal() < State.Closing.ordinal(); + } + + public boolean isClosingOrClosed() { + return this.state == State.Closing || this.state == State.Closed; } @Override - public void close() { - started = false; - if (producer != null) { - producer.closeAsync(); - producer = null; + public synchronized CompletableFuture closeAsync() { + int tryChangeStateCounter = 0; + while (true) { + if (isClosingOrClosed()) { + return closeFuture; + } + if (STATE_UPDATER.compareAndSet(this, State.Init, State.Closing) + || STATE_UPDATER.compareAndSet(this, State.Starting_Producer, State.Closing) + || STATE_UPDATER.compareAndSet(this, State.Starting_Consumer, State.Closing) + || STATE_UPDATER.compareAndSet(this, State.Started, State.Closing)) { + break; + } + // Just for avoid spinning loop which would cause 100% CPU consumption here. + if (++tryChangeStateCounter > 100) { + log.error("Unexpected error: the state can not be changed to closing {}, state: {}", topicName, state); + return CompletableFuture.failedFuture(new RuntimeException("Unexpected error," + + " the state can not be changed to closing")); + } } - if (consumer != null) { - consumer.closeAsync(); - consumer = null; + CompletableFuture closeProducer = new CompletableFuture<>(); + CompletableFuture closeConsumer = new CompletableFuture<>(); + if (producer == null) { + closeProducer.complete(null); + } else { + closeResource(() -> producer.closeAsync(), closeProducer); + } + if (consumer == null) { + closeConsumer.complete(null); + } else { + closeResource(() -> consumer.closeAsync(), closeConsumer); + } + + // Add logs. + closeProducer.thenRun(() -> log.info("Successfully close producer {}", topicName)); + closeConsumer.thenRun(() -> log.info("Successfully close consumer {}", topicName)); + + closeFuture = FutureUtil.waitForAll(Arrays.asList(closeProducer, closeConsumer)); + closeFuture.thenRun(() -> { + this.state = State.Closed; + log.info("Successfully close metadata store synchronizer {}", topicName); + }); + return closeFuture; + } + + private void closeResource(final Supplier> asyncCloseable, + final CompletableFuture future) { + if (asyncCloseable == null) { + future.complete(null); + return; } + asyncCloseable.get().whenComplete((ignore, ex) -> { + if (ex == null) { + backOff.reset(); + future.complete(null); + return; + } + // Retry. + long waitTimeMs = backOff.next(); + log.warn("[{}] Exception: '{}' occurred while trying to close the %s. Retrying again in {} s.", + topicName, ex.getMessage(), asyncCloseable.getClass().getSimpleName(), waitTimeMs / 1000.0, ex); + brokerService.executor().schedule(() -> closeResource(asyncCloseable, future), waitTimeMs, + TimeUnit.MILLISECONDS); + }); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/GeoReplicationWithConfigurationSyncTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/GeoReplicationWithConfigurationSyncTestBase.java new file mode 100644 index 0000000000000..9b4dd5192e1ec --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/GeoReplicationWithConfigurationSyncTestBase.java @@ -0,0 +1,234 @@ +/* + * 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.service; + +import com.google.common.collect.Sets; +import java.net.URL; +import java.util.Collections; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.tests.TestRetrySupport; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.apache.pulsar.zookeeper.ZookeeperServerTest; + +@Slf4j +public abstract class GeoReplicationWithConfigurationSyncTestBase extends TestRetrySupport { + + protected final String defaultTenant = "public"; + protected final String defaultNamespace = defaultTenant + "/default"; + + protected final String cluster1 = "r1"; + protected URL url1; + protected URL urlTls1; + protected ServiceConfiguration config1 = new ServiceConfiguration(); + protected ZookeeperServerTest brokerConfigZk1; + protected LocalBookkeeperEnsemble bkEnsemble1; + protected PulsarService pulsar1; + protected BrokerService ns1; + protected PulsarAdmin admin1; + protected PulsarClient client1; + + protected URL url2; + protected URL urlTls2; + protected final String cluster2 = "r2"; + protected ServiceConfiguration config2 = new ServiceConfiguration(); + protected ZookeeperServerTest brokerConfigZk2; + protected LocalBookkeeperEnsemble bkEnsemble2; + protected PulsarService pulsar2; + protected BrokerService ns2; + protected PulsarAdmin admin2; + protected PulsarClient client2; + + protected void startZKAndBK() throws Exception { + // Start ZK. + brokerConfigZk1 = new ZookeeperServerTest(0); + brokerConfigZk1.start(); + brokerConfigZk2 = new ZookeeperServerTest(0); + brokerConfigZk2.start(); + + // Start BK. + bkEnsemble1 = new LocalBookkeeperEnsemble(3, 0, () -> 0); + bkEnsemble1.start(); + bkEnsemble2 = new LocalBookkeeperEnsemble(3, 0, () -> 0); + bkEnsemble2.start(); + } + + protected void startBrokers() throws Exception { + // Start brokers. + setConfigDefaults(config1, cluster1, bkEnsemble1, brokerConfigZk1); + pulsar1 = new PulsarService(config1); + pulsar1.start(); + ns1 = pulsar1.getBrokerService(); + + url1 = new URL(pulsar1.getWebServiceAddress()); + urlTls1 = new URL(pulsar1.getWebServiceAddressTls()); + admin1 = PulsarAdmin.builder().serviceHttpUrl(url1.toString()).build(); + client1 = PulsarClient.builder().serviceUrl(url1.toString()).build(); + + // Start region 2 + setConfigDefaults(config2, cluster2, bkEnsemble2, brokerConfigZk2); + pulsar2 = new PulsarService(config2); + pulsar2.start(); + ns2 = pulsar2.getBrokerService(); + + url2 = new URL(pulsar2.getWebServiceAddress()); + urlTls2 = new URL(pulsar2.getWebServiceAddressTls()); + admin2 = PulsarAdmin.builder().serviceHttpUrl(url2.toString()).build(); + client2 = PulsarClient.builder().serviceUrl(url2.toString()).build(); + } + + protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { + admin1.clusters().createCluster(cluster1, ClusterData.builder() + .serviceUrl(url1.toString()) + .serviceUrlTls(urlTls1.toString()) + .brokerServiceUrl(pulsar1.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()) + .brokerClientTlsEnabled(false) + .build()); + admin1.clusters().createCluster(cluster2, ClusterData.builder() + .serviceUrl(url2.toString()) + .serviceUrlTls(urlTls2.toString()) + .brokerServiceUrl(pulsar2.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar2.getBrokerServiceUrlTls()) + .brokerClientTlsEnabled(false) + .build()); + admin2.clusters().createCluster(cluster1, ClusterData.builder() + .serviceUrl(url1.toString()) + .serviceUrlTls(urlTls1.toString()) + .brokerServiceUrl(pulsar1.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()) + .brokerClientTlsEnabled(false) + .build()); + admin2.clusters().createCluster(cluster2, ClusterData.builder() + .serviceUrl(url2.toString()) + .serviceUrlTls(urlTls2.toString()) + .brokerServiceUrl(pulsar2.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar2.getBrokerServiceUrlTls()) + .brokerClientTlsEnabled(false) + .build()); + + admin1.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(cluster1, cluster2))); + admin2.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(cluster1, cluster2))); + + admin1.namespaces().createNamespace(defaultNamespace); + admin2.namespaces().createNamespace(defaultNamespace); + } + + @Override + protected void setup() throws Exception { + incrementSetupNumber(); + + log.info("--- Starting OneWayReplicatorTestBase::setup ---"); + + startZKAndBK(); + + startBrokers(); + + createDefaultTenantsAndClustersAndNamespace(); + + Thread.sleep(100); + log.info("--- OneWayReplicatorTestBase::setup completed ---"); + } + + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, + LocalBookkeeperEnsemble bookkeeperEnsemble, ZookeeperServerTest brokerConfigZk) { + config.setClusterName(clusterName); + config.setAdvertisedAddress("localhost"); + config.setWebServicePort(Optional.of(0)); + config.setWebServicePortTls(Optional.of(0)); + config.setMetadataStoreUrl("zk:127.0.0.1:" + bookkeeperEnsemble.getZookeeperPort()); + config.setConfigurationMetadataStoreUrl("zk:127.0.0.1:" + brokerConfigZk.getZookeeperPort() + "/foo"); + config.setBrokerDeleteInactiveTopicsEnabled(false); + config.setBrokerDeleteInactiveTopicsFrequencySeconds(60); + config.setBrokerShutdownTimeoutMs(0L); + config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); + config.setBrokerServicePort(Optional.of(0)); + config.setBrokerServicePortTls(Optional.of(0)); + config.setBacklogQuotaCheckIntervalInSeconds(5); + config.setDefaultNumberOfNamespaceBundles(1); + config.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); + config.setEnableReplicatedSubscriptions(true); + config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); + config.setLoadBalancerSheddingEnabled(false); + } + + @Override + protected void cleanup() throws Exception { + // shutdown. + markCurrentSetupNumberCleaned(); + log.info("--- Shutting down ---"); + + // Stop brokers. + if (client1 != null) { + client1.close(); + client1 = null; + } + if (client2 != null) { + client2.close(); + client2 = null; + } + if (admin1 != null) { + admin1.close(); + admin1 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } + if (pulsar2 != null) { + pulsar2.close(); + pulsar2 = null; + } + if (pulsar1 != null) { + pulsar1.close(); + pulsar1 = null; + } + + // Stop ZK and BK. + if (bkEnsemble1 != null) { + bkEnsemble1.stop(); + bkEnsemble1 = null; + } + if (bkEnsemble2 != null) { + bkEnsemble2.stop(); + bkEnsemble2 = null; + } + if (brokerConfigZk1 != null) { + brokerConfigZk1.stop(); + brokerConfigZk1 = null; + } + if (brokerConfigZk2 != null) { + brokerConfigZk2.stop(); + brokerConfigZk2 = null; + } + + // Reset configs. + config1 = new ServiceConfiguration(); + config2 = new ServiceConfiguration(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SyncConfigStoreTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SyncConfigStoreTest.java new file mode 100644 index 0000000000000..577725f96ed34 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SyncConfigStoreTest.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import java.util.Arrays; +import java.util.HashSet; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.metadata.api.MetadataEvent; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.apache.pulsar.zookeeper.ZookeeperServerTest; +import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class SyncConfigStoreTest extends GeoReplicationWithConfigurationSyncTestBase { + + private static final String CONF_NAME_SYNC_EVENT_TOPIC = "configurationMetadataSyncEventTopic"; + private static final String SYNC_EVENT_TOPIC = TopicDomain.persistent.value() + "://" + SYSTEM_NAMESPACE + + "/__sync_config_meta"; + + @Override + @BeforeClass(alwaysRun = true, timeOut = 300000) + public void setup() throws Exception { + super.setup(); + TenantInfoImpl tenantInfo = new TenantInfoImpl(); + tenantInfo.setAllowedClusters(new HashSet<>(Arrays.asList(cluster1, cluster2))); + admin1.tenants().createTenant(TopicName.get(SYNC_EVENT_TOPIC).getTenant(), tenantInfo); + admin1.namespaces().createNamespace(TopicName.get(SYNC_EVENT_TOPIC).getNamespace()); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, + LocalBookkeeperEnsemble bookkeeperEnsemble, ZookeeperServerTest brokerConfigZk) { + super.setConfigDefaults(config, clusterName, bookkeeperEnsemble, brokerConfigZk); + } + + @Test + public void testDynamicEnableConfigurationMetadataSyncEventTopic() throws Exception { + // Verify the condition that supports synchronizer: the metadata store is a different one. + Awaitility.await().untilAsserted(() -> { + boolean shouldShutdownConfigurationMetadataStore = + WhiteboxImpl.getInternalState(pulsar1, "shouldShutdownConfigurationMetadataStore"); + assertTrue(shouldShutdownConfigurationMetadataStore); + }); + + // Verify the synchronizer will be created dynamically. + admin1.brokers().updateDynamicConfiguration(CONF_NAME_SYNC_EVENT_TOPIC, SYNC_EVENT_TOPIC); + Awaitility.await().untilAsserted(() -> { + assertEquals(pulsar1.getConfig().getConfigurationMetadataSyncEventTopic(), SYNC_EVENT_TOPIC); + PulsarMetadataEventSynchronizer synchronizer = + WhiteboxImpl.getInternalState(pulsar1, "configMetadataSynchronizer"); + assertNotNull(synchronizer); + assertEquals(synchronizer.getState(), PulsarMetadataEventSynchronizer.State.Started); + assertTrue(synchronizer.isStarted()); + }); + + PulsarMetadataEventSynchronizer synchronizerStarted = + WhiteboxImpl.getInternalState(pulsar1, "configMetadataSynchronizer"); + Producer producerStarted = + WhiteboxImpl.getInternalState(synchronizerStarted, "producer"); + Consumer consumerStarted = + WhiteboxImpl.getInternalState(synchronizerStarted, "consumer"); + + // Verify the synchronizer will be closed dynamically. + admin1.brokers().deleteDynamicConfiguration(CONF_NAME_SYNC_EVENT_TOPIC); + Awaitility.await().untilAsserted(() -> { + // The synchronizer that was started will be closed. + assertEquals(synchronizerStarted.getState(), PulsarMetadataEventSynchronizer.State.Closed); + assertTrue(synchronizerStarted.isClosingOrClosed()); + assertFalse(producerStarted.isConnected()); + assertFalse(consumerStarted.isConnected()); + // The synchronizer in memory will be null. + assertNull(pulsar1.getConfig().getConfigurationMetadataSyncEventTopic()); + PulsarMetadataEventSynchronizer synchronizer = + WhiteboxImpl.getInternalState(pulsar1, "configMetadataSynchronizer"); + assertNull(synchronizer); + }); + } +} diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataEventSynchronizer.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataEventSynchronizer.java index 9a735e0f15ab8..cababd0324627 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataEventSynchronizer.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataEventSynchronizer.java @@ -49,5 +49,5 @@ public interface MetadataEventSynchronizer { /** * close synchronizer resources. */ - void close(); + CompletableFuture closeAsync(); } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/extended/MetadataStoreExtended.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/extended/MetadataStoreExtended.java index e565ba30d3dfb..182c14ef601a4 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/extended/MetadataStoreExtended.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/extended/MetadataStoreExtended.java @@ -84,6 +84,8 @@ default Optional getMetadataEventSynchronizer() { return Optional.empty(); } + default void updateMetadataEventSynchronizer(MetadataEventSynchronizer synchronizer) {} + /** * Handles a metadata synchronizer event. * diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java index 7a495f78771b1..3909a89cf5eb2 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java @@ -82,8 +82,7 @@ public LocalMemoryMetadataStore(String metadataURL, MetadataStoreConfig metadata String name = metadataURL.substring(MEMORY_SCHEME_IDENTIFIER.length()); // Local means a private data set // update synchronizer and register sync listener - synchronizer = metadataStoreConfig.getSynchronizer(); - registerSyncListener(Optional.ofNullable(synchronizer)); + updateMetadataEventSynchronizer(metadataStoreConfig.getSynchronizer()); if ("local".equals(name)) { map = new TreeMap<>(); sequentialIdGenerator = new AtomicLong(); @@ -233,6 +232,12 @@ public Optional getMetadataEventSynchronizer() { return Optional.ofNullable(synchronizer); } + @Override + public void updateMetadataEventSynchronizer(MetadataEventSynchronizer synchronizer) { + this.synchronizer = synchronizer; + registerSyncListener(Optional.ofNullable(synchronizer)); + } + @Override public void close() throws Exception { if (isClosed.compareAndSet(false, true)) { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java index be985129f2ad1..39f7edd5ceed5 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java @@ -112,8 +112,7 @@ public static RocksdbMetadataStore get(String metadataStoreUri, MetadataStoreCon // Create a new store instance store = new RocksdbMetadataStore(metadataStoreUri, conf); // update synchronizer and register sync listener - store.synchronizer = conf.getSynchronizer(); - store.registerSyncListener(Optional.ofNullable(store.synchronizer)); + store.updateMetadataEventSynchronizer(conf.getSynchronizer()); instancesCache.put(metadataStoreUri, store); return store; } @@ -572,6 +571,12 @@ protected CompletableFuture storePut(String path, byte[] data, Optional getMetadataEventSynchronizer() { return Optional.ofNullable(synchronizer); } + + @Override + public void updateMetadataEventSynchronizer(MetadataEventSynchronizer synchronizer) { + this.synchronizer = synchronizer; + registerSyncListener(Optional.ofNullable(synchronizer)); + } } class RocksdbMetadataStoreProvider implements MetadataStoreProvider { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java index a164e4c246066..5b45530d2e20e 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java @@ -52,7 +52,7 @@ public abstract class AbstractBatchedMetadataStore extends AbstractMetadataStore private final int maxDelayMillis; private final int maxOperations; private final int maxSize; - private final MetadataEventSynchronizer synchronizer; + private MetadataEventSynchronizer synchronizer; private final BatchMetadataStoreStats batchMetadataStoreStats; protected AbstractBatchedMetadataStore(MetadataStoreConfig conf) { @@ -75,8 +75,7 @@ protected AbstractBatchedMetadataStore(MetadataStoreConfig conf) { } // update synchronizer and register sync listener - synchronizer = conf.getSynchronizer(); - registerSyncListener(Optional.ofNullable(synchronizer)); + updateMetadataEventSynchronizer(conf.getSynchronizer()); this.batchMetadataStoreStats = new BatchMetadataStoreStats(metadataStoreName, executor); } @@ -161,6 +160,12 @@ public Optional getMetadataEventSynchronizer() { return Optional.ofNullable(synchronizer); } + @Override + public void updateMetadataEventSynchronizer(MetadataEventSynchronizer synchronizer) { + this.synchronizer = synchronizer; + registerSyncListener(Optional.ofNullable(synchronizer)); + } + private void enqueue(MessagePassingQueue queue, MetadataOp op) { if (enabled) { if (!queue.offer(op)) { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java index 728bc1175b9ba..f85e3d2dc7562 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java @@ -55,7 +55,7 @@ public class OxiaMetadataStore extends AbstractMetadataStore { private final AsyncOxiaClient client; private final String identity; - private final Optional synchronizer; + private Optional synchronizer; OxiaMetadataStore( @NonNull String serviceAddress, @@ -69,7 +69,7 @@ public class OxiaMetadataStore extends AbstractMetadataStore { if (!metadataStoreConfig.isBatchingEnabled()) { linger = 0; } - this.synchronizer = Optional.ofNullable(metadataStoreConfig.getSynchronizer()); + updateMetadataEventSynchronizer(metadataStoreConfig.getSynchronizer()); identity = UUID.randomUUID().toString(); client = OxiaClientBuilder.create(serviceAddress) @@ -286,5 +286,11 @@ public Optional getMetadataEventSynchronizer() { return synchronizer; } + @Override + public void updateMetadataEventSynchronizer(MetadataEventSynchronizer synchronizer) { + this.synchronizer = Optional.ofNullable(synchronizer); + registerSyncListener(this.synchronizer); + } + private record PathWithPutResult(String path, PutResult result) {} } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStoreTest.java index 3fabe9647eb34..caca16ff538a4 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStoreTest.java @@ -206,8 +206,8 @@ public String getClusterName() { } @Override - public void close() { - // No-op + public CompletableFuture closeAsync() { + return CompletableFuture.completedFuture(null); } } From 774a5d42e8342ee50395cf3626b9e7af27da849e Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 10 May 2024 10:37:44 +0800 Subject: [PATCH 176/580] [fix][broker] Fix cursor should use latest ledger config (#22644) Signed-off-by: Zixuan Liu --- .../mledger/impl/ManagedCursorImpl.java | 61 +++++++++---------- .../mledger/impl/ManagedCursorMXBeanImpl.java | 3 +- .../mledger/impl/ManagedLedgerImpl.java | 8 +-- .../mledger/impl/NonDurableCursorImpl.java | 5 +- .../bookkeeper/mledger/impl/OpReadEntry.java | 3 +- .../mledger/impl/RangeSetWrapper.java | 2 +- .../mledger/impl/ReadOnlyCursorImpl.java | 5 +- .../impl/ReadOnlyManagedLedgerImpl.java | 2 +- ...edCursorIndividualDeletedMessagesTest.java | 3 +- .../mledger/impl/ManagedCursorTest.java | 7 +-- .../mledger/impl/ManagedLedgerTest.java | 2 +- .../service/BrokerBkEnsemblesTests.java | 8 +-- .../persistent/PersistentTopicTest.java | 25 ++++++++ 13 files changed, 77 insertions(+), 57 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 3671385e60f75..35000361eca68 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -119,7 +119,6 @@ public class ManagedCursorImpl implements ManagedCursor { return 0; }; protected final BookKeeper bookkeeper; - protected final ManagedLedgerConfig config; protected final ManagedLedgerImpl ledger; private final String name; @@ -299,31 +298,30 @@ public interface VoidCallback { void operationFailed(ManagedLedgerException exception); } - ManagedCursorImpl(BookKeeper bookkeeper, ManagedLedgerConfig config, ManagedLedgerImpl ledger, String cursorName) { + ManagedCursorImpl(BookKeeper bookkeeper, ManagedLedgerImpl ledger, String cursorName) { this.bookkeeper = bookkeeper; this.cursorProperties = Collections.emptyMap(); - this.config = config; this.ledger = ledger; this.name = cursorName; this.individualDeletedMessages = new RangeSetWrapper<>(positionRangeConverter, positionRangeReverseConverter, this); - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { this.batchDeletedIndexes = new ConcurrentSkipListMap<>(); } else { this.batchDeletedIndexes = null; } - this.digestType = BookKeeper.DigestType.fromApiDigestType(config.getDigestType()); + this.digestType = BookKeeper.DigestType.fromApiDigestType(getConfig().getDigestType()); STATE_UPDATER.set(this, State.Uninitialized); PENDING_MARK_DELETED_SUBMITTED_COUNT_UPDATER.set(this, 0); PENDING_READ_OPS_UPDATER.set(this, 0); RESET_CURSOR_IN_PROGRESS_UPDATER.set(this, FALSE); WAITING_READ_OP_UPDATER.set(this, null); - this.clock = config.getClock(); + this.clock = getConfig().getClock(); this.lastActive = this.clock.millis(); this.lastLedgerSwitchTimestamp = this.clock.millis(); - if (config.getThrottleMarkDelete() > 0.0) { - markDeleteLimiter = RateLimiter.create(config.getThrottleMarkDelete()); + if (getConfig().getThrottleMarkDelete() > 0.0) { + markDeleteLimiter = RateLimiter.create(getConfig().getThrottleMarkDelete()); } else { // Disable mark-delete rate limiter markDeleteLimiter = null; @@ -343,7 +341,7 @@ public Map getProperties() { @Override public boolean isCursorDataFullyPersistable() { - return individualDeletedMessages.size() <= config.getMaxUnackedRangesToPersist(); + return individualDeletedMessages.size() <= getConfig().getMaxUnackedRangesToPersist(); } @Override @@ -607,7 +605,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (positionInfo.getIndividualDeletedMessagesCount() > 0) { recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); } - if (config.isDeletionAtBatchIndexLevelEnabled() + if (getConfig().isDeletionAtBatchIndexLevelEnabled() && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) { recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList()); } @@ -616,7 +614,8 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac }, null); }; try { - bookkeeper.asyncOpenLedger(ledgerId, digestType, config.getPassword(), openCallback, null); + bookkeeper.asyncOpenLedger(ledgerId, digestType, getConfig().getPassword(), openCallback, + null); } catch (Throwable t) { log.error("[{}] Encountered error on opening cursor ledger {} for cursor {}", ledger.getName(), ledgerId, name, t); @@ -973,10 +972,10 @@ public void asyncReadEntriesWithSkipOrWait(int maxEntries, long maxSizeBytes, Re // Check again for new entries after the configured time, then if still no entries are available register // to be notified - if (config.getNewEntriesCheckDelayInMillis() > 0) { + if (getConfig().getNewEntriesCheckDelayInMillis() > 0) { ledger.getScheduledExecutor() .schedule(() -> checkForNewEntries(op, callback, ctx), - config.getNewEntriesCheckDelayInMillis(), TimeUnit.MILLISECONDS); + getConfig().getNewEntriesCheckDelayInMillis(), TimeUnit.MILLISECONDS); } else { // If there's no delay, check directly from the same thread checkForNewEntries(op, callback, ctx); @@ -1324,7 +1323,7 @@ public void operationComplete() { lastMarkDeleteEntry = new MarkDeleteEntry(newMarkDeletePosition, isCompactionCursor() ? getProperties() : Collections.emptyMap(), null, null); individualDeletedMessages.clear(); - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { batchDeletedIndexes.values().forEach(BitSetRecyclable::recycle); batchDeletedIndexes.clear(); long[] resetWords = newReadPosition.ackSet; @@ -1583,7 +1582,7 @@ protected long getNumberOfEntries(Range range) { lock.readLock().lock(); try { - if (config.isUnackedRangesOpenCacheSetEnabled()) { + if (getConfig().isUnackedRangesOpenCacheSetEnabled()) { int cardinality = individualDeletedMessages.cardinality( range.lowerEndpoint().ledgerId, range.lowerEndpoint().entryId, range.upperEndpoint().ledgerId, range.upperEndpoint().entryId); @@ -1963,7 +1962,7 @@ public void asyncMarkDelete(final Position position, Map propertie PositionImpl newPosition = (PositionImpl) position; - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { if (newPosition.ackSet != null) { AtomicReference bitSetRecyclable = new AtomicReference<>(); BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(newPosition.ackSet); @@ -2146,7 +2145,7 @@ public void operationComplete() { try { individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()); - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { Map subMap = batchDeletedIndexes.subMap(PositionImpl.EARLIEST, false, PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()), true); @@ -2284,7 +2283,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } if (isMessageDeleted(position)) { - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); if (bitSetRecyclable != null) { bitSetRecyclable.recycle(); @@ -2296,7 +2295,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb continue; } if (position.ackSet == null) { - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); if (bitSetRecyclable != null) { bitSetRecyclable.recycle(); @@ -2313,7 +2312,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb log.debug("[{}] [{}] Individually deleted messages: {}", ledger.getName(), name, individualDeletedMessages); } - } else if (config.isDeletionAtBatchIndexLevelEnabled()) { + } else if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(position.ackSet); BitSetRecyclable bitSet = batchDeletedIndexes.computeIfAbsent(position, (v) -> givenBitSet); if (givenBitSet != bitSet) { @@ -2660,8 +2659,8 @@ public void operationFailed(MetaStoreException e) { private boolean shouldPersistUnackRangesToLedger() { return cursorLedger != null && !isCursorLedgerReadOnly - && config.getMaxUnackedRangesToPersist() > 0 - && individualDeletedMessages.size() > config.getMaxUnackedRangesToPersistInMetadataStore(); + && getConfig().getMaxUnackedRangesToPersist() > 0 + && individualDeletedMessages.size() > getConfig().getMaxUnackedRangesToPersistInMetadataStore(); } private void persistPositionMetaStore(long cursorsLedgerId, PositionImpl position, Map properties, @@ -2686,7 +2685,7 @@ private void persistPositionMetaStore(long cursorsLedgerId, PositionImpl positio info.addAllCursorProperties(buildStringPropertiesMap(cursorProperties)); if (persistIndividualDeletedMessageRanges) { info.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()); - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { info.addAllBatchedEntryDeletionIndexInfo(buildBatchEntryDeletionIndexInfoList()); } } @@ -2951,7 +2950,7 @@ public void operationFailed(ManagedLedgerException exception) { private CompletableFuture doCreateNewMetadataLedger() { CompletableFuture future = new CompletableFuture<>(); - ledger.asyncCreateLedger(bookkeeper, config, digestType, (rc, lh, ctx) -> { + ledger.asyncCreateLedger(bookkeeper, getConfig(), digestType, (rc, lh, ctx) -> { if (ledger.checkAndCompleteLedgerOpTask(rc, lh, ctx)) { future.complete(null); @@ -3056,7 +3055,7 @@ private List buildIndividualDeletedMessageRanges() { acksSerializedSize.addAndGet(messageRange.getSerializedSize()); rangeList.add(messageRange); - return rangeList.size() <= config.getMaxUnackedRangesToPersist(); + return rangeList.size() <= getConfig().getMaxUnackedRangesToPersist(); }); this.individualDeletedMessagesSerializedSize = acksSerializedSize.get(); @@ -3070,7 +3069,7 @@ private List buildIndividualDeletedMessageRanges() { private List buildBatchEntryDeletionIndexInfoList() { lock.readLock().lock(); try { - if (!config.isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) { + if (!getConfig().isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) { return Collections.emptyList(); } MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo @@ -3079,7 +3078,7 @@ private List buildBatchEntryDeletio .BatchedEntryDeletionIndexInfo.newBuilder(); List result = new ArrayList<>(); Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); - while (iterator.hasNext() && result.size() < config.getMaxBatchDeletedIndexToPersist()) { + while (iterator.hasNext() && result.size() < getConfig().getMaxBatchDeletedIndexToPersist()) { Map.Entry entry = iterator.next(); nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); @@ -3199,8 +3198,8 @@ public void operationFailed(MetaStoreException e) { boolean shouldCloseLedger(LedgerHandle lh) { long now = clock.millis(); if (ledger.getFactory().isMetadataServiceAvailable() - && (lh.getLastAddConfirmed() >= config.getMetadataMaxEntriesPerLedger() - || lastLedgerSwitchTimestamp < (now - config.getLedgerRolloverTimeout() * 1000)) + && (lh.getLastAddConfirmed() >= getConfig().getMetadataMaxEntriesPerLedger() + || lastLedgerSwitchTimestamp < (now - getConfig().getLedgerRolloverTimeout() * 1000)) && (STATE_UPDATER.get(this) != State.Closed && STATE_UPDATER.get(this) != State.Closing)) { // It's safe to modify the timestamp since this method will be only called from a callback, implying that // calls will be serialized on one single thread @@ -3556,7 +3555,7 @@ private ManagedCursorImpl cursorImpl() { @Override public long[] getDeletedBatchIndexesAsLongArray(PositionImpl position) { - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable bitSet = batchDeletedIndexes.get(position); return bitSet == null ? null : bitSet.toLongArray(); } else { @@ -3657,7 +3656,7 @@ public boolean isCacheReadEntry() { private static final Logger log = LoggerFactory.getLogger(ManagedCursorImpl.class); public ManagedLedgerConfig getConfig() { - return config; + return getManagedLedger().getConfig(); } /*** diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorMXBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorMXBeanImpl.java index 48465e6294b0e..a183c0d61ce16 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorMXBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorMXBeanImpl.java @@ -90,7 +90,8 @@ public long getPersistZookeeperErrors() { @Override public void addWriteCursorLedgerSize(final long size) { - writeCursorLedgerSize.add(size * ((ManagedCursorImpl) managedCursor).config.getWriteQuorumSize()); + writeCursorLedgerSize.add( + size * managedCursor.getManagedLedger().getConfig().getWriteQuorumSize()); writeCursorLedgerLogicalSize.add(size); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index b12346cadc96a..ab32806fbae84 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -575,7 +575,7 @@ public void operationComplete(List consumers, Stat s) { for (final String cursorName : consumers) { log.info("[{}] Loading cursor {}", name, cursorName); final ManagedCursorImpl cursor; - cursor = new ManagedCursorImpl(bookKeeper, config, ManagedLedgerImpl.this, cursorName); + cursor = new ManagedCursorImpl(bookKeeper, ManagedLedgerImpl.this, cursorName); cursor.recover(new VoidCallback() { @Override @@ -606,7 +606,7 @@ public void operationFailed(ManagedLedgerException exception) { log.debug("[{}] Recovering cursor {} lazily", name, cursorName); } final ManagedCursorImpl cursor; - cursor = new ManagedCursorImpl(bookKeeper, config, ManagedLedgerImpl.this, cursorName); + cursor = new ManagedCursorImpl(bookKeeper, ManagedLedgerImpl.this, cursorName); CompletableFuture cursorRecoveryFuture = new CompletableFuture<>(); uninitializedCursors.put(cursorName, cursorRecoveryFuture); @@ -988,7 +988,7 @@ public synchronized void asyncOpenCursor(final String cursorName, final InitialP if (log.isDebugEnabled()) { log.debug("[{}] Creating new cursor: {}", name, cursorName); } - final ManagedCursorImpl cursor = new ManagedCursorImpl(bookKeeper, config, this, cursorName); + final ManagedCursorImpl cursor = new ManagedCursorImpl(bookKeeper, this, cursorName); CompletableFuture cursorFuture = new CompletableFuture<>(); uninitializedCursors.put(cursorName, cursorFuture); PositionImpl position = InitialPosition.Earliest == initialPosition ? getFirstPosition() : getLastPosition(); @@ -1121,7 +1121,7 @@ public ManagedCursor newNonDurableCursor(Position startCursorPosition, String cu return cachedCursor; } - NonDurableCursorImpl cursor = new NonDurableCursorImpl(bookKeeper, config, this, cursorName, + NonDurableCursorImpl cursor = new NonDurableCursorImpl(bookKeeper, this, cursorName, (PositionImpl) startCursorPosition, initialPosition, isReadCompacted); cursor.setActive(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java index 77216ce2e4588..734eab20bc58e 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java @@ -25,7 +25,6 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCursorCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback; -import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.slf4j.Logger; @@ -35,10 +34,10 @@ public class NonDurableCursorImpl extends ManagedCursorImpl { private final boolean readCompacted; - NonDurableCursorImpl(BookKeeper bookkeeper, ManagedLedgerConfig config, ManagedLedgerImpl ledger, String cursorName, + NonDurableCursorImpl(BookKeeper bookkeeper, ManagedLedgerImpl ledger, String cursorName, PositionImpl startCursorPosition, CommandSubscribe.InitialPosition initialPosition, boolean isReadCompacted) { - super(bookkeeper, config, ledger, cursorName); + super(bookkeeper, ledger, cursorName); this.readCompacted = isReadCompacted; // Compare with "latest" position marker by using only the ledger id. Since the C++ client is using 48bits to diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index a79ba3fb5e23b..534ef3d76cb0d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -111,7 +111,8 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { callback.readEntriesComplete(entries, ctx); recycle(); }); - } else if (cursor.config.isAutoSkipNonRecoverableData() && exception instanceof NonRecoverableLedgerException) { + } else if (cursor.getConfig().isAutoSkipNonRecoverableData() + && exception instanceof NonRecoverableLedgerException) { log.warn("[{}][{}] read failed from ledger at position:{} : {}", cursor.ledger.getName(), cursor.getName(), readPosition, exception.getMessage()); final ManagedLedgerImpl ledger = (ManagedLedgerImpl) cursor.getManagedLedger(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index 02e43504482d8..f235ffc63ace5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -52,7 +52,7 @@ public RangeSetWrapper(LongPairConsumer rangeConverter, RangeBoundConsumer rangeBoundConsumer, ManagedCursorImpl managedCursor) { requireNonNull(managedCursor); - this.config = managedCursor.getConfig(); + this.config = managedCursor.getManagedLedger().getConfig(); this.rangeConverter = rangeConverter; this.rangeSet = config.isUnackedRangesOpenCacheSetEnabled() ? new ConcurrentOpenLongPairRangeSet<>(4096, rangeConverter) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java index 1661613f07d7d..2461bcf780e99 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java @@ -22,7 +22,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.mledger.AsyncCallbacks; -import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound; @@ -31,9 +30,9 @@ @Slf4j public class ReadOnlyCursorImpl extends ManagedCursorImpl implements ReadOnlyCursor { - public ReadOnlyCursorImpl(BookKeeper bookkeeper, ManagedLedgerConfig config, ManagedLedgerImpl ledger, + public ReadOnlyCursorImpl(BookKeeper bookkeeper, ManagedLedgerImpl ledger, PositionImpl startPosition, String cursorName) { - super(bookkeeper, config, ledger, cursorName); + super(bookkeeper, ledger, cursorName); if (startPosition.equals(PositionImpl.EARLIEST)) { readPosition = ledger.getFirstPosition().getNext(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java index 707b71c9d9f09..d844963599995 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java @@ -143,7 +143,7 @@ ReadOnlyCursor createReadOnlyCursor(PositionImpl startPosition) { } } - return new ReadOnlyCursorImpl(bookKeeper, config, this, startPosition, "read-only-cursor"); + return new ReadOnlyCursorImpl(bookKeeper, this, startPosition, "read-only-cursor"); } @Override diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java index aa0d04783d991..864c25c6c434b 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java @@ -56,8 +56,9 @@ void testRecoverIndividualDeletedMessages() throws Exception { ManagedLedgerImpl ledger = mock(ManagedLedgerImpl.class); doReturn(ledgersInfo).when(ledger).getLedgersInfo(); + doReturn(config).when(ledger).getConfig(); - ManagedCursorImpl cursor = spy(new ManagedCursorImpl(bookkeeper, config, ledger, "test-cursor")); + ManagedCursorImpl cursor = spy(new ManagedCursorImpl(bookkeeper, ledger, "test-cursor")); LongPairRangeSet deletedMessages = cursor.getIndividuallyDeletedMessagesSet(); Method recoverMethod = ManagedCursorImpl.class.getDeclaredMethod("recoverIndividualDeletedMessages", diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 5c10533e2476b..4c95454e33a92 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -3465,10 +3465,10 @@ public Object answer(InvocationOnMock invocation) { when(ml.getNextValidLedger(markDeleteLedgerId)).thenReturn(3L); when(ml.getNextValidPosition(lastPosition)).thenReturn(nextPosition); when(ml.ledgerExists(markDeleteLedgerId)).thenReturn(false); + when(ml.getConfig()).thenReturn(new ManagedLedgerConfig()); BookKeeper mockBookKeeper = mock(BookKeeper.class); - final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, new ManagedLedgerConfig(), ml, - cursorName); + final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, ml, cursorName); cursor.recover(new VoidCallback() { @Override @@ -4772,8 +4772,7 @@ public void testRecoverCursorWithTerminateManagedLedger() throws Exception { // Reopen the ledger. ledger = (ManagedLedgerImpl) factory.open(mlName, config); BookKeeper mockBookKeeper = mock(BookKeeper.class); - final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, new ManagedLedgerConfig(), ledger, - cursorName); + final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, ledger, cursorName); CompletableFuture recoverFuture = new CompletableFuture<>(); // Recover the cursor. diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index e983523c1b62e..122bada487a44 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -3159,7 +3159,7 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { // (2) test read-timeout for: ManagedLedger.asyncReadEntry(..) AtomicReference responseException2 = new AtomicReference<>(); PositionImpl readPositionRef = PositionImpl.EARLIEST; - ManagedCursorImpl cursor = new ManagedCursorImpl(bk, config, ledger, "cursor1"); + ManagedCursorImpl cursor = new ManagedCursorImpl(bk, ledger, "cursor1"); OpReadEntry opReadEntry = OpReadEntry.create(cursor, readPositionRef, 1, new ReadEntriesCallback() { @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java index 42b9358911a69..82892ad353aa1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java @@ -210,10 +210,8 @@ public void testSkipCorruptDataLedger() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topic1).get(); ManagedLedgerImpl ml = (ManagedLedgerImpl) topic.getManagedLedger(); ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().iterator().next(); - Field configField = ManagedCursorImpl.class.getDeclaredField("config"); - configField.setAccessible(true); // Create multiple data-ledger - ManagedLedgerConfig config = (ManagedLedgerConfig) configField.get(cursor); + ManagedLedgerConfig config = ml.getConfig(); config.setMaxEntriesPerLedger(entriesPerLedger); config.setMinimumRolloverTime(1, TimeUnit.MILLISECONDS); // bookkeeper client @@ -323,10 +321,8 @@ public void testTruncateCorruptDataLedger() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topic1).get(); ManagedLedgerImpl ml = (ManagedLedgerImpl) topic.getManagedLedger(); ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().iterator().next(); - Field configField = ManagedCursorImpl.class.getDeclaredField("config"); - configField.setAccessible(true); // Create multiple data-ledger - ManagedLedgerConfig config = (ManagedLedgerConfig) configField.get(cursor); + ManagedLedgerConfig config = ml.getConfig(); config.setMaxEntriesPerLedger(entriesPerLedger); config.setMinimumRolloverTime(1, TimeUnit.MILLISECONDS); // bookkeeper client diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index d523586c2e2d3..5b750a0b9c2e5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -66,6 +66,7 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.PrometheusMetricsTestUtil; @@ -754,6 +755,30 @@ public void testDynamicConfigurationAutoSkipNonRecoverableData() throws Exceptio admin.topics().delete(topicName); } + @Test + public void testCursorGetConfigAfterTopicPoliciesChanged() throws Exception { + final String topicName = "persistent://prop/ns-abc/" + UUID.randomUUID(); + final String subName = "test_sub"; + + @Cleanup + Consumer subscribe = pulsarClient.newConsumer().topic(topicName).subscriptionName(subName).subscribe(); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get(); + PersistentSubscription subscription = persistentTopic.getSubscription(subName); + + int maxConsumers = 100; + admin.topicPolicies().setMaxConsumers(topicName, 100); + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.topicPolicies().getMaxConsumers(topicName, false), maxConsumers); + }); + + ManagedCursorImpl cursor = (ManagedCursorImpl) subscription.getCursor(); + assertEquals(cursor.getConfig(), persistentTopic.getManagedLedger().getConfig()); + + subscribe.close(); + admin.topics().delete(topicName); + } + @Test public void testAddWaitingCursorsForNonDurable() throws Exception { final String ns = "prop/ns-test"; From b7ec89a908255f160d8337bdd96fa10f5772a265 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Fri, 10 May 2024 14:58:57 +0800 Subject: [PATCH 177/580] [fix] [doc] fix the class name of transaction exception. (#22687) --- .../transaction/buffer/TransactionBuffer.java | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java index ae0b9bbf1ca2a..3fe989acc9227 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java @@ -24,6 +24,8 @@ import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.transaction.exception.TransactionException; +import org.apache.pulsar.broker.transaction.exception.buffer.TransactionBufferException; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.common.policies.data.TransactionBufferStats; import org.apache.pulsar.common.policies.data.TransactionInBufferStats; @@ -56,8 +58,7 @@ public interface TransactionBuffer { * * @param txnID the transaction id * @return a future represents the result of the operation - * @throws org.apache.pulsar.broker.transaction.buffer.exceptions.TransactionNotFoundException if the transaction - * is not in the buffer. + * @throws TransactionBufferException.TransactionNotFoundException if the transaction is not in the buffer. */ CompletableFuture getTransactionMeta(TxnID txnID); @@ -70,8 +71,7 @@ public interface TransactionBuffer { * @param sequenceId the sequence id of the entry in this transaction buffer. * @param buffer the entry buffer * @return a future represents the result of the operation. - * @throws org.apache.pulsar.broker.transaction.buffer.exceptions.TransactionSealedException if the transaction - * has been sealed. + * @throws TransactionException.TransactionSealedException if the transaction has been sealed. */ CompletableFuture appendBufferToTxn(TxnID txnId, long sequenceId, ByteBuf buffer); @@ -82,8 +82,7 @@ public interface TransactionBuffer { * @param txnID transaction id * @param startSequenceId the sequence id to start read * @return a future represents the result of open operation. - * @throws org.apache.pulsar.broker.transaction.buffer.exceptions.TransactionNotFoundException if the transaction - * is not in the buffer. + * @throws TransactionBufferException.TransactionNotFoundException if the transaction is not in the buffer. */ CompletableFuture openTransactionBufferReader(TxnID txnID, long startSequenceId); @@ -95,8 +94,7 @@ public interface TransactionBuffer { * @param txnID the transaction id * @param lowWaterMark the low water mark of this transaction * @return a future represents the result of commit operation. - * @throws org.apache.pulsar.broker.transaction.buffer.exceptions.TransactionNotFoundException if the transaction - * is not in the buffer. + * @throws TransactionBufferException.TransactionNotFoundException if the transaction is not in the buffer. */ CompletableFuture commitTxn(TxnID txnID, long lowWaterMark); @@ -107,8 +105,7 @@ public interface TransactionBuffer { * @param txnID the transaction id * @param lowWaterMark the low water mark of this transaction * @return a future represents the result of abort operation. - * @throws org.apache.pulsar.broker.transaction.buffer.exceptions.TransactionNotFoundException if the transaction - * is not in the buffer. + * @throws TransactionBufferException.TransactionNotFoundException if the transaction is not in the buffer. */ CompletableFuture abortTxn(TxnID txnID, long lowWaterMark); From b56f238f6aaffdc0b37b9f6e2185b219f8708570 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 10 May 2024 04:10:31 -0700 Subject: [PATCH 178/580] [fix][client] Fix ReaderBuilder doest not give illegalArgument on connection failure retry (#22639) --- .../apache/pulsar/client/impl/ReaderTest.java | 27 +++++++++++++++++++ .../pulsar/client/impl/ReaderBuilderImpl.java | 5 ++-- .../pulsar/client/impl/BuildersTest.java | 2 +- 3 files changed, 31 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java index 2d3e8d4c6e978..12228220b18bd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java @@ -36,6 +36,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -48,6 +50,7 @@ import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Range; import org.apache.pulsar.client.api.Reader; @@ -902,4 +905,28 @@ public void testHasMessageAvailableAfterSeekTimestamp(boolean initializeLastMess assertTrue(reader.hasMessageAvailable()); } } + + @Test + public void testReaderBuilderStateOnRetryFailure() throws Exception { + String ns = "my-property/my-ns"; + String topic = "persistent://" + ns + "/testRetryReader"; + RetentionPolicies retention = new RetentionPolicies(-1, -1); + admin.namespaces().setRetention(ns, retention); + String badUrl = "pulsar://bad-host:8080"; + + PulsarClient client = PulsarClient.builder().serviceUrl(badUrl).build(); + + ReaderBuilder readerBuilder = client.newReader().topic(topic).startMessageFromRollbackDuration(100, + TimeUnit.SECONDS); + + for (int i = 0; i < 3; i++) { + try { + readerBuilder.createAsync().get(1, TimeUnit.SECONDS); + } catch (TimeoutException e) { + log.info("It should time out due to invalid url"); + } catch (IllegalArgumentException e) { + fail("It should not fail with corrupt reader state"); + } + } + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java index 2860cda0ceef1..ef230475be53b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java @@ -86,8 +86,9 @@ public CompletableFuture> createAsync() { .failedFuture(new IllegalArgumentException("Topic name must be set on the reader builder")); } - if (conf.getStartMessageId() != null && conf.getStartMessageFromRollbackDurationInSec() > 0 - || conf.getStartMessageId() == null && conf.getStartMessageFromRollbackDurationInSec() <= 0) { + boolean isStartMsgIdExist = conf.getStartMessageId() != null && conf.getStartMessageId() != MessageId.earliest; + if ((isStartMsgIdExist && conf.getStartMessageFromRollbackDurationInSec() > 0) + || (conf.getStartMessageId() == null && conf.getStartMessageFromRollbackDurationInSec() <= 0)) { return FutureUtil .failedFuture(new IllegalArgumentException( "Start message id or start message from roll back must be specified but they cannot be" diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BuildersTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BuildersTest.java index 607689e0e2b3b..5f52f86d8b014 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BuildersTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BuildersTest.java @@ -106,7 +106,7 @@ public void readerBuilderLoadConfTest() throws Exception { @Test(expectedExceptions = {PulsarClientException.class}, expectedExceptionsMessageRegExp = ".* must be specified but they cannot be specified at the same time.*") public void shouldNotSetTwoOptAtTheSameTime() throws Exception { PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://localhost:6650").build(); - try (Reader reader = client.newReader().topic("abc").startMessageId(MessageId.earliest) + try (Reader reader = client.newReader().topic("abc").startMessageId(MessageId.latest) .startMessageFromRollbackDuration(10, TimeUnit.HOURS).create()) { // no-op } finally { From 2cfd9597676828bae68c9dac74e41d65a1a29864 Mon Sep 17 00:00:00 2001 From: Nikhil Erigila <60037808+nikhilerigila09@users.noreply.github.com> Date: Fri, 10 May 2024 16:41:20 +0530 Subject: [PATCH 179/580] [fix][sec] Upgrade postgresql version to avoid CVE-2024-1597 (#22635) --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index c2f563eb60edc..63b44788f1410 100644 --- a/pom.xml +++ b/pom.xml @@ -190,7 +190,7 @@ flexible messaging model and an intuitive client API. 5.1.0 3.42.0.0 8.0.11 - 42.5.1 + 42.5.5 0.4.6 2.7.5 0.4.4-hotfix1 @@ -199,7 +199,7 @@ flexible messaging model and an intuitive client API. 1.2.4 8.12.1 1.9.7.Final - 42.5.0 + 42.5.5 8.0.30 1.15.16.Final From d77c5de5d713043237773dc057caa1920134bfe3 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sat, 11 May 2024 01:27:52 +0800 Subject: [PATCH 180/580] [improve] [log] Print source client addr when enabled haProxyProtocolEnabled (#22686) --- .../pulsar/broker/service/Consumer.java | 2 +- .../pulsar/broker/service/Producer.java | 2 +- .../pulsar/broker/service/ServerCnx.java | 35 ++++++++++++++++--- .../service/ServerCnxThrottleTracker.java | 2 +- .../broker/service/TopicListService.java | 20 +++++------ .../pulsar/common/protocol/PulsarDecoder.java | 2 +- .../pulsar/common/protocol/PulsarHandler.java | 31 +++++++++++----- 7 files changed, 67 insertions(+), 27 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index b1c3687b3a0f6..89a9bab497d68 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -934,7 +934,7 @@ public KeySharedMeta getKeySharedMeta() { public String toString() { if (subscription != null && cnx != null) { return MoreObjects.toStringHelper(this).add("subscription", subscription).add("consumerId", consumerId) - .add("consumerName", consumerName).add("address", this.cnx.clientAddress()).toString(); + .add("consumerName", consumerName).add("address", this.cnx.toString()).toString(); } else { return MoreObjects.toStringHelper(this).add("consumerId", consumerId) .add("consumerName", consumerName).toString(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java index 9cfde67802bb0..c10e33818ed3a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java @@ -666,7 +666,7 @@ public Map getMetadata() { @Override public String toString() { - return MoreObjects.toStringHelper(this).add("topic", topic).add("client", cnx.clientAddress()) + return MoreObjects.toStringHelper(this).add("topic", topic).add("client", cnx.toString()) .add("producerName", producerName).add("producerId", producerId).toString(); } 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 5ccdbfbe715c5..59411aec0405f 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 @@ -1201,7 +1201,7 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { remoteAddress, getPrincipal()); } - log.info("[{}] Subscribing on topic {} / {}. consumerId: {}", this.ctx().channel().toString(), + log.info("[{}] Subscribing on topic {} / {}. consumerId: {}", this.toString(), topicName, subscriptionName, consumerId); try { Metadata.validateMetadata(metadata, @@ -1921,7 +1921,7 @@ protected void handleAck(CommandAck ack) { if (log.isDebugEnabled()) { log.debug("Consumer future is not complete(not complete or error), but received command ack. so discard" + " this command. consumerId: {}, cnx: {}, messageIdCount: {}", ack.getConsumerId(), - this.ctx().channel().toString(), ack.getMessageIdsCount()); + this.toString(), ack.getMessageIdsCount()); } } } @@ -2267,7 +2267,7 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { @Override public String toString() { return String.format("ServerCnx [%s] get largest batch index when possible", - ServerCnx.this.ctx.channel()); + ServerCnx.this.toString()); } }, null); @@ -3301,7 +3301,7 @@ private void disableTcpNoDelayIfNeeded(String topic, String producerName) { } } catch (Throwable t) { log.warn("[{}] [{}] Failed to remove TCP no-delay property on client cnx {}", topic, producerName, - ctx.channel()); + this.toString()); } } } @@ -3364,6 +3364,31 @@ public SocketAddress getRemoteAddress() { return remoteAddress; } + /** + * Demo: [id: 0x2561bcd1, L:/10.0.136.103:6650 ! R:/240.240.0.5:58038] [SR:/240.240.0.5:58038]. + * L: local Address. + * R: remote address. + * SR: source remote address. It is the source address when enabled "haProxyProtocolEnabled". + */ + @Override + public String toString() { + ChannelHandlerContext ctx = ctx(); + // ctx.channel(): 96. + // clientSourceAddress: 5 + 46(ipv6). + // state: 19. + // Len = 166. + StringBuilder buf = new StringBuilder(166); + if (ctx == null) { + buf.append("[ctx: null]"); + } else { + buf.append(ctx.channel().toString()); + } + String clientSourceAddr = clientSourceAddress(); + buf.append(" [SR:").append(clientSourceAddr == null ? "-" : clientSourceAddr) + .append(", state:").append(state).append("]"); + return buf.toString(); + } + @Override public BrokerService getBrokerService() { return service; @@ -3510,7 +3535,7 @@ public CompletableFuture> checkConnectionLiveness() { ctx.executor().schedule(() -> { if (finalConnectionCheckInProgress == connectionCheckInProgress && !finalConnectionCheckInProgress.isDone()) { - log.warn("[{}] Connection check timed out. Closing connection.", remoteAddress); + log.warn("[{}] Connection check timed out. Closing connection.", this.toString()); ctx.close(); } }, connectionLivenessCheckTimeoutMillis, TimeUnit.MILLISECONDS); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnxThrottleTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnxThrottleTracker.java index f223d6eee3795..7e55397022d5e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnxThrottleTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnxThrottleTracker.java @@ -87,7 +87,7 @@ public void decrementThrottleCount() { private void changeAutoRead(boolean autoRead) { if (isChannelActive()) { if (log.isDebugEnabled()) { - log.debug("[{}] Setting auto read to {}", serverCnx.ctx().channel(), autoRead); + log.debug("[{}] Setting auto read to {}", serverCnx.toString(), autoRead); } // change the auto read flag on the channel serverCnx.ctx().channel().config().setAutoRead(autoRead); 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 aea5b9fc65b46..b18286ee06259 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 @@ -131,7 +131,7 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo } else { msg += "Pattern longer than maximum: " + maxSubscriptionPatternLength; } - log.warn("[{}] {} on namespace {}", connection.getRemoteAddress(), msg, namespaceName); + log.warn("[{}] {} on namespace {}", connection.toString(), msg, namespaceName); connection.getCommandSender().sendErrorResponse(requestId, ServerError.NotAllowedError, msg); lookupSemaphore.release(); return; @@ -144,14 +144,14 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo TopicListWatcher watcher = existingWatcherFuture.getNow(null); log.info("[{}] Watcher with the same id is already created:" + " watcherId={}, watcher={}", - connection.getRemoteAddress(), 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.getRemoteAddress(), watcherId); + + " consumerId={}", connection.toString(), watcherId); ServerError error; if (!existingWatcherFuture.isDone()) { error = ServerError.ServiceNotReady; @@ -179,14 +179,14 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo if (log.isDebugEnabled()) { log.debug( "[{}] Received WatchTopicList for namespace [//{}] by {}", - connection.getRemoteAddress(), namespaceName, requestId); + connection.toString(), namespaceName, requestId); } connection.getCommandSender().sendWatchTopicListSuccess(requestId, watcherId, hash, topicList); lookupSemaphore.release(); }) .exceptionally(ex -> { log.warn("[{}] Error WatchTopicList for namespace [//{}] by {}", - connection.getRemoteAddress(), namespaceName, requestId); + connection.toString(), namespaceName, requestId); connection.getCommandSender().sendErrorResponse(requestId, BrokerServiceException.getClientErrorCode( new BrokerServiceException.ServerMetadataException(ex)), ex.getMessage()); @@ -213,7 +213,7 @@ public void initializeTopicsListWatcher(CompletableFuture watc } else { if (!watcherFuture.complete(watcher)) { log.warn("[{}] Watcher future was already completed. Deregistering watcherId={}.", - connection.getRemoteAddress(), watcherId); + connection.toString(), watcherId); topicResources.deregisterPersistentTopicListener(watcher); } } @@ -232,7 +232,7 @@ public void deleteTopicListWatcher(Long watcherId) { CompletableFuture watcherFuture = watchers.get(watcherId); if (watcherFuture == null) { log.info("[{}] TopicListWatcher was not registered on the connection: {}", - watcherId, connection.getRemoteAddress()); + watcherId, connection.toString()); return; } @@ -242,14 +242,14 @@ public void deleteTopicListWatcher(Long watcherId) { // watcher future as failed and we can tell the client the close operation was successful. When the actual // create operation will complete, the new watcher will be discarded. log.info("[{}] Closed watcher before its creation was completed. watcherId={}", - connection.getRemoteAddress(), watcherId); + connection.toString(), watcherId); watchers.remove(watcherId); return; } if (watcherFuture.isCompletedExceptionally()) { log.info("[{}] Closed watcher that already failed to be created. watcherId={}", - connection.getRemoteAddress(), watcherId); + connection.toString(), watcherId); watchers.remove(watcherId); return; } @@ -257,7 +257,7 @@ public void deleteTopicListWatcher(Long watcherId) { // Proceed with normal watcher close topicResources.deregisterPersistentTopicListener(watcherFuture.getNow(null)); watchers.remove(watcherId); - log.info("[{}] Closed watcher, watcherId={}", connection.getRemoteAddress(), watcherId); + log.info("[{}] Closed watcher, watcherId={}", connection.toString(), watcherId); } /** diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/PulsarDecoder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/PulsarDecoder.java index c1c1ebe355bb9..c05b1d796dfdd 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/PulsarDecoder.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/PulsarDecoder.java @@ -122,7 +122,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception cmd.parseFrom(buffer, cmdSize); if (log.isDebugEnabled()) { - log.debug("[{}] Received cmd {}", ctx.channel().remoteAddress(), cmd.getType()); + log.debug("[{}] Received cmd {}", ctx.channel(), cmd.getType()); } messageReceived(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/PulsarHandler.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/PulsarHandler.java index 51cd61afd6362..d5c741be01e22 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/PulsarHandler.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/PulsarHandler.java @@ -67,7 +67,7 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { this.ctx = ctx; if (log.isDebugEnabled()) { - log.debug("[{}] Scheduling keep-alive task every {} s", ctx.channel(), keepAliveIntervalSeconds); + log.debug("[{}] Scheduling keep-alive task every {} s", this.toString(), keepAliveIntervalSeconds); } if (keepAliveIntervalSeconds > 0) { this.keepAliveTask = ctx.executor() @@ -85,13 +85,13 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { protected final void handlePing(CommandPing ping) { // Immediately reply success to ping requests if (log.isDebugEnabled()) { - log.debug("[{}] Replying back to ping message", ctx.channel()); + log.debug("[{}] Replying back to ping message", this.toString()); } ctx.writeAndFlush(Commands.newPong()) .addListener(future -> { if (!future.isSuccess()) { log.warn("[{}] Forcing connection to close since cannot send a pong message.", - ctx.channel(), future.cause()); + toString(), future.cause()); ctx.close(); } }); @@ -107,24 +107,24 @@ private void handleKeepAliveTimeout() { } if (!isHandshakeCompleted()) { - log.warn("[{}] Pulsar Handshake was not completed within timeout, closing connection", ctx.channel()); + log.warn("[{}] Pulsar Handshake was not completed within timeout, closing connection", this.toString()); ctx.close(); } else if (waitingForPingResponse && ctx.channel().config().isAutoRead()) { // We were waiting for a response and another keep-alive just completed. // If auto-read was disabled, it means we stopped reading from the connection, so we might receive the Ping // response later and thus not enforce the strict timeout here. - log.warn("[{}] Forcing connection to close after keep-alive timeout", ctx.channel()); + log.warn("[{}] Forcing connection to close after keep-alive timeout", this.toString()); ctx.close(); } else if (getRemoteEndpointProtocolVersion() >= ProtocolVersion.v1.getValue()) { // Send keep alive probe to peer only if it supports the ping/pong commands, added in v1 if (log.isDebugEnabled()) { - log.debug("[{}] Sending ping message", ctx.channel()); + log.debug("[{}] Sending ping message", this.toString()); } waitingForPingResponse = true; sendPing(); } else { if (log.isDebugEnabled()) { - log.debug("[{}] Peer doesn't support keep-alive", ctx.channel()); + log.debug("[{}] Peer doesn't support keep-alive", this.toString()); } } } @@ -134,7 +134,7 @@ protected ChannelFuture sendPing() { .addListener(future -> { if (!future.isSuccess()) { log.warn("[{}] Forcing connection to close since cannot send a ping message.", - ctx.channel(), future.cause()); + this.toString(), future.cause()); ctx.close(); } }); @@ -152,5 +152,20 @@ public void cancelKeepAliveTask() { */ protected abstract boolean isHandshakeCompleted(); + /** + * Demo: [id: 0x2561bcd1, L:/10.0.136.103:6650 ! R:/240.240.0.5:58038]. + * L: local Address. + * R: remote address. + */ + @Override + public String toString() { + ChannelHandlerContext ctx = this.ctx; + if (ctx == null) { + return "[ctx: null]"; + } else { + return ctx.channel().toString(); + } + } + private static final Logger log = LoggerFactory.getLogger(PulsarHandler.class); } From e558cfe9836256065befb3ff6d6043eca10aa5ef Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 10 May 2024 15:35:03 -0700 Subject: [PATCH 181/580] [feat][broker] PIP-264: Add OpenTelemetry consumer metrics (#22693) --- .../apache/pulsar/broker/PulsarService.java | 8 + .../pulsar/broker/service/Consumer.java | 32 +++- .../stats/OpenTelemetryConsumerStats.java | 170 ++++++++++++++++++ .../stats/OpenTelemetryConsumerStatsTest.java | 151 ++++++++++++++++ .../broker/testcontext/PulsarTestContext.java | 1 + .../client/api/BrokerServiceLookupTest.java | 1 + .../OpenTelemetryAttributes.java | 46 +++++ 7 files changed, 408 insertions(+), 1 deletion(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStats.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStatsTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index ac37aca531af9..6ee35ad295fb5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -109,6 +109,7 @@ import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.SchemaStorageFactory; import org.apache.pulsar.broker.stats.MetricsGenerator; +import org.apache.pulsar.broker.stats.OpenTelemetryConsumerStats; import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; import org.apache.pulsar.broker.stats.PulsarBrokerOpenTelemetry; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; @@ -254,6 +255,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private MetricsGenerator metricsGenerator; private final PulsarBrokerOpenTelemetry openTelemetry; private OpenTelemetryTopicStats openTelemetryTopicStats; + private OpenTelemetryConsumerStats openTelemetryConsumerStats; private TransactionMetadataStoreService transactionMetadataStoreService; private TransactionBufferProvider transactionBufferProvider; @@ -630,8 +632,13 @@ public CompletableFuture closeAsync() { brokerClientSharedTimer.stop(); monotonicSnapshotClock.close(); + if (openTelemetryConsumerStats != null) { + openTelemetryConsumerStats.close(); + openTelemetryConsumerStats = null; + } if (openTelemetryTopicStats != null) { openTelemetryTopicStats.close(); + openTelemetryTopicStats = null; } asyncCloseFutures.add(EventLoopUtil.shutdownGracefully(ioEventLoopGroup)); @@ -775,6 +782,7 @@ public void start() throws PulsarServerException { } openTelemetryTopicStats = new OpenTelemetryTopicStats(this); + openTelemetryConsumerStats = new OpenTelemetryConsumerStats(this); localMetadataSynchronizer = StringUtils.isNotBlank(config.getMetadataSyncEventTopic()) ? new PulsarMetadataEventSynchronizer(this, config.getMetadataSyncEventTopic()) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index 89a9bab497d68..fe9fbe6a4000c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -25,6 +25,7 @@ import com.google.common.util.concurrent.AtomicDouble; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.Promise; +import java.time.Instant; import java.util.ArrayList; import java.util.BitSet; import java.util.Collections; @@ -90,7 +91,9 @@ public class Consumer { private final Rate msgOut; private final Rate msgRedeliver; private final LongAdder msgOutCounter; + private final LongAdder msgRedeliverCounter; private final LongAdder bytesOutCounter; + private final LongAdder messageAckCounter; private final Rate messageAckRate; private volatile long lastConsumedTimestamp; @@ -152,6 +155,9 @@ public class Consumer { @Getter private final SchemaType schemaType; + @Getter + private final Instant connectedSince = Instant.now(); + public Consumer(Subscription subscription, SubType subType, String topicName, long consumerId, int priorityLevel, String consumerName, boolean isDurable, TransportCnx cnx, String appId, @@ -182,8 +188,10 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo this.msgOut = new Rate(); this.chunkedMessageRate = new Rate(); this.msgRedeliver = new Rate(); + this.msgRedeliverCounter = new LongAdder(); this.bytesOutCounter = new LongAdder(); this.msgOutCounter = new LongAdder(); + this.messageAckCounter = new LongAdder(); this.messageAckRate = new Rate(); this.appId = appId; @@ -200,7 +208,7 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo stats = new ConsumerStatsImpl(); stats.setAddress(cnx.clientSourceAddressAndPort()); stats.consumerName = consumerName; - stats.setConnectedSince(DateFormatter.now()); + stats.setConnectedSince(DateFormatter.format(connectedSince)); stats.setClientVersion(cnx.getClientVersion()); stats.metadata = this.metadata; @@ -238,8 +246,10 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo this.consumerName = consumerName; this.msgOut = null; this.msgRedeliver = null; + this.msgRedeliverCounter = null; this.msgOutCounter = null; this.bytesOutCounter = null; + this.messageAckCounter = null; this.messageAckRate = null; this.pendingAcks = null; this.stats = null; @@ -502,6 +512,7 @@ public CompletableFuture messageAcked(CommandAck ack) { return future .thenApply(v -> { this.messageAckRate.recordEvent(v); + this.messageAckCounter.add(v); return null; }); } @@ -922,6 +933,14 @@ public long getBytesOutCounter() { return bytesOutCounter.longValue(); } + public long getMessageAckCounter() { + return messageAckCounter.sum(); + } + + public long getMessageRedeliverCounter() { + return msgRedeliverCounter.sum(); + } + public int getUnackedMessages() { return unackedMessages; } @@ -1059,6 +1078,8 @@ public void redeliverUnacknowledgedMessages(long consumerEpoch) { } msgRedeliver.recordMultipleEvents(totalRedeliveryMessages.intValue(), totalRedeliveryMessages.intValue()); + msgRedeliverCounter.add(totalRedeliveryMessages.intValue()); + subscription.redeliverUnacknowledgedMessages(this, pendingPositions); } else { subscription.redeliverUnacknowledgedMessages(this, consumerEpoch); @@ -1091,6 +1112,7 @@ public void redeliverUnacknowledgedMessages(List messageIds) { subscription.redeliverUnacknowledgedMessages(this, pendingPositions); msgRedeliver.recordMultipleEvents(totalRedeliveryMessages, totalRedeliveryMessages); + msgRedeliverCounter.add(totalRedeliveryMessages); int numberOfBlockedPermits = PERMITS_RECEIVED_WHILE_CONSUMER_BLOCKED_UPDATER.getAndSet(this, 0); @@ -1153,6 +1175,14 @@ public String getClientAddress() { return clientAddress; } + public String getClientAddressAndPort() { + return cnx.clientSourceAddressAndPort(); + } + + public String getClientVersion() { + return cnx.getClientVersion(); + } + public MessageId getStartMessageId() { return startMessageId; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStats.java new file mode 100644 index 0000000000000..25af3959db32d --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStats.java @@ -0,0 +1,170 @@ +/* + * 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.stats; + +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.BatchCallback; +import io.opentelemetry.api.metrics.ObservableLongMeasurement; +import java.util.Collection; +import java.util.Optional; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; + +public class OpenTelemetryConsumerStats implements AutoCloseable { + + // Replaces pulsar_consumer_msg_rate_out + public static final String MESSAGE_OUT_COUNTER = "pulsar.broker.consumer.message.outgoing.count"; + private final ObservableLongMeasurement messageOutCounter; + + // Replaces pulsar_consumer_msg_throughput_out + public static final String BYTES_OUT_COUNTER = "pulsar.broker.consumer.message.outgoing.size"; + private final ObservableLongMeasurement bytesOutCounter; + + // Replaces pulsar_consumer_msg_ack_rate + public static final String MESSAGE_ACK_COUNTER = "pulsar.broker.consumer.message.ack.count"; + private final ObservableLongMeasurement messageAckCounter; + + // Replaces pulsar_consumer_msg_rate_redeliver + public static final String MESSAGE_REDELIVER_COUNTER = "pulsar.broker.consumer.message.redeliver.count"; + private final ObservableLongMeasurement messageRedeliverCounter; + + // Replaces pulsar_consumer_unacked_messages + public static final String MESSAGE_UNACKNOWLEDGED_COUNTER = "pulsar.broker.consumer.message.unack.count"; + private final ObservableLongMeasurement messageUnacknowledgedCounter; + + // Replaces pulsar_consumer_available_permits + public static final String MESSAGE_PERMITS_COUNTER = "pulsar.broker.consumer.permit.count"; + private final ObservableLongMeasurement messagePermitsCounter; + + private final BatchCallback batchCallback; + + public OpenTelemetryConsumerStats(PulsarService pulsar) { + var meter = pulsar.getOpenTelemetry().getMeter(); + + messageOutCounter = meter + .counterBuilder(MESSAGE_OUT_COUNTER) + .setUnit("{message}") + .setDescription("The total number of messages dispatched to this consumer.") + .buildObserver(); + + bytesOutCounter = meter + .counterBuilder(BYTES_OUT_COUNTER) + .setUnit("By") + .setDescription("The total number of messages bytes dispatched to this consumer.") + .buildObserver(); + + messageAckCounter = meter + .counterBuilder(MESSAGE_ACK_COUNTER) + .setUnit("{ack}") + .setDescription("The total number of message acknowledgments received from this consumer.") + .buildObserver(); + + messageRedeliverCounter = meter + .counterBuilder(MESSAGE_REDELIVER_COUNTER) + .setUnit("{message}") + .setDescription("The total number of messages that have been redelivered to this consumer.") + .buildObserver(); + + messageUnacknowledgedCounter = meter + .upDownCounterBuilder(MESSAGE_UNACKNOWLEDGED_COUNTER) + .setUnit("{message}") + .setDescription("The total number of messages unacknowledged by this consumer.") + .buildObserver(); + + messagePermitsCounter = meter + .upDownCounterBuilder(MESSAGE_PERMITS_COUNTER) + .setUnit("{permit}") + .setDescription("The number of permits currently available for this consumer.") + .buildObserver(); + + batchCallback = meter.batchCallback(() -> pulsar.getBrokerService() + .getTopics() + .values() + .stream() + .map(topicFuture -> topicFuture.getNow(Optional.empty())) + .filter(Optional::isPresent) + .map(Optional::get) + .map(Topic::getSubscriptions) + .flatMap(s -> s.values().stream()) + .map(Subscription::getConsumers) + .flatMap(Collection::stream) + .forEach(this::recordMetricsForConsumer), + messageOutCounter, + bytesOutCounter, + messageAckCounter, + messageRedeliverCounter, + messageUnacknowledgedCounter, + messagePermitsCounter); + } + + @Override + public void close() { + batchCallback.close(); + } + + private void recordMetricsForConsumer(Consumer consumer) { + var subscription = consumer.getSubscription(); + var topicName = TopicName.get(subscription.getTopic().getName()); + + var builder = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_CONSUMER_NAME, consumer.consumerName()) + .put(OpenTelemetryAttributes.PULSAR_CONSUMER_ID, consumer.consumerId()) + .put(OpenTelemetryAttributes.PULSAR_CONSUMER_CONNECTED_SINCE, + consumer.getConnectedSince().getEpochSecond()) + .put(OpenTelemetryAttributes.PULSAR_SUBSCRIPTION_NAME, subscription.getName()) + .put(OpenTelemetryAttributes.PULSAR_SUBSCRIPTION_TYPE, consumer.subType().toString()) + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, topicName.getDomain().toString()) + .put(OpenTelemetryAttributes.PULSAR_TENANT, topicName.getTenant()) + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, topicName.getNamespace()) + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName.getPartitionedTopicName()); + if (topicName.isPartitioned()) { + builder.put(OpenTelemetryAttributes.PULSAR_PARTITION_INDEX, topicName.getPartitionIndex()); + } + var clientAddress = consumer.getClientAddressAndPort(); + if (clientAddress != null) { + builder.put(OpenTelemetryAttributes.PULSAR_CLIENT_ADDRESS, clientAddress); + } + var clientVersion = consumer.getClientVersion(); + if (clientVersion != null) { + builder.put(OpenTelemetryAttributes.PULSAR_CLIENT_VERSION, clientVersion); + } + var metadataList = consumer.getMetadata() + .entrySet() + .stream() + .map(e -> String.format("%s:%s", e.getKey(), e.getValue())) + .toList(); + builder.put(OpenTelemetryAttributes.PULSAR_CONSUMER_METADATA, metadataList); + var attributes = builder.build(); + + messageOutCounter.record(consumer.getMsgOutCounter(), attributes); + bytesOutCounter.record(consumer.getBytesOutCounter(), attributes); + messageAckCounter.record(consumer.getMessageAckCounter(), attributes); + messageRedeliverCounter.record(consumer.getMessageRedeliverCounter(), attributes); + messageUnacknowledgedCounter.record(consumer.getUnackedMessages(), + Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_CONSUMER_BLOCKED, consumer.isBlocked()) + .build()); + messagePermitsCounter.record(consumer.getAvailablePermits(), attributes); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStatsTest.java new file mode 100644 index 0000000000000..5fcc6754b08fd --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStatsTest.java @@ -0,0 +1,151 @@ +/* + * 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.stats; + +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.Mockito.doAnswer; +import io.opentelemetry.api.common.Attributes; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import lombok.Cleanup; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.intercept.BrokerInterceptor; +import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; +import org.awaitility.Awaitility; +import org.mockito.Mockito; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class OpenTelemetryConsumerStatsTest extends BrokerTestBase { + + private BrokerInterceptor brokerInterceptor; + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + brokerInterceptor = + Mockito.mock(BrokerInterceptor.class, Mockito.withSettings().defaultAnswer(Mockito.CALLS_REAL_METHODS)); + super.baseSetup(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder builder) { + super.customizeMainPulsarTestContextBuilder(builder); + builder.enableOpenTelemetry(true); + builder.brokerInterceptor(brokerInterceptor); + } + + @Test(timeOut = 30_000) + public void testMessagingMetrics() throws Exception { + var topicName = BrokerTestUtil.newUniqueName("persistent://prop/ns-abc/testConsumerMessagingMetrics"); + admin.topics().createNonPartitionedTopic(topicName); + + var messageCount = 5; + var ackCount = 3; + + var subscriptionName = BrokerTestUtil.newUniqueName("test"); + var receiverQueueSize = 100; + + // Intercept calls to create consumer, in order to fetch client information. + var consumerRef = new AtomicReference(); + doAnswer(invocation -> { + consumerRef.compareAndSet(null, invocation.getArgument(1)); + return null; + }).when(brokerInterceptor) + .consumerCreated(any(), argThat(arg -> arg.getSubscription().getName().equals(subscriptionName)), any()); + + @Cleanup + var consumer = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionName(subscriptionName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionType(SubscriptionType.Shared) + .ackTimeout(1, TimeUnit.SECONDS) + .receiverQueueSize(receiverQueueSize) + .property("prop1", "value1") + .subscribe(); + + Awaitility.await().until(() -> consumerRef.get() != null); + var serverConsumer = consumerRef.get(); + + @Cleanup + var producer = pulsarClient.newProducer() + .topic(topicName) + .create(); + for (int i = 0; i < messageCount; i++) { + producer.send(String.format("msg-%d", i).getBytes()); + var message = consumer.receive(); + if (i < ackCount) { + consumer.acknowledge(message); + } + } + + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, "persistent") + .put(OpenTelemetryAttributes.PULSAR_TENANT, "prop") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "prop/ns-abc") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName) + .put(OpenTelemetryAttributes.PULSAR_SUBSCRIPTION_NAME, subscriptionName) + .put(OpenTelemetryAttributes.PULSAR_SUBSCRIPTION_TYPE, SubscriptionType.Shared.toString()) + .put(OpenTelemetryAttributes.PULSAR_CONSUMER_NAME, consumer.getConsumerName()) + .put(OpenTelemetryAttributes.PULSAR_CONSUMER_ID, 0) + .put(OpenTelemetryAttributes.PULSAR_CONSUMER_CONNECTED_SINCE, + serverConsumer.getConnectedSince().getEpochSecond()) + .put(OpenTelemetryAttributes.PULSAR_CLIENT_ADDRESS, serverConsumer.getClientAddressAndPort()) + .put(OpenTelemetryAttributes.PULSAR_CLIENT_VERSION, serverConsumer.getClientVersion()) + .put(OpenTelemetryAttributes.PULSAR_CONSUMER_METADATA, List.of("prop1:value1")) + .build(); + + Awaitility.await().untilAsserted(() -> { + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + + assertMetricLongSumValue(metrics, OpenTelemetryConsumerStats.MESSAGE_OUT_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + assertMetricLongSumValue(metrics, OpenTelemetryConsumerStats.BYTES_OUT_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + + assertMetricLongSumValue(metrics, OpenTelemetryConsumerStats.MESSAGE_ACK_COUNTER, attributes, ackCount); + assertMetricLongSumValue(metrics, OpenTelemetryConsumerStats.MESSAGE_PERMITS_COUNTER, attributes, + actual -> assertThat(actual).isGreaterThanOrEqualTo(receiverQueueSize - messageCount - ackCount)); + + var unAckCount = messageCount - ackCount; + assertMetricLongSumValue(metrics, OpenTelemetryConsumerStats.MESSAGE_UNACKNOWLEDGED_COUNTER, + attributes.toBuilder().put(OpenTelemetryAttributes.PULSAR_CONSUMER_BLOCKED, false).build(), + unAckCount); + assertMetricLongSumValue(metrics, OpenTelemetryConsumerStats.MESSAGE_REDELIVER_COUNTER, attributes, + actual -> assertThat(actual).isGreaterThanOrEqualTo(unAckCount)); + }); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java index dceb18cbeaa9a..09cd4f7cb1a93 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java @@ -746,6 +746,7 @@ protected void initializePulsarServices(SpyConfig spyConfig, Builder builder) { if (builder.enableOpenTelemetry) { var reader = InMemoryMetricReader.create(); openTelemetryMetricReader(reader); + registerCloseable(reader); openTelemetrySdkBuilderCustomizer = BrokerOpenTelemetryTestUtil.getOpenTelemetrySdkBuilderConsumer(reader); } else { openTelemetrySdkBuilderCustomizer = null; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index 2d2019b38eddf..0ad0b01dc1c99 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -191,6 +191,7 @@ public void testMultipleBrokerLookup() throws Exception { // Disable collecting topic stats during this test, as it deadlocks on access to map BrokerService.topics. pulsar2.getOpenTelemetryTopicStats().close(); + pulsar2.getOpenTelemetryConsumerStats().close(); var metricReader = pulsarTestContext.getOpenTelemetryMetricReader(); var lookupRequestSemaphoreField = BrokerService.class.getDeclaredField("lookupRequestSemaphore"); diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index 6088f52f72c61..4f898b382e633 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -19,6 +19,7 @@ package org.apache.pulsar.opentelemetry; import io.opentelemetry.api.common.AttributeKey; +import java.util.List; /** * Common OpenTelemetry attributes to be used by Pulsar components. @@ -55,6 +56,51 @@ public interface OpenTelemetryAttributes { */ AttributeKey PULSAR_PARTITION_INDEX = AttributeKey.longKey("pulsar.partition.index"); + /** + * The name of the Pulsar subscription. + */ + AttributeKey PULSAR_SUBSCRIPTION_NAME = AttributeKey.stringKey("pulsar.subscription.name"); + + /** + * The type of the Pulsar subscription. + */ + AttributeKey PULSAR_SUBSCRIPTION_TYPE = AttributeKey.stringKey("pulsar.subscription.type"); + + /** + * The name of the Pulsar consumer. + */ + AttributeKey PULSAR_CONSUMER_NAME = AttributeKey.stringKey("pulsar.consumer.name"); + + /** + * The ID of the Pulsar consumer. + */ + AttributeKey PULSAR_CONSUMER_ID = AttributeKey.longKey("pulsar.consumer.id"); + + /** + * Indicates whether the consumer is currently blocked on unacknowledged messages or not. + */ + AttributeKey PULSAR_CONSUMER_BLOCKED = AttributeKey.booleanKey("pulsar.consumer.blocked"); + + /** + * The consumer metadata properties, as a list of "key:value" pairs. + */ + AttributeKey> PULSAR_CONSUMER_METADATA = AttributeKey.stringArrayKey("pulsar.consumer.metadata"); + + /** + * The UTC timestamp of the Pulsar consumer creation. + */ + AttributeKey PULSAR_CONSUMER_CONNECTED_SINCE = AttributeKey.longKey("pulsar.consumer.connected_since"); + + /** + * The address of the Pulsar client. + */ + AttributeKey PULSAR_CLIENT_ADDRESS = AttributeKey.stringKey("pulsar.client.address"); + + /** + * The version of the Pulsar client. + */ + AttributeKey PULSAR_CLIENT_VERSION = AttributeKey.stringKey("pulsar.client.version"); + /** * The status of the Pulsar transaction. */ From 3b24b6e0b7250f531c86e5ee2635a9b23467419c Mon Sep 17 00:00:00 2001 From: jito Date: Mon, 13 May 2024 09:29:38 +0900 Subject: [PATCH 182/580] [fix][misc] Correct the description of patternAutoDiscoveryPeriod (#22615) Signed-off-by: jitokim --- .../java/org/apache/pulsar/client/api/ConsumerBuilder.java | 5 +++-- .../pulsar/client/impl/conf/ConsumerConfigurationData.java | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java index 863432b478fb2..6f3c3be972735 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java @@ -464,7 +464,7 @@ public interface ConsumerBuilder extends Cloneable { ConsumerBuilder readCompacted(boolean readCompacted); /** - * Sets topic's auto-discovery period when using a pattern for topics consumer. + * Sets topic's auto-discovery period when using a pattern for topic's consumer. * The period is in minutes, and the default and minimum values are 1 minute. * * @param periodInMinutes @@ -476,7 +476,8 @@ public interface ConsumerBuilder extends Cloneable { /** - * Sets topic's auto-discovery period when using a pattern for topics consumer. + * Sets topic's auto-discovery period when using a pattern for topic's consumer. + * The default value of period is 1 minute, with a minimum of 1 second. * * @param interval * the amount of delay between checks for diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java index 3ae0e977d13c4..18529276c9c04 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java @@ -310,7 +310,7 @@ public int getMaxPendingChuckedMessage() { name = "patternAutoDiscoveryPeriod", value = "Topic auto discovery period when using a pattern for topic's consumer.\n" + "\n" - + "The default and minimum value is 1 minute." + + "The default value is 1 minute, with a minimum of 1 second." ) private int patternAutoDiscoveryPeriod = 60; From 16556faf41f803497adae42de66e2e9f139b2b83 Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Mon, 13 May 2024 11:30:07 +0800 Subject: [PATCH 183/580] [improve] [pip] PIP-348: Trigger offload on topic load stage (#22650) --- pip/pip-348.md | 40 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) create mode 100644 pip/pip-348.md diff --git a/pip/pip-348.md b/pip/pip-348.md new file mode 100644 index 0000000000000..7661ef3685867 --- /dev/null +++ b/pip/pip-348.md @@ -0,0 +1,40 @@ +# PIP-348: Trigger offload on topic load stage + +# Background knowledge + +Pulsar tiered storage is introduced by [PIP-17](https://github.com/apache/pulsar/wiki/PIP-17:-Tiered-storage-for-Pulsar-topics) to offload cold data from BookKeeper to external storage. Ledger is the basic offload unit, and one ledger will trigger offload only when the ledger rollover. Pulsar topic offload can be triggered by the following ways: +- Manually trigger offload by using the `bin/pulsar-admin` command. +- Automatically trigger offload by the offload policy. + + +# Motivation +For triggering offload, the offload policy is the most common way. The offload policy can be defined in cluster level, namespace level and topic level, and the offload policy is triggered by the following ways: +- One ledger is closed or rollover +- Check the offload policy +- Trigger offload if the offload policy is satisfied + +If one topic has multiple ledgers and the latest ledgers rollover triggered offload, all the previous ledgers will be added into pending offload queue and trigger offload one by one. However, if the topic is unloaded and loaded again, the offload process will be interrupted and needs to waiting for the next ledger rollover to trigger offload. This will cause the offload process is not efficient and the offload process is not triggered in time. + + +# Goals + +## In Scope + +Trigger offload on topic load stage to improve the offload process efficiency and make sure the offload process is triggered in time. + + +# Detailed Design + +## Design & Implementation Details + +When the topic is loaded, we can check the offload policy to see if the offload policy is satisfied. If the offload policy is satisfied, we can trigger offload immediately. This will improve the offload process efficiency and make sure the offload process is triggered in time. + +In order to reduce the impact on topic load when Pulsar is upgraded from the old versions, I introduce a flag named `triggerOffloadOnTopicLoad` to control whether enable this feature or not. + +# Backward & Forward Compatibility + +Fully compatible. + +# Links +* Mailing List discussion thread: https://lists.apache.org/thread/2ndomp8v4wkcykzthhlyjqfmswor88kv +* Mailing List voting thread: https://lists.apache.org/thread/q4mfn8x69hbgv19nmqx4dmknl3vsn9y8 From 936afecede8374b14d13e9d48e9372fec1c27447 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 13 May 2024 11:50:39 +0200 Subject: [PATCH 184/580] [improve][broker]Ensure namespace deletion doesn't fail (#22627) --- .../broker/resources/BaseResources.java | 27 +++++++------- .../resources/LocalPoliciesResources.java | 2 +- .../broker/resources/NamespaceResources.java | 17 +++++++-- .../broker/resources/TopicResources.java | 35 ++++--------------- .../broker/admin/impl/NamespacesBase.java | 16 +++++++-- .../SystemTopicBasedTopicPoliciesService.java | 3 +- .../pulsar/metadata/api/MetadataStore.java | 22 ++++++++++++ .../metadata/impl/AbstractMetadataStore.java | 13 ++++--- 8 files changed, 78 insertions(+), 57 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java index 4011a48207512..00e381e07292f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java @@ -197,22 +197,21 @@ protected CompletableFuture deleteAsync(String path) { } protected CompletableFuture deleteIfExistsAsync(String path) { - return cache.exists(path).thenCompose(exists -> { - if (!exists) { - return CompletableFuture.completedFuture(null); + log.info("Deleting path: {}", path); + CompletableFuture future = new CompletableFuture<>(); + cache.delete(path).whenComplete((ignore, ex) -> { + if (ex != null && ex.getCause() instanceof MetadataStoreException.NotFoundException) { + log.info("Path {} did not exist in metadata store", path); + future.complete(null); + } else if (ex != null) { + log.info("Failed to delete path from metadata store: {}", path, ex); + future.completeExceptionally(ex); + } else { + log.info("Deleted path from metadata store: {}", path); + future.complete(null); } - CompletableFuture future = new CompletableFuture<>(); - cache.delete(path).whenComplete((ignore, ex) -> { - if (ex != null && ex.getCause() instanceof MetadataStoreException.NotFoundException) { - future.complete(null); - } else if (ex != null) { - future.completeExceptionally(ex); - } else { - future.complete(null); - } - }); - return future; }); + return future; } protected boolean exists(String path) throws MetadataStoreException { diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LocalPoliciesResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LocalPoliciesResources.java index c6b658c3bd025..ae3479fde59b8 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LocalPoliciesResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LocalPoliciesResources.java @@ -79,7 +79,7 @@ public void deleteLocalPolicies(NamespaceName ns) throws MetadataStoreException } public CompletableFuture deleteLocalPoliciesAsync(NamespaceName ns) { - return deleteAsync(joinPath(LOCAL_POLICIES_ROOT, ns.toString())); + return deleteIfExistsAsync(joinPath(LOCAL_POLICIES_ROOT, ns.toString())); } public CompletableFuture deleteLocalPoliciesTenantAsync(String tenant) { diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java index 975b23192f949..9d7c60cd34453 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java @@ -115,7 +115,7 @@ public void deletePolicies(NamespaceName ns) throws MetadataStoreException{ } public CompletableFuture deletePoliciesAsync(NamespaceName ns){ - return deleteAsync(joinPath(BASE_POLICIES_PATH, ns.toString())); + return deleteIfExistsAsync(joinPath(BASE_POLICIES_PATH, ns.toString())); } public Optional getPolicies(NamespaceName ns) throws MetadataStoreException{ @@ -155,10 +155,18 @@ public static boolean pathIsNamespaceLocalPolicies(String path) { && path.substring(LOCAL_POLICIES_ROOT.length() + 1).contains("/"); } - // clear resource of `/namespace/{namespaceName}` for zk-node + /** + * Clear resource of `/namespace/{namespaceName}` for zk-node. + * @param ns the namespace name + * @return a handle to the results of the operation + * */ + // public CompletableFuture deleteNamespaceAsync(NamespaceName ns) { final String namespacePath = joinPath(NAMESPACE_BASE_PATH, ns.toString()); - return deleteIfExistsAsync(namespacePath); + // please beware that this will delete all the children of the namespace + // including the ownership nodes (ephemeral nodes) + // see ServiceUnitUtils.path(ns) for the ownership node path + return getStore().deleteRecursive(namespacePath); } // clear resource of `/namespace/{tenant}` for zk-node @@ -303,11 +311,14 @@ public CompletableFuture deletePartitionedTopicAsync(TopicName tn) { public CompletableFuture clearPartitionedTopicMetadataAsync(NamespaceName namespaceName) { final String globalPartitionedPath = joinPath(PARTITIONED_TOPIC_PATH, namespaceName.toString()); + log.info("Clearing partitioned topic metadata for namespace {}, path is {}", + namespaceName, globalPartitionedPath); return getStore().deleteRecursive(globalPartitionedPath); } public CompletableFuture clearPartitionedTopicTenantAsync(String tenant) { final String partitionedTopicPath = joinPath(PARTITIONED_TOPIC_PATH, tenant); + log.info("Clearing partitioned topic metadata for tenant {}, path is {}", tenant, partitionedTopicPath); return deleteIfExistsAsync(partitionedTopicPath); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java index 413184764f52b..f607da76b3c11 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java @@ -75,11 +75,6 @@ public CompletableFuture> getExistingPartitions(NamespaceName ns, T ); } - public CompletableFuture deletePersistentTopicAsync(TopicName topic) { - String path = MANAGED_LEDGER_PATH + "/" + topic.getPersistenceNamingEncoding(); - return store.delete(path, Optional.of(-1L)); - } - public CompletableFuture createPersistentTopicAsync(TopicName topic) { String path = MANAGED_LEDGER_PATH + "/" + topic.getPersistenceNamingEncoding(); return store.put(path, new byte[0], Optional.of(-1L)) @@ -93,38 +88,20 @@ public CompletableFuture persistentTopicExists(TopicName topic) { public CompletableFuture clearNamespacePersistence(NamespaceName ns) { String path = MANAGED_LEDGER_PATH + "/" + ns; - return store.exists(path) - .thenCompose(exists -> { - if (exists) { - return store.delete(path, Optional.empty()); - } else { - return CompletableFuture.completedFuture(null); - } - }); + log.info("Clearing namespace persistence for namespace: {}, path {}", ns, path); + return store.deleteIfExists(path, Optional.empty()); } public CompletableFuture clearDomainPersistence(NamespaceName ns) { String path = MANAGED_LEDGER_PATH + "/" + ns + "/persistent"; - return store.exists(path) - .thenCompose(exists -> { - if (exists) { - return store.delete(path, Optional.empty()); - } else { - return CompletableFuture.completedFuture(null); - } - }); + log.info("Clearing domain persistence for namespace: {}, path {}", ns, path); + return store.deleteIfExists(path, Optional.empty()); } public CompletableFuture clearTenantPersistence(String tenant) { String path = MANAGED_LEDGER_PATH + "/" + tenant; - return store.exists(path) - .thenCompose(exists -> { - if (exists) { - return store.deleteRecursive(path); - } else { - return CompletableFuture.completedFuture(null); - } - }); + log.info("Clearing tenant persistence for tenant: {}, path {}", tenant, path); + return store.deleteRecursive(path); } void handleNotification(Notification notification) { 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 5f2dccc3e9c24..ca67a24460721 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 @@ -309,8 +309,14 @@ private void internalRetryableDeleteNamespaceAsync0(boolean force, int retryTime clientAppId(), ex); return FutureUtil.failedFuture(ex); } + log.info("[{}] Deleting namespace bundle {}/{}", clientAppId(), + namespaceName, bundle.getBundleRange()); return admin.namespaces().deleteNamespaceBundleAsync(namespaceName.toString(), bundle.getBundleRange(), force); + } else { + log.warn("[{}] Skipping deleting namespace bundle {}/{} " + + "as it's not owned by any broker", + clientAppId(), namespaceName, bundle.getBundleRange()); } return CompletableFuture.completedFuture(null); }) @@ -321,8 +327,11 @@ private void internalRetryableDeleteNamespaceAsync0(boolean force, int retryTime final Throwable rc = FutureUtil.unwrapCompletionException(error); if (rc instanceof MetadataStoreException) { if (rc.getCause() != null && rc.getCause() instanceof KeeperException.NotEmptyException) { + KeeperException.NotEmptyException ne = + (KeeperException.NotEmptyException) rc.getCause(); log.info("[{}] There are in-flight topics created during the namespace deletion, " - + "retry to delete the namespace again.", namespaceName); + + "retry to delete the namespace again. (path {} is not empty on metadata)", + namespaceName, ne.getPath()); final int next = retryTimes - 1; if (next > 0) { // async recursive @@ -330,7 +339,8 @@ private void internalRetryableDeleteNamespaceAsync0(boolean force, int retryTime } else { callback.completeExceptionally( new RestException(Status.CONFLICT, "The broker still have in-flight topics" - + " created during namespace deletion, please try again.")); + + " created during namespace deletion (path " + ne.getPath() + ") " + + "is not empty on metadata store, please try again.")); // drop out recursive } return; @@ -476,6 +486,8 @@ protected CompletableFuture internalClearZkSources() { @SuppressWarnings("deprecation") protected CompletableFuture internalDeleteNamespaceBundleAsync(String bundleRange, boolean authoritative, boolean force) { + log.info("[{}] Deleting namespace bundle {}/{} authoritative:{} force:{}", + clientAppId(), namespaceName, bundleRange, authoritative, force); return validateNamespaceOperationAsync(namespaceName, NamespaceOperation.DELETE_BUNDLE) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) .thenCompose(__ -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index 6d18d6d61b08e..5156246bb5efb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -543,7 +543,8 @@ private void readMorePoliciesAsync(SystemTopicClient.Reader reader) } else { Throwable cause = FutureUtil.unwrapCompletionException(ex); if (cause instanceof PulsarClientException.AlreadyClosedException) { - log.warn("Read more topic policies exception, close the read now!", ex); + log.info("Closing the topic policies reader for {}", + reader.getSystemTopic().getTopicName()); cleanCacheAndCloseReader( reader.getSystemTopic().getTopicName().getNamespaceObject(), false); } else { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStore.java index 33942c19520a3..89b0e7a6fe1c0 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStore.java @@ -23,9 +23,12 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.function.Consumer; import org.apache.pulsar.metadata.api.MetadataStoreException.BadVersionException; import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Metadata store client interface. @@ -36,6 +39,8 @@ @Beta public interface MetadataStore extends AutoCloseable { + Logger LOGGER = LoggerFactory.getLogger(MetadataStore.class); + /** * Read the value of one key, identified by the path * @@ -121,6 +126,23 @@ default CompletableFuture sync(String path) { */ CompletableFuture delete(String path, Optional expectedVersion); + default CompletableFuture deleteIfExists(String path, Optional expectedVersion) { + return delete(path, expectedVersion) + .exceptionally(e -> { + if (e.getCause() instanceof NotFoundException) { + LOGGER.info("Path {} not found while deleting (this is not a problem)", path); + return null; + } else { + if (expectedVersion.isEmpty()) { + LOGGER.info("Failed to delete path {}", path, e); + } else { + LOGGER.info("Failed to delete path {} with expected version {}", path, expectedVersion, e); + } + throw new CompletionException(e); + } + }); + } + /** * Delete a key-value pair and all the children nodes. * diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java index 0a35664391455..fa827bb40e706 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java @@ -360,6 +360,7 @@ public void accept(Notification n) { @Override public final CompletableFuture delete(String path, Optional expectedVersion) { + log.info("Deleting path: {} (v. {})", path, expectedVersion); if (isClosed()) { return FutureUtil.failedFuture( new MetadataStoreException.AlreadyClosedException()); @@ -405,11 +406,13 @@ private CompletableFuture deleteInternal(String path, Optional expec } metadataCaches.forEach(c -> c.invalidate(path)); + log.info("Deleted path: {} (v. {})", path, expectedVersion); }); } @Override public CompletableFuture deleteRecursive(String path) { + log.info("Deleting recursively path: {}", path); if (isClosed()) { return FutureUtil.failedFuture( new MetadataStoreException.AlreadyClosedException()); @@ -419,13 +422,9 @@ public CompletableFuture deleteRecursive(String path) { children.stream() .map(child -> deleteRecursive(path + "/" + child)) .collect(Collectors.toList()))) - .thenCompose(__ -> exists(path)) - .thenCompose(exists -> { - if (exists) { - return delete(path, Optional.empty()); - } else { - return CompletableFuture.completedFuture(null); - } + .thenCompose(__ -> { + log.info("After deleting all children, now deleting path: {}", path); + return deleteIfExists(path, Optional.empty()); }); } From 9fd1b61fc45d06348af0241f002966087f1822a0 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Mon, 13 May 2024 20:04:55 +0800 Subject: [PATCH 185/580] [fix][broker] fix replicated subscriptions for transactional messages (#22452) --- .../service/persistent/PersistentTopic.java | 21 +- .../ReplicatedSubscriptionsController.java | 4 +- .../transaction/buffer/TransactionBuffer.java | 3 +- .../buffer/impl/InMemTransactionBuffer.java | 13 +- .../buffer/impl/TopicTransactionBuffer.java | 70 +++++-- .../buffer/impl/TransactionBufferDisable.java | 13 +- .../broker/service/PersistentTopicTest.java | 4 +- .../service/ReplicatorSubscriptionTest.java | 25 +++ ...ransactionalReplicateSubscriptionTest.java | 182 ++++++++++++++++++ .../transaction/TransactionProduceTest.java | 36 ++++ .../broker/transaction/TransactionTest.java | 2 +- 11 files changed, 342 insertions(+), 31 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionalReplicateSubscriptionTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 28bc27f796157..69c7f404fdd57 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -134,6 +134,7 @@ import org.apache.pulsar.broker.stats.NamespaceStats; import org.apache.pulsar.broker.stats.ReplicationMetrics; import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; +import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferDisable; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.client.admin.LongRunningProcessStatus; @@ -272,10 +273,13 @@ protected TopicStatsHelper initialValue() { @Getter protected final TransactionBuffer transactionBuffer; + @Getter + private final TopicTransactionBuffer.MaxReadPositionCallBack maxReadPositionCallBack = + (oldPosition, newPosition) -> updateMaxReadPositionMovedForwardTimestamp(); - // Record the last time a data message (ie: not an internal Pulsar marker) is published on the topic + // Record the last time max read position is moved forward, unless it's a marker message. @Getter - private volatile long lastDataMessagePublishedTimestamp = 0; + private volatile long lastMaxReadPositionMovedForwardTimestamp = 0; @Getter private final ExecutorService orderedExecutor; @@ -410,7 +414,7 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS } else { this.transactionBuffer = new TransactionBufferDisable(this); } - transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry()); + transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry(), true); if (ledger instanceof ShadowManagedLedgerImpl) { shadowSourceTopic = TopicName.get(ledger.getConfig().getShadowSource()); } else { @@ -719,6 +723,10 @@ private void decrementPendingWriteOpsAndCheck() { } } + private void updateMaxReadPositionMovedForwardTimestamp() { + lastMaxReadPositionMovedForwardTimestamp = Clock.systemUTC().millis(); + } + @Override public void addComplete(Position pos, ByteBuf entryData, Object ctx) { PublishContext publishContext = (PublishContext) ctx; @@ -727,12 +735,9 @@ public void addComplete(Position pos, ByteBuf entryData, Object ctx) { // Message has been successfully persisted messageDeduplication.recordMessagePersisted(publishContext, position); - if (!publishContext.isMarkerMessage()) { - lastDataMessagePublishedTimestamp = Clock.systemUTC().millis(); - } - // in order to sync the max position when cursor read entries - transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry()); + transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry(), + publishContext.isMarkerMessage()); publishContext.setMetadataFromEntryData(entryData); publishContext.completed(null, position.getLedgerId(), position.getEntryId()); decrementPendingWriteOpsAndCheck(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java index e011ed8d660f6..3a796b3e96dd4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java @@ -206,8 +206,8 @@ private void receiveSubscriptionUpdated(ReplicatedSubscriptionsUpdate update) { private void startNewSnapshot() { cleanupTimedOutSnapshots(); - if (topic.getLastDataMessagePublishedTimestamp() < lastCompletedSnapshotStartTime - || topic.getLastDataMessagePublishedTimestamp() == 0) { + if (topic.getLastMaxReadPositionMovedForwardTimestamp() < lastCompletedSnapshotStartTime + || topic.getLastMaxReadPositionMovedForwardTimestamp() == 0) { // There was no message written since the last snapshot, we can skip creating a new snapshot if (log.isDebugEnabled()) { log.debug("[{}] There is no new data in topic. Skipping snapshot creation.", topic.getName()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java index 3fe989acc9227..092638abf5bba 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java @@ -148,8 +148,9 @@ public interface TransactionBuffer { /** * Sync max read position for normal publish. * @param position {@link PositionImpl} the position to sync. + * @param isMarkerMessage whether the message is marker message. */ - void syncMaxReadPositionForNormalPublish(PositionImpl position); + void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage); /** * Get the can read max position. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java index 978536c5f4e36..bab7b64c608c4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java @@ -33,6 +33,7 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.TransactionBufferReader; @@ -213,11 +214,17 @@ public TransactionBufferReader newReader(long sequenceId) throws final ConcurrentMap buffers; final Map> txnIndex; private final Topic topic; + private final TopicTransactionBuffer.MaxReadPositionCallBack maxReadPositionCallBack; public InMemTransactionBuffer(Topic topic) { this.buffers = new ConcurrentHashMap<>(); this.txnIndex = new HashMap<>(); this.topic = topic; + if (topic instanceof PersistentTopic) { + this.maxReadPositionCallBack = ((PersistentTopic) topic).getMaxReadPositionCallBack(); + } else { + this.maxReadPositionCallBack = null; + } } @Override @@ -369,8 +376,10 @@ public boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { } @Override - public void syncMaxReadPositionForNormalPublish(PositionImpl position) { - //no-op + public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { + if (!isMarkerMessage && maxReadPositionCallBack != null) { + maxReadPositionCallBack.maxReadPositionMovedForward(null, position); + } } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index 81c9ecfc728e9..dfb73815e08d7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -103,6 +103,7 @@ public class TopicTransactionBuffer extends TopicTransactionBufferState implemen private final AbortedTxnProcessor snapshotAbortedTxnProcessor; private final AbortedTxnProcessor.SnapshotType snapshotType; + private final MaxReadPositionCallBack maxReadPositionCallBack; public TopicTransactionBuffer(PersistentTopic topic) { super(State.None); @@ -120,6 +121,7 @@ public TopicTransactionBuffer(PersistentTopic topic) { snapshotAbortedTxnProcessor = new SingleSnapshotAbortedTxnProcessorImpl(topic); snapshotType = AbortedTxnProcessor.SnapshotType.Single; } + this.maxReadPositionCallBack = topic.getMaxReadPositionCallBack(); this.recover(); } @@ -175,7 +177,7 @@ public void handleTxnEntry(Entry entry) { if (Markers.isTxnAbortMarker(msgMetadata)) { snapshotAbortedTxnProcessor.putAbortedTxnAndPosition(txnID, position); } - updateMaxReadPosition(txnID); + removeTxnAndUpdateMaxReadPosition(txnID); } else { handleTransactionMessage(txnID, position); } @@ -290,7 +292,8 @@ private void handleTransactionMessage(TxnID txnId, Position position) { ongoingTxns.put(txnId, (PositionImpl) position); PositionImpl firstPosition = ongoingTxns.get(ongoingTxns.firstKey()); // max read position is less than first ongoing transaction message position - maxReadPosition = ((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(firstPosition); + updateMaxReadPosition(((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(firstPosition), + false); } } @@ -314,7 +317,7 @@ public CompletableFuture commitTxn(TxnID txnID, long lowWaterMark) { @Override public void addComplete(Position position, ByteBuf entryData, Object ctx) { synchronized (TopicTransactionBuffer.this) { - updateMaxReadPosition(txnID); + removeTxnAndUpdateMaxReadPosition(txnID); handleLowWaterMark(txnID, lowWaterMark); snapshotAbortedTxnProcessor.trimExpiredAbortedTxns(); takeSnapshotByChangeTimes(); @@ -361,7 +364,7 @@ public CompletableFuture abortTxn(TxnID txnID, long lowWaterMark) { public void addComplete(Position position, ByteBuf entryData, Object ctx) { synchronized (TopicTransactionBuffer.this) { snapshotAbortedTxnProcessor.putAbortedTxnAndPosition(txnID, (PositionImpl) position); - updateMaxReadPosition(txnID); + removeTxnAndUpdateMaxReadPosition(txnID); snapshotAbortedTxnProcessor.trimExpiredAbortedTxns(); takeSnapshotByChangeTimes(); txnAbortedCounter.increment(); @@ -444,17 +447,39 @@ private void takeSnapshotByTimeout() { takeSnapshotIntervalTime, TimeUnit.MILLISECONDS); } - void updateMaxReadPosition(TxnID txnID) { - PositionImpl preMaxReadPosition = this.maxReadPosition; + /** + * remove the specified transaction from ongoing transaction list and update the max read position. + * @param txnID + */ + void removeTxnAndUpdateMaxReadPosition(TxnID txnID) { ongoingTxns.remove(txnID); if (!ongoingTxns.isEmpty()) { PositionImpl position = ongoingTxns.get(ongoingTxns.firstKey()); - maxReadPosition = ((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(position); + updateMaxReadPosition(((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(position), false); } else { - maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(); + updateMaxReadPosition((PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(), false); } - if (preMaxReadPosition.compareTo(this.maxReadPosition) != 0) { - this.changeMaxReadPositionCount.getAndIncrement(); + } + + /** + * update the max read position. if the new position is greater than the current max read position, + * we will trigger the callback, unless the disableCallback is true. + * Currently, we only use the callback to update the lastMaxReadPositionMovedForwardTimestamp. + * For non-transactional production, some marker messages will be sent to the topic, in which case we don't need + * to trigger the callback. + * @param newPosition new max read position to update. + * @param disableCallback whether disable the callback. + */ + void updateMaxReadPosition(PositionImpl newPosition, boolean disableCallback) { + PositionImpl preMaxReadPosition = this.maxReadPosition; + this.maxReadPosition = newPosition; + if (preMaxReadPosition.compareTo(this.maxReadPosition) < 0) { + if (!checkIfNoSnapshot()) { + this.changeMaxReadPositionCount.getAndIncrement(); + } + if (!disableCallback) { + maxReadPositionCallBack.maxReadPositionMovedForward(preMaxReadPosition, this.maxReadPosition); + } } } @@ -479,17 +504,22 @@ public synchronized boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) return snapshotAbortedTxnProcessor.checkAbortedTransaction(txnID); } + /** + * Sync max read position for normal publish. + * @param position {@link PositionImpl} the position to sync. + * @param isMarkerMessage whether the message is marker message, in such case, we + * don't need to trigger the callback to update lastMaxReadPositionMovedForwardTimestamp. + */ @Override - public void syncMaxReadPositionForNormalPublish(PositionImpl position) { + public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { // when ongoing transaction is empty, proved that lastAddConfirm is can read max position, because callback // thread is the same tread, in this time the lastAddConfirm don't content transaction message. synchronized (TopicTransactionBuffer.this) { if (checkIfNoSnapshot()) { - this.maxReadPosition = position; + updateMaxReadPosition(position, isMarkerMessage); } else if (checkIfReady()) { if (ongoingTxns.isEmpty()) { - maxReadPosition = position; - changeMaxReadPositionCount.incrementAndGet(); + updateMaxReadPosition(position, isMarkerMessage); } } } @@ -674,6 +704,18 @@ private void closeReader(SystemTopicClient.Reader rea } } + /** + * A functional interface to handle the max read position move forward. + */ + public interface MaxReadPositionCallBack { + /** + * callback method when max read position move forward. + * @param oldPosition the old max read position. + * @param newPosition the new max read position. + */ + void maxReadPositionMovedForward(PositionImpl oldPosition, PositionImpl newPosition); + } + static class FillEntryQueueCallback implements AsyncCallbacks.ReadEntriesCallback { private final AtomicLong outstandingReadsRequests = new AtomicLong(0); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java index 9de0888ae5b0b..ebd61dbaa82ec 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java @@ -26,6 +26,7 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.TransactionBufferReader; @@ -42,8 +43,14 @@ public class TransactionBufferDisable implements TransactionBuffer { private final Topic topic; + private final TopicTransactionBuffer.MaxReadPositionCallBack maxReadPositionCallBack; public TransactionBufferDisable(Topic topic) { this.topic = topic; + if (topic instanceof PersistentTopic) { + this.maxReadPositionCallBack = ((PersistentTopic) topic).getMaxReadPositionCallBack(); + } else { + this.maxReadPositionCallBack = null; + } } @Override @@ -91,8 +98,10 @@ public boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { } @Override - public void syncMaxReadPositionForNormalPublish(PositionImpl position) { - //no-op + public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { + if (!isMarkerMessage && maxReadPositionCallBack != null) { + maxReadPositionCallBack.maxReadPositionMovedForward(null, position); + } } @Override 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 de9d0272fc002..1118b71456e84 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 @@ -298,6 +298,8 @@ public void testPublishMessage() throws Exception { }).when(ledgerMock).asyncAddEntry(any(ByteBuf.class), any(AddEntryCallback.class), any()); PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); + long lastMaxReadPositionMovedForwardTimestamp = topic.getLastMaxReadPositionMovedForwardTimestamp(); + /* * MessageMetadata.Builder messageMetadata = MessageMetadata.newBuilder(); * messageMetadata.setPublishTime(System.currentTimeMillis()); messageMetadata.setProducerName("producer-name"); @@ -322,10 +324,10 @@ public void setMetadataFromEntryData(ByteBuf entryData) { assertEquals(entryData.array(), payload.array()); } }; - topic.publishMessage(payload, publishContext); assertTrue(latch.await(1, TimeUnit.SECONDS)); + assertTrue(topic.getLastMaxReadPositionMovedForwardTimestamp() > lastMaxReadPositionMovedForwardTimestamp); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java index 25b09f965498d..647b7b28281c4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java @@ -26,6 +26,8 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.google.common.collect.Sets; + +import java.lang.reflect.Field; import java.lang.reflect.Method; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -40,8 +42,10 @@ import lombok.Cleanup; import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.persistent.ReplicatedSubscriptionsController; +import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBufferState; import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -728,6 +732,21 @@ public void testReplicatedSubscriptionRestApi3() throws Exception { consumer4.close(); } + /** + * before sending message, we should wait for transaction buffer recover complete, + * or the MaxReadPosition will not move forward when the message is sent, and the + * MaxReadPositionMovedForwardTimestamp will not be updated, then the replication will not be triggered. + * @param topicName + * @throws Exception + */ + private void waitTBRecoverComplete(PulsarService pulsarService, String topicName) throws Exception { + TopicTransactionBufferState buffer = (TopicTransactionBufferState) ((PersistentTopic) pulsarService.getBrokerService() + .getTopic(topicName, false).get().get()).getTransactionBuffer(); + Field stateField = TopicTransactionBufferState.class.getDeclaredField("state"); + stateField.setAccessible(true); + Awaitility.await().until(() -> !stateField.get(buffer).toString().equals("Initializing")); + } + /** * Tests replicated subscriptions when replicator producer is closed */ @@ -755,6 +774,9 @@ public void testReplicatedSubscriptionWhenReplicatorProducerIsClosed() throws Ex .subscribe(); // send one message to trigger replication + if (config1.isTransactionCoordinatorEnabled()) { + waitTBRecoverComplete(pulsar1, topicName); + } @Cleanup Producer producer = client1.newProducer().topic(topicName) .enableBatching(false) @@ -917,6 +939,9 @@ public void testReplicatedSubscriptionWithCompaction() throws Exception { .statsInterval(0, TimeUnit.SECONDS).build(); Producer producer = client.newProducer(Schema.STRING).topic(topicName).create(); + if (config1.isTransactionCoordinatorEnabled()) { + waitTBRecoverComplete(pulsar1, topicName); + } producer.newMessage().key("K1").value("V1").send(); producer.newMessage().key("K1").value("V2").send(); producer.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionalReplicateSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionalReplicateSubscriptionTest.java new file mode 100644 index 0000000000000..2d348f8259746 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionalReplicateSubscriptionTest.java @@ -0,0 +1,182 @@ +/* + * 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.service; + +import com.google.common.collect.Sets; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +public class TransactionalReplicateSubscriptionTest extends ReplicatorTestBase { + @Override + @BeforeClass(timeOut = 300000) + public void setup() throws Exception { + super.setup(); + admin1.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString()); + createTransactionCoordinatorAssign(16, pulsar1); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + /** + * enable transaction coordinator for the cluster1 + */ + @Override + public void setConfig1DefaultValue(){ + super.setConfig1DefaultValue(); + config1.setTransactionCoordinatorEnabled(true); + } + + protected void createTransactionCoordinatorAssign(int numPartitionsOfTC, PulsarService pulsarService) throws MetadataStoreException { + pulsarService.getPulsarResources() + .getNamespaceResources() + .getPartitionedTopicResources() + .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, + new PartitionedTopicMetadata(numPartitionsOfTC)); + } + + /** + * Test replicated subscription with transaction. + * @throws Exception + */ + @Test + public void testReplicatedSubscribeAndSwitchToStandbyClusterWithTransaction() throws Exception { + final String namespace = BrokerTestUtil.newUniqueName("pulsar/ns_"); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/tp_"); + final String subscriptionName = "s1"; + final boolean isReplicatedSubscription = true; + final int messagesCount = 20; + final LinkedHashSet sentMessages = new LinkedHashSet<>(); + final Set receivedMessages = Collections.synchronizedSet(new LinkedHashSet<>()); + admin1.namespaces().createNamespace(namespace); + admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r2")); + admin1.topics().createNonPartitionedTopic(topicName); + admin1.topics().createSubscription(topicName, subscriptionName, MessageId.earliest, isReplicatedSubscription); + final PersistentTopic topic1 = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + + // Send messages + // Wait for the topic created on the cluster2. + // Wait for the snapshot created. + final PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()).enableTransaction(true).build(); + Producer producer1 = client1.newProducer(Schema.STRING).topic(topicName).enableBatching(false).create(); + Consumer consumer1 = client1.newConsumer(Schema.STRING).topic(topicName) + .subscriptionName(subscriptionName).replicateSubscriptionState(isReplicatedSubscription).subscribe(); + Transaction txn1 = client1.newTransaction() + .withTransactionTimeout(5, TimeUnit.SECONDS) + .build().get(); + for (int i = 0; i < messagesCount / 2; i++) { + String msg = i + ""; + producer1.newMessage(txn1).value(msg).send(); + sentMessages.add(msg); + } + txn1.commit().get(); + Awaitility.await().untilAsserted(() -> { + ConcurrentOpenHashMap replicators = topic1.getReplicators(); + assertTrue(replicators != null && replicators.size() == 1, "Replicator should started"); + assertTrue(replicators.values().iterator().next().isConnected(), "Replicator should be connected"); + assertTrue(topic1.getReplicatedSubscriptionController().get().getLastCompletedSnapshotId().isPresent(), + "One snapshot should be finished"); + }); + final PersistentTopic topic2 = + (PersistentTopic) pulsar2.getBrokerService().getTopic(topicName, false).join().get(); + Awaitility.await().untilAsserted(() -> { + assertTrue(topic2.getReplicatedSubscriptionController().isPresent(), + "Replicated subscription controller should created"); + }); + Transaction txn2 = client1.newTransaction() + .withTransactionTimeout(5, TimeUnit.SECONDS) + .build().get(); + for (int i = messagesCount / 2; i < messagesCount; i++) { + String msg = i + ""; + producer1.newMessage(txn2).value(msg).send(); + sentMessages.add(msg); + } + txn2.commit().get(); + + // Consume half messages and wait the subscription created on the cluster2. + for (int i = 0; i < messagesCount / 2; i++){ + Message message = consumer1.receive(2, TimeUnit.SECONDS); + if (message == null) { + fail("Should not receive null."); + } + receivedMessages.add(message.getValue()); + consumer1.acknowledge(message); + } + Awaitility.await().untilAsserted(() -> { + assertNotNull(topic2.getSubscriptions().get(subscriptionName), "Subscription should created"); + }); + + // Switch client to cluster2. + // Since the cluster1 was not crash, all messages will be replicated to the cluster2. + consumer1.close(); + final PulsarClient client2 = PulsarClient.builder().serviceUrl(url2.toString()).build(); + final Consumer consumer2 = client2.newConsumer(Schema.AUTO_CONSUME()).topic(topicName) + .subscriptionName(subscriptionName).replicateSubscriptionState(isReplicatedSubscription).subscribe(); + + // Verify all messages will be consumed. + Awaitility.await().untilAsserted(() -> { + while (true) { + Message message = consumer2.receive(2, TimeUnit.SECONDS); + if (message != null) { + receivedMessages.add(message.getValue().toString()); + consumer2.acknowledge(message); + } else { + break; + } + } + assertEquals(receivedMessages.size(), sentMessages.size()); + }); + + consumer2.close(); + producer1.close(); + client1.close(); + client2.close(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java index 32ffd2938939f..b375ab7d95429 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java @@ -19,11 +19,14 @@ package org.apache.pulsar.broker.transaction; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.testng.Assert.assertTrue; + import java.lang.reflect.Field; import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -38,7 +41,9 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -180,6 +185,37 @@ private void produceTest(boolean endAction) throws Exception { log.info("produce and {} test finished.", endAction ? "commit" : "abort"); } + @Test + public void testUpdateLastMaxReadPositionMovedForwardTimestampForTransactionalPublish() throws Exception { + final String topic = NAMESPACE1 + "/testUpdateLastMaxReadPositionMovedForwardTimestampForTransactionalPublish"; + PulsarClient pulsarClient = this.pulsarClient; + Transaction txn = pulsarClient.newTransaction() + .withTransactionTimeout(5, TimeUnit.SECONDS) + .build().get(); + @Cleanup + Producer producer = pulsarClient + .newProducer() + .topic(topic) + .sendTimeout(0, TimeUnit.SECONDS) + .create(); + PersistentTopic persistentTopic = getTopic(topic); + long lastMaxReadPositionMovedForwardTimestamp = persistentTopic.getLastMaxReadPositionMovedForwardTimestamp(); + + // transactional publish will not update lastMaxReadPositionMovedForwardTimestamp + producer.newMessage(txn).value("hello world".getBytes()).send(); + assertTrue(persistentTopic.getLastMaxReadPositionMovedForwardTimestamp() == lastMaxReadPositionMovedForwardTimestamp); + + // commit transaction will update lastMaxReadPositionMovedForwardTimestamp + txn.commit().get(); + assertTrue(persistentTopic.getLastMaxReadPositionMovedForwardTimestamp() > lastMaxReadPositionMovedForwardTimestamp); + } + + private PersistentTopic getTopic(String topic) throws ExecutionException, InterruptedException { + Optional optionalTopic = getPulsarServiceList().get(0).getBrokerService() + .getTopic(topic, true).get(); + return (PersistentTopic) optionalTopic.get(); + } + private void checkMessageId(List> futureList, boolean isFinished) { futureList.forEach(messageIdFuture -> { try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index 5e806bb9ceee2..55a3e09896557 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -1119,7 +1119,7 @@ public void testNotChangeMaxReadPositionCountWhenCheckIfNoSnapshot() throws Exce }); Assert.assertEquals(changeMaxReadPositionCount.get(), 0L); - buffer.syncMaxReadPositionForNormalPublish(new PositionImpl(1, 1)); + buffer.syncMaxReadPositionForNormalPublish(new PositionImpl(1, 1), false); Assert.assertEquals(changeMaxReadPositionCount.get(), 0L); } From 9668674b361aa1b7b5e72c457fc0fa9d7b324f05 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 14 May 2024 00:22:55 +0800 Subject: [PATCH 186/580] [improve][build] Improve docker-push (#22702) Signed-off-by: Zixuan Liu --- docker/pulsar-all/pom.xml | 23 ++++------------------- docker/pulsar/pom.xml | 23 ++++------------------- pom.xml | 1 + 3 files changed, 9 insertions(+), 38 deletions(-) diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 3da14ea84bcb3..6aa783ee9c85f 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -146,6 +146,7 @@ build tag + push @@ -180,25 +181,9 @@ docker-push - - - - io.fabric8 - docker-maven-plugin - - - default - package - - build - tag - push - - - - - - + + false + diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 79ff4bd33b10c..5d83c8b547759 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -73,6 +73,7 @@ build tag + push @@ -124,25 +125,9 @@ docker-push - - - - io.fabric8 - docker-maven-plugin - - - default - package - - build - tag - push - - - - - - + + false + diff --git a/pom.xml b/pom.xml index 63b44788f1410..2254d6a187540 100644 --- a/pom.xml +++ b/pom.xml @@ -103,6 +103,7 @@ flexible messaging model and an intuitive client API. To create multi-arch image, pass -Ddocker.platforms=linux/arm64,linux/amd64 --> + true From 1a7ada8805630f974cebc029e5ef12550c217ece Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Tue, 14 May 2024 11:58:48 +0800 Subject: [PATCH 187/580] [improve][build] Bump version to 3.4.0-SNAPSHOT (#22700) --- bouncy-castle/bc/pom.xml | 2 +- bouncy-castle/bcfips-include-test/pom.xml | 2 +- bouncy-castle/bcfips/pom.xml | 2 +- bouncy-castle/pom.xml | 2 +- buildtools/pom.xml | 4 ++-- distribution/io/pom.xml | 2 +- distribution/offloaders/pom.xml | 2 +- distribution/pom.xml | 2 +- distribution/server/pom.xml | 2 +- distribution/shell/pom.xml | 2 +- docker/pom.xml | 2 +- docker/pulsar-all/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- microbench/pom.xml | 2 +- pom.xml | 4 ++-- pulsar-bom/pom.xml | 4 ++-- pulsar-broker-auth-athenz/pom.xml | 2 +- pulsar-broker-auth-oidc/pom.xml | 2 +- pulsar-broker-auth-sasl/pom.xml | 2 +- pulsar-broker-common/pom.xml | 2 +- pulsar-broker/pom.xml | 2 +- pulsar-cli-utils/pom.xml | 2 +- pulsar-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml | 2 +- pulsar-client-admin-api/pom.xml | 2 +- pulsar-client-admin-shaded/pom.xml | 2 +- pulsar-client-admin/pom.xml | 2 +- pulsar-client-all/pom.xml | 2 +- pulsar-client-api/pom.xml | 2 +- pulsar-client-auth-athenz/pom.xml | 2 +- pulsar-client-auth-sasl/pom.xml | 2 +- pulsar-client-messagecrypto-bc/pom.xml | 2 +- pulsar-client-shaded/pom.xml | 2 +- pulsar-client-tools-api/pom.xml | 2 +- pulsar-client-tools-customcommand-example/pom.xml | 2 +- pulsar-client-tools-test/pom.xml | 2 +- pulsar-client-tools/pom.xml | 2 +- pulsar-client/pom.xml | 2 +- pulsar-common/pom.xml | 2 +- pulsar-config-validation/pom.xml | 2 +- pulsar-docs-tools/pom.xml | 2 +- pulsar-functions/api-java/pom.xml | 2 +- pulsar-functions/instance/pom.xml | 2 +- pulsar-functions/java-examples-builtin/pom.xml | 2 +- pulsar-functions/java-examples/pom.xml | 2 +- pulsar-functions/localrun-shaded/pom.xml | 2 +- pulsar-functions/localrun/pom.xml | 2 +- pulsar-functions/pom.xml | 2 +- pulsar-functions/proto/pom.xml | 2 +- pulsar-functions/runtime-all/pom.xml | 2 +- pulsar-functions/runtime/pom.xml | 2 +- pulsar-functions/secrets/pom.xml | 2 +- pulsar-functions/utils/pom.xml | 2 +- pulsar-functions/worker/pom.xml | 2 +- pulsar-io/aerospike/pom.xml | 2 +- pulsar-io/alluxio/pom.xml | 2 +- pulsar-io/aws/pom.xml | 2 +- pulsar-io/azure-data-explorer/pom.xml | 2 +- pulsar-io/batch-data-generator/pom.xml | 2 +- pulsar-io/batch-discovery-triggerers/pom.xml | 2 +- pulsar-io/canal/pom.xml | 2 +- pulsar-io/cassandra/pom.xml | 2 +- pulsar-io/common/pom.xml | 2 +- pulsar-io/core/pom.xml | 2 +- pulsar-io/data-generator/pom.xml | 2 +- pulsar-io/debezium/core/pom.xml | 2 +- pulsar-io/debezium/mongodb/pom.xml | 2 +- pulsar-io/debezium/mssql/pom.xml | 2 +- pulsar-io/debezium/mysql/pom.xml | 2 +- pulsar-io/debezium/oracle/pom.xml | 2 +- pulsar-io/debezium/pom.xml | 2 +- pulsar-io/debezium/postgres/pom.xml | 2 +- pulsar-io/docs/pom.xml | 2 +- pulsar-io/dynamodb/pom.xml | 2 +- pulsar-io/elastic-search/pom.xml | 2 +- pulsar-io/file/pom.xml | 2 +- pulsar-io/flume/pom.xml | 2 +- pulsar-io/hbase/pom.xml | 2 +- pulsar-io/hdfs2/pom.xml | 2 +- pulsar-io/hdfs3/pom.xml | 2 +- pulsar-io/http/pom.xml | 2 +- pulsar-io/influxdb/pom.xml | 2 +- pulsar-io/jdbc/clickhouse/pom.xml | 2 +- pulsar-io/jdbc/core/pom.xml | 2 +- pulsar-io/jdbc/mariadb/pom.xml | 2 +- pulsar-io/jdbc/openmldb/pom.xml | 2 +- pulsar-io/jdbc/pom.xml | 2 +- pulsar-io/jdbc/postgres/pom.xml | 2 +- pulsar-io/jdbc/sqlite/pom.xml | 2 +- pulsar-io/kafka-connect-adaptor-nar/pom.xml | 2 +- pulsar-io/kafka-connect-adaptor/pom.xml | 2 +- pulsar-io/kafka/pom.xml | 2 +- pulsar-io/kinesis/pom.xml | 2 +- pulsar-io/mongo/pom.xml | 2 +- pulsar-io/netty/pom.xml | 2 +- pulsar-io/nsq/pom.xml | 2 +- pulsar-io/pom.xml | 2 +- pulsar-io/rabbitmq/pom.xml | 2 +- pulsar-io/redis/pom.xml | 2 +- pulsar-io/solr/pom.xml | 2 +- pulsar-io/twitter/pom.xml | 2 +- pulsar-metadata/pom.xml | 2 +- pulsar-opentelemetry/pom.xml | 2 +- pulsar-package-management/bookkeeper-storage/pom.xml | 2 +- pulsar-package-management/core/pom.xml | 2 +- pulsar-package-management/filesystem-storage/pom.xml | 2 +- pulsar-package-management/pom.xml | 2 +- pulsar-proxy/pom.xml | 2 +- pulsar-testclient/pom.xml | 2 +- pulsar-transaction/common/pom.xml | 2 +- pulsar-transaction/coordinator/pom.xml | 2 +- pulsar-transaction/pom.xml | 2 +- pulsar-websocket/pom.xml | 2 +- structured-event-log/pom.xml | 2 +- testmocks/pom.xml | 2 +- tests/bc_2_0_0/pom.xml | 2 +- tests/bc_2_0_1/pom.xml | 2 +- tests/bc_2_6_0/pom.xml | 2 +- tests/docker-images/java-test-functions/pom.xml | 2 +- tests/docker-images/java-test-image/pom.xml | 2 +- tests/docker-images/java-test-plugins/pom.xml | 2 +- tests/docker-images/latest-version-image/pom.xml | 2 +- tests/docker-images/pom.xml | 2 +- tests/integration/pom.xml | 2 +- tests/pom.xml | 2 +- tests/pulsar-client-admin-shade-test/pom.xml | 2 +- tests/pulsar-client-all-shade-test/pom.xml | 2 +- tests/pulsar-client-shade-test/pom.xml | 2 +- tiered-storage/file-system/pom.xml | 2 +- tiered-storage/jcloud/pom.xml | 2 +- tiered-storage/pom.xml | 2 +- 134 files changed, 137 insertions(+), 137 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index a20f9146b76b4..a5bfb9e113b9d 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 99495aa987c5a..266517434f799 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar bouncy-castle-parent - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index 410dd14a260dd..b47b9fbacf675 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index c78b8fd6d8ea1..57b98ee547941 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 3b0b24d1d53a1..4a44b35066583 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ org.apache.pulsar buildtools - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT jar Pulsar Build Tools - 2023-12-28T19:33:08Z + 2024-05-13T09:56:11Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 22f8de5e15497..e9dd44d0d5950 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 6e489ceb81b75..32b197c5c7f6f 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT .. diff --git a/distribution/pom.xml b/distribution/pom.xml index f248b49f1f32a..666ed0514e3a9 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 1c9ea68685308..7a48245d800be 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 144f7b1ff6d83..457b0d6fb01ee 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT .. diff --git a/docker/pom.xml b/docker/pom.xml index 21ed4de940826..90a845400d3e6 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 6aa783ee9c85f..659153a82e6ab 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 5d83c8b547759..523a4b1bb3f77 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index 998fe98aa0f8d..bac1cf6b1e046 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index 59bb82911f2b5..b32cfb154277a 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT .. diff --git a/microbench/pom.xml b/microbench/pom.xml index a568e716ba0fa..98f32899888af 100644 --- a/microbench/pom.xml +++ b/microbench/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 2254d6a187540..576c1b4e42299 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ org.apache.pulsar pulsar - 3.3.0-SNAPSHOT + 3.4.0-SNAPSHOT Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -96,7 +96,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2023-12-28T19:33:08Z + 2024-05-13T09:56:12Z true + + +# General Notes + +# Links + +Issue: https://github.com/apache/pulsar/issues/21751 +Discuss thread: https://lists.apache.org/thread/w7w91xztdyy07otw0dh71nl2rn3yy45p +Vote thread: https://lists.apache.org/thread/hh9t6nz0pqjo7tbfn12nbwtylrvq4f43 From 3d260799e372138b95c573a105cab2d673076007 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Tue, 14 May 2024 19:22:41 +0800 Subject: [PATCH 190/580] [improve][pip] PIP-347: add role field in consumer's stat (#22564) --- pip/pip-347.md | 37 +++++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) create mode 100644 pip/pip-347.md diff --git a/pip/pip-347.md b/pip/pip-347.md new file mode 100644 index 0000000000000..5326fed353374 --- /dev/null +++ b/pip/pip-347.md @@ -0,0 +1,37 @@ + +# PIP-347: add role field in consumer's stat + +# Background knowledge + +During the operation and maintenance process, there are many users asking administrator for help to find out the consumers of a topic and notify them about the business change. +Administrators can call `bin/pulsar-admin topics partitioned-stats` to find out the `ip:port` of the consumers, but no role info. So administrators need to take a lot of time to +communicate with users to find out the owner based on the `ip:port`. It's a troublesome work and low efficiency, or even can't find out the owner. + +# Motivation + +This pip can help to solve such kind of problem. By adding a field `appId` in the consumer's stat. +For cluster with JWT-based authentication, the administrator can find out the owner of the consumer directly. +It can save a lot of time and improve the efficiency of the operation and maintenance process. + +# Goals + +- help administrator to find out the owner of the consumer for cluster with JWT-based authentication. + +# Detailed Design + +## Design & Implementation Details +- Add a field `appId` in the consumer's stat, which can show the owner of this consumer for JWT-based authentication users. + +# Backward & Forward Compatibility + +Fully compatible. + +# General Notes + +# Links + + +* Mailing List discussion thread: https://lists.apache.org/thread/p9y9r8pb7ygk8f0jd121c1121phvzd09 +* Mailing List voting thread: From 361156e74cb29786a88d796982333efd1b214231 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 15 May 2024 12:44:18 +0800 Subject: [PATCH 191/580] [improve][pip] PIP-350: Allow to disable the managedLedgerOffloadDeletionLagInMillis (#22688) --- pip/pip-350.md | 36 ++++++++++++++++++++++++++++++++++++ 1 file changed, 36 insertions(+) create mode 100644 pip/pip-350.md diff --git a/pip/pip-350.md b/pip/pip-350.md new file mode 100644 index 0000000000000..f48771e7ee17d --- /dev/null +++ b/pip/pip-350.md @@ -0,0 +1,36 @@ +# PIP-350: Allow to disable the managedLedgerOffloadDeletionLagInMillis + +# Background knowledge + +https://pulsar.apache.org/docs/3.2.x/tiered-storage-overview/ +Pulsar provides the ability to offload the data from bookkeeper to the cloud storage with the tiered storage. +Once the data is offloaded to the cloud storage, the data in the bookkeeper can be deleted after a certain period of time. +We use the managedLedgerOffloadDeletionLagInMillis to control the deletion lag time for the offloaded data. +The default value of managedLedgerOffloadDeletionLagInMillis is 4 hours. It means the offloaded data will be deleted after 4 hours by default. + +# Motivation + +In some test scenarios, we want to disable the deletionLag and never delete the data from the bookkeeper. +Then when the tiered storage data is broken, we can still read the data from the bookkeeper. + +# Goals + +## In Scope + +Never deletes the bookkeeper data when the managedLedgerOffloadDeletionLagInMillis is set to -1. + +# Detailed Design + +## Design & Implementation Details + +Only need to check the value of managedLedgerOffloadDeletionLagInMillis in the ManagedLedgerImpl when it is going to delete the bookkeeper data. +https://github.com/apache/pulsar/blob/774a5d42e8342ee50395cf3626b9e7af27da849e/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L2579 + +# Backward & Forward Compatibility + +Fully compatible. + +# Links + +* Mailing List discussion thread: https://lists.apache.org/thread/7tlpkcm2933ddg95kgrb42943r4gq3v9 +* Mailing List voting thread: https://lists.apache.org/thread/c3rh530dlwo6nhrdflpw0mjck85hhfbx From 22a9023acd231efea34e4f2cd7389b89eaec5435 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 15 May 2024 12:57:17 +0800 Subject: [PATCH 192/580] [cleanup][broker] Remove warn logs when changing the state from Owned to Free (Extensible LB) (#22708) --- .../channel/ServiceUnitStateChannelImpl.java | 36 +++++-------------- 1 file changed, 9 insertions(+), 27 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index bf6266482f8f0..9821ce56420ed 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -1284,34 +1284,16 @@ private void scheduleCleanup(String broker, long delayInSecs) { } - private ServiceUnitStateData getOverrideInactiveBrokerStateData(ServiceUnitStateData orphanData, - Optional selectedBroker, - String inactiveBroker) { - - - if (selectedBroker.isEmpty()) { - return new ServiceUnitStateData(Free, null, inactiveBroker, - true, getNextVersionId(orphanData)); - } - - if (orphanData.state() == Splitting) { - return new ServiceUnitStateData(Splitting, orphanData.dstBroker(), selectedBroker.get(), - Map.copyOf(orphanData.splitServiceUnitToDestBroker()), - true, getNextVersionId(orphanData)); - } else { - return new ServiceUnitStateData(Owned, selectedBroker.get(), inactiveBroker, - true, getNextVersionId(orphanData)); - } - } - private void overrideOwnership(String serviceUnit, ServiceUnitStateData orphanData, String inactiveBroker) { - Optional selectedBroker = selectBroker(serviceUnit, inactiveBroker); - if (selectedBroker.isEmpty()) { - log.warn("Empty selected broker for ownership serviceUnit:{} orphanData:{}." - + "totalCleanupErrorCnt:{}", - serviceUnit, orphanData, totalCleanupErrorCnt.incrementAndGet()); - } - var override = getOverrideInactiveBrokerStateData(orphanData, selectedBroker, inactiveBroker); + final var version = getNextVersionId(orphanData); + final var override = selectBroker(serviceUnit, inactiveBroker).map(selectedBroker -> { + if (orphanData.state() == Splitting) { + return new ServiceUnitStateData(Splitting, orphanData.dstBroker(), selectedBroker, + Map.copyOf(orphanData.splitServiceUnitToDestBroker()), true, version); + } else { + return new ServiceUnitStateData(Owned, selectedBroker, inactiveBroker, true, version); + } + }).orElseGet(() -> new ServiceUnitStateData(Free, null, inactiveBroker, true, version)); log.info("Overriding ownership serviceUnit:{} from orphanData:{} to overrideData:{}", serviceUnit, orphanData, override); publishOverrideEventAsync(serviceUnit, orphanData, override) From 7befb8df945899ef4f047d503aa7fcbde5df245c Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 15 May 2024 22:31:12 +0800 Subject: [PATCH 193/580] [improve][build] Support custom image and label names (#22703) Signed-off-by: Zixuan Liu Co-authored-by: Lari Hotari --- docker/pulsar-all/pom.xml | 6 +++--- docker/pulsar/pom.xml | 4 ++-- pom.xml | 2 ++ tests/docker-images/java-test-image/pom.xml | 4 ++-- tests/docker-images/latest-version-image/Dockerfile | 6 ++++-- tests/docker-images/latest-version-image/pom.xml | 8 ++++++-- 6 files changed, 19 insertions(+), 11 deletions(-) diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 659153a82e6ab..f54e95fd8857c 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -151,17 +151,17 @@ - ${docker.organization}/pulsar-all + ${docker.organization}/${docker.image}-all ${project.basedir} - latest + ${docker.tag} ${project.version}-${git.commit.id.abbrev} target/apache-pulsar-io-connectors-${project.version}-bin target/pulsar-offloader-distribution-${project.version}-bin.tar.gz - ${docker.organization}/pulsar:${project.version}-${git.commit.id.abbrev} + ${docker.organization}/${docker.image}:${project.version}-${git.commit.id.abbrev} diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 523a4b1bb3f77..f9393ee343d93 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -78,7 +78,7 @@ - ${docker.organization}/pulsar + ${docker.organization}/${docker.image} target/pulsar-server-distribution-${project.version}-bin.tar.gz @@ -86,7 +86,7 @@ ${project.basedir} - latest + ${docker.tag} ${project.version}-${git.commit.id.abbrev} diff --git a/pom.xml b/pom.xml index 576c1b4e42299..4af94ee984a3a 100644 --- a/pom.xml +++ b/pom.xml @@ -131,6 +131,8 @@ flexible messaging model and an intuitive client API. /tmp kill apachepulsar + pulsar + latest false false package diff --git a/tests/docker-images/java-test-image/pom.xml b/tests/docker-images/java-test-image/pom.xml index 053ca3da6e0a6..1ea3a0cd205b0 100644 --- a/tests/docker-images/java-test-image/pom.xml +++ b/tests/docker-images/java-test-image/pom.xml @@ -151,11 +151,11 @@ ${docker.organization}/java-test-image - ${docker.organization}/pulsar:${project.version}-${git.commit.id.abbrev} + ${docker.organization}/${docker.image}:${project.version}-${git.commit.id.abbrev} ${project.basedir} - latest + ${docker.tag} ${project.version} true diff --git a/tests/docker-images/latest-version-image/Dockerfile b/tests/docker-images/latest-version-image/Dockerfile index c23341c0748a2..0645dd2e78aab 100644 --- a/tests/docker-images/latest-version-image/Dockerfile +++ b/tests/docker-images/latest-version-image/Dockerfile @@ -18,6 +18,8 @@ # # build go lang examples first in a separate layer +ARG PULSAR_ALL_IMAGE +ARG PULSAR_IMAGE FROM golang:1.21-alpine as pulsar-function-go @@ -27,12 +29,12 @@ RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go/pf && go install RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go/examples && go install ./... # Reference pulsar-all to copy connectors from there -FROM apachepulsar/pulsar-all:latest as pulsar-all +FROM $PULSAR_ALL_IMAGE as pulsar-all ######################################## ###### Main image build ######################################## -FROM apachepulsar/pulsar:latest +FROM $PULSAR_IMAGE # Switch to run as the root user to simplify building container and then running # supervisord. Each of the pulsar components are spawned by supervisord and their diff --git a/tests/docker-images/latest-version-image/pom.xml b/tests/docker-images/latest-version-image/pom.xml index cd6ca46951f1f..63ff82a0c2b81 100644 --- a/tests/docker-images/latest-version-image/pom.xml +++ b/tests/docker-images/latest-version-image/pom.xml @@ -152,11 +152,15 @@ - ${docker.organization}/pulsar-test-latest-version + ${docker.organization}/${docker.image}-test-latest-version ${project.basedir} + + ${docker.organization}/${docker.image}:${project.version}-${git.commit.id.abbrev} + ${docker.organization}/${docker.image}-all:${project.version}-${git.commit.id.abbrev} + - latest + ${docker.tag} ${project.version} true From f07b3a030179c38f9786b3e26c82aa13e00b34a6 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 16 May 2024 02:47:26 +0800 Subject: [PATCH 194/580] [improve] [broker] [break change] Do not create partitioned DLQ/Retry topic automatically (#22705) --- .../pulsar/broker/service/BrokerService.java | 6 + ...LetterTopicDefaultMultiPartitionsTest.java | 251 ++++++++++++++++++ 2 files changed, 257 insertions(+) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicDefaultMultiPartitionsTest.java 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 566ad1ff377e1..6603e240ee7d9 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 @@ -24,6 +24,8 @@ import static org.apache.bookkeeper.mledger.ManagedLedgerConfig.PROPERTY_SOURCE_TOPIC_KEY; import static org.apache.commons.collections4.CollectionUtils.isEmpty; import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.pulsar.client.util.RetryMessageUtil.DLQ_GROUP_TOPIC_SUFFIX; +import static org.apache.pulsar.client.util.RetryMessageUtil.RETRY_GROUP_TOPIC_SUFFIX; import static org.apache.pulsar.common.naming.SystemTopicNames.isTransactionInternalName; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Queues; @@ -3493,6 +3495,10 @@ private CompletableFuture isAllowAutoTopicCreationAsync(final TopicName } public boolean isDefaultTopicTypePartitioned(final TopicName topicName, final Optional policies) { + if (topicName.getPartitionedTopicName().endsWith(DLQ_GROUP_TOPIC_SUFFIX) + || topicName.getPartitionedTopicName().endsWith(RETRY_GROUP_TOPIC_SUFFIX)) { + return false; + } AutoTopicCreationOverride autoTopicCreationOverride = getAutoTopicCreationOverride(topicName, policies); if (autoTopicCreationOverride != null) { return TopicType.PARTITIONED.toString().equals(autoTopicCreationOverride.getTopicType()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicDefaultMultiPartitionsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicDefaultMultiPartitionsTest.java new file mode 100644 index 0000000000000..b8bccb793724b --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicDefaultMultiPartitionsTest.java @@ -0,0 +1,251 @@ +/* + * 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 static org.apache.pulsar.client.util.RetryMessageUtil.DLQ_GROUP_TOPIC_SUFFIX; +import static org.testng.Assert.assertFalse; +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.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.TopicType; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-impl") +public class DeadLetterTopicDefaultMultiPartitionsTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + this.conf.setMaxMessageSize(5 * 1024); + this.conf.setAllowAutoTopicCreation(true); + this.conf.setDefaultNumPartitions(2); + this.conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + private void triggerDLQGenerate(String topic, String subscription) throws Exception { + String DLQ = getDLQName(topic, subscription); + String p0OfDLQ = TopicName.get(DLQ).getPartition(0).toString(); + Consumer consumer = pulsarClient.newConsumer().topic(topic).subscriptionName(subscription) + .ackTimeout(1000, TimeUnit.MILLISECONDS) + .subscriptionType(SubscriptionType.Shared) + .receiverQueueSize(10) + .negativeAckRedeliveryDelay(100, TimeUnit.MILLISECONDS) + .deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(1).build()) + .subscribe(); + Producer producer = pulsarClient.newProducer().topic(topic).create(); + producer.newMessage().value(new byte[]{1}).send(); + + Message message1 = consumer.receive(); + consumer.negativeAcknowledge(message1); + Message message2 = consumer.receive(); + consumer.negativeAcknowledge(message2); + + Awaitility.await().atMost(Duration.ofSeconds(1500)).until(() -> { + Message message3 = consumer.receive(2, TimeUnit.SECONDS); + if (message3 != null) { + log.info("===> {}", message3.getRedeliveryCount()); + consumer.negativeAcknowledge(message3); + } + List topicList = pulsar.getPulsarResources().getTopicResources() + .listPersistentTopicsAsync(TopicName.get(topic).getNamespaceObject()).join(); + if (topicList.contains(DLQ) || topicList.contains(p0OfDLQ)) { + return true; + } + int partitions = admin.topics().getPartitionedTopicMetadata(topic).partitions; + for (int i = 0; i < partitions; i++) { + for (int j = -1; j < pulsar.getConfig().getDefaultNumPartitions(); j++) { + String p0OfDLQ2; + if (j == -1) { + p0OfDLQ2 = TopicName + .get(getDLQName(TopicName.get(topic).getPartition(i).toString(), subscription)) + .toString(); + } else { + p0OfDLQ2 = TopicName + .get(getDLQName(TopicName.get(topic).getPartition(i).toString(), subscription)) + .getPartition(j).toString(); + } + if (topicList.contains(p0OfDLQ2)) { + return true; + } + } + } + return false; + }); + producer.close(); + consumer.close(); + admin.topics().unload(topic); + } + + private static String getDLQName(String primaryTopic, String subscription) { + String domain = TopicName.get(primaryTopic).getDomain().toString(); + return domain + "://" + TopicName.get(primaryTopic) + .toString().substring(( domain + "://").length()) + + "-" + subscription + DLQ_GROUP_TOPIC_SUFFIX; + } + + @DataProvider(name = "topicCreationTypes") + public Object[][] topicCreationTypes() { + return new Object[][]{ + //{TopicType.NON_PARTITIONED}, + {TopicType.PARTITIONED} + }; + } + + @Test(dataProvider = "topicCreationTypes") + public void testGenerateNonPartitionedDLQ(TopicType topicType) throws Exception { + final String topic = BrokerTestUtil.newUniqueName( "persistent://public/default/tp"); + final String subscription = "s1"; + switch (topicType) { + case PARTITIONED: { + admin.topics().createPartitionedTopic(topic, 2); + break; + } + case NON_PARTITIONED: { + admin.topics().createNonPartitionedTopic(topic); + } + } + + triggerDLQGenerate(topic, subscription); + + // Verify: no partitioned DLQ. + List partitionedTopics = pulsar.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources() + .listPartitionedTopicsAsync(TopicName.get(topic).getNamespaceObject(), TopicDomain.persistent).join(); + for (String tp : partitionedTopics) { + assertFalse(tp.endsWith("-DLQ")); + } + // Verify: non-partitioned DLQ exists. + List partitions = pulsar.getPulsarResources().getTopicResources() + .listPersistentTopicsAsync(TopicName.get(topic).getNamespaceObject()).join(); + List DLQCreated = new ArrayList<>(); + for (String tp : partitions) { + if (tp.endsWith("-DLQ")) { + DLQCreated.add(tp); + } + assertFalse(tp.endsWith("-partition-0-DLQ")); + } + assertTrue(!DLQCreated.isEmpty()); + + // cleanup. + switch (topicType) { + case PARTITIONED: { + admin.topics().deletePartitionedTopic(topic); + break; + } + case NON_PARTITIONED: { + admin.topics().delete(topic, false); + } + } + for (String t : DLQCreated) { + try { + admin.topics().delete(TopicName.get(t).getPartitionedTopicName(), false); + } catch (Exception ex) {} + try { + admin.topics().deletePartitionedTopic(TopicName.get(t).getPartitionedTopicName(), false); + } catch (Exception ex) {} + } + } + + @Test + public void testManuallyCreatePartitionedDLQ() throws Exception { + final String topic = BrokerTestUtil.newUniqueName( "persistent://public/default/tp"); + final String subscription = "s1"; + String DLQ = getDLQName(topic, subscription); + String p0OfDLQ = TopicName.get(DLQ).getPartition(0).toString(); + String p1OfDLQ = TopicName.get(DLQ).getPartition(1).toString(); + admin.topics().createNonPartitionedTopic(topic); + admin.topics().createPartitionedTopic(DLQ, 2); + + Awaitility.await().untilAsserted(() -> { + // Verify: partitioned DLQ exists. + List partitionedTopics = pulsar.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources() + .listPartitionedTopicsAsync(TopicName.get(topic).getNamespaceObject(), TopicDomain.persistent).join(); + assertTrue(partitionedTopics.contains(DLQ)); + assertFalse(partitionedTopics.contains(p0OfDLQ)); + // Verify: DLQ partitions exists. + List partitions = pulsar.getPulsarResources().getTopicResources() + .listPersistentTopicsAsync(TopicName.get(topic).getNamespaceObject()).join(); + assertFalse(partitions.contains(DLQ)); + assertTrue(partitions.contains(p0OfDLQ)); + assertTrue(partitions.contains(p1OfDLQ)); + }); + + // cleanup. + admin.topics().delete(topic, false); + admin.topics().deletePartitionedTopic(DLQ, false); + } + + @Test + public void testManuallyCreatePartitionedDLQ2() throws Exception { + final String topic = BrokerTestUtil.newUniqueName( "persistent://public/default/tp"); + final String subscription = "s1"; + final String p0OfTopic = TopicName.get(topic).getPartition(0).toString(); + String DLQ = getDLQName(p0OfTopic, subscription); + String p0OfDLQ = TopicName.get(DLQ).getPartition(0).toString(); + admin.topics().createPartitionedTopic(topic, 10); + try { + admin.topics().createPartitionedTopic(DLQ, 2); + } catch (Exception ex) { + // Keep multiple versions compatible. + if (ex.getMessage().contains("Partitioned Topic Name should not contain '-partition-'")){ + return; + } else { + fail("Failed to create partitioned DLQ"); + } + } + + Awaitility.await().untilAsserted(() -> { + // Verify: partitioned DLQ exists. + List partitionedTopics = pulsar.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources() + .listPartitionedTopicsAsync(TopicName.get(topic).getNamespaceObject(), TopicDomain.persistent).join(); + assertTrue(partitionedTopics.contains(DLQ)); + assertFalse(partitionedTopics.contains(p0OfDLQ)); + // Verify: DLQ partitions exists. + List partitions = pulsar.getPulsarResources().getTopicResources() + .listPersistentTopicsAsync(TopicName.get(topic).getNamespaceObject()).join(); + assertFalse(partitions.contains(DLQ)); + }); + + // cleanup. + admin.topics().deletePartitionedTopic(topic, false); + admin.topics().deletePartitionedTopic(DLQ, false); + } +} From f3e52b568ec7e86e7582bdc425321fe172bc4deb Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Thu, 16 May 2024 13:29:26 +0800 Subject: [PATCH 195/580] [improve][pip] PIP-347: Add role field in consumer's stat (#22562) --- pip/pip-347.md | 2 +- .../pulsar/broker/service/Consumer.java | 1 + .../stats/AuthenticatedConsumerStatsTest.java | 169 ++++++++++++++++++ .../broker/stats/ConsumerStatsTest.java | 2 +- .../common/policies/data/ConsumerStats.java | 3 + .../data/stats/ConsumerStatsImpl.java | 3 + 6 files changed, 178 insertions(+), 2 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/AuthenticatedConsumerStatsTest.java diff --git a/pip/pip-347.md b/pip/pip-347.md index 5326fed353374..a5d5d76ae1700 100644 --- a/pip/pip-347.md +++ b/pip/pip-347.md @@ -34,4 +34,4 @@ Fully compatible. Updated afterwards --> * Mailing List discussion thread: https://lists.apache.org/thread/p9y9r8pb7ygk8f0jd121c1121phvzd09 -* Mailing List voting thread: +* Mailing List voting thread: https://lists.apache.org/thread/sfv0vq498dnjx6k6zdrnn0cw8f22tz05 diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index fe9fbe6a4000c..c9f417c4bc4f7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -208,6 +208,7 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo stats = new ConsumerStatsImpl(); stats.setAddress(cnx.clientSourceAddressAndPort()); stats.consumerName = consumerName; + stats.appId = appId; stats.setConnectedSince(DateFormatter.format(connectedSince)); stats.setClientVersion(cnx.getClientVersion()); stats.metadata = this.metadata; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/AuthenticatedConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/AuthenticatedConsumerStatsTest.java new file mode 100644 index 0000000000000..e8cadb72e1e04 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/AuthenticatedConsumerStatsTest.java @@ -0,0 +1,169 @@ +/* + * 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.stats; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Sets; +import io.jsonwebtoken.Jwts; +import io.jsonwebtoken.SignatureAlgorithm; +import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; +import org.apache.pulsar.client.admin.PulsarAdminBuilder; +import org.apache.pulsar.client.api.AuthenticationFactory; +import org.apache.pulsar.client.api.ClientBuilder; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.policies.data.ConsumerStats; +import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.security.KeyPair; +import java.security.KeyPairGenerator; +import java.security.NoSuchAlgorithmException; +import java.security.PrivateKey; +import java.time.Duration; +import java.util.Base64; +import java.util.Date; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Properties; +import java.util.Set; + +public class AuthenticatedConsumerStatsTest extends ConsumerStatsTest{ + private final String ADMIN_TOKEN; + private final String TOKEN_PUBLIC_KEY; + private final KeyPair kp; + + AuthenticatedConsumerStatsTest() throws NoSuchAlgorithmException { + KeyPairGenerator kpg = KeyPairGenerator.getInstance("RSA"); + kp = kpg.generateKeyPair(); + + byte[] encodedPublicKey = kp.getPublic().getEncoded(); + TOKEN_PUBLIC_KEY = "data:;base64," + Base64.getEncoder().encodeToString(encodedPublicKey); + ADMIN_TOKEN = generateToken(kp, "admin"); + } + + + private String generateToken(KeyPair kp, String subject) { + PrivateKey pkey = kp.getPrivate(); + long expMillis = System.currentTimeMillis() + Duration.ofHours(1).toMillis(); + Date exp = new Date(expMillis); + + return Jwts.builder() + .setSubject(subject) + .setExpiration(exp) + .signWith(pkey, SignatureAlgorithm.forSigningKey(pkey)) + .compact(); + } + + @Override + protected void customizeNewPulsarClientBuilder(ClientBuilder clientBuilder) { + clientBuilder.authentication(AuthenticationFactory.token(ADMIN_TOKEN)); + } + + @Override + protected void customizeNewPulsarAdminBuilder(PulsarAdminBuilder pulsarAdminBuilder) { + pulsarAdminBuilder.authentication(AuthenticationFactory.token(ADMIN_TOKEN)); + } + + @BeforeMethod + @Override + protected void setup() throws Exception { + conf.setAuthenticationEnabled(true); + conf.setAuthorizationEnabled(true); + + Set superUserRoles = new HashSet<>(); + superUserRoles.add("admin"); + conf.setSuperUserRoles(superUserRoles); + + Set providers = new HashSet<>(); + providers.add(AuthenticationProviderToken.class.getName()); + conf.setAuthenticationProviders(providers); + conf.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); + conf.setBrokerClientAuthenticationParameters("token:" + ADMIN_TOKEN); + + conf.setClusterName("test"); + + // Set provider domain name + Properties properties = new Properties(); + properties.setProperty("tokenPublicKey", TOKEN_PUBLIC_KEY); + conf.setProperties(properties); + + super.internalSetup(); + super.producerBaseSetup(); + } + + @Test + public void testConsumerStatsOutput() throws Exception { + Set allowedFields = Sets.newHashSet( + "msgRateOut", + "msgThroughputOut", + "bytesOutCounter", + "msgOutCounter", + "messageAckRate", + "msgRateRedeliver", + "chunkedMessageRate", + "consumerName", + "availablePermits", + "unackedMessages", + "avgMessagesPerEntry", + "blockedConsumerOnUnackedMsgs", + "readPositionWhenJoining", + "lastAckedTime", + "lastAckedTimestamp", + "lastConsumedTime", + "lastConsumedTimestamp", + "lastConsumedFlowTimestamp", + "keyHashRanges", + "metadata", + "address", + "connectedSince", + "clientVersion", + "appId"); + + final String topicName = "persistent://public/default/testConsumerStatsOutput"; + final String subName = "my-subscription"; + + Consumer consumer = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionType(SubscriptionType.Shared) + .subscriptionName(subName) + .subscribe(); + + TopicStats stats = admin.topics().getStats(topicName); + ObjectMapper mapper = ObjectMapperFactory.create(); + ConsumerStats consumerStats = stats.getSubscriptions() + .get(subName).getConsumers().get(0); + Assert.assertTrue(consumerStats.getLastConsumedFlowTimestamp() > 0); + JsonNode node = mapper.readTree(mapper.writer().writeValueAsString(consumerStats)); + Iterator itr = node.fieldNames(); + while (itr.hasNext()) { + String field = itr.next(); + Assert.assertTrue(allowedFields.contains(field), field + " should not be exposed"); + } + // assert that role is exposed + Assert.assertEquals(consumerStats.getAppId(), "admin"); + consumer.close(); + } + +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index 512a5cfcab661..024d8582fa213 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -195,7 +195,7 @@ public void testAckStatsOnPartitionedTopicForExclusiveSubscription() throws Puls @Test public void testUpdateStatsForActiveConsumerAndSubscription() throws Exception { - final String topicName = "persistent://prop/use/ns-abc/testUpdateStatsForActiveConsumerAndSubscription"; + final String topicName = "persistent://public/default/testUpdateStatsForActiveConsumerAndSubscription"; pulsarClient.newConsumer() .topic(topicName) .subscriptionType(SubscriptionType.Shared) diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java index 8c9a615d6d01c..d2d3600df96ed 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java @@ -25,6 +25,9 @@ * Consumer statistics. */ public interface ConsumerStats { + /** the app id. */ + String getAppId(); + /** Total rate of messages delivered to the consumer (msg/s). */ double getMsgRateOut(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java index 548abdc9ada33..de36b330b7f1a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java @@ -30,6 +30,9 @@ */ @Data public class ConsumerStatsImpl implements ConsumerStats { + /** the app id. */ + public String appId; + /** Total rate of messages delivered to the consumer (msg/s). */ public double msgRateOut; From 101aee4543fb66035165d8744def630f9a9c3a59 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 16 May 2024 17:36:57 +0800 Subject: [PATCH 196/580] [fix][schema] Error checking schema compatibility on a schema-less topic via REST API (#22720) --- .../AvroSchemaBasedCompatibilityCheck.java | 6 ++-- ...rotobufNativeSchemaCompatibilityCheck.java | 4 ++- .../schema/SchemaRegistryServiceImpl.java | 2 +- .../IncompatibleSchemaException.java | 4 +++ .../broker/admin/AdminApiSchemaTest.java | 30 +++++++++++++++++++ .../service/schema/SchemaServiceTest.java | 4 +-- 6 files changed, 43 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/AvroSchemaBasedCompatibilityCheck.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/AvroSchemaBasedCompatibilityCheck.java index 1e75834a12988..e5fc7800c5170 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/AvroSchemaBasedCompatibilityCheck.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/AvroSchemaBasedCompatibilityCheck.java @@ -64,8 +64,10 @@ public void checkCompatible(Iterable from, SchemaData to, SchemaComp log.warn("Error during schema parsing: {}", e.getMessage()); throw new IncompatibleSchemaException(e); } catch (SchemaValidationException e) { - log.warn("Error during schema compatibility check: {}", e.getMessage()); - throw new IncompatibleSchemaException(e); + String msg = String.format("Error during schema compatibility check with strategy %s: %s: %s", + strategy, e.getClass().getName(), e.getMessage()); + log.warn(msg); + throw new IncompatibleSchemaException(msg, e); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/ProtobufNativeSchemaCompatibilityCheck.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/ProtobufNativeSchemaCompatibilityCheck.java index 16b3b33ec7894..fc935e80dca36 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/ProtobufNativeSchemaCompatibilityCheck.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/ProtobufNativeSchemaCompatibilityCheck.java @@ -67,7 +67,9 @@ public void checkCompatible(Iterable from, SchemaData to, SchemaComp private void checkRootMessageChange(Descriptor fromDescriptor, Descriptor toDescriptor, SchemaCompatibilityStrategy strategy) throws IncompatibleSchemaException { if (!fromDescriptor.getFullName().equals(toDescriptor.getFullName())) { - throw new IncompatibleSchemaException("Protobuf root message isn't allow change!"); + throw new IncompatibleSchemaException("Protobuf root message change is not allowed under the '" + + strategy + "' strategy. Original message name: '" + fromDescriptor.getFullName() + + "', new message name: '" + toDescriptor.getFullName() + "'."); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java index ae56df248d85d..903f57cb7803a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java @@ -473,7 +473,7 @@ private CompletableFuture checkCompatibilityWithLatest(String schemaId, Sc } return result; } else { - return FutureUtils.exception(new IncompatibleSchemaException("Do not have existing schema.")); + return CompletableFuture.completedFuture(null); } }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/IncompatibleSchemaException.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/IncompatibleSchemaException.java index c1a2d9fd703fd..bbe2f4111d759 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/IncompatibleSchemaException.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/IncompatibleSchemaException.java @@ -33,6 +33,10 @@ public IncompatibleSchemaException(String message) { super(message); } + public IncompatibleSchemaException(String message, Throwable e) { + super(message, e); + } + public IncompatibleSchemaException(Throwable e) { super(e); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaTest.java index f67bd6fcfce5b..34d7dbeb8183c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiSchemaTest.java @@ -467,4 +467,34 @@ public void testCompatibility() throws Exception { assertTrue(e.getMessage().contains("Incompatible schema: exists schema type STRING, new schema type INT8")); } } + + @Test + public void testCompatibilityWithEmpty() throws Exception { + List> checkSchemas = List.of( + Schema.STRING, + Schema.JSON(SchemaDefinition.builder().withPojo(Foo.class).withProperties(PROPS).build()), + Schema.AVRO(SchemaDefinition.builder().withPojo(Foo.class).withProperties(PROPS).build()), + Schema.KeyValue(Schema.STRING, Schema.STRING) + ); + for (Schema schema : checkSchemas) { + SchemaInfo schemaInfo = schema.getSchemaInfo(); + String topicName = schemaCompatibilityNamespace + "/testCompatibilityWithEmpty"; + PostSchemaPayload postSchemaPayload = new PostSchemaPayload(schemaInfo.getType().toString(), + schemaInfo.getSchemaDefinition(), new HashMap<>()); + + // check compatibility with empty schema + IsCompatibilityResponse isCompatibilityResponse = + admin.schemas().testCompatibility(topicName, postSchemaPayload); + assertTrue(isCompatibilityResponse.isCompatibility()); + assertEquals(isCompatibilityResponse.getSchemaCompatibilityStrategy(), SchemaCompatibilityStrategy.FULL.name()); + + // set schema compatibility strategy is FULL_TRANSITIVE to cover checkCompatibilityWithAll + admin.namespaces().setSchemaCompatibilityStrategy(schemaCompatibilityNamespace, SchemaCompatibilityStrategy.FULL_TRANSITIVE); + isCompatibilityResponse = admin.schemas().testCompatibility(topicName, postSchemaPayload); + assertTrue(isCompatibilityResponse.isCompatibility()); + assertEquals(isCompatibilityResponse.getSchemaCompatibilityStrategy(), SchemaCompatibilityStrategy.FULL_TRANSITIVE.name()); + // set back to FULL + admin.namespaces().setSchemaCompatibilityStrategy(schemaCompatibilityNamespace, SchemaCompatibilityStrategy.FULL); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java index 3a4016eb79c21..fbf8c5cc15444 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java @@ -20,7 +20,6 @@ import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; -import static org.testng.Assert.assertThrows; import static org.testng.Assert.fail; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertFalse; @@ -48,7 +47,6 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; -import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; @@ -407,7 +405,7 @@ public void testKeyValueSchema() throws Exception { .build(), SchemaInfo.builder().type(SchemaType.BOOLEAN).schema(new byte[0]) .build(), KeyValueEncodingType.SEPARATED); - assertThrows(PulsarAdminException.ServerSideErrorException.class, () -> admin.schemas().testCompatibility(topicName, schemaInfo)); + Assert.assertTrue(admin.schemas().testCompatibility(topicName, schemaInfo).isCompatibility()); admin.schemas().createSchema(topicName, schemaInfo); final IsCompatibilityResponse isCompatibilityResponse = admin.schemas().testCompatibility(topicName, schemaInfo); From 73fd61db646da44d51347e3dcc194ea23b64b4ff Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Thu, 16 May 2024 20:54:32 +0800 Subject: [PATCH 197/580] [improve][offload] Allow to disable the managedLedgerOffloadDeletionLagInMillis (#22689) --- .../org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java | 3 ++- .../bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java | 4 ++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index ab32806fbae84..b3426692df308 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -2578,6 +2578,7 @@ boolean isOffloadedNeedsDelete(OffloadContext offload, Optional long elapsedMs = clock.millis() - offload.getTimestamp(); return offloadPolicies.filter(policies -> offload.getComplete() && !offload.getBookkeeperDeleted() && policies.getManagedLedgerOffloadDeletionLagInMillis() != null + && policies.getManagedLedgerOffloadDeletionLagInMillis() >= 0 && elapsedMs > policies.getManagedLedgerOffloadDeletionLagInMillis()).isPresent(); } @@ -4559,4 +4560,4 @@ public Position getTheSlowestNonDurationReadPosition() { } return theSlowestNonDurableReadPosition; } -} \ No newline at end of file +} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java index 56da315553ea4..b46f06106cf4c 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java @@ -383,6 +383,10 @@ public void isOffloadedNeedsDeleteTest() throws Exception { needsDelete = managedLedger.isOffloadedNeedsDelete(offloadContext, Optional.of(offloadPolicies)); Assert.assertTrue(needsDelete); + offloadPolicies.setManagedLedgerOffloadDeletionLagInMillis(-1L); + needsDelete = managedLedger.isOffloadedNeedsDelete(offloadContext, Optional.of(offloadPolicies)); + Assert.assertFalse(needsDelete); + offloadPolicies.setManagedLedgerOffloadDeletionLagInMillis(1000L * 2); needsDelete = managedLedger.isOffloadedNeedsDelete(offloadContext, Optional.of(offloadPolicies)); Assert.assertFalse(needsDelete); From 4e132d3f2919767cef6fa935a59721937cb668e7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 17 May 2024 01:18:49 +0800 Subject: [PATCH 198/580] [improve] [test] Add a test to guarantee the TNX topics will not be replicated (#22721) --- .../broker/service/OneWayReplicatorTest.java | 9 - .../service/OneWayReplicatorTestBase.java | 44 ++- .../broker/service/ReplicationTxnTest.java | 262 ++++++++++++++++++ 3 files changed, 297 insertions(+), 18 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index 99fd4d877c173..fae72e8eac242 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -96,15 +96,6 @@ public void cleanup() throws Exception { super.cleanup(); } - private void waitReplicatorStarted(String topicName) { - Awaitility.await().untilAsserted(() -> { - Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); - assertTrue(topicOptional2.isPresent()); - PersistentTopic persistentTopic2 = (PersistentTopic) topicOptional2.get(); - assertFalse(persistentTopic2.getProducers().isEmpty()); - }); - } - private void waitReplicatorStopped(String topicName) { Awaitility.await().untilAsserted(() -> { Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index b4eed00c4470f..317e43306e356 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.service; import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; import com.google.common.collect.Sets; import java.net.URL; import java.time.Duration; @@ -29,6 +31,7 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.policies.data.ClusterData; @@ -55,7 +58,7 @@ public abstract class OneWayReplicatorTestBase extends TestRetrySupport { protected ZookeeperServerTest brokerConfigZk1; protected LocalBookkeeperEnsemble bkEnsemble1; protected PulsarService pulsar1; - protected BrokerService ns1; + protected BrokerService broker1; protected PulsarAdmin admin1; protected PulsarClient client1; @@ -66,7 +69,7 @@ public abstract class OneWayReplicatorTestBase extends TestRetrySupport { protected ZookeeperServerTest brokerConfigZk2; protected LocalBookkeeperEnsemble bkEnsemble2; protected PulsarService pulsar2; - protected BrokerService ns2; + protected BrokerService broker2; protected PulsarAdmin admin2; protected PulsarClient client2; @@ -89,23 +92,29 @@ protected void startBrokers() throws Exception { setConfigDefaults(config1, cluster1, bkEnsemble1, brokerConfigZk1); pulsar1 = new PulsarService(config1); pulsar1.start(); - ns1 = pulsar1.getBrokerService(); - + broker1 = pulsar1.getBrokerService(); url1 = new URL(pulsar1.getWebServiceAddress()); urlTls1 = new URL(pulsar1.getWebServiceAddressTls()); - admin1 = PulsarAdmin.builder().serviceHttpUrl(url1.toString()).build(); - client1 = PulsarClient.builder().serviceUrl(url1.toString()).build(); // Start region 2 setConfigDefaults(config2, cluster2, bkEnsemble2, brokerConfigZk2); pulsar2 = new PulsarService(config2); pulsar2.start(); - ns2 = pulsar2.getBrokerService(); - + broker2 = pulsar2.getBrokerService(); url2 = new URL(pulsar2.getWebServiceAddress()); urlTls2 = new URL(pulsar2.getWebServiceAddressTls()); + } + + protected void startAdminClient() throws Exception { + admin1 = PulsarAdmin.builder().serviceHttpUrl(url1.toString()).build(); admin2 = PulsarAdmin.builder().serviceHttpUrl(url2.toString()).build(); - client2 = PulsarClient.builder().serviceUrl(url2.toString()).build(); + } + + protected void startPulsarClient() throws Exception{ + ClientBuilder clientBuilder1 = PulsarClient.builder().serviceUrl(url1.toString()); + client1 = initClient(clientBuilder1); + ClientBuilder clientBuilder2 = PulsarClient.builder().serviceUrl(url2.toString()); + client2 = initClient(clientBuilder2); } protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { @@ -196,8 +205,12 @@ protected void setup() throws Exception { startBrokers(); + startAdminClient(); + createDefaultTenantsAndClustersAndNamespace(); + startPulsarClient(); + Thread.sleep(100); log.info("--- OneWayReplicatorTestBase::setup completed ---"); } @@ -287,4 +300,17 @@ protected void cleanup() throws Exception { config1 = new ServiceConfiguration(); config2 = new ServiceConfiguration(); } + + protected void waitReplicatorStarted(String topicName) { + Awaitility.await().untilAsserted(() -> { + Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); + assertTrue(topicOptional2.isPresent()); + PersistentTopic persistentTopic2 = (PersistentTopic) topicOptional2.get(); + assertFalse(persistentTopic2.getProducers().isEmpty()); + }); + } + + protected PulsarClient initClient(ClientBuilder clientBuilder) throws Exception { + return clientBuilder.build(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java new file mode 100644 index 0000000000000..3caf4a1f2398c --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java @@ -0,0 +1,262 @@ +/* + * 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.service; + +import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; +import static org.apache.pulsar.transaction.coordinator.impl.MLTransactionLogImpl.TRANSACTION_LOG_PREFIX; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import com.google.common.collect.Sets; +import java.util.Collections; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; +import org.apache.pulsar.client.api.ClientBuilder; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.apache.pulsar.zookeeper.ZookeeperServerTest; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class ReplicationTxnTest extends OneWayReplicatorTestBase { + + private boolean transactionBufferSegmentedSnapshotEnabled = false; + private int txnLogPartitions = 4; + + @Override + @BeforeClass(alwaysRun = true, timeOut = 300000) + public void setup() throws Exception { + super.setup(); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + @Override + protected PulsarClient initClient(ClientBuilder clientBuilder) throws Exception { + return clientBuilder.enableTransaction(true).build(); + } + + @Override + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, + LocalBookkeeperEnsemble bookkeeperEnsemble, ZookeeperServerTest brokerConfigZk) { + super.setConfigDefaults(config, clusterName, bookkeeperEnsemble, brokerConfigZk); + config.setSystemTopicEnabled(true); + config.setTopicLevelPoliciesEnabled(true); + config.setTransactionCoordinatorEnabled(true); + config.setTransactionLogBatchedWriteEnabled(true); + config.setTransactionPendingAckBatchedWriteEnabled(true); + config.setTransactionBufferSegmentedSnapshotEnabled(transactionBufferSegmentedSnapshotEnabled); + } + + @Override + protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { + super.createDefaultTenantsAndClustersAndNamespace(); + + // Create resource that transaction function relies on. + admin1.tenants().createTenant(SYSTEM_NAMESPACE.getTenant(), new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(cluster1, cluster2))); + admin1.namespaces().createNamespace(SYSTEM_NAMESPACE.toString(), 4); + pulsar1.getPulsarResources().getNamespaceResources().getPartitionedTopicResources().createPartitionedTopic( + SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, new PartitionedTopicMetadata(txnLogPartitions)); + //admin1.topics().createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN.toString(), 4); + + admin2.tenants().createTenant(SYSTEM_NAMESPACE.getTenant(), new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(cluster1, cluster2))); + admin2.namespaces().createNamespace(SYSTEM_NAMESPACE.toString(), 4); + pulsar2.getPulsarResources().getNamespaceResources().getPartitionedTopicResources().createPartitionedTopic( + SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, new PartitionedTopicMetadata(txnLogPartitions)); + } + + private void pubAndSubOneMsg(String topic, String subscription) throws Exception { + Consumer consumer1 = client1.newConsumer(Schema.STRING).topic(topic).subscriptionName(subscription) + .isAckReceiptEnabled(true).subscribe(); + Producer producer1 = client1.newProducer(Schema.STRING).topic(topic).create(); + producer1.newMessage().value("msg1").send(); + // start txn. + Transaction txn = client1.newTransaction().withTransactionTimeout(1, TimeUnit.MINUTES).build().get(); + // consume. + Message c1Msg1 = consumer1.receive(5, TimeUnit.SECONDS); + assertNotNull(c1Msg1); + assertEquals(c1Msg1.getValue(), "msg1"); + consumer1.acknowledgeAsync(c1Msg1.getMessageId(), txn).join(); + // send. + producer1.newMessage(txn).value("msg2").send(); + // commit. + txn.commit().get(); + + // Consume the msg with TXN. + Message c1Msg2 = consumer1.receive(5, TimeUnit.SECONDS); + assertNotNull(c1Msg2); + assertEquals(c1Msg2.getValue(), "msg2"); + consumer1.acknowledgeAsync(c1Msg2.getMessageId()).join(); + + // Consume messages on the remote cluster. + Consumer consumer2 = client2.newConsumer(Schema.STRING).topic(topic).subscriptionName(subscription).subscribe(); + Message c2Msg1 = consumer2.receive(15, TimeUnit.SECONDS); + assertNotNull(c2Msg1); + MessageMetadata msgMetadata1 = WhiteboxImpl.getInternalState(c2Msg1, "msgMetadata"); + // Verify: the messages replicated has no TXN id. + assertFalse(msgMetadata1.hasTxnidMostBits()); + assertFalse(msgMetadata1.hasTxnidLeastBits()); + consumer2.acknowledge(c2Msg1); + Message c2Msg2 = consumer2.receive(15, TimeUnit.SECONDS); + assertNotNull(c2Msg2); + MessageMetadata msgMetadata2 = WhiteboxImpl.getInternalState(c2Msg2, "msgMetadata"); + // Verify: the messages replicated has no TXN id. + assertFalse(msgMetadata2.hasTxnidMostBits()); + assertFalse(msgMetadata2.hasTxnidLeastBits()); + consumer2.acknowledge(c2Msg2); + + // cleanup. + producer1.close(); + consumer1.close(); + consumer2.close(); + } + + private void verifyNoReplicator(BrokerService broker, TopicName topicName) throws Exception { + String tpStr = topicName.toString(); + CompletableFuture> future = broker.getTopic(tpStr, true); + if (future == null) { + return; + } + PersistentTopic persistentTopic = (PersistentTopic) future.join().get(); + assertTrue(persistentTopic.getReplicators().isEmpty()); + } + + @Test + public void testTxnLogNotBeReplicated() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp"); + final String subscription = "s1"; + admin1.topics().createNonPartitionedTopic(topic); + waitReplicatorStarted(topic); + admin1.topics().createSubscription(topic, subscription, MessageId.earliest); + admin2.topics().createSubscription(topic, subscription, MessageId.earliest); + // Pub & Sub. + pubAndSubOneMsg(topic, subscription); + // To cover more cases, sleep 3s. + Thread.sleep(3000); + + // Verify: messages on the TXN system topic did not been replicated. + // __transaction_log_: it only uses ML, will not create topic. + for (int i = 0; i < txnLogPartitions; i++) { + TopicName txnLog = TopicName.get(TopicDomain.persistent.value(), + NamespaceName.SYSTEM_NAMESPACE, TRANSACTION_LOG_PREFIX + i); + assertNotNull(pulsar1.getManagedLedgerFactory() + .getManagedLedgerInfo(txnLog.getPersistenceNamingEncoding())); + assertFalse(broker1.getTopics().containsKey(txnLog.toString())); + } + // __transaction_pending_ack: it only uses ML, will not create topic. + TopicName pendingAck = TopicName.get( + MLPendingAckStore.getTransactionPendingAckStoreSuffix(topic, subscription)); + assertNotNull(pulsar1.getManagedLedgerFactory() + .getManagedLedgerInfo(pendingAck.getPersistenceNamingEncoding())); + assertFalse(broker1.getTopics().containsKey(pendingAck.toString())); + // __transaction_buffer_snapshot. + verifyNoReplicator(broker1, TopicName.get(TopicDomain.persistent.value(), + TopicName.get(topic).getNamespaceObject(), + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT)); + verifyNoReplicator(broker1, TopicName.get(TopicDomain.persistent.value(), + TopicName.get(topic).getNamespaceObject(), + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS)); + verifyNoReplicator(broker1, TopicName.get(TopicDomain.persistent.value(), + TopicName.get(topic).getNamespaceObject(), + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES)); + + // cleanup. + cleanupTopics(() -> { + admin1.topics().delete(topic); + admin2.topics().delete(topic); + try { + admin1.topics().delete(pendingAck.toString()); + } catch (Exception ex) {} + try { + admin2.topics().delete(pendingAck.toString()); + } catch (Exception ex) {} + }); + } + + @Test + public void testOngoingMessagesWillNotBeReplicated() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp"); + final String subscription = "s1"; + admin1.topics().createNonPartitionedTopic(topic); + waitReplicatorStarted(topic); + admin1.topics().createSubscription(topic, subscription, MessageId.earliest); + admin2.topics().createSubscription(topic, subscription, MessageId.earliest); + // Pub without commit. + Producer producer1 = client1.newProducer(Schema.STRING).topic(topic).create(); + Transaction txn = client1.newTransaction().withTransactionTimeout(1, TimeUnit.HOURS).build().get(); + producer1.newMessage(txn).value("msg1").send(); + // Verify: receive nothing on the remote cluster. + Consumer consumer2 = client2.newConsumer(Schema.STRING).topic(topic).subscriptionName(subscription).subscribe(); + Message msg = consumer2.receive(15, TimeUnit.SECONDS); + assertNull(msg); + // Verify: the repl cursor is not end of the topic. + PersistentTopic persistentTopic = (PersistentTopic) broker1.getTopic(topic, false).join().get(); + GeoPersistentReplicator replicator = + (GeoPersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + assertTrue(replicator.getCursor().hasMoreEntries()); + + // cleanup. + producer1.close(); + consumer2.close(); + cleanupTopics(() -> { + admin1.topics().delete(topic); + admin2.topics().delete(topic); + TopicName pendingAck = TopicName.get( + MLPendingAckStore.getTransactionPendingAckStoreSuffix(topic, subscription)); + try { + admin1.topics().delete(pendingAck.toString()); + } catch (Exception ex) {} + try { + admin2.topics().delete(pendingAck.toString()); + } catch (Exception ex) {} + }); + } +} From fd5916cca6ee2041efa3947d19910e16d94d1bee Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Thu, 16 May 2024 22:17:38 -0700 Subject: [PATCH 199/580] [fix][broker] Make ExtensibleLoadManagerImpl.getOwnedServiceUnits async (#22727) --- .../extensions/ExtensibleLoadManagerImpl.java | 47 ++++++++----------- .../broker/namespace/NamespaceService.java | 19 +++++--- .../ExtensibleLoadManagerImplTest.java | 17 ++++--- 3 files changed, 43 insertions(+), 40 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 41832fb60075d..c22a4086a639d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -204,13 +204,14 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS /** * Get all the bundles that are owned by this broker. */ - public Set getOwnedServiceUnits() { + public CompletableFuture> getOwnedServiceUnitsAsync() { if (!started) { log.warn("Failed to get owned service units, load manager is not started."); - return Collections.emptySet(); + return CompletableFuture.completedFuture(Collections.emptySet()); } - Set> entrySet = serviceUnitStateChannel.getOwnershipEntrySet(); + String brokerId = brokerRegistry.getBrokerId(); + Set> entrySet = serviceUnitStateChannel.getOwnershipEntrySet(); Set ownedServiceUnits = entrySet.stream() .filter(entry -> { var stateData = entry.getValue(); @@ -223,34 +224,26 @@ public Set getOwnedServiceUnits() { }).collect(Collectors.toSet()); // Add heartbeat and SLA monitor namespace bundle. NamespaceName heartbeatNamespace = NamespaceService.getHeartbeatNamespace(brokerId, pulsar.getConfiguration()); - try { - NamespaceBundle fullBundle = pulsar.getNamespaceService().getNamespaceBundleFactory() - .getFullBundle(heartbeatNamespace); - ownedServiceUnits.add(fullBundle); - } catch (Exception e) { - log.warn("Failed to get heartbeat namespace bundle.", e); - } NamespaceName heartbeatNamespaceV2 = NamespaceService .getHeartbeatNamespaceV2(brokerId, pulsar.getConfiguration()); - try { - NamespaceBundle fullBundle = pulsar.getNamespaceService().getNamespaceBundleFactory() - .getFullBundle(heartbeatNamespaceV2); - ownedServiceUnits.add(fullBundle); - } catch (Exception e) { - log.warn("Failed to get heartbeat namespace V2 bundle.", e); - } - NamespaceName slaMonitorNamespace = NamespaceService .getSLAMonitorNamespace(brokerId, pulsar.getConfiguration()); - try { - NamespaceBundle fullBundle = pulsar.getNamespaceService().getNamespaceBundleFactory() - .getFullBundle(slaMonitorNamespace); - ownedServiceUnits.add(fullBundle); - } catch (Exception e) { - log.warn("Failed to get SLA Monitor namespace bundle.", e); - } - - return ownedServiceUnits; + return pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundleAsync(heartbeatNamespace) + .thenAccept(fullBundle -> ownedServiceUnits.add(fullBundle)).exceptionally(e -> { + log.warn("Failed to get heartbeat namespace bundle.", e); + return null; + }).thenCompose(__ -> pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundleAsync(heartbeatNamespaceV2)) + .thenAccept(fullBundle -> ownedServiceUnits.add(fullBundle)).exceptionally(e -> { + log.warn("Failed to get heartbeat namespace V2 bundle.", e); + return null; + }).thenCompose(__ -> pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundleAsync(slaMonitorNamespace)) + .thenAccept(fullBundle -> ownedServiceUnits.add(fullBundle)).exceptionally(e -> { + log.warn("Failed to get SLA Monitor namespace bundle.", e); + return null; + }).thenApply(__ -> ownedServiceUnits); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 44cdd6368fe79..96936b3a5c05c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -865,12 +865,12 @@ public CompletableFuture> getOwnedNameSpac if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); - var statusMap = extensibleLoadManager.getOwnedServiceUnits().stream() - .collect(Collectors.toMap(NamespaceBundle::toString, - bundle -> getNamespaceOwnershipStatus(true, - namespaceIsolationPolicies.getPolicyByNamespace( - bundle.getNamespaceObject())))); - return CompletableFuture.completedFuture(statusMap); + return extensibleLoadManager.getOwnedServiceUnitsAsync() + .thenApply(OwnedServiceUnits -> OwnedServiceUnits.stream() + .collect(Collectors.toMap(NamespaceBundle::toString, + bundle -> getNamespaceOwnershipStatus(true, + namespaceIsolationPolicies.getPolicyByNamespace( + bundle.getNamespaceObject()))))); } Collection> futures = ownershipCache.getOwnedBundlesAsync().values(); @@ -1187,7 +1187,12 @@ public OwnershipCache getOwnershipCache() { public Set getOwnedServiceUnits() { if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); - return extensibleLoadManager.getOwnedServiceUnits(); + try { + return extensibleLoadManager.getOwnedServiceUnitsAsync() + .get(config.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } } return ownershipCache.getOwnedBundles().values().stream().map(OwnedBundle::getNamespaceBundle) .collect(Collectors.toSet()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index a385b0d3c5cca..8b96ed04f64de 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -1579,13 +1579,15 @@ public void testGetOwnedServiceUnitsAndGetOwnedNamespaceStatus() throws Exceptio .getFullBundle(slaMonitorNamespacePulsar2); - Set ownedServiceUnitsByPulsar1 = primaryLoadManager.getOwnedServiceUnits(); + Set ownedServiceUnitsByPulsar1 = primaryLoadManager.getOwnedServiceUnitsAsync() + .get(5, TimeUnit.SECONDS); log.info("Owned service units: {}", ownedServiceUnitsByPulsar1); // heartbeat namespace bundle will own by pulsar1 assertTrue(ownedServiceUnitsByPulsar1.contains(bundle1)); assertTrue(ownedServiceUnitsByPulsar1.contains(bundle2)); assertTrue(ownedServiceUnitsByPulsar1.contains(slaBundle1)); - Set ownedServiceUnitsByPulsar2 = secondaryLoadManager.getOwnedServiceUnits(); + Set ownedServiceUnitsByPulsar2 = secondaryLoadManager.getOwnedServiceUnitsAsync() + .get(5, TimeUnit.SECONDS); log.info("Owned service units: {}", ownedServiceUnitsByPulsar2); assertTrue(ownedServiceUnitsByPulsar2.contains(bundle3)); assertTrue(ownedServiceUnitsByPulsar2.contains(bundle4)); @@ -1621,7 +1623,8 @@ private void assertOwnedServiceUnits( ExtensibleLoadManagerImpl extensibleLoadManager, NamespaceBundle bundle) throws PulsarAdminException { Awaitility.await().untilAsserted(() -> { - Set ownedBundles = extensibleLoadManager.getOwnedServiceUnits(); + Set ownedBundles = extensibleLoadManager.getOwnedServiceUnitsAsync() + .get(5, TimeUnit.SECONDS); assertTrue(ownedBundles.contains(bundle)); }); Map ownedNamespaces = @@ -1634,9 +1637,11 @@ private void assertOwnedServiceUnits( } @Test(timeOut = 30 * 1000) - public void testGetOwnedServiceUnitsWhenLoadManagerNotStart() { + public void testGetOwnedServiceUnitsWhenLoadManagerNotStart() + throws Exception { ExtensibleLoadManagerImpl loadManager = new ExtensibleLoadManagerImpl(); - Set ownedServiceUnits = loadManager.getOwnedServiceUnits(); + Set ownedServiceUnits = loadManager.getOwnedServiceUnitsAsync() + .get(5, TimeUnit.SECONDS); assertNotNull(ownedServiceUnits); assertTrue(ownedServiceUnits.isEmpty()); } @@ -1651,7 +1656,7 @@ public void testTryAcquiringOwnership() NamespaceEphemeralData namespaceEphemeralData = primaryLoadManager.tryAcquiringOwnership(bundle).get(); assertTrue(Set.of(pulsar1.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrl()) .contains(namespaceEphemeralData.getNativeUrl())); - admin.namespaces().deleteNamespace(namespace, true); + admin.namespaces().deleteNamespace(namespace); } @Test(timeOut = 30 * 1000) From 23d5e123d3bd11b79c24628b358806e7ef032cc3 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 17 May 2024 13:50:48 +0800 Subject: [PATCH 200/580] [improve] [broker] Do not call cursor.isCursorDataFullyPersistable if disabled dispatcherPauseOnAckStatePersistentEnabled (#22729) --- ...PersistentDispatcherMultipleConsumers.java | 9 ++++ ...SubscriptionPauseOnAckStatPersistTest.java | 50 +++++++++++++++++++ 2 files changed, 59 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 49a19c0fe3138..f20750fa0c20d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -1087,6 +1087,15 @@ public void afterAckMessages(Throwable exOfDeletion, Object ctxOfDeletion) { @Override public boolean checkAndResumeIfPaused() { boolean paused = blockedDispatcherOnCursorDataCanNotFullyPersist == TRUE; + // Calling "cursor.isCursorDataFullyPersistable()" will loop the collection "individualDeletedMessages". It is + // not a light method. + // If never enabled "dispatcherPauseOnAckStatePersistentEnabled", skip the following checks to improve + // performance. + if (!paused && !topic.isDispatcherPauseOnAckStatePersistentEnabled()){ + // "true" means no need to pause. + return true; + } + // Enabled "dispatcherPauseOnAckStatePersistentEnabled" before. boolean shouldPauseNow = !cursor.isCursorDataFullyPersistable() && topic.isDispatcherPauseOnAckStatePersistentEnabled(); // No need to change. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java index 9a4de8ecf21cc..36c36735c067e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java @@ -23,8 +23,12 @@ import java.util.Arrays; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService; @@ -38,6 +42,7 @@ import org.apache.pulsar.common.policies.data.TopicPolicies; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -549,4 +554,49 @@ public void testMultiConsumersPauseOnAckStatPersistNotAffectReplayRead(Subscript c4.close(); admin.topics().delete(tpName, false); } + + @Test(dataProvider = "multiConsumerSubscriptionTypes") + public void testNeverCallCursorIsCursorDataFullyPersistableIfDisabledTheFeature(SubscriptionType subscriptionType) + throws Exception { + final String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String mlName = TopicName.get(tpName).getPersistenceNamingEncoding(); + final String subscription = "s1"; + final int msgSendCount = 100; + // Inject a injection to record the counter of calling "cursor.isCursorDataFullyPersistable". + final ManagedLedgerImpl ml = (ManagedLedgerImpl) pulsar.getBrokerService().getManagedLedgerFactory().open(mlName); + final ManagedCursorImpl cursor = (ManagedCursorImpl) ml.openCursor(subscription); + final ManagedCursorImpl spyCursor = Mockito.spy(cursor); + AtomicInteger callingIsCursorDataFullyPersistableCounter = new AtomicInteger(); + Mockito.doAnswer(invocation -> { + callingIsCursorDataFullyPersistableCounter.incrementAndGet(); + return invocation.callRealMethod(); + }).when(spyCursor).isCursorDataFullyPersistable(); + final ManagedCursorContainer cursors = WhiteboxImpl.getInternalState(ml, "cursors"); + final ManagedCursorContainer activeCursors = WhiteboxImpl.getInternalState(ml, "activeCursors"); + cursors.removeCursor(cursor.getName()); + activeCursors.removeCursor(cursor.getName()); + cursors.add(spyCursor, null); + activeCursors.add(spyCursor, null); + + // Pub & Sub. + Consumer c1 = pulsarClient.newConsumer(Schema.STRING).topic(tpName).subscriptionName(subscription) + .isAckReceiptEnabled(true).subscriptionType(subscriptionType).subscribe(); + Producer p1 = pulsarClient.newProducer(Schema.STRING).topic(tpName).enableBatching(false).create(); + for (int i = 0; i < msgSendCount; i++) { + p1.send(Integer.valueOf(i).toString()); + } + for (int i = 0; i < msgSendCount; i++) { + Message m = c1.receive(2, TimeUnit.SECONDS); + Assert.assertNotNull(m); + c1.acknowledge(m); + } + // Verify: the counter of calling "cursor.isCursorDataFullyPersistable". + // In expected the counter should be "0", to avoid flaky, verify it is less than 5. + Assert.assertTrue(callingIsCursorDataFullyPersistableCounter.get() < 5); + + // cleanup. + p1.close(); + c1.close(); + admin.topics().delete(tpName, false); + } } From 528c4b0fdb0207ac33d7715829761d0a83830544 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 17 May 2024 00:10:52 -0700 Subject: [PATCH 201/580] [fix][build] Fix CVE-2024-2511 by upgrading to OpenSSL in docker image (#22731) --- docker/pulsar/Dockerfile | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index f586a9dd4f9d7..a1825524ce481 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -100,6 +100,10 @@ RUN apk add --no-cache \ ca-certificates \ procps +# Fix CVE-2024-2511 by upgrading to OpenSSL 3.1.4-r6 +# We can remove once new Alpine image is released +RUN apk upgrade --no-cache libssl3 libcrypto3 + # Install GLibc compatibility library COPY --from=glibc /root/packages /root/packages RUN apk add --allow-untrusted --force-overwrite /root/packages/glibc-*.apk From edde408f781215fcb85b890795affc89612586f9 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Fri, 17 May 2024 15:26:47 +0800 Subject: [PATCH 202/580] [fix][build] Add curl command for pulsar image (#22732) --- docker/pulsar/Dockerfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index a1825524ce481..e1f1dd1e06f76 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -98,7 +98,8 @@ RUN apk add --no-cache \ py3-pip \ gcompat \ ca-certificates \ - procps + procps \ + curl # Fix CVE-2024-2511 by upgrading to OpenSSL 3.1.4-r6 # We can remove once new Alpine image is released From 0c6f2480685b38cd131876c11a32e7c16890575b Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 17 May 2024 00:39:43 -0700 Subject: [PATCH 203/580] [fix][build] Fixed creation of `packages-storage` directory in docker image (#22730) --- docker/pulsar/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index e1f1dd1e06f76..4c22a033d83e4 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -39,7 +39,7 @@ COPY scripts/* /pulsar/bin/ # container when gid=0 is prohibited. In that case, the container must be run with uid 10000 with # any group id != 0 (for example 10001). # The file permissions are preserved when copying files from this builder image to the target image. -RUN for SUBDIRECTORY in conf data download logs instances/deps; do \ +RUN for SUBDIRECTORY in conf data download logs instances/deps packages-storage; do \ mkdir -p /pulsar/$SUBDIRECTORY; \ chmod -R ug+rwx /pulsar/$SUBDIRECTORY; \ chown -R 10000:0 /pulsar/$SUBDIRECTORY; \ From e35c00e9c09e08422d1cb6d3f1261f4fa2bece42 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 17 May 2024 19:13:52 +0800 Subject: [PATCH 204/580] [fix][offload] Break the fillbuffer loop when met EOF (#22722) --- .../impl/BlobStoreBackedInputStreamImpl.java | 18 +++++-- .../impl/BlobStoreBackedInputStreamTest.java | 51 +++++++++++++++++++ 2 files changed, 66 insertions(+), 3 deletions(-) create mode 100644 tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamTest.java diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java index 6cb60e14984f9..6ebbe5bce582a 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java @@ -107,9 +107,7 @@ private boolean refillBufferIfNeeded() throws IOException { bufferOffsetEnd = endRange; long bytesRead = endRange - startRange + 1; int bytesToCopy = (int) bytesRead; - while (bytesToCopy > 0) { - bytesToCopy -= buffer.writeBytes(stream, bytesToCopy); - } + fillBuffer(stream, bytesToCopy); cursor += buffer.readableBytes(); } @@ -135,6 +133,20 @@ private boolean refillBufferIfNeeded() throws IOException { return true; } + void fillBuffer(InputStream is, int bytesToCopy) throws IOException { + while (bytesToCopy > 0) { + int writeBytes = buffer.writeBytes(is, bytesToCopy); + if (writeBytes < 0) { + break; + } + bytesToCopy -= writeBytes; + } + } + + ByteBuf getBuffer() { + return buffer; + } + @Override public int read() throws IOException { if (refillBufferIfNeeded()) { diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamTest.java new file mode 100644 index 0000000000000..951180e4e18c8 --- /dev/null +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamTest.java @@ -0,0 +1,51 @@ +/* + * 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.bookkeeper.mledger.offload.jcloud.impl; + +import static org.testng.Assert.assertEquals; + +import java.io.IOException; +import java.io.InputStream; +import org.apache.bookkeeper.mledger.offload.jcloud.BlobStoreTestBase; +import org.testng.annotations.Test; + +public class BlobStoreBackedInputStreamTest extends BlobStoreTestBase { + + @Test + public void testFillBuffer() throws Exception { + BlobStoreBackedInputStreamImpl bis = new BlobStoreBackedInputStreamImpl( + blobStore, BUCKET, "testFillBuffer", (k, md) -> { + }, 2048, 512); + + InputStream is = new InputStream() { + int count = 10; + + @Override + public int read() throws IOException { + if (count-- > 0) { + return 1; + } else { + return -1; + } + } + }; + bis.fillBuffer(is, 20); + assertEquals(bis.getBuffer().readableBytes(), 10); + } +} From 400a286ca09d4f45f388616fd996f0605f19e5a4 Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Fri, 17 May 2024 21:59:42 +0800 Subject: [PATCH 205/580] [improve] [broker] Trigger offload on topic load (#22652) --- conf/broker.conf | 3 +++ conf/standalone.conf | 3 +++ .../bookkeeper/mledger/ManagedLedgerConfig.java | 17 +++++++++++++++++ .../mledger/impl/ManagedLedgerFactoryImpl.java | 5 +++++ .../mledger/impl/ManagedLedgerImpl.java | 4 ++-- .../pulsar/broker/ServiceConfiguration.java | 9 +++++++-- .../pulsar/broker/service/BrokerService.java | 1 + 7 files changed, 38 insertions(+), 4 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 1ef68a0395cef..2a9641b5b90b8 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1211,6 +1211,9 @@ managedLedgerDataReadPriority=tiered-storage-first # (default is -1, which is disabled) managedLedgerOffloadThresholdInSeconds=-1 +# Trigger offload on topic load or not. Default is false. +# triggerOffloadOnTopicLoad=false + # Max number of entries to append to a cursor ledger managedLedgerCursorMaxEntriesPerLedger=50000 diff --git a/conf/standalone.conf b/conf/standalone.conf index a8615b70293d6..7c6aeb6815d6b 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -835,6 +835,9 @@ managedLedgerPrometheusStatsLatencyRolloverSeconds=60 # Whether trace managed ledger task execution time managedLedgerTraceTaskExecution=true +# Trigger offload on topic load or not. Default is false. +# triggerOffloadOnTopicLoad=false + # If you want to custom bookie ID or use a dynamic network address for the bookie, # you can set this option. # Bookie advertises itself using bookieId rather than diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 0c93a5b642cf6..fb2c6de3c7423 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -85,6 +85,7 @@ public class ManagedLedgerConfig { private int minimumBacklogCursorsForCaching = 0; private int minimumBacklogEntriesForCaching = 1000; private int maxBacklogBetweenCursorsForCaching = 1000; + private boolean triggerOffloadOnTopicLoad = false; @Getter @Setter @@ -748,6 +749,22 @@ public void setMaxBacklogBetweenCursorsForCaching(int maxBacklogBetweenCursorsFo this.maxBacklogBetweenCursorsForCaching = maxBacklogBetweenCursorsForCaching; } + /** + * Trigger offload on topic load. + * @return + */ + public boolean isTriggerOffloadOnTopicLoad() { + return triggerOffloadOnTopicLoad; + } + + /** + * Set trigger offload on topic load. + * @param triggerOffloadOnTopicLoad + */ + public void setTriggerOffloadOnTopicLoad(boolean triggerOffloadOnTopicLoad) { + this.triggerOffloadOnTopicLoad = triggerOffloadOnTopicLoad; + } + public String getShadowSource() { return MapUtils.getString(properties, PROPERTY_SOURCE_TOPIC_KEY); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 5ce84b3ed850a..d867f2f4c0221 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.bookkeeper.mledger.ManagedLedgerException.getManagedLedgerException; +import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.NULL_OFFLOAD_PROMISE; import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; import com.google.common.base.Predicates; import com.google.common.collect.Maps; @@ -395,6 +396,10 @@ public void initializeComplete() { // May need to update the cursor position newledger.maybeUpdateCursorBeforeTrimmingConsumedLedger(); + // May need to trigger offloading + if (config.isTriggerOffloadOnTopicLoad()) { + newledger.maybeOffloadInBackground(NULL_OFFLOAD_PROMISE); + } } @Override diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index b3426692df308..681441bf73839 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -212,7 +212,7 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { private final CallbackMutex trimmerMutex = new CallbackMutex(); private final CallbackMutex offloadMutex = new CallbackMutex(); - private static final CompletableFuture NULL_OFFLOAD_PROMISE = CompletableFuture + public static final CompletableFuture NULL_OFFLOAD_PROMISE = CompletableFuture .completedFuture(PositionImpl.LATEST); protected volatile LedgerHandle currentLedger; protected volatile long currentLedgerEntries = 0; @@ -2469,7 +2469,7 @@ private void scheduleDeferredTrimming(boolean isTruncate, CompletableFuture p 100, TimeUnit.MILLISECONDS); } - private void maybeOffloadInBackground(CompletableFuture promise) { + public void maybeOffloadInBackground(CompletableFuture promise) { if (config.getLedgerOffloader() == null || config.getLedgerOffloader() == NullLedgerOffloader.INSTANCE || config.getLedgerOffloader().getOffloadPolicies() == null) { return; 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 9efe185650969..6f03bef30548e 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 @@ -2090,10 +2090,15 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece ) private long managedLedgerOffloadAutoTriggerSizeThresholdBytes = -1L; @FieldContext( - category = CATEGORY_STORAGE_OFFLOADING, - doc = "The threshold to triggering automatic offload to long term storage" + category = CATEGORY_STORAGE_OFFLOADING, + doc = "The threshold to triggering automatic offload to long term storage" ) private long managedLedgerOffloadThresholdInSeconds = -1L; + @FieldContext( + category = CATEGORY_STORAGE_OFFLOADING, + doc = "Trigger offload on topic load or not. Default is false" + ) + private boolean triggerOffloadOnTopicLoad = false; @FieldContext( category = CATEGORY_STORAGE_ML, doc = "Max number of entries to append to a cursor ledger" 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 6603e240ee7d9..9a08578ee4088 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 @@ -1986,6 +1986,7 @@ private CompletableFuture getManagedLedgerConfig(@Nonnull T .setLedgerOffloader(pulsar.getManagedLedgerOffloader(namespace, offloadPolicies)); } } + managedLedgerConfig.setTriggerOffloadOnTopicLoad(serviceConfig.isTriggerOffloadOnTopicLoad()); managedLedgerConfig.setDeletionAtBatchIndexLevelEnabled( serviceConfig.isAcknowledgmentAtBatchIndexLevelEnabled()); From 158960a7e2ad20dc084294f3b85ba4f409659cbb Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 17 May 2024 17:17:17 +0300 Subject: [PATCH 206/580] [fix][test] Fix flaky AuthorizationTest.testGetListWithGetBundleOp (#22713) Co-authored-by: Jiwe Guo --- .../java/org/apache/pulsar/broker/auth/AuthorizationTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java index 6c913d4290897..6b0ff3333bbc7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthorizationTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.auth; -import static org.mockito.Mockito.when; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -322,7 +321,6 @@ public void testGetListWithGetBundleOp() throws Exception { : brokerUrlTls.toString()) .authentication(new MockAuthentication("pass.pass2")) .build(); - when(pulsar.getAdminClient()).thenReturn(admin2); Assert.assertEquals(admin2.topics().getList(namespaceV1, TopicDomain.non_persistent).size(), 0); Assert.assertEquals(admin2.topics().getList(namespaceV2, TopicDomain.non_persistent).size(), 0); } From 2308f27c555e2d6aa5bc4d135e48249cab62c34e Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Sat, 18 May 2024 00:26:29 +0800 Subject: [PATCH 207/580] [fix][build] Fix pulsar-client-python installation on ARM arch (#22733) Signed-off-by: Zixuan Liu --- docker/pulsar/Dockerfile | 51 ++++++++++++++++++++-------------------- 1 file changed, 26 insertions(+), 25 deletions(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 4c22a033d83e4..5553f13b8799f 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -48,28 +48,6 @@ RUN for SUBDIRECTORY in conf data download logs instances/deps packages-storage; RUN chmod -R g+rx /pulsar/bin RUN chmod -R o+rx /pulsar -## Create 2nd stage to build the Python dependencies -## Since it needs to have GCC available, we're doing it in a different layer -FROM alpine:3.19 AS python-deps - -RUN apk add --no-cache \ - bash \ - python3-dev \ - g++ \ - musl-dev \ - libffi-dev \ - py3-pip \ - py3-grpcio \ - py3-yaml - -RUN pip3 install --break-system-packages \ - kazoo - -ARG PULSAR_CLIENT_PYTHON_VERSION -RUN pip3 install --break-system-packages \ - pulsar-client[all]==${PULSAR_CLIENT_PYTHON_VERSION} - - ### Create one stage to include JVM distribution FROM alpine AS jvm @@ -96,6 +74,8 @@ RUN apk add --no-cache \ bash \ python3 \ py3-pip \ + py3-grpcio \ + py3-yaml \ gcompat \ ca-certificates \ procps \ @@ -105,6 +85,30 @@ RUN apk add --no-cache \ # We can remove once new Alpine image is released RUN apk upgrade --no-cache libssl3 libcrypto3 +# Python dependencies + +# The grpcio@1.59.3 is installed by apk, and Pulsar-client@3.4.0 requires grpcio>=1.60.0, which causes the grocio to be reinstalled by pip. +# If pip cannot find the grpcio wheel that the doesn't match the OS, the grpcio will be compiled locally. +# Once https://github.com/apache/pulsar-client-python/pull/211 is released, keep only the pulsar-client[all] and kazoo dependencies, and remove comments. +ARG PULSAR_CLIENT_PYTHON_VERSION +RUN echo -e "\ +#pulsar-client[all]==${PULSAR_CLIENT_PYTHON_VERSION}\n\ +pulsar-client==${PULSAR_CLIENT_PYTHON_VERSION}\n\ +# Zookeeper\n\ +kazoo\n\ +# functions\n\ +protobuf>=3.6.1,<=3.20.3\n\ +grpcio>=1.59.3\n\ +apache-bookkeeper-client>=4.16.1\n\ +prometheus_client\n\ +ratelimit\n\ +# avro\n\ +fastavro>=1.9.2\n\ +" > /requirements.txt + +RUN pip3 install --break-system-packages --no-cache-dir --only-binary grpcio -r /requirements.txt +RUN rm /requirements.txt + # Install GLibc compatibility library COPY --from=glibc /root/packages /root/packages RUN apk add --allow-untrusted --force-overwrite /root/packages/glibc-*.apk @@ -115,9 +119,6 @@ ENV JAVA_HOME=/opt/jvm # The default is /pulsat/bin and cannot be written. ENV PULSAR_PID_DIR=/pulsar/logs -# Copy Python depedencies from the other stage -COPY --from=python-deps /usr/lib/python3.11/site-packages /usr/lib/python3.11/site-packages - ENV PULSAR_ROOT_LOGGER=INFO,CONSOLE COPY --from=pulsar /pulsar /pulsar From 4593cc33f9326b040eada84af69aa44c667b3fdd Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 18 May 2024 06:03:22 +0300 Subject: [PATCH 208/580] [improve][build] Remove invalid relativePath definitions in pom.xml files (#22741) ### Motivation Using `..` is invalid, it should be `../pom.xml`. Since the default for `relativePath` value is `../pom.xml`, there's no need to include the relativePath definition when it's `../pom.xml`. ### Modifications - remove `relativePath` elements from `pom.xml` files when the value is `..` or `../pom.xml`. --- bouncy-castle/bc/pom.xml | 1 - bouncy-castle/bcfips-include-test/pom.xml | 1 - bouncy-castle/bcfips/pom.xml | 1 - bouncy-castle/pom.xml | 1 - distribution/io/pom.xml | 1 - distribution/offloaders/pom.xml | 1 - distribution/pom.xml | 1 - distribution/server/pom.xml | 1 - distribution/shell/pom.xml | 1 - jclouds-shaded/pom.xml | 1 - managed-ledger/pom.xml | 3 +-- microbench/pom.xml | 1 - pulsar-broker/pom.xml | 1 - pulsar-cli-utils/pom.xml | 5 ++--- pulsar-client-1x-base/pom.xml | 1 - pulsar-client-1x-base/pulsar-client-1x/pom.xml | 1 - pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml | 1 - pulsar-client-admin-api/pom.xml | 3 +-- pulsar-client-admin-shaded/pom.xml | 1 - pulsar-client-admin/pom.xml | 1 - pulsar-client-all/pom.xml | 1 - pulsar-client-api/pom.xml | 1 - pulsar-client-auth-athenz/pom.xml | 1 - pulsar-client-auth-sasl/pom.xml | 1 - pulsar-client-messagecrypto-bc/pom.xml | 1 - pulsar-client-shaded/pom.xml | 1 - pulsar-client-tools-api/pom.xml | 1 - pulsar-client-tools-customcommand-example/pom.xml | 1 - pulsar-client-tools-test/pom.xml | 1 - pulsar-client-tools/pom.xml | 1 - pulsar-client/pom.xml | 1 - pulsar-common/pom.xml | 1 - pulsar-config-validation/pom.xml | 3 +-- pulsar-functions/localrun-shaded/pom.xml | 1 - pulsar-functions/localrun/pom.xml | 1 - pulsar-functions/runtime-all/pom.xml | 1 - pulsar-functions/worker/pom.xml | 3 +-- pulsar-metadata/pom.xml | 1 - pulsar-package-management/pom.xml | 1 - pulsar-testclient/pom.xml | 1 - pulsar-websocket/pom.xml | 1 - structured-event-log/pom.xml | 1 - tiered-storage/file-system/pom.xml | 1 - tiered-storage/jcloud/pom.xml | 1 - tiered-storage/pom.xml | 1 - 45 files changed, 6 insertions(+), 51 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index a5bfb9e113b9d..afe3f5e4a6312 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar bouncy-castle-parent 3.4.0-SNAPSHOT - .. bouncy-castle-bc diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 266517434f799..775a82861ccfb 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -25,7 +25,6 @@ org.apache.pulsar bouncy-castle-parent 3.4.0-SNAPSHOT - .. bcfips-include-test diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index b47b9fbacf675..a5cab68961ed2 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar bouncy-castle-parent 3.4.0-SNAPSHOT - .. bouncy-castle-bcfips diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index 57b98ee547941..7641e79a48942 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index e9dd44d0d5950..bd65d5a81232b 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar distribution 3.4.0-SNAPSHOT - .. pulsar-io-distribution diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 32b197c5c7f6f..131eacf986af9 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar distribution 3.4.0-SNAPSHOT - .. pulsar-offloader-distribution diff --git a/distribution/pom.xml b/distribution/pom.xml index 666ed0514e3a9..0ed2219ec3aef 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. distribution diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 7a48245d800be..adabddfa31da4 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar distribution 3.4.0-SNAPSHOT - .. pulsar-server-distribution diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 457b0d6fb01ee..905bcc747450a 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar distribution 3.4.0-SNAPSHOT - .. pulsar-shell-distribution diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index bac1cf6b1e046..f0e456b5c00f8 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -27,7 +27,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. jclouds-shaded diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index b32cfb154277a..d8b31220d51be 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. managed-ledger @@ -147,7 +146,7 @@ - + org.apache.maven.plugins maven-jar-plugin diff --git a/microbench/pom.xml b/microbench/pom.xml index 98f32899888af..62561339e8879 100644 --- a/microbench/pom.xml +++ b/microbench/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - ../pom.xml microbench diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 100ed62d773c4..73f55710c4f79 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - ../pom.xml pulsar-broker diff --git a/pulsar-cli-utils/pom.xml b/pulsar-cli-utils/pom.xml index 76c8635993167..7fe1485a379b7 100644 --- a/pulsar-cli-utils/pom.xml +++ b/pulsar-cli-utils/pom.xml @@ -27,7 +27,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-cli-utils @@ -73,7 +72,7 @@ - + pl.project13.maven git-commit-id-plugin @@ -112,7 +111,7 @@ - + com.github.spotbugs spotbugs-maven-plugin diff --git a/pulsar-client-1x-base/pom.xml b/pulsar-client-1x-base/pom.xml index 5c79f4d6b9a24..cc9bb9df093fb 100644 --- a/pulsar-client-1x-base/pom.xml +++ b/pulsar-client-1x-base/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-1x-base diff --git a/pulsar-client-1x-base/pulsar-client-1x/pom.xml b/pulsar-client-1x-base/pulsar-client-1x/pom.xml index bec564d46aaf6..fb938f7432fc6 100644 --- a/pulsar-client-1x-base/pulsar-client-1x/pom.xml +++ b/pulsar-client-1x-base/pulsar-client-1x/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar-client-1x-base 3.4.0-SNAPSHOT - .. pulsar-client-1x diff --git a/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml b/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml index dbdb1fe9df560..e4de42d52738a 100644 --- a/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml +++ b/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar-client-1x-base 3.4.0-SNAPSHOT - .. pulsar-client-2x-shaded diff --git a/pulsar-client-admin-api/pom.xml b/pulsar-client-admin-api/pom.xml index e17dd8bedf6db..0f58dcef24ce4 100644 --- a/pulsar-client-admin-api/pom.xml +++ b/pulsar-client-admin-api/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-admin-api @@ -43,7 +42,7 @@ org.slf4j slf4j-api - + diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index adc376b079781..7370ea42a4a5a 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -27,7 +27,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-admin diff --git a/pulsar-client-admin/pom.xml b/pulsar-client-admin/pom.xml index 289fec65ea22a..bdcbeeef59c3c 100644 --- a/pulsar-client-admin/pom.xml +++ b/pulsar-client-admin/pom.xml @@ -27,7 +27,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-admin-original diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index cd6748dd536c5..b73c495ec1b69 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-all diff --git a/pulsar-client-api/pom.xml b/pulsar-client-api/pom.xml index b275dca835f93..409cfdd9c5187 100644 --- a/pulsar-client-api/pom.xml +++ b/pulsar-client-api/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-api diff --git a/pulsar-client-auth-athenz/pom.xml b/pulsar-client-auth-athenz/pom.xml index f29cf6bd186cf..54590092d8fe5 100644 --- a/pulsar-client-auth-athenz/pom.xml +++ b/pulsar-client-auth-athenz/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-auth-athenz diff --git a/pulsar-client-auth-sasl/pom.xml b/pulsar-client-auth-sasl/pom.xml index 864967fb72edd..023436aca44e0 100644 --- a/pulsar-client-auth-sasl/pom.xml +++ b/pulsar-client-auth-sasl/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-auth-sasl diff --git a/pulsar-client-messagecrypto-bc/pom.xml b/pulsar-client-messagecrypto-bc/pom.xml index 8f2431793c7b9..89239b5ba891c 100644 --- a/pulsar-client-messagecrypto-bc/pom.xml +++ b/pulsar-client-messagecrypto-bc/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-messagecrypto-bc diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index 37063f06f6388..be2dc028498d8 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client diff --git a/pulsar-client-tools-api/pom.xml b/pulsar-client-tools-api/pom.xml index dba8215418a85..e8e3de2e46f21 100644 --- a/pulsar-client-tools-api/pom.xml +++ b/pulsar-client-tools-api/pom.xml @@ -25,7 +25,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-tools-api diff --git a/pulsar-client-tools-customcommand-example/pom.xml b/pulsar-client-tools-customcommand-example/pom.xml index 0140abbdac3a2..cea79f16506fc 100644 --- a/pulsar-client-tools-customcommand-example/pom.xml +++ b/pulsar-client-tools-customcommand-example/pom.xml @@ -23,7 +23,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. 4.0.0 pulsar-client-tools-customcommand-example diff --git a/pulsar-client-tools-test/pom.xml b/pulsar-client-tools-test/pom.xml index 5f28e11a8078d..2c3c9c7bd2dc5 100644 --- a/pulsar-client-tools-test/pom.xml +++ b/pulsar-client-tools-test/pom.xml @@ -25,7 +25,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-tools-test diff --git a/pulsar-client-tools/pom.xml b/pulsar-client-tools/pom.xml index 74b3877513014..7616b8d3d96f6 100644 --- a/pulsar-client-tools/pom.xml +++ b/pulsar-client-tools/pom.xml @@ -25,7 +25,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-tools diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index 8dd76ef061f93..c1db14beaed21 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -26,7 +26,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-client-original diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 61e22946306a4..31f425e8b4181 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -27,7 +27,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-common diff --git a/pulsar-config-validation/pom.xml b/pulsar-config-validation/pom.xml index 2d5bbfc4aadf0..9d9adf7a579a0 100644 --- a/pulsar-config-validation/pom.xml +++ b/pulsar-config-validation/pom.xml @@ -27,7 +27,6 @@ org.apache.pulsar pulsar 3.4.0-SNAPSHOT - .. pulsar-config-validation @@ -59,7 +58,7 @@ - + com.github.spotbugs spotbugs-maven-plugin diff --git a/pulsar-functions/localrun-shaded/pom.xml b/pulsar-functions/localrun-shaded/pom.xml index fd93fb6b2588b..f82c6e88e9e68 100644 --- a/pulsar-functions/localrun-shaded/pom.xml +++ b/pulsar-functions/localrun-shaded/pom.xml @@ -27,7 +27,6 @@ org.apache.pulsar pulsar-functions 3.4.0-SNAPSHOT - .. pulsar-functions-local-runner diff --git a/pulsar-functions/localrun/pom.xml b/pulsar-functions/localrun/pom.xml index a8a798a8b6f6c..2427332cb80d1 100644 --- a/pulsar-functions/localrun/pom.xml +++ b/pulsar-functions/localrun/pom.xml @@ -27,7 +27,6 @@ org.apache.pulsar pulsar-functions 3.4.0-SNAPSHOT - .. pulsar-functions-local-runner-original diff --git a/pulsar-functions/runtime-all/pom.xml b/pulsar-functions/runtime-all/pom.xml index 062e8102c2a1d..0313a4e2026b8 100644 --- a/pulsar-functions/runtime-all/pom.xml +++ b/pulsar-functions/runtime-all/pom.xml @@ -27,7 +27,6 @@ org.apache.pulsar pulsar-functions 3.4.0-SNAPSHOT - .. +* Mailing List discussion thread: https://lists.apache.org/thread/wwybg8og80yz9gvj6bfdbv1znx2dfp4w +* Mailing List voting thread: https://lists.apache.org/thread/67r3nv33gfoxhvo74ql41dydh2rmyvjw From 9b3876df70f3b1d8bc01a34308d718c456f1781b Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 27 May 2024 21:37:24 +0800 Subject: [PATCH 228/580] [fix][admin] Clearly define REST API on Open API for Namesaces@v2 (#22775) --- .../broker/admin/impl/NamespacesBase.java | 2 +- .../pulsar/broker/admin/v2/Namespaces.java | 453 +++++++++++++----- 2 files changed, 329 insertions(+), 126 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 ca67a24460721..afcf4e646fa2c 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 @@ -2711,7 +2711,7 @@ protected CompletableFuture internalSetDispatcherPauseOnAckStatePersistent })); } - protected CompletableFuture internalGetDispatcherPauseOnAckStatePersistentAsync() { + protected CompletableFuture internalGetDispatcherPauseOnAckStatePersistentAsync() { return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.DISPATCHER_PAUSE_ON_ACK_STATE_PERSISTENT, PolicyOperation.READ) .thenCompose(__ -> namespaceResources().getPoliciesAsync(namespaceName)) 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 0e270ed34f7c3..3a7c614a7c6f8 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 @@ -23,6 +23,8 @@ import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; +import io.swagger.annotations.Example; +import io.swagger.annotations.ExampleProperty; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; import java.util.HashSet; @@ -58,9 +60,11 @@ import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.BookieAffinityGroupData; import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; +import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.EntryFilters; import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.OffloadPolicies; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.PersistencePolicies; import org.apache.pulsar.common.policies.data.Policies; @@ -72,6 +76,12 @@ import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; +import org.apache.pulsar.common.policies.data.TopicHashPositions; +import org.apache.pulsar.common.policies.data.impl.AutoSubscriptionCreationOverrideImpl; +import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl; +import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; +import org.apache.pulsar.common.policies.data.impl.BookieAffinityGroupDataImpl; +import org.apache.pulsar.common.policies.data.impl.BundlesDataImpl; import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.MetadataStoreException; @@ -151,7 +161,9 @@ public void getPolicies(@Suspended AsyncResponse response, @PUT @Path("/{tenant}/{namespace}") @ApiOperation(value = "Creates a new namespace with the specified policies") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster doesn't exist"), @ApiResponse(code = 409, message = "Namespace already exists"), @ApiResponse(code = 412, message = "Namespace name is not valid") }) @@ -179,6 +191,7 @@ public void createNamespace(@Suspended AsyncResponse response, @Path("/{tenant}/{namespace}") @ApiOperation(value = "Delete a namespace and all the topics under it.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @@ -207,6 +220,7 @@ public void deleteNamespace(@Suspended final AsyncResponse asyncResponse, @PathP @Path("/{tenant}/{namespace}/{bundle}") @ApiOperation(value = "Delete a namespace bundle and all the topics under it.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @@ -230,7 +244,10 @@ public void deleteNamespaceBundle(@Suspended AsyncResponse response, @PathParam( @GET @Path("/{tenant}/{namespace}/permissions") - @ApiOperation(value = "Retrieve the permissions for a namespace.") + @ApiOperation(value = "Retrieve the permissions for a namespace.", + notes = "Returns a nested map structure which Swagger does not fully support for display. " + + "Structure: Map>. Please refer to this structure for details.", + response = AuthAction.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Namespace is not empty") }) @@ -250,7 +267,10 @@ public void getPermissions(@Suspended AsyncResponse response, @GET @Path("/{tenant}/{namespace}/permissions/subscription") - @ApiOperation(value = "Retrieve the permissions for a subscription.") + @ApiOperation(value = "Retrieve the permissions for a subscription.", + notes = "Returns a nested map structure which Swagger does not fully support for display. " + + "Structure: Map>. Please refer to this structure for details.", + response = String.class, responseContainer = "Map") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Namespace is not empty")}) @@ -272,7 +292,9 @@ public void getPermissionOnSubscription(@Suspended AsyncResponse response, @POST @Path("/{tenant}/{namespace}/permissions/{role}") @ApiOperation(value = "Grant a new permission to a role on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 501, message = "Authorization is not enabled")}) @@ -296,7 +318,9 @@ public void grantPermissionOnNamespace(@Suspended AsyncResponse asyncResponse, @Path("/{property}/{namespace}/permissions/subscription/{subscription}") @ApiOperation(hidden = true, value = "Grant a new permission to roles for a subscription." + "[Tenant admin is allowed to perform this operation]") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 501, message = "Authorization is not enabled") }) @@ -320,7 +344,9 @@ public void grantPermissionOnSubscription(@Suspended AsyncResponse asyncResponse @DELETE @Path("/{tenant}/{namespace}/permissions/{role}") @ApiOperation(value = "Revoke all permissions to a role on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void revokePermissionsOnNamespace(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -339,7 +365,9 @@ public void revokePermissionsOnNamespace(@Suspended AsyncResponse asyncResponse, @DELETE @Path("/{property}/{namespace}/permissions/{subscription}/{role}") @ApiOperation(hidden = true, value = "Revoke subscription admin-api access permission for a role.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) public void revokePermissionOnSubscription(@Suspended AsyncResponse asyncResponse, @PathParam("property") String property, @@ -359,7 +387,7 @@ public void revokePermissionOnSubscription(@Suspended AsyncResponse asyncRespons @GET @Path("/{tenant}/{namespace}/replication") @ApiOperation(value = "Get the replication clusters for a namespace.", - response = String.class, responseContainer = "List") + response = String.class, responseContainer = "Set") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Namespace is not global")}) @@ -380,7 +408,9 @@ public void getNamespaceReplicationClusters(@Suspended AsyncResponse asyncRespon @POST @Path("/{tenant}/{namespace}/replication") @ApiOperation(value = "Set the replication clusters for a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Peer-cluster can't be part of replication-cluster"), @ApiResponse(code = 412, message = "Namespace is not global or invalid cluster ids") }) @@ -421,7 +451,9 @@ public void getNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @Path @POST @Path("/{tenant}/{namespace}/messageTTL") @ApiOperation(value = "Set message TTL in seconds for namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid TTL") }) public void setNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -441,7 +473,9 @@ public void setNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @Path @DELETE @Path("/{tenant}/{namespace}/messageTTL") @ApiOperation(value = "Remove message TTL in seconds for namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid TTL")}) public void removeNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @@ -459,7 +493,7 @@ public void removeNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @GET @Path("/{tenant}/{namespace}/subscriptionExpirationTime") - @ApiOperation(value = "Get the subscription expiration time for the namespace") + @ApiOperation(value = "Get the subscription expiration time for the namespace", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void getSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse, @@ -481,7 +515,9 @@ public void getSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse @POST @Path("/{tenant}/{namespace}/subscriptionExpirationTime") @ApiOperation(value = "Set subscription expiration time in minutes for namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid expiration time")}) public void setSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse, @@ -504,7 +540,9 @@ public void setSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse @DELETE @Path("/{tenant}/{namespace}/subscriptionExpirationTime") @ApiOperation(value = "Remove subscription expiration time for namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist")}) public void removeSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -522,7 +560,7 @@ public void removeSubscriptionExpirationTime(@Suspended AsyncResponse asyncRespo @GET @Path("/{tenant}/{namespace}/deduplication") - @ApiOperation(value = "Get broker side deduplication for all topics in a namespace") + @ApiOperation(value = "Get broker side deduplication for all topics in a namespace", response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void getDeduplication(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -540,7 +578,9 @@ public void getDeduplication(@Suspended AsyncResponse asyncResponse, @PathParam( @POST @Path("/{tenant}/{namespace}/deduplication") @ApiOperation(value = "Enable or disable broker side deduplication for all topics in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void modifyDeduplication(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -560,7 +600,9 @@ public void modifyDeduplication(@Suspended AsyncResponse asyncResponse, @PathPar @DELETE @Path("/{tenant}/{namespace}/deduplication") @ApiOperation(value = "Remove broker side deduplication for all topics in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void removeDeduplication(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -577,7 +619,7 @@ public void removeDeduplication(@Suspended AsyncResponse asyncResponse, @PathPar @GET @Path("/{tenant}/{namespace}/autoTopicCreation") - @ApiOperation(value = "Get autoTopicCreation info in a namespace") + @ApiOperation(value = "Get autoTopicCreation info in a namespace", response = AutoTopicCreationOverrideImpl.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist")}) public void getAutoTopicCreation(@Suspended AsyncResponse asyncResponse, @@ -596,7 +638,9 @@ public void getAutoTopicCreation(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/autoTopicCreation") @ApiOperation(value = "Override broker's allowAutoTopicCreation setting for a namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 406, message = "The number of partitions should be less than or" + " equal to maxNumPartitionsPerPartitionedTopic"), @@ -632,7 +676,9 @@ public void setAutoTopicCreation( @DELETE @Path("/{tenant}/{namespace}/autoTopicCreation") @ApiOperation(value = "Remove override of broker's allowAutoTopicCreation in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void removeAutoTopicCreation(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -660,7 +706,9 @@ public void removeAutoTopicCreation(@Suspended final AsyncResponse asyncResponse @POST @Path("/{tenant}/{namespace}/autoSubscriptionCreation") @ApiOperation(value = "Override broker's allowAutoSubscriptionCreation setting for a namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 400, message = "Invalid autoSubscriptionCreation override")}) public void setAutoSubscriptionCreation( @@ -690,7 +738,8 @@ public void setAutoSubscriptionCreation( @GET @Path("/{tenant}/{namespace}/autoSubscriptionCreation") - @ApiOperation(value = "Get autoSubscriptionCreation info in a namespace") + @ApiOperation(value = "Get autoSubscriptionCreation info in a namespace", + response = AutoSubscriptionCreationOverrideImpl.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist")}) public void getAutoSubscriptionCreation(@Suspended final AsyncResponse asyncResponse, @@ -709,7 +758,9 @@ public void getAutoSubscriptionCreation(@Suspended final AsyncResponse asyncResp @DELETE @Path("/{tenant}/{namespace}/autoSubscriptionCreation") @ApiOperation(value = "Remove override of broker's allowAutoSubscriptionCreation in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void removeAutoSubscriptionCreation(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -735,7 +786,7 @@ public void removeAutoSubscriptionCreation(@Suspended final AsyncResponse asyncR @GET @Path("/{tenant}/{namespace}/bundles") - @ApiOperation(value = "Get the bundles split data.") + @ApiOperation(value = "Get the bundles split data.", response = BundlesDataImpl.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Namespace is not setup to split in bundles") }) @@ -767,6 +818,7 @@ public void getBundlesData(@Suspended final AsyncResponse asyncResponse, + " since it wouldresult in non-persistent message loss and" + " unexpected connection closure to the clients.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), @@ -799,6 +851,7 @@ public void unloadNamespace(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/{bundle}/unload") @ApiOperation(value = "Unload a namespace bundle") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 403, message = "Don't have admin permission") }) @@ -828,6 +881,7 @@ public void unloadNamespaceBundle(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/{bundle}/split") @ApiOperation(value = "Split a namespace bundle") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 403, message = "Don't have admin permission") }) @@ -864,7 +918,7 @@ public void splitNamespaceBundle( @GET @Path("/{tenant}/{namespace}/{bundle}/topicHashPositions") - @ApiOperation(value = "Get hash positions for topics") + @ApiOperation(value = "Get hash positions for topics", response = TopicHashPositions.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) @@ -890,7 +944,9 @@ public void getTopicHashPositions( @POST @Path("/{property}/{namespace}/publishRate") @ApiOperation(hidden = true, value = "Set publish-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void setPublishRate(@Suspended AsyncResponse asyncResponse, @PathParam("property") String property, @PathParam("namespace") String namespace, @ApiParam(value = "Publish rate for all topics of the specified namespace") PublishRate publishRate) { @@ -906,7 +962,9 @@ public void setPublishRate(@Suspended AsyncResponse asyncResponse, @PathParam("p @DELETE @Path("/{property}/{namespace}/publishRate") @ApiOperation(hidden = true, value = "Set publish-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void removePublishRate(@Suspended AsyncResponse asyncResponse, @PathParam("property") String property, @PathParam("namespace") String namespace) { validateNamespaceName(property, namespace); @@ -924,7 +982,8 @@ public void removePublishRate(@Suspended AsyncResponse asyncResponse, @PathParam @Path("/{property}/{namespace}/publishRate") @ApiOperation(hidden = true, value = "Get publish-rate configured for the namespace, null means publish-rate not configured, " - + "-1 means msg-publish-rate or byte-publish-rate not configured in publish-rate yet") + + "-1 means msg-publish-rate or byte-publish-rate not configured in publish-rate yet", + response = PublishRate.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) public void getPublishRate(@Suspended AsyncResponse asyncResponse, @@ -943,7 +1002,9 @@ public void getPublishRate(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/dispatchRate") @ApiOperation(value = "Set dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void setDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @ApiParam(value = "Dispatch rate for all topics of the specified namespace") @@ -962,7 +1023,9 @@ public void setDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam(" @DELETE @Path("/{tenant}/{namespace}/dispatchRate") @ApiOperation(value = "Delete dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void deleteDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { validateNamespaceName(tenant, namespace); @@ -979,7 +1042,8 @@ public void deleteDispatchRate(@Suspended AsyncResponse asyncResponse, @PathPara @GET @Path("/{tenant}/{namespace}/dispatchRate") @ApiOperation(value = "Get dispatch-rate configured for the namespace, null means dispatch-rate not configured, " - + "-1 means msg-dispatch-rate or byte-dispatch-rate not configured in dispatch-rate yet") + + "-1 means msg-dispatch-rate or byte-dispatch-rate not configured in dispatch-rate yet", + response = DispatchRate.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -996,7 +1060,9 @@ public void getDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam(" @POST @Path("/{tenant}/{namespace}/subscriptionDispatchRate") @ApiOperation(value = "Set Subscription dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void setSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -1018,7 +1084,7 @@ public void setSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/subscriptionDispatchRate") @ApiOperation(value = "Get subscription dispatch-rate configured for the namespace, null means subscription " + "dispatch-rate not configured, -1 means msg-dispatch-rate or byte-dispatch-rate not configured " - + "in dispatch-rate yet") + + "in dispatch-rate yet", response = DispatchRate.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) public void getSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @@ -1038,7 +1104,9 @@ public void getSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/subscriptionDispatchRate") @ApiOperation(value = "Delete Subscription dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void deleteSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1056,7 +1124,9 @@ public void deleteSubscriptionDispatchRate(@Suspended AsyncResponse asyncRespons @DELETE @Path("/{tenant}/{namespace}/subscribeRate") @ApiOperation(value = "Delete subscribe-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void deleteSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { validateNamespaceName(tenant, namespace); @@ -1073,7 +1143,9 @@ public void deleteSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathPar @POST @Path("/{tenant}/{namespace}/subscribeRate") @ApiOperation(value = "Set subscribe-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void setSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @ApiParam(value = "Subscribe rate for all topics of the specified namespace") @@ -1091,7 +1163,7 @@ public void setSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam( @GET @Path("/{tenant}/{namespace}/subscribeRate") - @ApiOperation(value = "Get subscribe-rate configured for the namespace") + @ApiOperation(value = "Get subscribe-rate configured for the namespace", response = SubscribeRate.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) public void getSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -1109,7 +1181,9 @@ public void getSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam( @DELETE @Path("/{tenant}/{namespace}/replicatorDispatchRate") @ApiOperation(value = "Remove replicator dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void removeReplicatorDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1120,7 +1194,9 @@ public void removeReplicatorDispatchRate(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/replicatorDispatchRate") @ApiOperation(value = "Set replicator dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void setReplicatorDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -1134,7 +1210,7 @@ public void setReplicatorDispatchRate(@Suspended AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/replicatorDispatchRate") @ApiOperation(value = "Get replicator dispatch-rate configured for the namespace, null means replicator " + "dispatch-rate not configured, -1 means msg-dispatch-rate or byte-dispatch-rate not configured " - + "in dispatch-rate yet") + + "in dispatch-rate yet", response = DispatchRateImpl.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getReplicatorDispatchRate(@Suspended final AsyncResponse asyncResponse, @@ -1146,7 +1222,8 @@ public void getReplicatorDispatchRate(@Suspended final AsyncResponse asyncRespon @GET @Path("/{tenant}/{namespace}/backlogQuotaMap") - @ApiOperation(value = "Get backlog quota map on a namespace.") + @ApiOperation(value = "Get backlog quota map on a namespace.", + response = BacklogQuotaImpl.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getBacklogQuotaMap( @@ -1160,7 +1237,9 @@ public void getBacklogQuotaMap( @POST @Path("/{tenant}/{namespace}/backlogQuota") @ApiOperation(value = " Set a backlog quota for all the topics on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, @@ -1178,7 +1257,9 @@ public void setBacklogQuota( @DELETE @Path("/{tenant}/{namespace}/backlogQuota") @ApiOperation(value = "Remove a backlog quota policy from a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeBacklogQuota( @@ -1191,7 +1272,7 @@ public void removeBacklogQuota( @GET @Path("/{tenant}/{namespace}/retention") - @ApiOperation(value = "Get retention config on a namespace.") + @ApiOperation(value = "Get retention config on a namespace.", response = RetentionPolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getRetention(@Suspended final AsyncResponse asyncResponse, @@ -1212,7 +1293,9 @@ public void getRetention(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/retention") @ApiOperation(value = " Set retention configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "Retention Quota must exceed backlog quota") }) @@ -1225,7 +1308,9 @@ public void setRetention(@PathParam("tenant") String tenant, @PathParam("namespa @DELETE @Path("/{tenant}/{namespace}/retention") @ApiOperation(value = " Remove retention configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "Retention Quota must exceed backlog quota") }) @@ -1238,7 +1323,9 @@ public void removeRetention(@PathParam("tenant") String tenant, @PathParam("name @POST @Path("/{tenant}/{namespace}/persistence") @ApiOperation(value = "Set the persistence configuration for all the topics on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 400, message = "Invalid persistence policies")}) @@ -1260,7 +1347,9 @@ public void setPersistence(@Suspended final AsyncResponse asyncResponse, @PathPa @DELETE @Path("/{tenant}/{namespace}/persistence") @ApiOperation(value = "Delete the persistence configuration for all topics on a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void deletePersistence(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { validateNamespaceName(tenant, namespace); @@ -1278,6 +1367,7 @@ public void deletePersistence(@Suspended final AsyncResponse asyncResponse, @Pat @Path("/{tenant}/{namespace}/persistence/bookieAffinity") @ApiOperation(value = "Set the bookie-affinity-group to namespace-persistent policy.") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @@ -1291,7 +1381,8 @@ public void setBookieAffinityGroup(@PathParam("tenant") String tenant, @PathPara @GET @Path("/{property}/{namespace}/persistence/bookieAffinity") - @ApiOperation(value = "Get the bookie-affinity-group from namespace-local policy.") + @ApiOperation(value = "Get the bookie-affinity-group from namespace-local policy.", + response = BookieAffinityGroupDataImpl.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1306,7 +1397,9 @@ public BookieAffinityGroupData getBookieAffinityGroup(@PathParam("property") Str @DELETE @Path("/{property}/{namespace}/persistence/bookieAffinity") @ApiOperation(value = "Delete the bookie-affinity-group from namespace-local policy.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void deleteBookieAffinityGroup(@PathParam("property") String property, @@ -1317,7 +1410,7 @@ public void deleteBookieAffinityGroup(@PathParam("property") String property, @GET @Path("/{tenant}/{namespace}/persistence") - @ApiOperation(value = "Get the persistence configuration for a namespace.") + @ApiOperation(value = "Get the persistence configuration for a namespace.", response = PersistencePolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -1341,6 +1434,7 @@ public void getPersistence( @Path("/{tenant}/{namespace}/clearBacklog") @ApiOperation(value = "Clear backlog for all topics on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void clearNamespaceBacklog(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -1360,6 +1454,7 @@ public void clearNamespaceBacklog(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/{bundle}/clearBacklog") @ApiOperation(value = "Clear backlog for all topics on a namespace bundle.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) @@ -1374,6 +1469,7 @@ public void clearNamespaceBundleBacklog(@PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}/clearBacklog/{subscription}") @ApiOperation(value = "Clear backlog for a given subscription on all topics on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void clearNamespaceBacklogForSubscription(@Suspended final AsyncResponse asyncResponse, @@ -1394,6 +1490,7 @@ public void clearNamespaceBacklogForSubscription(@Suspended final AsyncResponse @Path("/{tenant}/{namespace}/{bundle}/clearBacklog/{subscription}") @ApiOperation(value = "Clear backlog for a given subscription on all topics on a namespace bundle.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) @@ -1409,6 +1506,7 @@ public void clearNamespaceBundleBacklogForSubscription(@PathParam("tenant") Stri @Path("/{tenant}/{namespace}/unsubscribe/{subscription}") @ApiOperation(value = "Unsubscribes the given subscription on all topics on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespacen"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void unsubscribeNamespace(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -1429,6 +1527,7 @@ public void unsubscribeNamespace(@Suspended final AsyncResponse asyncResponse, @ @Path("/{tenant}/{namespace}/{bundle}/unsubscribe/{subscription}") @ApiOperation(value = "Unsubscribes the given subscription on all topics on a namespace bundle.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void unsubscribeNamespaceBundle(@PathParam("tenant") String tenant, @@ -1442,7 +1541,9 @@ public void unsubscribeNamespaceBundle(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/subscriptionAuthMode") @ApiOperation(value = " Set a subscription auth mode for all the topics on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setSubscriptionAuthMode(@PathParam("tenant") String tenant, @@ -1455,7 +1556,7 @@ public void setSubscriptionAuthMode(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/subscriptionAuthMode") - @ApiOperation(value = "Get subscription auth mode in a namespace") + @ApiOperation(value = "Get subscription auth mode in a namespace", response = SubscriptionAuthMode.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist")}) public void getSubscriptionAuthMode( @@ -1477,7 +1578,9 @@ public void getSubscriptionAuthMode( @POST @Path("/{tenant}/{namespace}/encryptionRequired") @ApiOperation(value = "Message encryption is required or not for all topics in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), }) public void modifyEncryptionRequired( @@ -1491,7 +1594,7 @@ public void modifyEncryptionRequired( @GET @Path("/{tenant}/{namespace}/encryptionRequired") - @ApiOperation(value = "Get message encryption required status in a namespace") + @ApiOperation(value = "Get message encryption required status in a namespace", response = Boolean.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist")}) public void getEncryptionRequired(@Suspended AsyncResponse asyncResponse, @@ -1511,7 +1614,8 @@ public void getEncryptionRequired(@Suspended AsyncResponse asyncResponse, @GET @Path("/{tenant}/{namespace}/delayedDelivery") - @ApiOperation(value = "Get delayed delivery messages config on a namespace.") + @ApiOperation(value = "Get delayed delivery messages config on a namespace.", + response = DelayedDeliveryPolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), }) @@ -1533,7 +1637,9 @@ public void getDelayedDeliveryPolicies(@Suspended final AsyncResponse asyncRespo @POST @Path("/{tenant}/{namespace}/delayedDelivery") @ApiOperation(value = "Set delayed delivery messages config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), }) public void setDelayedDeliveryPolicies(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -1546,7 +1652,9 @@ public void setDelayedDeliveryPolicies(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/delayedDelivery") @ApiOperation(value = "Delete delayed delivery messages config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), }) public void removeDelayedDeliveryPolicies(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1556,7 +1664,7 @@ public void removeDelayedDeliveryPolicies(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/inactiveTopicPolicies") - @ApiOperation(value = "Get inactive topic policies config on a namespace.") + @ApiOperation(value = "Get inactive topic policies config on a namespace.", response = InactiveTopicPolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), }) @@ -1578,7 +1686,9 @@ public void getInactiveTopicPolicies(@Suspended final AsyncResponse asyncRespons @DELETE @Path("/{tenant}/{namespace}/inactiveTopicPolicies") @ApiOperation(value = "Remove inactive topic policies from a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void removeInactiveTopicPolicies(@PathParam("tenant") String tenant, @@ -1590,7 +1700,9 @@ public void removeInactiveTopicPolicies(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/inactiveTopicPolicies") @ApiOperation(value = "Set inactive topic policies config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), }) public void setInactiveTopicPolicies(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -1602,7 +1714,7 @@ public void setInactiveTopicPolicies(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/maxProducersPerTopic") - @ApiOperation(value = "Get maxProducersPerTopic config on a namespace.") + @ApiOperation(value = "Get maxProducersPerTopic config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxProducersPerTopic( @@ -1624,7 +1736,9 @@ public void getMaxProducersPerTopic( @POST @Path("/{tenant}/{namespace}/maxProducersPerTopic") @ApiOperation(value = " Set maxProducersPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxProducersPerTopic value is not valid") }) @@ -1637,7 +1751,9 @@ public void setMaxProducersPerTopic(@PathParam("tenant") String tenant, @PathPar @DELETE @Path("/{tenant}/{namespace}/maxProducersPerTopic") @ApiOperation(value = "Remove maxProducersPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeMaxProducersPerTopic(@PathParam("tenant") String tenant, @@ -1648,7 +1764,7 @@ public void removeMaxProducersPerTopic(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/deduplicationSnapshotInterval") - @ApiOperation(value = "Get deduplicationSnapshotInterval config on a namespace.") + @ApiOperation(value = "Get deduplicationSnapshotInterval config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getDeduplicationSnapshotInterval( @@ -1670,7 +1786,9 @@ public void getDeduplicationSnapshotInterval( @POST @Path("/{tenant}/{namespace}/deduplicationSnapshotInterval") @ApiOperation(value = "Set deduplicationSnapshotInterval config on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) public void setDeduplicationSnapshotInterval(@PathParam("tenant") String tenant , @PathParam("namespace") String namespace @@ -1682,7 +1800,7 @@ public void setDeduplicationSnapshotInterval(@PathParam("tenant") String tenant @GET @Path("/{tenant}/{namespace}/maxConsumersPerTopic") - @ApiOperation(value = "Get maxConsumersPerTopic config on a namespace.") + @ApiOperation(value = "Get maxConsumersPerTopic config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxConsumersPerTopic( @@ -1704,7 +1822,9 @@ public void getMaxConsumersPerTopic( @POST @Path("/{tenant}/{namespace}/maxConsumersPerTopic") @ApiOperation(value = " Set maxConsumersPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxConsumersPerTopic value is not valid") }) @@ -1717,7 +1837,9 @@ public void setMaxConsumersPerTopic(@PathParam("tenant") String tenant, @PathPar @DELETE @Path("/{tenant}/{namespace}/maxConsumersPerTopic") @ApiOperation(value = "Remove maxConsumersPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeMaxConsumersPerTopic(@PathParam("tenant") String tenant, @@ -1728,7 +1850,7 @@ public void removeMaxConsumersPerTopic(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/maxConsumersPerSubscription") - @ApiOperation(value = "Get maxConsumersPerSubscription config on a namespace.") + @ApiOperation(value = "Get maxConsumersPerSubscription config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxConsumersPerSubscription( @@ -1750,7 +1872,9 @@ public void getMaxConsumersPerSubscription( @POST @Path("/{tenant}/{namespace}/maxConsumersPerSubscription") @ApiOperation(value = " Set maxConsumersPerSubscription configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxConsumersPerSubscription value is not valid")}) @@ -1766,7 +1890,9 @@ public void setMaxConsumersPerSubscription(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/maxConsumersPerSubscription") @ApiOperation(value = " Set maxConsumersPerSubscription configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxConsumersPerSubscription value is not valid")}) @@ -1778,7 +1904,7 @@ public void removeMaxConsumersPerSubscription(@PathParam("tenant") String tenant @GET @Path("/{tenant}/{namespace}/maxUnackedMessagesPerConsumer") - @ApiOperation(value = "Get maxUnackedMessagesPerConsumer config on a namespace.") + @ApiOperation(value = "Get maxUnackedMessagesPerConsumer config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxUnackedMessagesPerConsumer(@Suspended final AsyncResponse asyncResponse, @@ -1799,7 +1925,9 @@ public void getMaxUnackedMessagesPerConsumer(@Suspended final AsyncResponse asyn @POST @Path("/{tenant}/{namespace}/maxUnackedMessagesPerConsumer") @ApiOperation(value = " Set maxConsumersPerTopic configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxUnackedMessagesPerConsumer value is not valid")}) @@ -1815,7 +1943,9 @@ public void setMaxUnackedMessagesPerConsumer(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/maxUnackedMessagesPerConsumer") @ApiOperation(value = "Remove maxUnackedMessagesPerConsumer config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void removeMaxUnackedmessagesPerConsumer(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1825,7 +1955,7 @@ public void removeMaxUnackedmessagesPerConsumer(@PathParam("tenant") String tena @GET @Path("/{tenant}/{namespace}/maxUnackedMessagesPerSubscription") - @ApiOperation(value = "Get maxUnackedMessagesPerSubscription config on a namespace.") + @ApiOperation(value = "Get maxUnackedMessagesPerSubscription config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxUnackedmessagesPerSubscription( @@ -1847,7 +1977,9 @@ public void getMaxUnackedmessagesPerSubscription( @POST @Path("/{tenant}/{namespace}/maxUnackedMessagesPerSubscription") @ApiOperation(value = " Set maxUnackedMessagesPerSubscription configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxUnackedMessagesPerSubscription value is not valid")}) @@ -1862,7 +1994,9 @@ public void setMaxUnackedMessagesPerSubscription( @DELETE @Path("/{tenant}/{namespace}/maxUnackedMessagesPerSubscription") @ApiOperation(value = "Remove maxUnackedMessagesPerSubscription config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void removeMaxUnackedmessagesPerSubscription(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1872,7 +2006,7 @@ public void removeMaxUnackedmessagesPerSubscription(@PathParam("tenant") String @GET @Path("/{tenant}/{namespace}/maxSubscriptionsPerTopic") - @ApiOperation(value = "Get maxSubscriptionsPerTopic config on a namespace.") + @ApiOperation(value = "Get maxSubscriptionsPerTopic config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResponse, @@ -1893,7 +2027,9 @@ public void getMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResp @POST @Path("/{tenant}/{namespace}/maxSubscriptionsPerTopic") @ApiOperation(value = " Set maxSubscriptionsPerTopic configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxUnackedMessagesPerSubscription value is not valid")}) @@ -1908,7 +2044,9 @@ public void setMaxSubscriptionsPerTopic( @DELETE @Path("/{tenant}/{namespace}/maxSubscriptionsPerTopic") @ApiOperation(value = "Remove maxSubscriptionsPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeMaxSubscriptionsPerTopic(@PathParam("tenant") String tenant, @@ -1920,7 +2058,9 @@ public void removeMaxSubscriptionsPerTopic(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/antiAffinity") @ApiOperation(value = "Set anti-affinity group for a namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid antiAffinityGroup")}) public void setNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @@ -1934,7 +2074,7 @@ public void setNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/antiAffinity") - @ApiOperation(value = "Get anti-affinity group of a namespace.") + @ApiOperation(value = "Get anti-affinity group of a namespace.", response = String.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public String getNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @@ -1946,7 +2086,9 @@ public String getNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/antiAffinity") @ApiOperation(value = "Remove anti-affinity group of a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @@ -1958,7 +2100,8 @@ public void removeNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @GET @Path("{cluster}/antiAffinity/{group}") @ApiOperation(value = "Get all namespaces that are grouped by given anti-affinity group in a given cluster." - + " api can be only accessed by admin of any of the existing tenant") + + " api can be only accessed by admin of any of the existing tenant", + response = String.class, responseContainer = "List") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 412, message = "Cluster not exist/Anti-affinity group can't be empty.")}) public List getAntiAffinityNamespaces(@PathParam("cluster") String cluster, @@ -1970,7 +2113,7 @@ public List getAntiAffinityNamespaces(@PathParam("cluster") String clust @Path("/{tenant}/{namespace}/compactionThreshold") @ApiOperation(value = "Maximum number of uncompacted bytes in topics before compaction is triggered.", notes = "The backlog size is compared to the threshold periodically. " - + "A threshold of 0 disabled automatic compaction") + + "A threshold of 0 disabled automatic compaction", response = Long.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public void getCompactionThreshold( @@ -1994,7 +2137,9 @@ public void getCompactionThreshold( @ApiOperation(value = "Set maximum number of uncompacted bytes in a topic before compaction is triggered.", notes = "The backlog size is compared to the threshold periodically. " + "A threshold of 0 disabled automatic compaction") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "compactionThreshold value is not valid")}) @@ -2012,7 +2157,9 @@ public void setCompactionThreshold(@PathParam("tenant") String tenant, @ApiOperation(value = "Delete maximum number of uncompacted bytes in a topic before compaction is triggered.", notes = "The backlog size is compared to the threshold periodically. " + "A threshold of 0 disabled automatic compaction") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void deleteCompactionThreshold(@PathParam("tenant") String tenant, @@ -2025,7 +2172,7 @@ public void deleteCompactionThreshold(@PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}/offloadThreshold") @ApiOperation(value = "Maximum number of bytes stored on the pulsar cluster for a topic," + " before the broker will start offloading to longterm storage", - notes = "A negative value disables automatic offloading") + notes = "A negative value disables automatic offloading", response = Long.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public void getOffloadThreshold( @@ -2056,7 +2203,9 @@ public void getOffloadThreshold( + " before the broker will start offloading to longterm storage", notes = "-1 will revert to using the cluster default." + " A negative value disables automatic offloading. ") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "offloadThreshold value is not valid")}) @@ -2074,7 +2223,7 @@ public void setOffloadThreshold(@PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}/offloadThresholdInSeconds") @ApiOperation(value = "Maximum number of bytes stored on the pulsar cluster for a topic," + " before the broker will start offloading to longterm storage", - notes = "A negative value disables automatic offloading") + notes = "A negative value disables automatic offloading", response = Long.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public void getOffloadThresholdInSeconds( @@ -2104,7 +2253,9 @@ public void getOffloadThresholdInSeconds( @ApiOperation(value = "Set maximum number of seconds stored on the pulsar cluster for a topic," + " before the broker will start offloading to longterm storage", notes = "A negative value disables automatic offloading") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "offloadThresholdInSeconds value is not valid") }) @@ -2128,7 +2279,7 @@ public void setOffloadThresholdInSeconds( + " from the Pulsar cluster's local storage (i.e. BookKeeper)", notes = "A negative value denotes that deletion has been completely disabled." + " 'null' denotes that the topics in the namespace will fall back to the" - + " broker default for deletion lag.") + + " broker default for deletion lag.", response = Long.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public void getOffloadDeletionLag( @@ -2158,7 +2309,9 @@ public void getOffloadDeletionLag( @ApiOperation(value = "Set number of milliseconds to wait before deleting a ledger segment which has been offloaded" + " from the Pulsar cluster's local storage (i.e. BookKeeper)", notes = "A negative value disables the deletion completely.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "offloadDeletionLagMs value is not valid")}) @@ -2177,6 +2330,7 @@ public void setOffloadDeletionLag(@PathParam("tenant") String tenant, @ApiOperation(value = "Clear the namespace configured offload deletion lag. The topics in the namespace" + " will fallback to using the default configured deletion lag for the broker") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void clearOffloadDeletionLag(@PathParam("tenant") String tenant, @@ -2190,7 +2344,8 @@ public void clearOffloadDeletionLag(@PathParam("tenant") String tenant, @ApiOperation(value = "The strategy used to check the compatibility of new schemas," + " provided by producers, before automatically updating the schema", notes = "The value AutoUpdateDisabled prevents producers from updating the schema. " - + " If set to AutoUpdateDisabled, schemas must be updated through the REST api") + + " If set to AutoUpdateDisabled, schemas must be updated through the REST api", + response = SchemaAutoUpdateCompatibilityStrategy.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -2207,7 +2362,9 @@ public SchemaAutoUpdateCompatibilityStrategy getSchemaAutoUpdateCompatibilityStr + " provided by producers, before automatically updating the schema", notes = "The value AutoUpdateDisabled prevents producers from updating the schema. " + " If set to AutoUpdateDisabled, schemas must be updated through the REST api") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setSchemaAutoUpdateCompatibilityStrategy( @@ -2221,7 +2378,8 @@ public void setSchemaAutoUpdateCompatibilityStrategy( @GET @Path("/{tenant}/{namespace}/schemaCompatibilityStrategy") - @ApiOperation(value = "The strategy of the namespace schema compatibility ") + @ApiOperation(value = "The strategy of the namespace schema compatibility ", + response = SchemaCompatibilityStrategy.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -2245,7 +2403,9 @@ public void getSchemaCompatibilityStrategy( @PUT @Path("/{tenant}/{namespace}/schemaCompatibilityStrategy") @ApiOperation(value = "Update the strategy used to check the compatibility of new schema") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setSchemaCompatibilityStrategy( @@ -2259,7 +2419,7 @@ public void setSchemaCompatibilityStrategy( @GET @Path("/{tenant}/{namespace}/isAllowAutoUpdateSchema") - @ApiOperation(value = "The flag of whether allow auto update schema") + @ApiOperation(value = "The flag of whether allow auto update schema", response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -2289,7 +2449,9 @@ public void getIsAllowAutoUpdateSchema( @POST @Path("/{tenant}/{namespace}/isAllowAutoUpdateSchema") @ApiOperation(value = "Update flag of whether allow auto update schema") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setIsAllowAutoUpdateSchema( @@ -2303,7 +2465,8 @@ public void setIsAllowAutoUpdateSchema( @GET @Path("/{tenant}/{namespace}/subscriptionTypesEnabled") - @ApiOperation(value = "The set of whether allow subscription types") + @ApiOperation(value = "The set of whether allow subscription types", + response = SubscriptionType.class, responseContainer = "Set") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -2331,7 +2494,9 @@ public void getSubscriptionTypesEnabled( @POST @Path("/{tenant}/{namespace}/subscriptionTypesEnabled") @ApiOperation(value = "Update set of whether allow share sub type") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setSubscriptionTypesEnabled( @@ -2362,7 +2527,8 @@ public void removeSubscriptionTypesEnabled(@PathParam("tenant") String tenant, notes = "If the flag is set to true, when a producer without a schema attempts to produce to a topic" + " with schema in this namespace, the producer will be failed to connect. PLEASE be" + " carefully on using this, since non-java clients don't support schema.if you enable" - + " this setting, it will cause non-java clients failed to produce.") + + " this setting, it will cause non-java clients failed to produce.", + response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenants or Namespace doesn't exist") }) public void getSchemaValidtionEnforced( @@ -2397,7 +2563,9 @@ public void getSchemaValidtionEnforced( + " with schema in this namespace, the producer will be failed to connect. PLEASE be" + " carefully on using this, since non-java clients don't support schema.if you enable" + " this setting, it will cause non-java clients failed to produce.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or Namespace doesn't exist"), @ApiResponse(code = 412, message = "schemaValidationEnforced value is not valid")}) public void setSchemaValidationEnforced(@PathParam("tenant") String tenant, @@ -2412,8 +2580,9 @@ public void setSchemaValidationEnforced(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/offloadPolicies") - @ApiOperation(value = " Set offload configuration on a namespace.") + @ApiOperation(value = "Set offload configuration on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @@ -2437,6 +2606,7 @@ public void setOffloadPolicies(@PathParam("tenant") String tenant, @PathParam("n @Path("/{tenant}/{namespace}/removeOffloadPolicies") @ApiOperation(value = " Set offload configuration on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @@ -2456,7 +2626,7 @@ public void removeOffloadPolicies(@PathParam("tenant") String tenant, @PathParam @GET @Path("/{tenant}/{namespace}/offloadPolicies") - @ApiOperation(value = "Get offload configuration on a namespace.") + @ApiOperation(value = "Get offload configuration on a namespace.", response = OffloadPolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) @@ -2478,7 +2648,7 @@ public void getOffloadPolicies( @GET @Path("/{tenant}/{namespace}/maxTopicsPerNamespace") - @ApiOperation(value = "Get maxTopicsPerNamespace config on a namespace.") + @ApiOperation(value = "Get maxTopicsPerNamespace config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace does not exist") }) public void getMaxTopicsPerNamespace(@Suspended final AsyncResponse asyncResponse, @@ -2503,7 +2673,9 @@ public void getMaxTopicsPerNamespace(@Suspended final AsyncResponse asyncRespons @POST @Path("/{tenant}/{namespace}/maxTopicsPerNamespace") @ApiOperation(value = "Set maxTopicsPerNamespace config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void setMaxTopicsPerNamespace(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -2516,7 +2688,9 @@ public void setMaxTopicsPerNamespace(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/maxTopicsPerNamespace") @ApiOperation(value = "Remove maxTopicsPerNamespace config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void removeMaxTopicsPerNamespace(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -2527,7 +2701,9 @@ public void removeMaxTopicsPerNamespace(@PathParam("tenant") String tenant, @PUT @Path("/{tenant}/{namespace}/property/{key}/{value}") @ApiOperation(value = "Put a key value pair property on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void setProperty( @Suspended final AsyncResponse asyncResponse, @@ -2541,7 +2717,7 @@ public void setProperty( @GET @Path("/{tenant}/{namespace}/property/{key}") - @ApiOperation(value = "Get property value for a given key on a namespace.") + @ApiOperation(value = "Get property value for a given key on a namespace.", response = String.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void getProperty( @@ -2556,7 +2732,9 @@ public void getProperty( @DELETE @Path("/{tenant}/{namespace}/property/{key}") @ApiOperation(value = "Remove property value for a given key on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void removeProperty( @Suspended final AsyncResponse asyncResponse, @@ -2570,7 +2748,9 @@ public void removeProperty( @PUT @Path("/{tenant}/{namespace}/properties") @ApiOperation(value = "Put key value pairs property on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void setProperties( @Suspended final AsyncResponse asyncResponse, @@ -2584,7 +2764,8 @@ public void setProperties( @GET @Path("/{tenant}/{namespace}/properties") - @ApiOperation(value = "Get key value pair properties for a given namespace.") + @ApiOperation(value = "Get key value pair properties for a given namespace.", + response = String.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void getProperties( @@ -2598,7 +2779,9 @@ public void getProperties( @DELETE @Path("/{tenant}/{namespace}/properties") @ApiOperation(value = "Clear properties on a given namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void clearProperties( @Suspended final AsyncResponse asyncResponse, @@ -2610,7 +2793,7 @@ public void clearProperties( @GET @Path("/{tenant}/{namespace}/resourcegroup") - @ApiOperation(value = "Get the resource group attached to the namespace") + @ApiOperation(value = "Get the resource group attached to the namespace", response = String.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void getNamespaceResourceGroup( @@ -2632,7 +2815,9 @@ public void getNamespaceResourceGroup( @POST @Path("/{tenant}/{namespace}/resourcegroup/{resourcegroup}") @ApiOperation(value = "Set resourcegroup for a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid resourcegroup") }) public void setNamespaceResourceGroup(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -2644,7 +2829,9 @@ public void setNamespaceResourceGroup(@PathParam("tenant") String tenant, @PathP @DELETE @Path("/{tenant}/{namespace}/resourcegroup") @ApiOperation(value = "Delete resourcegroup for a namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid resourcegroup")}) public void removeNamespaceResourceGroup(@PathParam("tenant") String tenant, @@ -2656,7 +2843,13 @@ public void removeNamespaceResourceGroup(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/scanOffloadedLedgers") @ApiOperation(value = "Trigger the scan of offloaded Ledgers on the LedgerOffloader for the given namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Successful get of offloaded ledger data", response = String.class, + examples = @Example(value = { @ExampleProperty(mediaType = "application/json", + value = "{\"objects\":[{\"key1\":\"value1\",\"key2\":\"value2\"}]," + + "\"total\":100,\"errors\":5,\"unknown\":3}") + })), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public Response scanOffloadedLedgers(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -2705,7 +2898,7 @@ public void finished(int total, int errors, int unknown) throws Exception { @GET @Path("/{tenant}/{namespace}/entryFilters") - @ApiOperation(value = "Get maxConsumersPerSubscription config on a namespace.") + @ApiOperation(value = "Get maxConsumersPerSubscription config on a namespace.", response = EntryFilters.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getEntryFiltersPerTopic( @@ -2728,6 +2921,7 @@ public void getEntryFiltersPerTopic( @Path("/{tenant}/{namespace}/entryFilters") @ApiOperation(value = "Set entry filters for namespace") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 400, message = "Specified entry filters are not valid"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") @@ -2749,7 +2943,9 @@ public void setEntryFiltersPerTopic(@Suspended AsyncResponse asyncResponse, @Pat @DELETE @Path("/{tenant}/{namespace}/entryFilters") @ApiOperation(value = "Remove entry filters for namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid TTL")}) public void removeNamespaceEntryFilters(@Suspended AsyncResponse asyncResponse, @@ -2768,7 +2964,9 @@ public void removeNamespaceEntryFilters(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/migration") @ApiOperation(hidden = true, value = "Update migration for all topics in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) public void enableMigration(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -2780,7 +2978,9 @@ public void enableMigration(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/dispatcherPauseOnAckStatePersistent") @ApiOperation(value = "Set dispatcher pause on ack state persistent configuration for specified namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setDispatcherPauseOnAckStatePersistent(@Suspended final AsyncResponse asyncResponse, @@ -2802,7 +3002,9 @@ public void setDispatcherPauseOnAckStatePersistent(@Suspended final AsyncRespons @DELETE @Path("/{tenant}/{namespace}/dispatcherPauseOnAckStatePersistent") @ApiOperation(value = "Remove dispatcher pause on ack state persistent configuration for specified namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void removeDispatcherPauseOnAckStatePersistent(@Suspended final AsyncResponse asyncResponse, @@ -2823,7 +3025,8 @@ public void removeDispatcherPauseOnAckStatePersistent(@Suspended final AsyncResp @GET @Path("/{tenant}/{namespace}/dispatcherPauseOnAckStatePersistent") - @ApiOperation(value = "Get dispatcher pause on ack state persistent config on a namespace.") + @ApiOperation(value = "Get dispatcher pause on ack state persistent config on a namespace.", + response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void getDispatcherPauseOnAckStatePersistent(@Suspended final AsyncResponse asyncResponse, From ba20e02f01d75f0d4ec38393841bcf5c417e9363 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 27 May 2024 21:45:51 +0800 Subject: [PATCH 229/580] [fix][admin] Clearly define REST API on Open API (#22783) --- .../broker/admin/impl/FunctionsBase.java | 27 ++++++++----- .../pulsar/broker/admin/impl/SinksBase.java | 18 ++++++--- .../pulsar/broker/admin/impl/SourcesBase.java | 20 ++++++---- .../pulsar/broker/admin/impl/TenantsBase.java | 12 ++++-- .../broker/admin/v2/ResourceGroups.java | 5 ++- .../broker/admin/v2/ResourceQuotas.java | 4 +- .../apache/pulsar/broker/admin/v2/Worker.java | 10 ++++- .../pulsar/broker/admin/v3/Packages.java | 9 +++-- .../pulsar/broker/admin/v3/Transactions.java | 39 +++++++++++++------ 9 files changed, 99 insertions(+), 45 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java index 4350316e2f011..42971ae231c05 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java @@ -39,7 +39,6 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.StreamingOutput; import org.apache.pulsar.broker.admin.AdminResource; -import org.apache.pulsar.client.api.Message; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.FunctionDefinition; import org.apache.pulsar.common.functions.FunctionState; @@ -486,7 +485,7 @@ public List listFunctions( @POST @ApiOperation( value = "Triggers a Pulsar Function with a user-specified value or file data", - response = Message.class + response = String.class ) @ApiResponses(value = { @ApiResponse(code = 400, message = "Invalid request"), @@ -541,6 +540,7 @@ public FunctionState getFunctionState( value = "Put the state associated with a Pulsar Function" ) @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @@ -557,8 +557,9 @@ public void putFunctionState(final @PathParam("tenant") String tenant, } @POST - @ApiOperation(value = "Restart an instance of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Restart an instance of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this function"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @@ -578,8 +579,9 @@ public void restartFunction( } @POST - @ApiOperation(value = "Restart all instances of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Restart all instances of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -597,8 +599,9 @@ public void restartFunction( } @POST - @ApiOperation(value = "Stop an instance of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Stop an instance of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -617,8 +620,9 @@ public void stopFunction( } @POST - @ApiOperation(value = "Stop all instances of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Stop all instances of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -636,8 +640,9 @@ public void stopFunction( } @POST - @ApiOperation(value = "Start an instance of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Start an instance of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -656,8 +661,9 @@ public void startFunction( } @POST - @ApiOperation(value = "Start all instances of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Start all instances of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -718,7 +724,8 @@ public StreamingOutput downloadFunction( @GET @ApiOperation( value = "Fetches a list of supported Pulsar IO connectors currently running in cluster mode", - response = List.class + response = ConnectorDefinition.class, + responseContainer = "List" ) @ApiResponses(value = { @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @@ -739,6 +746,7 @@ public List getConnectorsList() throws IOException { value = "Reload the built-in Functions" ) @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 401, message = "This operation requires super-user access"), @ApiResponse(code = 503, message = "Function worker service is now initializing. Please try again later."), @ApiResponse(code = 500, message = "Internal server error") @@ -768,6 +776,7 @@ public List getBuiltinFunction() { @PUT @ApiOperation(value = "Updates a Pulsar Function on the worker leader", hidden = true) @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 403, message = "The requester doesn't have super-user permissions"), @ApiResponse(code = 404, message = "The function does not exist"), @ApiResponse(code = 400, message = "Invalid request"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java index 80ad72d6f9aa9..0a76fe27e0a35 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java @@ -389,8 +389,9 @@ public List listSinks(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Restart an instance of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Restart an instance of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this sink"), @ApiResponse(code = 400, message = "Invalid restart request"), @ApiResponse(code = 401, message = "The client is not authorized to perform this operation"), @@ -415,8 +416,9 @@ public void restartSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Restart all instances of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Restart all instances of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid restart request"), @ApiResponse(code = 401, message = "The client is not authorized to perform this operation"), @ApiResponse(code = 404, message = "The Pulsar Sink does not exist"), @@ -436,8 +438,9 @@ public void restartSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Stop an instance of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Stop an instance of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid stop request"), @ApiResponse(code = 404, message = "The Pulsar Sink instance does not exist"), @ApiResponse(code = 500, message = @@ -460,8 +463,9 @@ public void stopSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Stop all instances of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Stop all instances of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid stop request"), @ApiResponse(code = 404, message = "The Pulsar Sink does not exist"), @ApiResponse(code = 500, message = @@ -481,8 +485,9 @@ public void stopSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Start an instance of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Start an instance of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid start request"), @ApiResponse(code = 404, message = "The Pulsar Sink does not exist"), @ApiResponse(code = 500, message = @@ -505,8 +510,9 @@ public void startSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Start all instances of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Start all instances of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid start request"), @ApiResponse(code = 404, message = "The Pulsar Sink does not exist"), @ApiResponse(code = 500, message = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java index 4af0afc0d6ec5..0d037dd42362f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java @@ -323,7 +323,7 @@ public SourceStatus getSourceStatus( @ApiOperation( value = "Lists all Pulsar Sources currently deployed in a given namespace", response = String.class, - responseContainer = "Collection" + responseContainer = "List" ) @ApiResponses(value = { @ApiResponse(code = 400, message = "Invalid request"), @@ -342,8 +342,9 @@ public List listSources( } @POST - @ApiOperation(value = "Restart an instance of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Restart an instance of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this source"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @@ -365,8 +366,9 @@ public void restartSource( } @POST - @ApiOperation(value = "Restart all instances of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Restart all instances of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), @@ -386,8 +388,9 @@ public void restartSource( } @POST - @ApiOperation(value = "Stop instance of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Stop instance of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), @@ -407,8 +410,9 @@ public void stopSource( } @POST - @ApiOperation(value = "Stop all instances of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Stop all instances of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), @@ -428,8 +432,9 @@ public void stopSource( } @POST - @ApiOperation(value = "Start an instance of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Start an instance of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), @@ -449,8 +454,9 @@ public void startSource( } @POST - @ApiOperation(value = "Start all instances of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Start all instances of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java index 93eb7f33faf5f..0d1f79a09dc14 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java @@ -103,7 +103,9 @@ public void getTenantAdmin(@Suspended final AsyncResponse asyncResponse, @PUT @Path("/{tenant}") @ApiOperation(value = "Create a new tenant.", notes = "This operation requires Pulsar super-user privileges.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 409, message = "Tenant already exists"), @ApiResponse(code = 412, message = "Tenant name is not valid"), @ApiResponse(code = 412, message = "Clusters can not be empty"), @@ -156,7 +158,9 @@ public void createTenant(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}") @ApiOperation(value = "Update the admins for a tenant.", notes = "This operation requires Pulsar super-user privileges.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 404, message = "Tenant does not exist"), @ApiResponse(code = 409, message = "Tenant already exists"), @ApiResponse(code = 412, message = "Clusters can not be empty"), @@ -192,7 +196,9 @@ public void updateTenant(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}") @ApiOperation(value = "Delete a tenant and all namespaces and topics under it.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 404, message = "Tenant does not exist"), @ApiResponse(code = 405, message = "Broker doesn't allow forced deletion of tenants"), @ApiResponse(code = 409, message = "The tenant still has active namespaces")}) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceGroups.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceGroups.java index 52fd03b18ed0b..58f593e20ce3b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceGroups.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceGroups.java @@ -60,7 +60,9 @@ public ResourceGroup getResourceGroup(@PathParam("resourcegroup") String resourc @PUT @Path("/{resourcegroup}") @ApiOperation(value = "Creates a new resourcegroup with the specified rate limiters") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "cluster doesn't exist")}) public void createOrUpdateResourceGroup(@PathParam("resourcegroup") String name, @ApiParam(value = "Rate limiters for the resourcegroup") @@ -72,6 +74,7 @@ public void createOrUpdateResourceGroup(@PathParam("resourcegroup") String name, @Path("/{resourcegroup}") @ApiOperation(value = "Delete a resourcegroup.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "ResourceGroup doesn't exist"), @ApiResponse(code = 409, message = "ResourceGroup is in use")}) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java index 58ccc1c10288c..d2884e8ea6f7e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java @@ -75,7 +75,7 @@ public void setDefaultResourceQuota( @GET @Path("/{tenant}/{namespace}/{bundle}") - @ApiOperation(value = "Get resource quota of a namespace bundle.") + @ApiOperation(value = "Get resource quota of a namespace bundle.", response = ResourceQuota.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -103,6 +103,7 @@ public void getNamespaceBundleResourceQuota( @Path("/{tenant}/{namespace}/{bundle}") @ApiOperation(value = "Set resource quota on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -133,6 +134,7 @@ public void setNamespaceBundleResourceQuota( @Path("/{tenant}/{namespace}/{bundle}") @ApiOperation(value = "Remove resource quota for a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 409, message = "Concurrent modification") }) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java index 3813790e4f428..7178b565719ca 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java @@ -87,7 +87,9 @@ public WorkerInfo getClusterLeader() { @GET @ApiOperation( value = "Fetches information about which Pulsar Functions are assigned to which Pulsar clusters", - response = Map.class + response = Map.class, + notes = "Returns a nested map structure which Swagger does not fully support for display." + + "Structure: Map>. Please refer to this structure for details." ) @ApiResponses(value = { @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @@ -102,7 +104,8 @@ public Map> getAssignments() { @GET @ApiOperation( value = "Fetches a list of supported Pulsar IO connectors currently running in cluster mode", - response = List.class + response = ConnectorDefinition.class, + responseContainer = "List" ) @ApiResponses(value = { @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @@ -120,6 +123,7 @@ public List getConnectorsList() throws IOException { value = "Triggers a rebalance of functions to workers" ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 408, message = "Request timeout") @@ -134,6 +138,7 @@ public void rebalance() { value = "Drains the specified worker, i.e., moves its work-assignments to other workers" ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 408, message = "Request timeout"), @@ -150,6 +155,7 @@ public void drainAtLeader(@QueryParam("workerId") String workerId) { value = "Drains this worker, i.e., moves its work-assignments to other workers" ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 408, message = "Request timeout"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Packages.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Packages.java index 15e7b69554dc7..4ca7e3948ff5a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Packages.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Packages.java @@ -80,7 +80,7 @@ public void getMeta( ) @ApiResponses( value = { - @ApiResponse(code = 200, message = "Update the metadata of the specified package successfully."), + @ApiResponse(code = 204, message = "Update the metadata of the specified package successfully."), @ApiResponse(code = 404, message = "The specified package is not existent."), @ApiResponse(code = 412, message = "The package name is illegal."), @ApiResponse(code = 500, message = "Internal server error."), @@ -113,7 +113,7 @@ public void updateMeta( ) @ApiResponses( value = { - @ApiResponse(code = 200, message = "Upload the specified package successfully."), + @ApiResponse(code = 204, message = "Upload the specified package successfully."), @ApiResponse(code = 412, message = "The package name is illegal."), @ApiResponse(code = 500, message = "Internal server error."), @ApiResponse(code = 503, message = "Package Management Service is not enabled in the broker.") @@ -169,7 +169,7 @@ public StreamingOutput download( @Path("/{type}/{tenant}/{namespace}/{packageName}/{version}") @ApiResponses( value = { - @ApiResponse(code = 200, message = "Delete the specified package successfully."), + @ApiResponse(code = 204, message = "Delete the specified package successfully."), @ApiResponse(code = 404, message = "The specified package is not existent."), @ApiResponse(code = 412, message = "The package name is illegal."), @ApiResponse(code = 500, message = "Internal server error."), @@ -218,7 +218,8 @@ public void listPackageVersion( @Path("/{type}/{tenant}/{namespace}") @ApiOperation( value = "Get all the specified type packages in a namespace.", - response = PackageMetadata.class + response = PackageMetadata.class, + responseContainer = "List" ) @ApiResponses( value = { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java index 19a93db0b5146..7e3806aa9b47b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java @@ -43,6 +43,17 @@ import org.apache.pulsar.broker.admin.impl.TransactionsBase; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.common.policies.data.TransactionBufferInternalStats; +import org.apache.pulsar.common.policies.data.TransactionBufferStats; +import org.apache.pulsar.common.policies.data.TransactionCoordinatorInfo; +import org.apache.pulsar.common.policies.data.TransactionCoordinatorInternalStats; +import org.apache.pulsar.common.policies.data.TransactionCoordinatorStats; +import org.apache.pulsar.common.policies.data.TransactionInBufferStats; +import org.apache.pulsar.common.policies.data.TransactionInPendingAckStats; +import org.apache.pulsar.common.policies.data.TransactionMetadata; +import org.apache.pulsar.common.policies.data.TransactionPendingAckInternalStats; +import org.apache.pulsar.common.policies.data.TransactionPendingAckStats; +import org.apache.pulsar.common.stats.PositionInPendingAckStats; import org.apache.pulsar.common.util.FutureUtil; import org.jetbrains.annotations.Nullable; @@ -55,7 +66,8 @@ public class Transactions extends TransactionsBase { @GET @Path("/coordinators") - @ApiOperation(value = "List transaction coordinators.") + @ApiOperation(value = "List transaction coordinators.", + response = TransactionCoordinatorInfo.class, responseContainer = "List") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 503, message = "This Broker is not " + "configured with transactionCoordinatorEnabled=true.")}) @@ -66,7 +78,7 @@ public void listCoordinators(@Suspended final AsyncResponse asyncResponse) { @GET @Path("/coordinatorStats") - @ApiOperation(value = "Get transaction coordinator stats.") + @ApiOperation(value = "Get transaction coordinator stats.", response = TransactionCoordinatorStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 503, message = "This Broker is not " + "configured with transactionCoordinatorEnabled=true."), @@ -82,7 +94,7 @@ public void getCoordinatorStats(@Suspended final AsyncResponse asyncResponse, @GET @Path("/transactionInBufferStats/{tenant}/{namespace}/{topic}/{mostSigBits}/{leastSigBits}") - @ApiOperation(value = "Get transaction state in transaction buffer.") + @ApiOperation(value = "Get transaction state in transaction buffer.", response = TransactionInBufferStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " @@ -119,7 +131,7 @@ public void getTransactionInBufferStats(@Suspended final AsyncResponse asyncResp @GET @Path("/transactionInPendingAckStats/{tenant}/{namespace}/{topic}/{subName}/{mostSigBits}/{leastSigBits}") - @ApiOperation(value = "Get transaction state in pending ack.") + @ApiOperation(value = "Get transaction state in pending ack.", response = TransactionInPendingAckStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " @@ -157,7 +169,7 @@ public void getTransactionInPendingAckStats(@Suspended final AsyncResponse async @GET @Path("/transactionBufferStats/{tenant}/{namespace}/{topic}") - @ApiOperation(value = "Get transaction buffer stats in topic.") + @ApiOperation(value = "Get transaction buffer stats in topic.", response = TransactionBufferStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " @@ -195,7 +207,7 @@ public void getTransactionBufferStats(@Suspended final AsyncResponse asyncRespon @GET @Path("/pendingAckStats/{tenant}/{namespace}/{topic}/{subName}") - @ApiOperation(value = "Get transaction pending ack stats in topic.") + @ApiOperation(value = "Get transaction pending ack stats in topic.", response = TransactionPendingAckStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic or subName doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " @@ -231,7 +243,7 @@ public void getPendingAckStats(@Suspended final AsyncResponse asyncResponse, @GET @Path("/transactionMetadata/{mostSigBits}/{leastSigBits}") - @ApiOperation(value = "Get transaction metadata") + @ApiOperation(value = "Get transaction metadata", response = TransactionMetadata.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or coordinator or transaction doesn't exist"), @@ -252,7 +264,7 @@ public void getTransactionMetadata(@Suspended final AsyncResponse asyncResponse, @GET @Path("/slowTransactions/{timeout}") - @ApiOperation(value = "Get slow transactions.") + @ApiOperation(value = "Get slow transactions.", response = TransactionMetadata.class, responseContainer = "Map") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or coordinator or transaction doesn't exist"), @@ -272,7 +284,7 @@ public void getSlowTransactions(@Suspended final AsyncResponse asyncResponse, @GET @Path("/coordinatorInternalStats/{coordinatorId}") - @ApiOperation(value = "Get coordinator internal stats.") + @ApiOperation(value = "Get coordinator internal stats.", response = TransactionCoordinatorInternalStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 503, message = "This Broker is not " + "configured with transactionCoordinatorEnabled=true."), @@ -290,7 +302,8 @@ public void getCoordinatorInternalStats(@Suspended final AsyncResponse asyncResp @GET @Path("/pendingAckInternalStats/{tenant}/{namespace}/{topic}/{subName}") - @ApiOperation(value = "Get transaction pending ack internal stats.") + @ApiOperation(value = "Get transaction pending ack internal stats.", + response = TransactionPendingAckInternalStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or subscription name doesn't exist"), @@ -343,7 +356,7 @@ private Void resumeAsyncResponseWithBrokerException(@Suspended AsyncResponse asy @GET @Path("/transactionBufferInternalStats/{tenant}/{namespace}/{topic}") - @ApiOperation(value = "Get transaction buffer internal stats.") + @ApiOperation(value = "Get transaction buffer internal stats.", response = TransactionBufferInternalStats.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @@ -379,6 +392,7 @@ public void getTransactionBufferInternalStats(@Suspended final AsyncResponse asy @POST @Path("/transactionCoordinator/replicas") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 503, message = "This Broker is not configured " + "with transactionCoordinatorEnabled=true."), @ApiResponse(code = 406, message = "The number of replicas should be more than " @@ -401,7 +415,7 @@ public void scaleTransactionCoordinators(@Suspended final AsyncResponse asyncRes @GET @Path("/positionStatsInPendingAck/{tenant}/{namespace}/{topic}/{subName}/{ledgerId}/{entryId}") - @ApiOperation(value = "Get position stats in pending ack.") + @ApiOperation(value = "Get position stats in pending ack.", response = PositionInPendingAckStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or subscription name doesn't exist"), @@ -443,6 +457,7 @@ public void getPositionStatsInPendingAck(@Suspended final AsyncResponse asyncRes @Path("/abortTransaction/{mostSigBits}/{leastSigBits}") @ApiOperation(value = "Abort transaction") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or coordinator or transaction doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " From c25d7b20b21e66f122f949b5a26fa32b433632b7 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 27 May 2024 23:45:50 +0800 Subject: [PATCH 230/580] [fix][admin] Clearly define REST API on Open API for Topics (#22782) --- .../broker/admin/v2/NonPersistentTopics.java | 19 +- .../broker/admin/v2/PersistentTopics.java | 327 ++++++++++++++---- 2 files changed, 270 insertions(+), 76 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index 7de7d7363c0b1..5a7ea1b7632c8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -52,8 +52,10 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.EntryFilters; import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.policies.data.TopicStats; @@ -74,7 +76,7 @@ public class NonPersistentTopics extends PersistentTopics { @GET @Path("/{tenant}/{namespace}/{topic}/partitions") - @ApiOperation(value = "Get partitioned topic metadata.") + @ApiOperation(value = "Get partitioned topic metadata.", response = PartitionedTopicMetadata.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to manage resources on this tenant"), @@ -102,7 +104,7 @@ public void getPartitionedMetadata( @GET @Path("{tenant}/{namespace}/{topic}/internalStats") - @ApiOperation(value = "Get the internal stats for the topic.") + @ApiOperation(value = "Get the internal stats for the topic.", response = PersistentTopicInternalStats.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to manage resources on this tenant"), @@ -145,6 +147,7 @@ public void getInternalStats( @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to manage resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -317,6 +320,7 @@ public void getPartitionedStats( @Path("/{tenant}/{namespace}/{topic}/unload") @ApiOperation(value = "Unload a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "This operation requires super-user access"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -502,6 +506,7 @@ public void getListFromBundle( @ApiOperation(value = "Truncate a topic.", notes = "NonPersistentTopic does not support truncate.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 412, message = "NonPersistentTopic does not support truncate.") }) public void truncateTopic( @@ -525,7 +530,7 @@ protected void validateAdminOperationOnTopic(TopicName topicName, boolean author @GET @Path("/{tenant}/{namespace}/{topic}/entryFilters") - @ApiOperation(value = "Get entry filters for a topic.") + @ApiOperation(value = "Get entry filters for a topic.", response = EntryFilters.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenants or Namespace doesn't exist") }) public void getEntryFilters(@Suspended AsyncResponse asyncResponse, @@ -553,7 +558,9 @@ public void getEntryFilters(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/entryFilters") @ApiOperation(value = "Set entry filters for specified topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -581,7 +588,9 @@ public void setEntryFilters(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/entryFilters") @ApiOperation(value = "Remove entry filters for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 7e138442ae228..8a1f4e0dc5600 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -51,7 +51,10 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.GetStatsOptions; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; +import org.apache.pulsar.client.admin.OffloadProcessStatus; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.ResetCursorData; @@ -164,7 +167,10 @@ public void getPartitionedTopicList( @ApiOperation(value = "Get permissions on a topic.", notes = "Retrieve the effective permissions for a topic." + " These permissions are defined by the permissions set at the" - + "namespace level combined (union) with any eventual specific permission set on the topic.") + + "namespace level combined (union) with any eventual specific permission set on the topic." + + "Returns a nested map structure which Swagger does not fully support for display. " + + "Structure: Map>. Please refer to this structure for details.", + response = AuthAction.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -197,6 +203,7 @@ public void getPermissionsOnTopic( @Path("/{tenant}/{namespace}/{topic}/permissions/{role}") @ApiOperation(value = "Grant a new permission to a role on a single topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -234,6 +241,7 @@ public void grantPermissionsOnTopic( + "level, but rather at the namespace level," + " this operation will return an error (HTTP status code 412).") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -265,6 +273,7 @@ public void revokePermissionsOnTopic( @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -307,6 +316,7 @@ public void createPartitionedTopic( @ApiOperation(value = "Create a non-partitioned topic.", notes = "This is the only REST endpoint from which non-partitioned topics could be created.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), @@ -346,7 +356,7 @@ public void createNonPartitionedTopic( @GET @Path("/{tenant}/{namespace}/{topic}/offloadPolicies") - @ApiOperation(value = "Get offload policies on a topic.") + @ApiOperation(value = "Get offload policies on a topic.", response = OffloadPoliciesImpl.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 500, message = "Internal server error"), }) @@ -372,7 +382,9 @@ public void getOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/offloadPolicies") @ApiOperation(value = "Set offload policies on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -397,7 +409,9 @@ public void setOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/offloadPolicies") @ApiOperation(value = "Delete offload policies on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void removeOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -444,7 +458,9 @@ public void getMaxUnackedMessagesOnConsumer(@Suspended final AsyncResponse async @POST @Path("/{tenant}/{namespace}/{topic}/maxUnackedMessagesOnConsumer") @ApiOperation(value = "Set max unacked messages per consumer config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setMaxUnackedMessagesOnConsumer( @Suspended final AsyncResponse asyncResponse, @@ -470,7 +486,9 @@ public void setMaxUnackedMessagesOnConsumer( @DELETE @Path("/{tenant}/{namespace}/{topic}/maxUnackedMessagesOnConsumer") @ApiOperation(value = "Delete max unacked messages per consumer config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteMaxUnackedMessagesOnConsumer(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -520,7 +538,9 @@ public void getDeduplicationSnapshotInterval(@Suspended final AsyncResponse asyn @POST @Path("/{tenant}/{namespace}/{topic}/deduplicationSnapshotInterval") @ApiOperation(value = "Set deduplicationSnapshotInterval config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setDeduplicationSnapshotInterval( @Suspended final AsyncResponse asyncResponse, @@ -546,7 +566,9 @@ public void setDeduplicationSnapshotInterval( @DELETE @Path("/{tenant}/{namespace}/{topic}/deduplicationSnapshotInterval") @ApiOperation(value = "Delete deduplicationSnapshotInterval config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteDeduplicationSnapshotInterval(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -593,7 +615,9 @@ public void getInactiveTopicPolicies(@Suspended final AsyncResponse asyncRespons @POST @Path("/{tenant}/{namespace}/{topic}/inactiveTopicPolicies") @ApiOperation(value = "Set inactive topic policies on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setInactiveTopicPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -618,7 +642,9 @@ public void setInactiveTopicPolicies(@Suspended final AsyncResponse asyncRespons @DELETE @Path("/{tenant}/{namespace}/{topic}/inactiveTopicPolicies") @ApiOperation(value = "Delete inactive topic policies on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteInactiveTopicPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -666,7 +692,9 @@ public void getMaxUnackedMessagesOnSubscription(@Suspended final AsyncResponse a @POST @Path("/{tenant}/{namespace}/{topic}/maxUnackedMessagesOnSubscription") @ApiOperation(value = "Set max unacked messages per subscription config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setMaxUnackedMessagesOnSubscription( @Suspended final AsyncResponse asyncResponse, @@ -694,7 +722,9 @@ public void setMaxUnackedMessagesOnSubscription( @DELETE @Path("/{tenant}/{namespace}/{topic}/maxUnackedMessagesOnSubscription") @ApiOperation(value = "Delete max unacked messages per subscription config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteMaxUnackedMessagesOnSubscription(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -742,7 +772,9 @@ public void getDelayedDeliveryPolicies(@Suspended final AsyncResponse asyncRespo @POST @Path("/{tenant}/{namespace}/{topic}/delayedDelivery") @ApiOperation(value = "Set delayed delivery messages config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setDelayedDeliveryPolicies( @Suspended final AsyncResponse asyncResponse, @@ -771,7 +803,9 @@ public void setDelayedDeliveryPolicies( @DELETE @Path("/{tenant}/{namespace}/{topic}/delayedDelivery") @ApiOperation(value = "Set delayed delivery messages config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteDelayedDeliveryPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -854,6 +888,7 @@ public void updatePartitionedTopic( @Path("/{tenant}/{namespace}/{topic}/createMissedPartitions") @ApiOperation(value = "Create missed partitions of an existing partitioned topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @@ -961,6 +996,7 @@ public void getProperties( @Path("/{tenant}/{namespace}/{topic}/properties") @ApiOperation(value = "Update the properties on the given topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -997,6 +1033,7 @@ public void updateProperties( @Path("/{tenant}/{namespace}/{topic}/properties") @ApiOperation(value = "Remove the key in properties on the given topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1033,6 +1070,7 @@ public void removeProperties( @ApiOperation(value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1073,6 +1111,7 @@ public void deletePartitionedTopic( @Path("/{tenant}/{namespace}/{topic}/unload") @ApiOperation(value = "Unload a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic does not exist"), @@ -1107,6 +1146,7 @@ public void unloadTopic( + "subscription or producer connected to the it. " + "Force delete ignores connected clients and deletes topic by explicitly closing them.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1382,6 +1422,7 @@ public void getPartitionedStatsInternal( + " there are any active consumers attached to it. " + "Force delete ignores connected consumers and deletes subscription by explicitly closing them.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1434,6 +1475,7 @@ public void deleteSubscription( @ApiOperation(value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1469,6 +1511,7 @@ public void skipAllMessages( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/skip/{numMessages}") @ApiOperation(value = "Skipping messages on a topic subscription.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1505,6 +1548,7 @@ public void skipMessages( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/expireMessages/{expireTimeInSeconds}") @ApiOperation(value = "Expiry messages on a topic subscription.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1542,6 +1586,7 @@ public void expireTopicMessages( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/expireMessages") @ApiOperation(value = "Expiry messages on a topic subscription.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1581,6 +1626,7 @@ public void expireTopicMessages( @Path("/{tenant}/{namespace}/{topic}/all_subscription/expireMessages/{expireTimeInSeconds}") @ApiOperation(value = "Expiry messages on all subscriptions of topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1617,6 +1663,7 @@ public void expireMessagesForAllSubscriptions( @ApiOperation(value = "Create a subscription on the topic.", notes = "Creates a subscription on the topic at the specified message id") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 400, message = "Create subscription on non persistent topic is not supported"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -1672,6 +1719,7 @@ public void createSubscription( @ApiOperation(value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before resetting cursor.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1722,6 +1770,7 @@ public void resetCursor( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/properties") @ApiOperation(value = "Replace all the properties on the given subscription") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1757,7 +1806,8 @@ public void updateSubscriptionProperties( @GET @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/properties") - @ApiOperation(value = "Return all the properties on the given subscription") + @ApiOperation(value = "Return all the properties on the given subscription", + response = String.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -1795,6 +1845,7 @@ public void getSubscriptionProperties( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/analyzeBacklog") @ApiOperation(value = "Analyse a subscription, by scanning all the unprocessed messages") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1841,6 +1892,7 @@ public void analyzeSubscriptionBacklog( @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before resetting cursor.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1879,6 +1931,13 @@ public void resetCursorOnPosition( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/position/{messagePosition}") @ApiOperation(value = "Peek nth message on a topic subscription.") @ApiResponses(value = { + @ApiResponse( + code = 200, + message = "Successfully retrieved the message. The response is a binary byte stream " + + "containing the message data. Clients need to parse this binary stream based" + + " on the message metadata provided in the response headers.", + response = byte[].class + ), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1921,6 +1980,13 @@ public void peekNthMessage( @ApiOperation(value = "Examine a specific message on a topic by position relative to the earliest or the latest message.") @ApiResponses(value = { + @ApiResponse( + code = 200, + message = "Successfully retrieved the message. The response is a binary byte stream " + + "containing the message data. Clients need to parse this binary stream based" + + " on the message metadata provided in the response headers.", + response = byte[].class + ), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic, the message position does not exist"), @@ -1962,6 +2028,13 @@ public void examineMessage( @Path("/{tenant}/{namespace}/{topic}/ledger/{ledgerId}/entry/{entryId}") @ApiOperation(value = "Get message by its messageId.") @ApiResponses(value = { + @ApiResponse( + code = 200, + message = "Successfully retrieved the message. The response is a binary byte stream " + + "containing the message data. Clients need to parse this binary stream based" + + " on the message metadata provided in the response headers.", + response = byte[].class + ), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -2002,7 +2075,8 @@ public void getMessageById( @GET @Path("/{tenant}/{namespace}/{topic}/messageid/{timestamp}") - @ApiOperation(value = "Get message ID published at or just after this absolute timestamp (in ms).") + @ApiOperation(value = "Get message ID published at or just after this absolute timestamp (in ms).", + response = MessageIdAdv.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -2132,7 +2206,9 @@ public void getBacklogQuotaMap( @POST @Path("/{tenant}/{namespace}/{topic}/backlogQuota") @ApiOperation(value = "Set a backlog quota for a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 405, @@ -2162,7 +2238,9 @@ public void setBacklogQuota( @DELETE @Path("/{tenant}/{namespace}/{topic}/backlogQuota") @ApiOperation(value = "Remove a backlog quota policy from a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2225,7 +2303,9 @@ public void getReplicationClusters(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/replication") @ApiOperation(value = "Set the replication clusters for a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 405, @@ -2252,7 +2332,9 @@ public void setReplicationClusters( @DELETE @Path("/{tenant}/{namespace}/{topic}/replication") @ApiOperation(value = "Remove the replication clusters from a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2311,7 +2393,9 @@ public void getMessageTTL(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/messageTTL") @ApiOperation(value = "Set message TTL in seconds for a topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Not authenticate to perform the request or policy is read only"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = @@ -2341,6 +2425,7 @@ public void setMessageTTL(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/{topic}/messageTTL") @ApiOperation(value = "Remove message TTL in seconds for a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Not authenticate to perform the request or policy is read only"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @@ -2395,7 +2480,9 @@ public void getDeduplication(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/deduplicationEnabled") @ApiOperation(value = "Set deduplication enabled on a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry")}) @@ -2423,7 +2510,9 @@ public void setDeduplication( @DELETE @Path("/{tenant}/{namespace}/{topic}/deduplicationEnabled") @ApiOperation(value = "Remove deduplication configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2476,7 +2565,9 @@ public void getRetention(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/retention") @ApiOperation(value = "Set retention configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2514,7 +2605,9 @@ public void setRetention(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/retention") @ApiOperation(value = "Remove retention configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2547,7 +2640,9 @@ public void removeRetention(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/dispatcherPauseOnAckStatePersistent") @ApiOperation(value = "Set dispatcher pause on ack state persistent configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2578,7 +2673,9 @@ public void setDispatcherPauseOnAckStatePersistent(@Suspended final AsyncRespons @DELETE @Path("/{tenant}/{namespace}/{topic}/dispatcherPauseOnAckStatePersistent") @ApiOperation(value = "Remove dispatcher pause on ack state persistent configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2664,7 +2761,9 @@ public void getPersistence(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/persistence") @ApiOperation(value = "Set configuration of persistence policies for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2704,7 +2803,9 @@ public void setPersistence(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/persistence") @ApiOperation(value = "Remove configuration of persistence policies for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2763,7 +2864,9 @@ public void getMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResp @POST @Path("/{tenant}/{namespace}/{topic}/maxSubscriptionsPerTopic") @ApiOperation(value = "Set maxSubscriptionsPerTopic config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2796,7 +2899,9 @@ public void setMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResp @DELETE @Path("/{tenant}/{namespace}/{topic}/maxSubscriptionsPerTopic") @ApiOperation(value = "Remove maxSubscriptionsPerTopic config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2853,7 +2958,9 @@ public void getReplicatorDispatchRate(@Suspended final AsyncResponse asyncRespon @POST @Path("/{tenant}/{namespace}/{topic}/replicatorDispatchRate") @ApiOperation(value = "Set replicatorDispatchRate config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2886,7 +2993,9 @@ public void setReplicatorDispatchRate(@Suspended final AsyncResponse asyncRespon @DELETE @Path("/{tenant}/{namespace}/{topic}/replicatorDispatchRate") @ApiOperation(value = "Remove replicatorDispatchRate config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2943,7 +3052,9 @@ public void getMaxProducers(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/maxProducers") @ApiOperation(value = "Set maxProducers config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2978,7 +3089,9 @@ public void setMaxProducers(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/maxProducers") @ApiOperation(value = "Remove maxProducers config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -3037,7 +3150,9 @@ public void getMaxConsumers(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/maxConsumers") @ApiOperation(value = "Set maxConsumers config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -3072,7 +3187,9 @@ public void setMaxConsumers(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/maxConsumers") @ApiOperation(value = "Remove maxConsumers config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -3132,7 +3249,9 @@ public void getMaxMessageSize(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/maxMessageSize") @ApiOperation(value = "Set maxMessageSize config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -3169,7 +3288,9 @@ public void setMaxMessageSize(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/maxMessageSize") @ApiOperation(value = "Remove maxMessageSize config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -3204,6 +3325,12 @@ public void removeMaxMessageSize(@Suspended final AsyncResponse asyncResponse, @ApiOperation(value = "Terminate a topic. A topic that is terminated will not accept any more " + "messages to be published and will let consumer to drain existing messages in backlog") @ApiResponses(value = { + @ApiResponse( + code = 200, + message = "Operation terminated successfully. The response includes the 'lastMessageId'," + + " which is the identifier of the last message processed.", + response = MessageIdAdv.class + ), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -3241,6 +3368,7 @@ public void terminate( @ApiOperation(value = "Terminate all partitioned topic. A topic that is terminated will not accept any more " + "messages to be published and will let consumer to drain existing messages in backlog") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -3267,6 +3395,7 @@ public void terminatePartitionedTopic(@Suspended final AsyncResponse asyncRespon @Path("/{tenant}/{namespace}/{topic}/compaction") @ApiOperation(value = "Trigger a compaction operation on a topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -3299,7 +3428,8 @@ public void compact( @GET @Path("/{tenant}/{namespace}/{topic}/compaction") - @ApiOperation(value = "Get the status of a compaction operation for a topic.") + @ApiOperation(value = "Get the status of a compaction operation for a topic.", + response = LongRunningProcessStatus.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -3337,6 +3467,7 @@ public void compactionStatus( @Path("/{tenant}/{namespace}/{topic}/offload") @ApiOperation(value = "Offload a prefix of a topic to long term storage") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 400, message = "Message ID is null"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -3374,7 +3505,7 @@ public void triggerOffload( @GET @Path("/{tenant}/{namespace}/{topic}/offload") - @ApiOperation(value = "Offload a prefix of a topic to long term storage") + @ApiOperation(value = "Offload a prefix of a topic to long term storage", response = OffloadProcessStatus.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -3407,7 +3538,7 @@ public void offloadStatus( @GET @Path("/{tenant}/{namespace}/{topic}/lastMessageId") - @ApiOperation(value = "Return the last commit message id of topic") + @ApiOperation(value = "Return the last commit message id of topic", response = MessageIdAdv.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -3440,6 +3571,7 @@ public void getLastMessageId( @Path("/{tenant}/{namespace}/{topic}/trim") @ApiOperation(value = " Trim a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -3476,7 +3608,7 @@ public void trimTopic( @GET @Path("/{tenant}/{namespace}/{topic}/dispatchRate") - @ApiOperation(value = "Get dispatch rate configuration for specified topic.") + @ApiOperation(value = "Get dispatch rate configuration for specified topic.", response = DispatchRateImpl.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, @@ -3504,7 +3636,9 @@ public void getDispatchRate(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/dispatchRate") @ApiOperation(value = "Set message dispatch rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3542,7 +3676,9 @@ public void setDispatchRate(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/dispatchRate") @ApiOperation(value = "Remove message dispatch rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3605,7 +3741,9 @@ public void getSubscriptionDispatchRate(@Suspended final AsyncResponse asyncResp @POST @Path("/{tenant}/{namespace}/{topic}/subscriptionDispatchRate") @ApiOperation(value = "Set subscription message dispatch rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3645,7 +3783,9 @@ public void setSubscriptionDispatchRate( @DELETE @Path("/{tenant}/{namespace}/{topic}/subscriptionDispatchRate") @ApiOperation(value = "Remove subscription message dispatch rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3677,7 +3817,8 @@ public void removeSubscriptionDispatchRate(@Suspended final AsyncResponse asyncR @GET @Path("/{tenant}/{namespace}/{topic}/{subName}/dispatchRate") - @ApiOperation(value = "Get message dispatch rate configuration for specified subscription.") + @ApiOperation(value = "Get message dispatch rate configuration for specified subscription.", + response = DispatchRate.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, @@ -3707,7 +3848,9 @@ public void getSubscriptionLevelDispatchRate(@Suspended final AsyncResponse asyn @POST @Path("/{tenant}/{namespace}/{topic}/{subName}/dispatchRate") @ApiOperation(value = "Set message dispatch rate configuration for specified subscription.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3748,7 +3891,9 @@ public void setSubscriptionLevelDispatchRate( @DELETE @Path("/{tenant}/{namespace}/{topic}/{subName}/dispatchRate") @ApiOperation(value = "Remove message dispatch rate configuration for specified subscription.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3809,7 +3954,9 @@ public void getCompactionThreshold(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/compactionThreshold") @ApiOperation(value = "Set compaction threshold configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3847,7 +3994,9 @@ public void setCompactionThreshold(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/compactionThreshold") @ApiOperation(value = "Remove compaction threshold configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3910,7 +4059,9 @@ public void getMaxConsumersPerSubscription(@Suspended final AsyncResponse asyncR @POST @Path("/{tenant}/{namespace}/{topic}/maxConsumersPerSubscription") @ApiOperation(value = "Set max consumers per subscription configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3949,7 +4100,9 @@ public void setMaxConsumersPerSubscription( @DELETE @Path("/{tenant}/{namespace}/{topic}/maxConsumersPerSubscription") @ApiOperation(value = "Remove max consumers per subscription configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4010,7 +4163,9 @@ public void getPublishRate(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/publishRate") @ApiOperation(value = "Set message publish rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4049,7 +4204,9 @@ public void setPublishRate(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/publishRate") @ApiOperation(value = "Remove message publish rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4116,7 +4273,9 @@ public void getSubscriptionTypesEnabled(@Suspended final AsyncResponse asyncResp @POST @Path("/{tenant}/{namespace}/{topic}/subscriptionTypesEnabled") @ApiOperation(value = "Set is enable sub types for specified topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4155,7 +4314,9 @@ public void setSubscriptionTypesEnabled(@Suspended final AsyncResponse asyncResp @DELETE @Path("/{tenant}/{namespace}/{topic}/subscriptionTypesEnabled") @ApiOperation(value = "Remove subscription types enabled for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -4213,7 +4374,9 @@ public void getSubscribeRate(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/subscribeRate") @ApiOperation(value = "Set subscribe rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4253,7 +4416,9 @@ public void setSubscribeRate( @DELETE @Path("/{tenant}/{namespace}/{topic}/subscribeRate") @ApiOperation(value = "Remove subscribe rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4292,6 +4457,7 @@ public void removeSubscribeRate(@Suspended final AsyncResponse asyncResponse, notes = "The truncate operation will move all cursors to the end of the topic " + "and delete all inactive ledgers.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -4327,6 +4493,7 @@ public void truncateTopic( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/replicatedSubscriptionStatus") @ApiOperation(value = "Enable or disable a replicated subscription on a topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or " + "subscriber is not authorized to access this operation"), @@ -4423,6 +4590,7 @@ public void getSchemaCompatibilityStrategy( @Path("/{tenant}/{namespace}/{topic}/schemaCompatibilityStrategy") @ApiOperation(value = "Set schema compatibility strategy on a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 405, message = "Operation not allowed on persistent topic"), @@ -4463,6 +4631,7 @@ public void setSchemaCompatibilityStrategy( @Path("/{tenant}/{namespace}/{topic}/schemaCompatibilityStrategy") @ApiOperation(value = "Remove schema compatibility strategy on a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 405, message = "Operation not allowed on persistent topic"), @@ -4501,7 +4670,7 @@ public void removeSchemaCompatibilityStrategy( @GET @Path("/{tenant}/{namespace}/{topic}/schemaValidationEnforced") - @ApiOperation(value = "Get schema validation enforced flag for topic.") + @ApiOperation(value = "Get schema validation enforced flag for topic.", response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenants or Namespace doesn't exist") }) public void getSchemaValidationEnforced(@Suspended AsyncResponse asyncResponse, @@ -4529,7 +4698,9 @@ public void getSchemaValidationEnforced(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/schemaValidationEnforced") @ApiOperation(value = "Set schema validation enforced flag on topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or Namespace doesn't exist"), @ApiResponse(code = 412, message = "schemaValidationEnforced value is not valid")}) public void setSchemaValidationEnforced(@Suspended AsyncResponse asyncResponse, @@ -4556,7 +4727,7 @@ public void setSchemaValidationEnforced(@Suspended AsyncResponse asyncResponse, @GET @Path("/{tenant}/{namespace}/{topic}/entryFilters") - @ApiOperation(value = "Get entry filters for a topic.") + @ApiOperation(value = "Get entry filters for a topic.", response = EntryFilters.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenants or Namespace doesn't exist") }) public void getEntryFilters(@Suspended AsyncResponse asyncResponse, @@ -4585,7 +4756,9 @@ public void getEntryFilters(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/entryFilters") @ApiOperation(value = "Set entry filters for specified topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4614,7 +4787,9 @@ public void setEntryFilters(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/entryFilters") @ApiOperation(value = "Remove entry filters for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4649,7 +4824,8 @@ public void removeEntryFilters(@Suspended final AsyncResponse asyncResponse, @GET @Path("/{tenant}/{namespace}/{topic}/shadowTopics") - @ApiOperation(value = "Get the shadow topic list for a topic") + @ApiOperation(value = "Get the shadow topic list for a topic", + response = String.class, responseContainer = "List") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = @@ -4675,7 +4851,9 @@ public void getShadowTopics( @PUT @Path("/{tenant}/{namespace}/{topic}/shadowTopics") @ApiOperation(value = "Set shadow topic list for a topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, enable the topic level policy and retry"), @@ -4703,7 +4881,9 @@ public void setShadowTopics( @DELETE @Path("/{tenant}/{namespace}/{topic}/shadowTopics") @ApiOperation(value = "Delete shadow topics for a topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, enable the topic level policy and retry"), @@ -4730,7 +4910,9 @@ public void deleteShadowTopics( @POST @Path("/{tenant}/{namespace}/{topic}/autoSubscriptionCreation") @ApiOperation(value = "Override namespace's allowAutoSubscriptionCreation setting for a topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, enable the topic level policy and retry"), @@ -4757,7 +4939,8 @@ public void setAutoSubscriptionCreation( @GET @Path("/{tenant}/{namespace}/{topic}/autoSubscriptionCreation") - @ApiOperation(value = "Get autoSubscriptionCreation info in a topic") + @ApiOperation(value = "Get autoSubscriptionCreation info in a topic", + response = AutoSubscriptionCreationOverrideImpl.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic does not exist"), @ApiResponse(code = 405, @@ -4784,7 +4967,9 @@ public void getAutoSubscriptionCreation( @DELETE @Path("/{tenant}/{namespace}/{topic}/autoSubscriptionCreation") @ApiOperation(value = "Remove autoSubscriptionCreation ina a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic does not exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), From f5a00d8c7a0264464db63f9d8442579886b2c1a3 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Tue, 28 May 2024 11:05:03 +0800 Subject: [PATCH 231/580] [improve] [pip] PIP-354: apply topK mechanism to ModularLoadManagerImpl (#22753) Implementation PR for https://github.com/apache/pulsar/pull/22765 ### Motivation `ModularLoadManagerImpl` rely on zk to store and synchronize metadata about load, which pose greate pressure on zk, threatening the stability of system. Every broker will upload its `LocalBrokerData` to zk, and leader broker will retrieve all `LocalBrokerData` from zk, generate all `BundleData` from each `LocalBrokerData`, and update all `BundleData` to zk. As every bundle in the cluster corresponds to a zk node, it is common that there are thousands of zk nodes in a cluster, which results into thousands of read/update operations to zk. This will cause a lot of pressure on zk. **As All Load Shedding Algorithm pick bundles from top to bottom based on throughput/msgRate, bundles with low throughput/msgRate are rarely be selected for shedding. So that we don't need to contain these bundles in the bundle load report.** ### Modifications Reuse the configuration loadBalancerMaxNumberOfBundlesInBundleLoadReport in ExtensibleLoadManager, apply the topK mechanism to ModularLoadManagerImpl. --- conf/broker.conf | 5 +- .../pulsar/broker/ServiceConfiguration.java | 5 +- .../extensions/models/TopKBundles.java | 2 +- .../impl/ModularLoadManagerImpl.java | 41 ++++++++++++-- .../impl/ModularLoadManagerImplTest.java | 56 +++++++++++++++++++ .../data/loadbalancer/BundleData.java | 14 ++++- .../loadbalancer/TimeAverageMessageData.java | 46 ++++++++++++++- 7 files changed, 159 insertions(+), 10 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index f1ec8e7a09f89..d68b6c6ca61de 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1470,7 +1470,10 @@ loadBalancerBrokerLoadDataTTLInSeconds=1800 # The load balancer distributes bundles across brokers, # based on topK bundle load data and other broker load data. # The bigger value will increase the overhead of reporting many bundles in load data. -# (only used in load balancer extension logics) +# Used for ExtensibleLoadManagerImpl and ModularLoadManagerImpl, default value is 10. +# User can disable the bundle filtering feature of ModularLoadManagerImpl by setting this value to -1. +# Enabling this feature can reduce the pressure on the zookeeper when doing load report. +# WARNING: too small value could result in a long load balance time. loadBalancerMaxNumberOfBundlesInBundleLoadReport=10 # Service units'(bundles) split interval. Broker periodically checks whether 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 c18ffe4bc1886..6e8820db27ca7 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 @@ -2662,7 +2662,10 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece + "The load balancer distributes bundles across brokers, " + "based on topK bundle load data and other broker load data." + "The bigger value will increase the overhead of reporting many bundles in load data. " - + "(only used in load balancer extension logics)" + + "Used for ExtensibleLoadManagerImpl and ModularLoadManagerImpl, default value is 10. " + + "User can disable the bundle filtering feature of ModularLoadManagerImpl by setting to -1." + + "Enabling this feature can reduce the pressure on the zookeeper when doing load report." + + "WARNING: too small value could result in a long load balance time." ) private int loadBalancerMaxNumberOfBundlesInBundleLoadReport = 10; @FieldContext( diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundles.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundles.java index 624546fdff837..ec26521af41f5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundles.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/models/TopKBundles.java @@ -99,7 +99,7 @@ public void update(Map bundleStats, int topk) { } } - static void partitionSort(List> arr, int k) { + public static void partitionSort(List> arr, int k) { int start = 0; int end = arr.size() - 1; int target = k - 1; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index 974d75d60b203..a3e6b1c3aebd3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -57,6 +57,7 @@ import org.apache.pulsar.broker.loadbalance.LoadSheddingStrategy; import org.apache.pulsar.broker.loadbalance.ModularLoadManager; import org.apache.pulsar.broker.loadbalance.ModularLoadManagerStrategy; +import org.apache.pulsar.broker.loadbalance.extensions.models.TopKBundles; import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared.BrokerTopicLoadingPredicate; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.stats.prometheus.metrics.Summary; @@ -187,6 +188,9 @@ public class ModularLoadManagerImpl implements ModularLoadManager { private final Lock lock = new ReentrantLock(); private final Set knownBrokers = new HashSet<>(); private Map bundleBrokerAffinityMap; + // array used for sorting and select topK bundles + private final List> bundleArr = new ArrayList<>(); + /** * Initializes fields which do not depend on PulsarService. initialize(PulsarService) should subsequently be called. @@ -1122,6 +1126,32 @@ public void writeBrokerDataOnZooKeeper(boolean force) { } } + /** + * sort bundles by load and select topK bundles for each broker. + * @return the number of bundles selected + */ + private int selectTopKBundle() { + bundleArr.clear(); + bundleArr.addAll(loadData.getBundleData().entrySet()); + + int maxNumberOfBundlesInBundleLoadReport = pulsar.getConfiguration() + .getLoadBalancerMaxNumberOfBundlesInBundleLoadReport(); + if (maxNumberOfBundlesInBundleLoadReport <= 0) { + // select all bundle + return bundleArr.size(); + } else { + // select topK bundle for each broker, so select topK * brokerCount bundle in total + int brokerCount = Math.max(1, loadData.getBrokerData().size()); + int updateBundleCount = Math.min(maxNumberOfBundlesInBundleLoadReport * brokerCount, bundleArr.size()); + if (updateBundleCount == 0) { + // no bundle to update + return 0; + } + TopKBundles.partitionSort(bundleArr, updateBundleCount); + return updateBundleCount; + } + } + /** * As the leader broker, write bundle data aggregated from all brokers to metadata store. */ @@ -1131,11 +1161,12 @@ public void writeBundleDataOnZooKeeper() { // Write the bundle data to metadata store. List> futures = new ArrayList<>(); - for (Map.Entry entry : loadData.getBundleData().entrySet()) { - final String bundle = entry.getKey(); - final BundleData data = entry.getValue(); - futures.add( - pulsarResources.getLoadBalanceResources().getBundleDataResources().updateBundleData(bundle, data)); + // use synchronized to protect bundleArr. + synchronized (bundleArr) { + int updateBundleCount = selectTopKBundle(); + bundleArr.stream().limit(updateBundleCount).forEach(entry -> futures.add( + pulsarResources.getLoadBalanceResources().getBundleDataResources().updateBundleData( + entry.getKey(), (BundleData) entry.getValue()))); } // Write the time average broker data to metadata store. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java index 1f9cd806e19b5..20a33a70bfa40 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java @@ -40,6 +40,7 @@ import java.lang.reflect.Method; import java.net.URL; import java.util.EnumSet; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -425,6 +426,61 @@ public void testMaxTopicDistributionToBroker() throws Exception { } } + /** + * It verifies that the load-manager of leader broker only write topK * brokerCount bundles to zk. + */ + @Test + public void testFilterBundlesWhileWritingToMetadataStore() throws Exception { + Map pulsarServices = new HashMap<>(); + pulsarServices.put(pulsar1.getWebServiceAddress(), pulsar1); + pulsarServices.put(pulsar2.getWebServiceAddress(), pulsar2); + MetadataCache metadataCache = pulsar1.getLocalMetadataStore().getMetadataCache(BundleData.class); + String protocol = "http://"; + PulsarService leaderBroker = pulsarServices.get(protocol + pulsar1.getLeaderElectionService().getCurrentLeader().get().getBrokerId()); + ModularLoadManagerImpl loadManager = (ModularLoadManagerImpl) getField( + leaderBroker.getLoadManager().get(), "loadManager"); + int topK = 1; + leaderBroker.getConfiguration().setLoadBalancerMaxNumberOfBundlesInBundleLoadReport(topK); + // there are two broker in cluster, so total bundle count will be topK * 2 + int exportBundleCount = topK * 2; + + // create and configure bundle-data + final int totalBundles = 5; + final NamespaceBundle[] bundles = LoadBalancerTestingUtils.makeBundles( + nsFactory, "test", "test", "test", totalBundles); + LoadData loadData = (LoadData) getField(loadManager, "loadData"); + for (int i = 0; i < totalBundles; i++) { + final BundleData bundleData = new BundleData(10, 1000); + final String bundleDataPath = String.format("%s/%s", BUNDLE_DATA_BASE_PATH, bundles[i]); + final TimeAverageMessageData longTermMessageData = new TimeAverageMessageData(1000); + longTermMessageData.setMsgThroughputIn(1000 * i); + longTermMessageData.setMsgThroughputOut(1000 * i); + longTermMessageData.setMsgRateIn(1000 * i); + longTermMessageData.setNumSamples(1000); + bundleData.setLongTermData(longTermMessageData); + loadData.getBundleData().put(bundles[i].toString(), bundleData); + loadData.getBrokerData().get(leaderBroker.getWebServiceAddress().substring(protocol.length())) + .getLocalData().getLastStats().put(bundles[i].toString(), new NamespaceBundleStats()); + metadataCache.create(bundleDataPath, bundleData).join(); + } + for (int i = 0; i < totalBundles; i++) { + final String bundleDataPath = String.format("%s/%s", BUNDLE_DATA_BASE_PATH, bundles[i]); + assertEquals(metadataCache.getWithStats(bundleDataPath).get().get().getStat().getVersion(), 0); + } + + // update bundle data to zk and verify + loadManager.writeBundleDataOnZooKeeper(); + int filterBundleCount = totalBundles - exportBundleCount; + for (int i = 0; i < filterBundleCount; i++) { + final String bundleDataPath = String.format("%s/%s", BUNDLE_DATA_BASE_PATH, bundles[i]); + assertEquals(metadataCache.getWithStats(bundleDataPath).get().get().getStat().getVersion(), 0); + } + for (int i = filterBundleCount; i < totalBundles; i++) { + final String bundleDataPath = String.format("%s/%s", BUNDLE_DATA_BASE_PATH, bundles[i]); + assertEquals(metadataCache.getWithStats(bundleDataPath).get().get().getStat().getVersion(), 1); + } + } + // Test that load shedding works @Test public void testLoadShedding() throws Exception { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/BundleData.java b/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/BundleData.java index e5e32046e4970..3c03b7b79bc07 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/BundleData.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/BundleData.java @@ -18,10 +18,13 @@ */ package org.apache.pulsar.policies.data.loadbalancer; +import lombok.EqualsAndHashCode; + /** * Data class comprising the short term and long term historical data for this bundle. */ -public class BundleData { +@EqualsAndHashCode +public class BundleData implements Comparable { // Short term data for this bundle. The time frame of this data is // determined by the number of short term samples // and the bundle update period. @@ -103,4 +106,13 @@ public int getTopics() { public void setTopics(int topics) { this.topics = topics; } + + @Override + public int compareTo(BundleData o) { + int result = this.shortTermData.compareTo(o.shortTermData); + if (result == 0) { + result = this.longTermData.compareTo(o.longTermData); + } + return result; + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/TimeAverageMessageData.java b/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/TimeAverageMessageData.java index 777a6684ce81e..b9c7a43c3a7a0 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/TimeAverageMessageData.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/TimeAverageMessageData.java @@ -18,10 +18,13 @@ */ package org.apache.pulsar.policies.data.loadbalancer; +import lombok.EqualsAndHashCode; + /** * Data class comprising the average message data over a fixed period of time. */ -public class TimeAverageMessageData { +@EqualsAndHashCode +public class TimeAverageMessageData implements Comparable { // The maximum number of samples this data will consider. private int maxSamples; @@ -41,6 +44,11 @@ public class TimeAverageMessageData { // The average message rate out per second. private double msgRateOut; + // Consider the throughput equal if difference is less than 100 KB/s + private static final double throughputDifferenceThreshold = 1e5; + // Consider the msgRate equal if the difference is less than 100 + private static final double msgRateDifferenceThreshold = 100; + // For JSON only. public TimeAverageMessageData() { } @@ -177,4 +185,40 @@ public double totalMsgRate() { public double totalMsgThroughput() { return msgThroughputIn + msgThroughputOut; } + + @Override + public int compareTo(TimeAverageMessageData other) { + int result = this.compareByBandwidthIn(other); + + if (result == 0) { + result = this.compareByBandwidthOut(other); + } + if (result == 0) { + result = this.compareByMsgRate(other); + } + return result; + } + + public int compareByMsgRate(TimeAverageMessageData other) { + double thisMsgRate = this.msgRateIn + this.msgRateOut; + double otherMsgRate = other.msgRateIn + other.msgRateOut; + if (Math.abs(thisMsgRate - otherMsgRate) > msgRateDifferenceThreshold) { + return Double.compare(thisMsgRate, otherMsgRate); + } + return 0; + } + + public int compareByBandwidthIn(TimeAverageMessageData other) { + if (Math.abs(this.msgThroughputIn - other.msgThroughputIn) > throughputDifferenceThreshold) { + return Double.compare(this.msgThroughputIn, other.msgThroughputIn); + } + return 0; + } + + public int compareByBandwidthOut(TimeAverageMessageData other) { + if (Math.abs(this.msgThroughputOut - other.msgThroughputOut) > throughputDifferenceThreshold) { + return Double.compare(this.msgThroughputOut, other.msgThroughputOut); + } + return 0; + } } From 55ad4b22ba2e94029c2e1c01b67b22cb237e5ecc Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 28 May 2024 11:13:12 +0800 Subject: [PATCH 232/580] [fix] [broker] fix topic partitions was expanded even if disabled topic level replication (#22769) --- .../admin/impl/PersistentTopicsBase.java | 11 +++- .../broker/service/OneWayReplicatorTest.java | 65 ++++++++++++++++++- .../service/OneWayReplicatorTestBase.java | 13 +++- .../OneWayReplicatorUsingGlobalZKTest.java | 10 +++ 4 files changed, 93 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 104a84d041d8a..fc47613810426 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -35,6 +35,7 @@ import java.util.Base64; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -451,7 +452,14 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean if (!policies.isPresent()) { return CompletableFuture.completedFuture(null); } - final Set replicationClusters = policies.get().replication_clusters; + // Combine namespace level policies and topic level policies. + Set replicationClusters = policies.get().replication_clusters; + TopicPolicies topicPolicies = + pulsarService.getTopicPoliciesService().getTopicPoliciesIfExists(topicName); + if (topicPolicies != null) { + replicationClusters = new HashSet<>(topicPolicies.getReplicationClusters()); + } + // Do check replicated clusters. if (replicationClusters.size() == 0) { return CompletableFuture.completedFuture(null); } @@ -467,6 +475,7 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean // The replication clusters just has the current cluster itself. return CompletableFuture.completedFuture(null); } + // Do sync operation to other clusters. List> futures = replicationClusters.stream() .map(replicationCluster -> admin.clusters().getClusterAsync(replicationCluster) .thenCompose(clusterData -> pulsarService.getBrokerService() diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index a5f1339e95fbf..3dcd787a0cd5e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -495,8 +495,17 @@ public void testPartitionedTopicLevelReplicationRemoteTopicExist() throws Except admin2.topics().createPartitionedTopic(topicName, 2); admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); // Check the partitioned topic has been created at the remote cluster. - PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); - assertEquals(topicMetadata2.partitions, 2); + Awaitility.await().untilAsserted(() -> { + PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); + assertEquals(topicMetadata2.partitions, 2); + }); + + // Expand partitions + admin2.topics().updatePartitionedTopic(topicName, 3); + Awaitility.await().untilAsserted(() -> { + PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); + assertEquals(topicMetadata2.partitions, 3); + }); // cleanup. admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); waitReplicatorStopped(partition0); @@ -748,4 +757,56 @@ public void testDeletePartitionedTopic() throws Exception { .persistentTopicExists(TopicName.get(topicName).getPartition(1)).join()); } } + + @Test + public void testNoExpandTopicPartitionsWhenDisableTopicLevelReplication() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createPartitionedTopic(topicName, 2); + + // Verify replicator works. + verifyReplicationWorks(topicName); + + // Disable topic level replication. + setTopicLevelClusters(topicName, Arrays.asList(cluster1), admin1, pulsar1); + setTopicLevelClusters(topicName, Arrays.asList(cluster2), admin2, pulsar2); + + // Expand topic. + admin1.topics().updatePartitionedTopic(topicName, 3); + assertEquals(admin1.topics().getPartitionedTopicMetadata(topicName).partitions, 3); + + // Wait for async tasks that were triggered by expanding topic partitions. + Thread.sleep(3 * 1000); + + + // Verify: the topics on the remote cluster did not been expanded. + assertEquals(admin2.topics().getPartitionedTopicMetadata(topicName).partitions, 2); + + cleanupTopics(() -> { + admin1.topics().deletePartitionedTopic(topicName, false); + admin2.topics().deletePartitionedTopic(topicName, false); + }); + } + + @Test + public void testExpandTopicPartitionsOnNamespaceLevelReplication() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createPartitionedTopic(topicName, 2); + + // Verify replicator works. + verifyReplicationWorks(topicName); + + // Expand topic. + admin1.topics().updatePartitionedTopic(topicName, 3); + assertEquals(admin1.topics().getPartitionedTopicMetadata(topicName).partitions, 3); + + // Verify: the topics on the remote cluster will be expanded. + Awaitility.await().untilAsserted(() -> { + assertEquals(admin2.topics().getPartitionedTopicMetadata(topicName).partitions, 3); + }); + + cleanupTopics(() -> { + admin1.topics().deletePartitionedTopic(topicName, false); + admin2.topics().deletePartitionedTopic(topicName, false); + }); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index 6a84432890cb5..7372b2e478475 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -196,9 +196,16 @@ protected void cleanupTopics(String namespace, CleanupTopicAction cleanupTopicAc } protected void waitChangeEventsInit(String namespace) { - PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService() - .getTopic(namespace + "/" + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME, false) - .join().get(); + CompletableFuture> future = pulsar1.getBrokerService() + .getTopic(namespace + "/" + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME, false); + if (future == null) { + return; + } + Optional optional = future.join(); + if (!optional.isPresent()) { + return; + } + PersistentTopic topic = (PersistentTopic) optional.get(); Awaitility.await().atMost(Duration.ofSeconds(180)).untilAsserted(() -> { TopicStatsImpl topicStats = topic.getStats(true, false, false); topicStats.getSubscriptions().entrySet().forEach(entry -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java index d827235bc326d..b4747a8bd0e47 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java @@ -94,4 +94,14 @@ public void testDeleteNonPartitionedTopic() throws Exception { public void testDeletePartitionedTopic() throws Exception { super.testDeletePartitionedTopic(); } + + @Test(enabled = false) + public void testNoExpandTopicPartitionsWhenDisableTopicLevelReplication() throws Exception { + super.testNoExpandTopicPartitionsWhenDisableTopicLevelReplication(); + } + + @Test(enabled = false) + public void testExpandTopicPartitionsOnNamespaceLevelReplication() throws Exception { + super.testExpandTopicPartitionsOnNamespaceLevelReplication(); + } } From 20e83b96c3fcf10010977ab785093e105e4e40d8 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Tue, 28 May 2024 12:40:52 +0800 Subject: [PATCH 233/580] [improve][cli] PIP-353: Improve transaction message visibility for peek-message (#22762) --- pip/pip-353.md | 51 +++---- .../admin/impl/PersistentTopicsBase.java | 30 +++-- .../admin/v3/AdminApiTransactionTest.java | 126 ++++++++++++++++++ .../apache/pulsar/client/admin/Topics.java | 60 ++++++++- .../client/admin/internal/TopicsImpl.java | 70 ++++++++-- .../client/api/TransactionIsolationLevel.java | 31 +++++ .../pulsar/admin/cli/PulsarAdminToolTest.java | 4 +- .../apache/pulsar/admin/cli/CmdTopics.java | 22 ++- 8 files changed, 343 insertions(+), 51 deletions(-) create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TransactionIsolationLevel.java diff --git a/pip/pip-353.md b/pip/pip-353.md index 4315bdab0eb2e..5944aaea1abf4 100644 --- a/pip/pip-353.md +++ b/pip/pip-353.md @@ -25,7 +25,7 @@ This behavior can confuse users and lead to incorrect data handling. The proposa ### In Scope -- Implement flags to selectively display `server markers`, `uncommitted messages`, and `aborted messages` in peek operations. +- Implement flags to selectively display `server markers`, `uncommitted messages(include aborted messages) for transaction` in peek operations. - Set the default behavior to only show messages from committed transactions to ensure data integrity. ### Out of Scope @@ -37,8 +37,9 @@ This behavior can confuse users and lead to incorrect data handling. The proposa The proposal introduces three new flags to the `peek-messages` command: 1. `--show-server-marker`: Controls the visibility of server markers (default: `false`). -2. `--show-txn-uncommitted`: Controls the visibility of messages from uncommitted transactions (default: `false`). -3. `--show-txn-aborted`: Controls the visibility of messages from aborted transactions (default: `false`). +2. `---transaction-isolation-level`: Controls the visibility of messages for transactions. (default: `READ_COMMITTED`). Options: + - READ_COMMITTED: Can only consume all transactional messages which have been committed. + - READ_UNCOMMITTED: Can consume all messages, even transactional messages which have been aborted. These flags will allow administrators and developers to tailor the peek functionality to their needs, improving the usability and security of message handling in transactional contexts. @@ -46,7 +47,7 @@ These flags will allow administrators and developers to tailor the peek function ### Design & Implementation Details -To support the `--show-server-marker` and `--show-txn-aborted`, `--show-txn-uncommitted` flags, needs to introduce specific tag into the `headers` of messages returned by the +To support the `--show-server-marker` and `---transaction-isolation-level` flags, needs to introduce specific tag into the `headers` of messages returned by the [peekNthMessage REST API](https://github.com/apache/pulsar/blob/8ca01cd42edfd4efd986f752f6f8538ea5bf4f94/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java#L1892-L1905). - `X-Pulsar-marker-type`: Already exists. @@ -62,11 +63,10 @@ see the following code: [https://github.com/shibd/pulsar/pull/34](https://github New command line flags added for the `bin/pulsar-admin topics peek-messages` command: -| Flag | Abbreviation | Type | Default | Description | -|--------------------------|--------------|---------|---------|----------------------------------------------------------------| -| `--show-server-marker` | `-ssm` | Boolean | `false` | Enables the display of internal server write markers. | -| `--show-txn-uncommitted` | `-stu` | Boolean | `false` | Enables the display of messages from uncommitted transactions. | -| `--show-txn-aborted` | `-sta` | Boolean | `false` | Enables the display of messages from aborted transactions. | +| Flag | Abbreviation | Type | Default | Description | +|----------------------------------|--------------|---------|---------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `--show-server-marker` | `-ssm` | Boolean | `false` | Enables the display of internal server write markers. | +| `---transaction-isolation-level` | `-til` | Enum | `false` | Enables theSets the isolation level for consuming messages within transactions.
- 'READ_COMMITTED' allows consuming only committed transactional messages.
- 'READ_UNCOMMITTED' allows consuming all messages, even transactional messages which have been aborted. | ## Public-facing Changes @@ -85,10 +85,11 @@ Add two methods to the admin.Topics() interface. * Number of messages * @param showServerMarker * Enables the display of internal server write markers - * @param showTxnAborted - * Enables the display of messages from aborted transactions - * @param showTxnUncommitted - * Enables the display of messages from uncommitted transactions + * @param transactionIsolationLevel + * Sets the isolation level for consuming messages within transactions. + * - 'READ_COMMITTED' allows consuming only committed transactional messages. + * - 'READ_UNCOMMITTED' allows consuming all messages, + * even transactional messages which have been aborted. * @return * @throws NotAuthorizedException * Don't have admin permission @@ -98,8 +99,9 @@ Add two methods to the admin.Topics() interface. * Unexpected error */ List> peekMessages(String topic, String subName, int numMessages, - boolean showServerMarker, boolean showTxnAborted, - boolean showTxnUncommitted) throws PulsarAdminException; + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel) + throws PulsarAdminException; + /** * Peek messages from a topic subscription asynchronously. @@ -112,15 +114,16 @@ Add two methods to the admin.Topics() interface. * Number of messages * @param showServerMarker * Enables the display of internal server write markers - * @param showTxnAborted - * Enables the display of messages from aborted transactions - * @param showTxnUncommitted - * Enables the display of messages from uncommitted transactions - * @return a future that can be used to track when the messages are returned + @param transactionIsolationLevel + * Sets the isolation level for consuming messages within transactions. + * - 'READ_COMMITTED' allows consuming only committed transactional messages. + * - 'READ_UNCOMMITTED' allows consuming all messages, + * even transactional messages which have been aborted. + * @return a future that can be used to track when the messages are returned */ - CompletableFuture>> peekMessagesAsync(String topic, String subName, int numMessages, - boolean showServerMarker, boolean showTxnAborted, - boolean showTxnUncommitted); + CompletableFuture>> peekMessagesAsync( + String topic, String subName, int numMessages, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel); ``` ## Backward & Forward Compatibility @@ -130,5 +133,5 @@ Reverting to a previous version of Pulsar without this feature will remove the a ### Upgrade While upgrading to the new version of Pulsar that includes these changes, the default behavior of the `peek-messages` command will change. -Existing scripts or commands that rely on the old behavior (where transaction markers and messages from uncommitted or aborted transactions are visible) will need to explicitly set the new flags (`--show-server-marker`, `--show-txn-uncommitted`, `--show-txn-aborted`) to `true` to maintain the old behavior. +Existing scripts or commands that rely on the old behavior (where transaction markers and messages from uncommitted or aborted transactions are visible) will need to explicitly set the new flags (`--show-server-marker true` and `--transaction-isolation-level READ_UNCOMMITTED` to maintain the old behavior. This change is necessary as the previous default behavior did not align with typical expectations around data visibility and integrity in transactional systems. \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index fc47613810426..bc933cc5c1adb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -98,6 +98,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; @@ -2726,7 +2727,7 @@ public void readEntryFailed(ManagedLedgerException exception, @Override public void readEntryComplete(Entry entry, Object ctx) { try { - results.complete(generateResponseWithEntry(entry)); + results.complete(generateResponseWithEntry(entry, (PersistentTopic) topic)); } catch (IOException exception) { throw new RestException(exception); } finally { @@ -2867,10 +2868,12 @@ protected CompletableFuture internalPeekNthMessageAsync(String subName entry = sub.peekNthMessage(messagePosition); } } - return entry; - }).thenCompose(entry -> { + return entry.thenApply(e -> Pair.of(e, (PersistentTopic) topic)); + }).thenCompose(entryTopicPair -> { + Entry entry = entryTopicPair.getLeft(); + PersistentTopic persistentTopic = entryTopicPair.getRight(); try { - Response response = generateResponseWithEntry(entry); + Response response = generateResponseWithEntry(entry, persistentTopic); return CompletableFuture.completedFuture(response); } catch (NullPointerException npe) { throw new RestException(Status.NOT_FOUND, "Message not found"); @@ -2949,17 +2952,18 @@ public String toString() { PersistentTopicsBase.this.topicName); } }, null); - return future; + return future.thenApply(entry -> Pair.of(entry, (PersistentTopic) topic)); } catch (ManagedLedgerException exception) { log.error("[{}] Failed to examine message at position {} from {} due to {}", clientAppId(), messagePosition, topicName, exception); throw new RestException(exception); } - - }).thenApply(entry -> { + }).thenApply(entryTopicPair -> { + Entry entry = entryTopicPair.getLeft(); + PersistentTopic persistentTopic = entryTopicPair.getRight(); try { - return generateResponseWithEntry(entry); + return generateResponseWithEntry(entry, persistentTopic); } catch (IOException exception) { throw new RestException(exception); } finally { @@ -2970,7 +2974,7 @@ public String toString() { }); } - private Response generateResponseWithEntry(Entry entry) throws IOException { + private Response generateResponseWithEntry(Entry entry, PersistentTopic persistentTopic) throws IOException { checkNotNull(entry); PositionImpl pos = (PositionImpl) entry.getPosition(); ByteBuf metadataAndPayload = entry.getDataBuffer(); @@ -3088,6 +3092,14 @@ private Response generateResponseWithEntry(Entry entry) throws IOException { if (metadata.hasNullPartitionKey()) { responseBuilder.header("X-Pulsar-null-partition-key", metadata.isNullPartitionKey()); } + if (metadata.hasTxnidMostBits() && metadata.hasTxnidLeastBits()) { + TxnID txnID = new TxnID(metadata.getTxnidMostBits(), metadata.getTxnidLeastBits()); + boolean isTxnAborted = persistentTopic.isTxnAborted(txnID, (PositionImpl) entry.getPosition()); + responseBuilder.header("X-Pulsar-txn-aborted", isTxnAborted); + } + boolean isTxnUncommitted = ((PositionImpl) entry.getPosition()) + .compareTo(persistentTopic.getMaxReadPosition()) > 0; + responseBuilder.header("X-Pulsar-txn-uncommitted", isTxnUncommitted); // Decode if needed CompressionCodec codec = CompressionCodecProvider.getCompressionCodec(metadata.getCompression()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java index adf810945de5f..5a192d0159a42 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java @@ -38,6 +38,7 @@ import lombok.Cleanup; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.http.HttpStatus; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; @@ -48,12 +49,16 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.transaction.TransactionImpl; +import org.apache.pulsar.common.api.proto.MarkerType; +import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicDomain; @@ -917,6 +922,127 @@ public void testAbortTransaction() throws Exception { } } + @Test + public void testPeekMessageForSkipTxnMarker() throws Exception { + initTransaction(1); + + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/peek_marker"); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + int n = 10; + for (int i = 0; i < n; i++) { + Transaction txn = pulsarClient.newTransaction().build().get(); + producer.newMessage(txn).value("msg").send(); + txn.commit().get(); + } + + List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", n, + false, TransactionIsolationLevel.READ_UNCOMMITTED); + assertEquals(peekMsgs.size(), n); + for (Message peekMsg : peekMsgs) { + assertEquals(new String(peekMsg.getValue()), "msg"); + } + } + + @Test + public void testPeekMessageFoReadCommittedMessages() throws Exception { + initTransaction(1); + + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/peek_txn"); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + int n = 10; + // Alternately sends `n` committed transactional messages and `n` abort transactional messages. + for (int i = 0; i < 2 * n; i++) { + Transaction txn = pulsarClient.newTransaction().build().get(); + if (i % 2 == 0) { + producer.newMessage(txn).value("msg").send(); + txn.commit().get(); + } else { + producer.newMessage(txn).value("msg-aborted").send(); + txn.abort(); + } + } + // Then sends 1 uncommitted transactional messages. + Transaction txn = pulsarClient.newTransaction().build().get(); + producer.newMessage(txn).value("msg-uncommitted").send(); + // Then sends n-1 no transaction messages. + for (int i = 0; i < n - 1; i++) { + producer.newMessage().value("msg-after-uncommitted").send(); + } + + // peek n message, all messages value should be "msg" + { + List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", n, + false, TransactionIsolationLevel.READ_COMMITTED); + assertEquals(peekMsgs.size(), n); + for (Message peekMsg : peekMsgs) { + assertEquals(new String(peekMsg.getValue()), "msg"); + } + } + + // peek 3 * n message, and still get n message, all messages value should be "msg" + { + List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", 2 * n, + false, TransactionIsolationLevel.READ_COMMITTED); + assertEquals(peekMsgs.size(), n); + for (Message peekMsg : peekMsgs) { + assertEquals(new String(peekMsg.getValue()), "msg"); + } + } + } + + @Test + public void testPeekMessageForShowAllMessages() throws Exception { + initTransaction(1); + + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/peek_all"); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + int n = 10; + // Alternately sends `n` committed transactional messages and `n` abort transactional messages. + for (int i = 0; i < 2 * n; i++) { + Transaction txn = pulsarClient.newTransaction().build().get(); + if (i % 2 == 0) { + producer.newMessage(txn).value("msg").send(); + txn.commit().get(); + } else { + producer.newMessage(txn).value("msg-aborted").send(); + txn.abort(); + } + } + // Then sends `n` uncommitted transactional messages. + Transaction txn = pulsarClient.newTransaction().build().get(); + for (int i = 0; i < n; i++) { + producer.newMessage(txn).value("msg-uncommitted").send(); + } + + // peek 5 * n message, will get 5 * n msg. + List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", 5 * n, + true, TransactionIsolationLevel.READ_UNCOMMITTED); + assertEquals(peekMsgs.size(), 5 * n); + + for (int i = 0; i < 4 * n; i++) { + Message peekMsg = peekMsgs.get(i); + MessageImpl peekMsgImpl = (MessageImpl) peekMsg; + MessageMetadata metadata = peekMsgImpl.getMessageBuilder(); + if (metadata.hasMarkerType()) { + assertTrue(metadata.getMarkerType() == MarkerType.TXN_COMMIT_VALUE || + metadata.getMarkerType() == MarkerType.TXN_ABORT_VALUE); + } else { + String value = new String(peekMsg.getValue()); + assertTrue(value.equals("msg") || value.equals("msg-aborted")); + } + } + for (int i = 4 * n; i < peekMsgs.size(); i++) { + Message peekMsg = peekMsgs.get(i); + assertEquals(new String(peekMsg.getValue()), "msg-uncommitted"); + } + } + private static void verifyCoordinatorStats(String state, long sequenceId, long lowWaterMark) { assertEquals(state, "Ready"); diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java index 574b859e82c80..c681bd1a7bca1 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; @@ -1653,7 +1654,53 @@ void expireMessagesForAllSubscriptions(String topic, long expireTimeInSeconds) * @throws PulsarAdminException * Unexpected error */ - List> peekMessages(String topic, String subName, int numMessages) throws PulsarAdminException; + default List> peekMessages(String topic, String subName, int numMessages) + throws PulsarAdminException { + return peekMessages(topic, subName, numMessages, false, TransactionIsolationLevel.READ_COMMITTED); + } + + /** + * Peek messages from a topic subscription. + * + * @param topic + * topic name + * @param subName + * Subscription name + * @param numMessages + * Number of messages + * @param showServerMarker + * Enables the display of internal server write markers + * @param transactionIsolationLevel + * Sets the isolation level for peeking messages within transactions. + * - 'READ_COMMITTED' allows peeking only committed transactional messages. + * - 'READ_UNCOMMITTED' allows peeking all messages, + * even transactional messages which have been aborted. + * @return + * @throws NotAuthorizedException + * Don't have admin permission + * @throws NotFoundException + * Topic or subscription does not exist + * @throws PulsarAdminException + * Unexpected error + */ + List> peekMessages(String topic, String subName, int numMessages, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel) + throws PulsarAdminException; + + /** + * Peek messages from a topic subscription asynchronously. + * + * @param topic + * topic name + * @param subName + * Subscription name + * @param numMessages + * Number of messages + * @return a future that can be used to track when the messages are returned + */ + default CompletableFuture>> peekMessagesAsync(String topic, String subName, int numMessages) { + return peekMessagesAsync(topic, subName, numMessages, false, TransactionIsolationLevel.READ_COMMITTED); + } /** * Peek messages from a topic subscription asynchronously. @@ -1664,9 +1711,18 @@ void expireMessagesForAllSubscriptions(String topic, long expireTimeInSeconds) * Subscription name * @param numMessages * Number of messages + * @param showServerMarker + * Enables the display of internal server write markers + @param transactionIsolationLevel + * Sets the isolation level for peeking messages within transactions. + * - 'READ_COMMITTED' allows peeking only committed transactional messages. + * - 'READ_UNCOMMITTED' allows peeking all messages, + * even transactional messages which have been aborted. * @return a future that can be used to track when the messages are returned */ - CompletableFuture>> peekMessagesAsync(String topic, String subName, int numMessages); + CompletableFuture>> peekMessagesAsync( + String topic, String subName, int numMessages, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel); /** * Get a message by its messageId via a topic subscription. diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index f76cfbcde985f..b7a8b87664075 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -56,6 +56,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; @@ -130,6 +131,8 @@ public class TopicsImpl extends BaseResource implements Topics { private static final String SCHEMA_VERSION = "X-Pulsar-Base64-schema-version-b64encoded"; private static final String ENCRYPTION_PARAM = "X-Pulsar-Base64-encryption-param"; private static final String ENCRYPTION_KEYS = "X-Pulsar-Base64-encryption-keys"; + public static final String TXN_ABORTED = "X-Pulsar-txn-aborted"; + public static final String TXN_UNCOMMITTED = "X-Pulsar-txn-uncommitted"; // CHECKSTYLE.ON: MemberName public static final String PROPERTY_SHADOW_SOURCE_KEY = "PULSAR.SHADOW_SOURCE"; @@ -867,7 +870,9 @@ public CompletableFuture expireMessagesForAllSubscriptionsAsync(String top return asyncPostRequest(path, Entity.entity("", MediaType.APPLICATION_JSON)); } - private CompletableFuture>> peekNthMessage(String topic, String subName, int messagePosition) { + private CompletableFuture>> peekNthMessage( + String topic, String subName, int messagePosition, boolean showServerMarker, + TransactionIsolationLevel transactionIsolationLevel) { TopicName tn = validateTopic(topic); String encodedSubName = Codec.encode(subName); WebTarget path = topicPath(tn, "subscription", encodedSubName, @@ -879,7 +884,8 @@ private CompletableFuture>> peekNthMessage(String topic, St @Override public void completed(Response response) { try { - future.complete(getMessagesFromHttpResponse(tn.toString(), response)); + future.complete(getMessagesFromHttpResponse(tn.toString(), response, + showServerMarker, transactionIsolationLevel)); } catch (Exception e) { future.completeExceptionally(getApiException(e)); } @@ -894,28 +900,35 @@ public void failed(Throwable throwable) { } @Override - public List> peekMessages(String topic, String subName, int numMessages) + public List> peekMessages(String topic, String subName, int numMessages, + boolean showServerMarker, + TransactionIsolationLevel transactionIsolationLevel) throws PulsarAdminException { - return sync(() -> peekMessagesAsync(topic, subName, numMessages)); + return sync(() -> peekMessagesAsync(topic, subName, numMessages, showServerMarker, transactionIsolationLevel)); } @Override - public CompletableFuture>> peekMessagesAsync(String topic, String subName, int numMessages) { + public CompletableFuture>> peekMessagesAsync( + String topic, String subName, int numMessages, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel) { checkArgument(numMessages > 0); CompletableFuture>> future = new CompletableFuture>>(); - peekMessagesAsync(topic, subName, numMessages, new ArrayList<>(), future, 1); + peekMessagesAsync(topic, subName, numMessages, new ArrayList<>(), + future, 1, showServerMarker, transactionIsolationLevel); return future; } private void peekMessagesAsync(String topic, String subName, int numMessages, - List> messages, CompletableFuture>> future, int nthMessage) { + List> messages, CompletableFuture>> future, int nthMessage, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel) { if (numMessages <= 0) { future.complete(messages); return; } // if peeking first message succeeds, we know that the topic and subscription exists - peekNthMessage(topic, subName, nthMessage).handle((r, ex) -> { + peekNthMessage(topic, subName, nthMessage, showServerMarker, transactionIsolationLevel) + .handle((r, ex) -> { if (ex != null) { // if we get a not found exception, it means that the position for the message we are trying to get // does not exist. At this point, we can return the already found messages. @@ -930,7 +943,8 @@ private void peekMessagesAsync(String topic, String subName, int numMessages, for (int i = 0; i < Math.min(r.size(), numMessages); i++) { messages.add(r.get(i)); } - peekMessagesAsync(topic, subName, numMessages - r.size(), messages, future, nthMessage + 1); + peekMessagesAsync(topic, subName, numMessages - r.size(), messages, future, + nthMessage + 1, showServerMarker, transactionIsolationLevel); return null; }); } @@ -1253,6 +1267,13 @@ private TopicName validateTopic(String topic) { } private List> getMessagesFromHttpResponse(String topic, Response response) throws Exception { + return getMessagesFromHttpResponse(topic, response, true, + TransactionIsolationLevel.READ_UNCOMMITTED); + } + + private List> getMessagesFromHttpResponse( + String topic, Response response, boolean showServerMarker, + TransactionIsolationLevel transactionIsolationLevel) throws Exception { if (response.getStatus() != Status.OK.getStatusCode()) { throw getApiException(response); @@ -1284,7 +1305,32 @@ private List> getMessagesFromHttpResponse(String topic, Response Map properties = new TreeMap<>(); MultivaluedMap headers = response.getHeaders(); - Object tmp = headers.getFirst(PUBLISH_TIME); + Object tmp = headers.getFirst(MARKER_TYPE); + if (tmp != null) { + if (!showServerMarker) { + return new ArrayList<>(); + } else { + messageMetadata.setMarkerType(Integer.parseInt(tmp.toString())); + } + } + + tmp = headers.getFirst(TXN_ABORTED); + if (tmp != null && Boolean.parseBoolean(tmp.toString())) { + properties.put(TXN_ABORTED, tmp.toString()); + if (transactionIsolationLevel == TransactionIsolationLevel.READ_COMMITTED) { + return new ArrayList<>(); + } + } + + tmp = headers.getFirst(TXN_UNCOMMITTED); + if (tmp != null && Boolean.parseBoolean(tmp.toString())) { + properties.put(TXN_UNCOMMITTED, tmp.toString()); + if (transactionIsolationLevel == TransactionIsolationLevel.READ_COMMITTED) { + return new ArrayList<>(); + } + } + + tmp = headers.getFirst(PUBLISH_TIME); if (tmp != null) { messageMetadata.setPublishTime(DateFormatter.parse(tmp.toString())); } @@ -1336,10 +1382,6 @@ private List> getMessagesFromHttpResponse(String topic, Response if (tmp != null) { messageMetadata.setPartitionKeyB64Encoded(Boolean.parseBoolean(tmp.toString())); } - tmp = headers.getFirst(MARKER_TYPE); - if (tmp != null) { - messageMetadata.setMarkerType(Integer.parseInt(tmp.toString())); - } tmp = headers.getFirst(TXNID_LEAST_BITS); if (tmp != null) { messageMetadata.setTxnidLeastBits(Long.parseLong(tmp.toString())); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TransactionIsolationLevel.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TransactionIsolationLevel.java new file mode 100644 index 0000000000000..ae385b20232c7 --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TransactionIsolationLevel.java @@ -0,0 +1,31 @@ +/* + * 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 org.apache.pulsar.common.classification.InterfaceAudience; +import org.apache.pulsar.common.classification.InterfaceStability; + +@InterfaceAudience.Public +@InterfaceStability.Stable +public enum TransactionIsolationLevel { + // Consumer can only consume all transactional messages which have been committed. + READ_COMMITTED, + // Consumer can consume all messages, even transactional messages which have been aborted. + READ_UNCOMMITTED; +} diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java index fd1bdf4799848..a3b1fa075cffc 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java @@ -81,6 +81,7 @@ import org.apache.pulsar.client.admin.internal.OffloadProcessStatusImpl; import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.api.transaction.TxnID; @@ -1744,7 +1745,8 @@ public void topics() throws Exception { verify(mockTopics).deletePartitionedTopic("persistent://myprop/clust/ns1/ds1", true); cmdTopics.run(split("peek-messages persistent://myprop/clust/ns1/ds1 -s sub1 -n 3")); - verify(mockTopics).peekMessages("persistent://myprop/clust/ns1/ds1", "sub1", 3); + verify(mockTopics).peekMessages("persistent://myprop/clust/ns1/ds1", "sub1", 3, + false, TransactionIsolationLevel.READ_COMMITTED); MessageImpl message = mock(MessageImpl.class); when(message.getData()).thenReturn(new byte[]{}); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index e1e85c68f7e5e..261bd81a5b7bd 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -62,9 +62,12 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; +import org.apache.pulsar.common.api.proto.MarkerType; +import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BacklogQuota; @@ -1097,10 +1100,23 @@ private class PeekMessages extends CliCommand { @Option(names = { "-n", "--count" }, description = "Number of messages (default 1)", required = false) private int numMessages = 1; + @Option(names = { "-ssm", "--show-server-marker" }, + description = "Enables the display of internal server write markers.", required = false) + private boolean showServerMarker = false; + + @Option(names = { "-til", "--transaction-isolation-level" }, + description = "Sets the isolation level for peeking messages within transactions. " + + "'READ_COMMITTED' allows peeking only committed transactional messages. " + + "'READ_UNCOMMITTED' allows peeking all messages, " + + "even transactional messages which have been aborted.", + required = false) + private TransactionIsolationLevel transactionIsolationLevel = TransactionIsolationLevel.READ_COMMITTED; + @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(topicName); - List> messages = getTopics().peekMessages(persistentTopic, subName, numMessages); + List> messages = getTopics().peekMessages(persistentTopic, subName, numMessages, + showServerMarker, transactionIsolationLevel); int position = 0; for (Message msg : messages) { MessageImpl message = (MessageImpl) msg; @@ -1122,6 +1138,10 @@ void run() throws PulsarAdminException { if (message.getDeliverAtTime() != 0) { System.out.println("Deliver at time: " + message.getDeliverAtTime()); } + MessageMetadata msgMetaData = message.getMessageBuilder(); + if (showServerMarker && msgMetaData.hasMarkerType()) { + System.out.println("Marker Type: " + MarkerType.valueOf(msgMetaData.getMarkerType())); + } if (message.getBrokerEntryMetadata() != null) { if (message.getBrokerEntryMetadata().hasBrokerTimestamp()) { From b0e8fe9f7ea765d4b580ab3eb6bf3c51c59e685f Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Tue, 28 May 2024 18:53:57 +0800 Subject: [PATCH 234/580] [fix] [broker] replace loadSheddingPipeline with loadSheddingStrategy. (#22786) --- .../impl/ModularLoadManagerImpl.java | 81 +++++++++---------- 1 file changed, 38 insertions(+), 43 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index a3e6b1c3aebd3..5d08ea9c3c3be 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -139,8 +139,8 @@ public class ModularLoadManagerImpl implements ModularLoadManager { // LocalBrokerData available before most recent update. private LocalBrokerData lastData; - // Pipeline used to determine what namespaces, if any, should be unloaded. - private final List loadSheddingPipeline; + // Used to determine what namespaces, if any, should be unloaded. + private LoadSheddingStrategy loadSheddingStrategy; // Local data for the broker this is running on. private LocalBrokerData localData; @@ -204,7 +204,6 @@ public ModularLoadManagerImpl() { defaultStats = new NamespaceBundleStats(); filterPipeline = new ArrayList<>(); loadData = new LoadData(); - loadSheddingPipeline = new ArrayList<>(); preallocatedBundleToBroker = new ConcurrentHashMap<>(); executors = Executors.newSingleThreadExecutor( new ExecutorProvider.ExtendedThreadFactory("pulsar-modular-load-manager")); @@ -270,7 +269,7 @@ public void initialize(final PulsarService pulsar) { () -> LoadManagerShared.refreshBrokerToFailureDomainMap(pulsar, brokerToFailureDomainMap)); }); - loadSheddingPipeline.add(createLoadSheddingStrategy()); + loadSheddingStrategy = createLoadSheddingStrategy(); } public void handleDataNotification(Notification t) { @@ -476,9 +475,7 @@ private synchronized void cleanupDeadBrokersData() { if (pulsar.getLeaderElectionService() != null && pulsar.getLeaderElectionService().isLeader()) { deadBrokers.forEach(this::deleteTimeAverageDataFromMetadataStoreAsync); - for (LoadSheddingStrategy loadSheddingStrategy : loadSheddingPipeline) { - loadSheddingStrategy.onActiveBrokersChange(activeBrokers); - } + loadSheddingStrategy.onActiveBrokersChange(activeBrokers); placementStrategy.onActiveBrokersChange(activeBrokers); } } @@ -632,47 +629,45 @@ public synchronized void doLoadShedding() { final Map recentlyUnloadedBundles = loadData.getRecentlyUnloadedBundles(); recentlyUnloadedBundles.keySet().removeIf(e -> recentlyUnloadedBundles.get(e) < timeout); - for (LoadSheddingStrategy strategy : loadSheddingPipeline) { - final Multimap bundlesToUnload = strategy.findBundlesForUnloading(loadData, conf); + final Multimap bundlesToUnload = loadSheddingStrategy.findBundlesForUnloading(loadData, conf); - bundlesToUnload.asMap().forEach((broker, bundles) -> { - bundles.forEach(bundle -> { - final String namespaceName = LoadManagerShared.getNamespaceNameFromBundleName(bundle); - final String bundleRange = LoadManagerShared.getBundleRangeFromBundleName(bundle); - if (!shouldNamespacePoliciesUnload(namespaceName, bundleRange, broker)) { - return; - } + bundlesToUnload.asMap().forEach((broker, bundles) -> { + bundles.forEach(bundle -> { + final String namespaceName = LoadManagerShared.getNamespaceNameFromBundleName(bundle); + final String bundleRange = LoadManagerShared.getBundleRangeFromBundleName(bundle); + if (!shouldNamespacePoliciesUnload(namespaceName, bundleRange, broker)) { + return; + } - if (!shouldAntiAffinityNamespaceUnload(namespaceName, bundleRange, broker)) { - return; - } - NamespaceBundle bundleToUnload = LoadManagerShared.getNamespaceBundle(pulsar, bundle); - Optional destBroker = this.selectBroker(bundleToUnload); - if (!destBroker.isPresent()) { - log.info("[{}] No broker available to unload bundle {} from broker {}", - strategy.getClass().getSimpleName(), bundle, broker); - return; - } - if (destBroker.get().equals(broker)) { - log.warn("[{}] The destination broker {} is the same as the current owner broker for Bundle {}", - strategy.getClass().getSimpleName(), destBroker.get(), bundle); - return; - } + if (!shouldAntiAffinityNamespaceUnload(namespaceName, bundleRange, broker)) { + return; + } + NamespaceBundle bundleToUnload = LoadManagerShared.getNamespaceBundle(pulsar, bundle); + Optional destBroker = this.selectBroker(bundleToUnload); + if (!destBroker.isPresent()) { + log.info("[{}] No broker available to unload bundle {} from broker {}", + loadSheddingStrategy.getClass().getSimpleName(), bundle, broker); + return; + } + if (destBroker.get().equals(broker)) { + log.warn("[{}] The destination broker {} is the same as the current owner broker for Bundle {}", + loadSheddingStrategy.getClass().getSimpleName(), destBroker.get(), bundle); + return; + } - log.info("[{}] Unloading bundle: {} from broker {} to dest broker {}", - strategy.getClass().getSimpleName(), bundle, broker, destBroker.get()); - try { - pulsar.getAdminClient().namespaces() - .unloadNamespaceBundle(namespaceName, bundleRange, destBroker.get()); - loadData.getRecentlyUnloadedBundles().put(bundle, System.currentTimeMillis()); - } catch (PulsarServerException | PulsarAdminException e) { - log.warn("Error when trying to perform load shedding on {} for broker {}", bundle, broker, e); - } - }); + log.info("[{}] Unloading bundle: {} from broker {} to dest broker {}", + loadSheddingStrategy.getClass().getSimpleName(), bundle, broker, destBroker.get()); + try { + pulsar.getAdminClient().namespaces() + .unloadNamespaceBundle(namespaceName, bundleRange, destBroker.get()); + loadData.getRecentlyUnloadedBundles().put(bundle, System.currentTimeMillis()); + } catch (PulsarServerException | PulsarAdminException e) { + log.warn("Error when trying to perform load shedding on {} for broker {}", bundle, broker, e); + } }); + }); - updateBundleUnloadingMetrics(bundlesToUnload); - } + updateBundleUnloadingMetrics(bundlesToUnload); } /** From 82025b8ea104937f933b45ede188fd196fa212a0 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Tue, 28 May 2024 22:45:30 +0800 Subject: [PATCH 235/580] [improve][broker] avoid creating new objects when intercepting (#22790) --- .../BrokerInterceptorWithClassLoader.java | 127 ++++++++++++++---- .../intercept/BrokerInterceptorUtilsTest.java | 2 +- .../BrokerInterceptorWithClassLoaderTest.java | 2 +- 3 files changed, 105 insertions(+), 26 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java index faee5799289d0..3997e214f4316 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoader.java @@ -29,7 +29,6 @@ import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; -import org.apache.pulsar.broker.ClassLoaderSwitcher; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Producer; @@ -51,16 +50,20 @@ public class BrokerInterceptorWithClassLoader implements BrokerInterceptor { private final BrokerInterceptor interceptor; - private final NarClassLoader classLoader; + private final NarClassLoader narClassLoader; @Override public void beforeSendMessage(Subscription subscription, Entry entry, long[] ackSet, MessageMetadata msgMetadata) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.beforeSendMessage( subscription, entry, ackSet, msgMetadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -70,25 +73,37 @@ public void beforeSendMessage(Subscription subscription, long[] ackSet, MessageMetadata msgMetadata, Consumer consumer) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.beforeSendMessage( subscription, entry, ackSet, msgMetadata, consumer); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onMessagePublish(Producer producer, ByteBuf headersAndPayload, Topic.PublishContext publishContext) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onMessagePublish(producer, headersAndPayload, publishContext); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void producerCreated(ServerCnx cnx, Producer producer, Map metadata){ - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.producerCreated(cnx, producer, metadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -96,8 +111,12 @@ public void producerCreated(ServerCnx cnx, Producer producer, public void producerClosed(ServerCnx cnx, Producer producer, Map metadata) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.producerClosed(cnx, producer, metadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -105,9 +124,12 @@ public void producerClosed(ServerCnx cnx, public void consumerCreated(ServerCnx cnx, Consumer consumer, Map metadata) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { - this.interceptor.consumerCreated( - cnx, consumer, metadata); + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); + this.interceptor.consumerCreated(cnx, consumer, metadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -115,8 +137,12 @@ public void consumerCreated(ServerCnx cnx, public void consumerClosed(ServerCnx cnx, Consumer consumer, Map metadata) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.consumerClosed(cnx, consumer, metadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -124,87 +150,140 @@ public void consumerClosed(ServerCnx cnx, @Override public void messageProduced(ServerCnx cnx, Producer producer, long startTimeNs, long ledgerId, long entryId, Topic.PublishContext publishContext) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.messageProduced(cnx, producer, startTimeNs, ledgerId, entryId, publishContext); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void messageDispatched(ServerCnx cnx, Consumer consumer, long ledgerId, long entryId, ByteBuf headersAndPayload) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.messageDispatched(cnx, consumer, ledgerId, entryId, headersAndPayload); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void messageAcked(ServerCnx cnx, Consumer consumer, CommandAck ackCmd) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.messageAcked(cnx, consumer, ackCmd); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void txnOpened(long tcId, String txnID) { - this.interceptor.txnOpened(tcId, txnID); + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); + this.interceptor.txnOpened(tcId, txnID); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); + } } @Override public void txnEnded(String txnID, long txnAction) { - this.interceptor.txnEnded(txnID, txnAction); + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); + this.interceptor.txnEnded(txnID, txnAction); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); + } } @Override public void onConnectionCreated(ServerCnx cnx) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onConnectionCreated(cnx); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onPulsarCommand(BaseCommand command, ServerCnx cnx) throws InterceptException { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onPulsarCommand(command, cnx); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onConnectionClosed(ServerCnx cnx) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onConnectionClosed(cnx); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onWebserviceRequest(ServletRequest request) throws IOException, ServletException, InterceptException { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onWebserviceRequest(request); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onWebserviceResponse(ServletRequest request, ServletResponse response) throws IOException, ServletException { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onWebserviceResponse(request, response); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void initialize(PulsarService pulsarService) throws Exception { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.initialize(pulsarService); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void close() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); interceptor.close(); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } + try { - classLoader.close(); + narClassLoader.close(); } catch (IOException e) { log.warn("Failed to close the broker interceptor class loader", e); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtilsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtilsTest.java index 5abe8a69ee499..979bf6cd0d5db 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtilsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtilsTest.java @@ -65,7 +65,7 @@ public void testLoadBrokerEventListener() throws Exception { BrokerInterceptorWithClassLoader returnedPhWithCL = BrokerInterceptorUtils.load(metadata, ""); BrokerInterceptor returnedPh = returnedPhWithCL.getInterceptor(); - assertSame(mockLoader, returnedPhWithCL.getClassLoader()); + assertSame(mockLoader, returnedPhWithCL.getNarClassLoader()); assertTrue(returnedPh instanceof MockBrokerInterceptor); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoaderTest.java index a2f97e16a76ae..64d4b5ee6cca5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoaderTest.java @@ -135,7 +135,7 @@ public void close() { new BrokerInterceptorWithClassLoader(interceptor, narLoader); ClassLoader curClassLoader = Thread.currentThread().getContextClassLoader(); // test class loader - assertEquals(brokerInterceptorWithClassLoader.getClassLoader(), narLoader); + assertEquals(brokerInterceptorWithClassLoader.getNarClassLoader(), narLoader); // test initialize brokerInterceptorWithClassLoader.initialize(mock(PulsarService.class)); assertEquals(Thread.currentThread().getContextClassLoader(), curClassLoader); From 60b9f870083ee78e4482e4014b7917d191bd8a34 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 29 May 2024 22:19:47 +0800 Subject: [PATCH 236/580] [improve][broker] Remove ClassLoaderSwitcher to avoid objects allocations and consistent the codestyle (#22796) --- .../pulsar/broker/ClassLoaderSwitcher.java | 37 ---------------- .../AdditionalServletWithClassLoader.java | 25 ++++++++--- .../ProtocolHandlerWithClassLoader.java | 44 +++++++++++++++---- 3 files changed, 55 insertions(+), 51 deletions(-) delete mode 100644 pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ClassLoaderSwitcher.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ClassLoaderSwitcher.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ClassLoaderSwitcher.java deleted file mode 100644 index 55cb9198da2bc..0000000000000 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ClassLoaderSwitcher.java +++ /dev/null @@ -1,37 +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.broker; - -/** - * Help to switch the class loader of current thread to the NarClassLoader, and change it back when it's done. - * With the help of try-with-resources statement, the code would be cleaner than using try finally every time. - */ -public class ClassLoaderSwitcher implements AutoCloseable { - private final ClassLoader prevClassLoader; - - public ClassLoaderSwitcher(ClassLoader classLoader) { - prevClassLoader = Thread.currentThread().getContextClassLoader(); - Thread.currentThread().setContextClassLoader(classLoader); - } - - @Override - public void close() { - Thread.currentThread().setContextClassLoader(prevClassLoader); - } -} \ No newline at end of file diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java index c2b4b90073391..bc1f25c5af933 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java @@ -22,7 +22,6 @@ import lombok.Data; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.ClassLoaderSwitcher; import org.apache.pulsar.common.configuration.PulsarConfiguration; import org.apache.pulsar.common.nar.NarClassLoader; import org.eclipse.jetty.servlet.ServletHolder; @@ -40,29 +39,45 @@ public class AdditionalServletWithClassLoader implements AdditionalServlet { @Override public void loadConfig(PulsarConfiguration pulsarConfiguration) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); servlet.loadConfig(pulsarConfiguration); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public String getBasePath() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return servlet.getBasePath(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public ServletHolder getServletHolder() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return servlet.getServletHolder(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public void close() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); servlet.close(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } try { classLoader.close(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java index d648c261403d4..eb4bcb0a9bf4b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java @@ -26,7 +26,6 @@ import lombok.Data; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.ClassLoaderSwitcher; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.common.nar.NarClassLoader; @@ -44,52 +43,79 @@ class ProtocolHandlerWithClassLoader implements ProtocolHandler { @Override public String protocolName() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return handler.protocolName(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public boolean accept(String protocol) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return handler.accept(protocol); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public void initialize(ServiceConfiguration conf) throws Exception { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); handler.initialize(conf); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public String getProtocolDataToAdvertise() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return handler.getProtocolDataToAdvertise(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public void start(BrokerService service) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); handler.start(service); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public Map> newChannelInitializers() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return handler.newChannelInitializers(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public void close() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); handler.close(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } - try { classLoader.close(); } catch (IOException e) { From 5a7efd8f3acce625e411fbd3a404ddfd36a65d20 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 29 May 2024 17:27:00 +0200 Subject: [PATCH 237/580] [fix][broker] EntryFilters fix NoClassDefFoundError due to closed classloader (#22767) --- .../service/plugin/EntryFilterProvider.java | 3 +- .../plugin/EntryFilterWithClassLoader.java | 29 +++++++++++++++---- .../service/plugin/FilterEntryTest.java | 12 ++++---- .../broker/stats/ConsumerStatsTest.java | 2 +- .../broker/stats/SubscriptionStatsTest.java | 2 +- .../pulsar/common/nar/NarClassLoader.java | 16 ++++++++++ 6 files changed, 49 insertions(+), 15 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterProvider.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterProvider.java index f93e561542eeb..53418744b5486 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterProvider.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterProvider.java @@ -197,7 +197,8 @@ protected EntryFilter load(EntryFilterMetaData metadata) + " does not implement entry filter interface"); } EntryFilter pi = (EntryFilter) filter; - return new EntryFilterWithClassLoader(pi, ncl); + // the classloader is shared with the broker, the instance doesn't own it + return new EntryFilterWithClassLoader(pi, ncl, false); } catch (Throwable e) { if (e instanceof IOException) { throw (IOException) e; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterWithClassLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterWithClassLoader.java index c5c5721087788..aab46c62acdb4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterWithClassLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterWithClassLoader.java @@ -30,15 +30,23 @@ public class EntryFilterWithClassLoader implements EntryFilter { private final EntryFilter entryFilter; private final NarClassLoader classLoader; + private final boolean classLoaderOwned; - public EntryFilterWithClassLoader(EntryFilter entryFilter, NarClassLoader classLoader) { + public EntryFilterWithClassLoader(EntryFilter entryFilter, NarClassLoader classLoader, boolean classLoaderOwned) { this.entryFilter = entryFilter; this.classLoader = classLoader; + this.classLoaderOwned = classLoaderOwned; } @Override public FilterResult filterEntry(Entry entry, FilterContext context) { - return entryFilter.filterEntry(entry, context); + ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); + return entryFilter.filterEntry(entry, context); + } finally { + Thread.currentThread().setContextClassLoader(currentClassLoader); + } } @VisibleForTesting @@ -48,11 +56,20 @@ public EntryFilter getEntryFilter() { @Override public void close() { - entryFilter.close(); + ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader(); try { - classLoader.close(); - } catch (IOException e) { - log.error("close EntryFilterWithClassLoader failed", e); + Thread.currentThread().setContextClassLoader(classLoader); + entryFilter.close(); + } finally { + Thread.currentThread().setContextClassLoader(currentClassLoader); + } + if (classLoaderOwned) { + log.info("Closing classloader {} for EntryFilter {}", classLoader, entryFilter.getClass().getName()); + try { + classLoader.close(); + } catch (IOException e) { + log.error("close EntryFilterWithClassLoader failed", e); + } } } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java index 7b3daddcd9da0..f7388ef9eb990 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java @@ -239,9 +239,9 @@ public void testFilter() throws Exception { hasFilterField.setAccessible(true); NarClassLoader narClassLoader = mock(NarClassLoader.class); EntryFilter filter1 = new EntryFilterTest(); - EntryFilterWithClassLoader loader1 = spyWithClassAndConstructorArgsRecordingInvocations(EntryFilterWithClassLoader.class, filter1, narClassLoader); + EntryFilterWithClassLoader loader1 = spyWithClassAndConstructorArgsRecordingInvocations(EntryFilterWithClassLoader.class, filter1, narClassLoader, false); EntryFilter filter2 = new EntryFilter2Test(); - EntryFilterWithClassLoader loader2 = spyWithClassAndConstructorArgsRecordingInvocations(EntryFilterWithClassLoader.class, filter2, narClassLoader); + EntryFilterWithClassLoader loader2 = spyWithClassAndConstructorArgsRecordingInvocations(EntryFilterWithClassLoader.class, filter2, narClassLoader, false); field.set(dispatcher, List.of(loader1, loader2)); hasFilterField.set(dispatcher, true); @@ -371,9 +371,9 @@ public void testFilteredMsgCount(String topic) throws Throwable { hasFilterField.setAccessible(true); NarClassLoader narClassLoader = mock(NarClassLoader.class); EntryFilter filter1 = new EntryFilterTest(); - EntryFilterWithClassLoader loader1 = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader); + EntryFilterWithClassLoader loader1 = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader, false); EntryFilter filter2 = new EntryFilter2Test(); - EntryFilterWithClassLoader loader2 = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter2, narClassLoader); + EntryFilterWithClassLoader loader2 = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter2, narClassLoader, false); field.set(dispatcher, List.of(loader1, loader2)); hasFilterField.set(dispatcher, true); @@ -463,10 +463,10 @@ public void testEntryFilterRescheduleMessageDependingOnConsumerSharedSubscriptio NarClassLoader narClassLoader = mock(NarClassLoader.class); EntryFilter filter1 = new EntryFilterTest(); EntryFilterWithClassLoader loader1 = - spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader); + spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader, false); EntryFilter filter2 = new EntryFilterTest(); EntryFilterWithClassLoader loader2 = - spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter2, narClassLoader); + spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter2, narClassLoader, false); field.set(dispatcher, List.of(loader1, loader2)); hasFilterField.set(dispatcher, true); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index 024d8582fa213..5b2998216e8e1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -409,7 +409,7 @@ public void testAvgMessagesPerEntry() throws Exception { EntryFilter filter = new EntryFilterProducerTest(); EntryFilterWithClassLoader loader = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter, - narClassLoader); + narClassLoader, false); Pair> entryFilters = Pair.of("filter", List.of(loader)); PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService() diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java index 3e71d8f211101..bc4cb73e5b6fe 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java @@ -208,7 +208,7 @@ public void testSubscriptionStats(final String topic, final String subName, bool NarClassLoader narClassLoader = mock(NarClassLoader.class); EntryFilter filter1 = new EntryFilterTest(); EntryFilterWithClassLoader loader1 = - spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader); + spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader, false); field.set(dispatcher, List.of(loader1)); hasFilterField.set(dispatcher, true); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java index 9736d8b47ef71..44cfc2872ef6b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java @@ -40,6 +40,7 @@ import java.util.Comparator; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -135,6 +136,7 @@ public class NarClassLoader extends URLClassLoader { * The NAR for which this ClassLoader is responsible. */ private final File narWorkingDirectory; + private final AtomicBoolean closed = new AtomicBoolean(); private static final String TMP_DIR_PREFIX = "pulsar-nar"; @@ -292,4 +294,18 @@ protected String findLibrary(final String libname) { public String toString() { return NarClassLoader.class.getName() + "[" + narWorkingDirectory.getPath() + "]"; } + + @Override + protected Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + if (closed.get()) { + log.warn("Loading class {} from a closed classloader ({})", name, this); + } + return super.loadClass(name, resolve); + } + + @Override + public void close() throws IOException { + closed.set(true); + super.close(); + } } From 43b20c3cfda0ca2c70b970161c572bcc19f1745f Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 30 May 2024 15:12:55 +0800 Subject: [PATCH 238/580] [improve] [pip] PIP-356: Support Geo-Replication starts at earliest position (#22791) --- pip/pip-356.md | 71 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 71 insertions(+) create mode 100644 pip/pip-356.md diff --git a/pip/pip-356.md b/pip/pip-356.md new file mode 100644 index 0000000000000..c922e212ad748 --- /dev/null +++ b/pip/pip-356.md @@ -0,0 +1,71 @@ +# PIP-356: Support Geo-Replication starts at earliest position + +# Background knowledge + +Replication reads messages from the source cluster, and copies them to the remote cluster. +- Registers a cursor named `pulsar.repl.{remote-cluster}` on the source cluster. Replicator reads messages relies on this cursor. +- Registers a producer on the remote cluster. Replicator writes messages relies on this producer. + +# Motivation + +If you have some older messages to migrate, the steps recommended are below, which was described at [pulsar doc](https://pulsar.apache.org/docs/3.2.x/administration-geo/#migrate-data-between-clusters-using-geo-replication). +1. Create the cursor that the replicator will use manually: `pulsar-admin topics create-subscription -s pulsar.repl.{remote-cluster} -m earliest `. +2. Enable namespace-level/topic-level Geo-Replication. + +The steps recommended are difficultly to use, for example: +- Create cursor `pulsar.repl.{remote-cluster}` manually. +- The namespace/topic was unloaded due to a re-balance. + - The broker will remove the `pulsar.repl.{remote-cluster}` automatically because the Geo-Replication feature is disabled at this moment. +- Enable namespace-level/topic-level Geo-Replication, but the cursor that was created manually has been deleted, the broker will create a new one with latest position, which is not expected. + + +# Goals +Add an optional config(broker level, namespace level, and topic level) to support Geo-Replication starting at the earliest position. + +### Configuration + +**broker.conf** +```properties +# The position that replication task start at, it can be set to "earliest" or "latest (default)". +replicationStartAt=latest +``` + +**ServiceConfiguration** +```java +@FieldContext( + category = CATEGORY_REPLICATION, + dynamic = true, + doc = "The position that replication task start at, it can be set to earliest or latest (default)." +) +String replicationStartAt = "latest"; +``` + +### Public API + +**V2/Namespaces.java** +```java +@POST +@Path("/{tenant}/{namespace}/replicationStartAt") +public void setNamespaceLevelReplicationStartAt( + @PathParam("tenant") String tenant, + @PathParam("namespace") String namespace, + @QueryParam("replicationStartAt") String replicationStartAt) { + ... + ... +} +``` + +**V2/PersistentTopics.java** +```java +@POST +@Path("/{tenant}/{namespace}/{topic}/replicationStartAt") +public void setNamespaceLevelReplicationStartAt( + @PathParam("tenant") String tenant, + @PathParam("namespace") String namespace, + @PathParam("topic") @Encoded String encodedTopic, + @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal, + @QueryParam("replicationStartAt") String replicationStartAt) { + ... + ... +} +``` From 2b1630e9f0bcb9f9cc30c14ea552f1b7238e1eb9 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 30 May 2024 15:21:43 +0800 Subject: [PATCH 239/580] Revert "[improve] [pip] PIP-356: Support Geo-Replication starts at earliest position" (#22805) --- pip/pip-356.md | 71 -------------------------------------------------- 1 file changed, 71 deletions(-) delete mode 100644 pip/pip-356.md diff --git a/pip/pip-356.md b/pip/pip-356.md deleted file mode 100644 index c922e212ad748..0000000000000 --- a/pip/pip-356.md +++ /dev/null @@ -1,71 +0,0 @@ -# PIP-356: Support Geo-Replication starts at earliest position - -# Background knowledge - -Replication reads messages from the source cluster, and copies them to the remote cluster. -- Registers a cursor named `pulsar.repl.{remote-cluster}` on the source cluster. Replicator reads messages relies on this cursor. -- Registers a producer on the remote cluster. Replicator writes messages relies on this producer. - -# Motivation - -If you have some older messages to migrate, the steps recommended are below, which was described at [pulsar doc](https://pulsar.apache.org/docs/3.2.x/administration-geo/#migrate-data-between-clusters-using-geo-replication). -1. Create the cursor that the replicator will use manually: `pulsar-admin topics create-subscription -s pulsar.repl.{remote-cluster} -m earliest `. -2. Enable namespace-level/topic-level Geo-Replication. - -The steps recommended are difficultly to use, for example: -- Create cursor `pulsar.repl.{remote-cluster}` manually. -- The namespace/topic was unloaded due to a re-balance. - - The broker will remove the `pulsar.repl.{remote-cluster}` automatically because the Geo-Replication feature is disabled at this moment. -- Enable namespace-level/topic-level Geo-Replication, but the cursor that was created manually has been deleted, the broker will create a new one with latest position, which is not expected. - - -# Goals -Add an optional config(broker level, namespace level, and topic level) to support Geo-Replication starting at the earliest position. - -### Configuration - -**broker.conf** -```properties -# The position that replication task start at, it can be set to "earliest" or "latest (default)". -replicationStartAt=latest -``` - -**ServiceConfiguration** -```java -@FieldContext( - category = CATEGORY_REPLICATION, - dynamic = true, - doc = "The position that replication task start at, it can be set to earliest or latest (default)." -) -String replicationStartAt = "latest"; -``` - -### Public API - -**V2/Namespaces.java** -```java -@POST -@Path("/{tenant}/{namespace}/replicationStartAt") -public void setNamespaceLevelReplicationStartAt( - @PathParam("tenant") String tenant, - @PathParam("namespace") String namespace, - @QueryParam("replicationStartAt") String replicationStartAt) { - ... - ... -} -``` - -**V2/PersistentTopics.java** -```java -@POST -@Path("/{tenant}/{namespace}/{topic}/replicationStartAt") -public void setNamespaceLevelReplicationStartAt( - @PathParam("tenant") String tenant, - @PathParam("namespace") String namespace, - @PathParam("topic") @Encoded String encodedTopic, - @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal, - @QueryParam("replicationStartAt") String replicationStartAt) { - ... - ... -} -``` From 34898e365764f4dd2f1cfbcb9d7381b8e4f104e9 Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Thu, 30 May 2024 16:38:08 +0800 Subject: [PATCH 240/580] [fix][broker] fix can not cleanup heartbeat data if scaling down broker (#22750) Co-authored-by: fanjianye --- .../apache/pulsar/broker/PulsarService.java | 42 +++++++++++++++++++ .../pulsar/broker/admin/impl/BrokersBase.java | 12 ++++-- 2 files changed, 50 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 6482ead1f5a2d..722bfda426dd7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.pulsar.broker.admin.impl.BrokersBase.getHeartbeatTopicName; import static org.apache.pulsar.broker.resourcegroup.ResourceUsageTransportManager.DISABLE_RESOURCE_USAGE_TRANSPORT_MANAGER; import static org.apache.pulsar.common.naming.SystemTopicNames.isTransactionInternalName; import com.google.common.annotations.VisibleForTesting; @@ -72,6 +73,7 @@ import org.apache.bookkeeper.mledger.LedgerOffloader; import org.apache.bookkeeper.mledger.LedgerOffloaderFactory; import org.apache.bookkeeper.mledger.LedgerOffloaderStats; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; import org.apache.bookkeeper.mledger.offload.Offloaders; @@ -414,6 +416,41 @@ private void closeLeaderElectionService() throws Exception { } } + private boolean isManagedLedgerNotFoundException(Throwable e) { + Throwable realCause = e.getCause(); + return realCause instanceof ManagedLedgerException.MetadataNotFoundException + || realCause instanceof MetadataStoreException.NotFoundException; + } + + private void deleteHeartbeatResource() { + if (this.brokerService != null) { + LOG.info("forcefully delete heartbeat topic when close broker"); + + String heartbeatTopicNameV1 = getHeartbeatTopicName(getBrokerId(), getConfiguration(), false); + String heartbeatTopicNameV2 = getHeartbeatTopicName(getBrokerId(), getConfiguration(), true); + + try { + this.brokerService.deleteTopic(heartbeatTopicNameV1, true).get(); + } catch (Exception e) { + if (!isManagedLedgerNotFoundException(e)) { + LOG.error("Closed with errors in delete heartbeat topic [{}]", + heartbeatTopicNameV1, e); + } + } + + try { + this.brokerService.deleteTopic(heartbeatTopicNameV2, true).get(); + } catch (Exception e) { + if (!isManagedLedgerNotFoundException(e)) { + LOG.error("Closed with errors in delete heartbeat topic [{}]", + heartbeatTopicNameV2, e); + } + } + + LOG.info("finish forcefully delete heartbeat topic when close broker"); + } + } + @Override public void close() throws PulsarServerException { try { @@ -460,6 +497,11 @@ public CompletableFuture closeAsync() { // It only tells the Pulsar clients that this service is not ready to serve for the lookup requests state = State.Closing; + if (brokerId != null) { + // forcefully delete heartbeat topic when close broker + deleteHeartbeatResource(); + } + // close the service in reverse order v.s. in which they are started if (this.resourceUsageTransportManager != null) { try { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index 7eeea66db7164..9db17f76a8dbe 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -407,13 +407,17 @@ private void checkDeadlockedThreads() { } } + public static String getHeartbeatTopicName(String brokerId, ServiceConfiguration configuration, boolean isV2) { + NamespaceName namespaceName = isV2 + ? NamespaceService.getHeartbeatNamespaceV2(brokerId, configuration) + : NamespaceService.getHeartbeatNamespace(brokerId, configuration); + return String.format("persistent://%s/%s", namespaceName, HEALTH_CHECK_TOPIC_SUFFIX); + } private CompletableFuture internalRunHealthCheck(TopicVersion topicVersion) { String brokerId = pulsar().getBrokerId(); - NamespaceName namespaceName = (topicVersion == TopicVersion.V2) - ? NamespaceService.getHeartbeatNamespaceV2(brokerId, pulsar().getConfiguration()) - : NamespaceService.getHeartbeatNamespace(brokerId, pulsar().getConfiguration()); - final String topicName = String.format("persistent://%s/%s", namespaceName, HEALTH_CHECK_TOPIC_SUFFIX); + final String topicName = + getHeartbeatTopicName(brokerId, pulsar().getConfiguration(), (topicVersion == TopicVersion.V2)); LOG.info("[{}] Running healthCheck with topic={}", clientAppId(), topicName); final String messageStr = UUID.randomUUID().toString(); final String subscriptionName = "healthCheck-" + messageStr; From 87a33399873ff1e9723a6ca3812cbf914d8c8eef Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 30 May 2024 16:42:26 +0800 Subject: [PATCH 241/580] [improve] [client] improve the class GetTopicsResult (#22766) --- .../pulsar/client/impl/LookupServiceTest.java | 128 ++++++++++++++++++ .../client/impl/BinaryProtoLookupService.java | 14 +- .../pulsar/client/impl/HttpLookupService.java | 13 +- .../pulsar/common/lookup/GetTopicsResult.java | 106 +++++++++++++-- 4 files changed, 225 insertions(+), 36 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java new file mode 100644 index 0000000000000..59cb7ae03d0e3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java @@ -0,0 +1,128 @@ +/* + * 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.impl; + +import static org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace.Mode; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import java.util.Collection; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Test(groups = "broker-admin") +@Slf4j +public class LookupServiceTest extends ProducerConsumerBase { + + private PulsarClientImpl clientWithHttpLookup; + private PulsarClientImpl clientWitBinaryLookup; + + private boolean enableBrokerSideSubscriptionPatternEvaluation = true; + private int subscriptionPatternMaxLength = 10_000; + + @Override + @BeforeClass + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + clientWithHttpLookup = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar.getWebServiceAddress()).build(); + clientWitBinaryLookup = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); + } + + @Override + @AfterClass(alwaysRun = true) + protected void cleanup() throws Exception { + super.internalCleanup(); + if (clientWithHttpLookup != null) { + clientWithHttpLookup.close(); + } + if (clientWitBinaryLookup != null) { + clientWitBinaryLookup.close(); + } + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setEnableBrokerSideSubscriptionPatternEvaluation(enableBrokerSideSubscriptionPatternEvaluation); + conf.setSubscriptionPatternMaxLength(subscriptionPatternMaxLength); + } + + private LookupService getLookupService(boolean isUsingHttpLookup) { + if (isUsingHttpLookup) { + return clientWithHttpLookup.getLookup(); + } else { + return clientWitBinaryLookup.getLookup(); + } + } + + @DataProvider(name = "isUsingHttpLookup") + public Object[][] isUsingHttpLookup() { + return new Object[][]{ + {true}, + {false} + }; + } + + @Test(dataProvider = "isUsingHttpLookup") + public void testGetTopicsOfGetTopicsResult(boolean isUsingHttpLookup) throws Exception { + LookupService lookupService = getLookupService(isUsingHttpLookup); + String nonPartitionedTopic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(nonPartitionedTopic); + String partitionedTopic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createPartitionedTopic(partitionedTopic, 3); + String nonPersistentTopic = BrokerTestUtil.newUniqueName("non-persistent://public/default/tp"); + + // Verify the new method "GetTopicsResult.getTopics" works as expected. + Collection topics = lookupService.getTopicsUnderNamespace(NamespaceName.get("public/default"), + Mode.PERSISTENT, "public/default/.*", null).join().getTopics(); + assertTrue(topics.contains(nonPartitionedTopic)); + assertTrue(topics.contains(partitionedTopic)); + assertFalse(topics.contains(nonPersistentTopic)); + assertFalse(topics.contains(TopicName.get(partitionedTopic).getPartition(0).toString())); + // Verify the new method "GetTopicsResult.nonPartitionedOrPartitionTopics" works as expected. + Collection nonPartitionedOrPartitionTopics = + lookupService.getTopicsUnderNamespace(NamespaceName.get("public/default"), + Mode.PERSISTENT, "public/default/.*", null).join() + .getNonPartitionedOrPartitionTopics(); + assertTrue(nonPartitionedOrPartitionTopics.contains(nonPartitionedTopic)); + assertFalse(nonPartitionedOrPartitionTopics.contains(partitionedTopic)); + assertFalse(nonPartitionedOrPartitionTopics.contains(nonPersistentTopic)); + assertTrue(nonPartitionedOrPartitionTopics.contains(TopicName.get(partitionedTopic).getPartition(0) + .toString())); + assertTrue(nonPartitionedOrPartitionTopics.contains(TopicName.get(partitionedTopic).getPartition(1) + .toString())); + assertTrue(nonPartitionedOrPartitionTopics.contains(TopicName.get(partitionedTopic).getPartition(2) + .toString())); + + // Cleanup. + admin.topics().deletePartitionedTopic(partitionedTopic, false); + admin.topics().delete(nonPartitionedTopic, false); + } + +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 080a04100e904..b363d6e4366ad 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -23,8 +23,6 @@ import io.opentelemetry.api.common.Attributes; import java.net.InetSocketAddress; import java.net.URI; -import java.util.ArrayList; -import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -378,17 +376,7 @@ private void getTopicsUnderNamespace(InetSocketAddress socketAddress, log.debug("[namespace: {}] Success get topics list in request: {}", namespace, requestId); } - // do not keep partition part of topic name - List result = new ArrayList<>(); - r.getTopics().forEach(topic -> { - String filtered = TopicName.get(topic).getPartitionedTopicName(); - if (!result.contains(filtered)) { - result.add(filtered); - } - }); - - getTopicsResultFuture.complete(new GetTopicsResult(result, r.getTopicsHash(), - r.isFiltered(), r.isChanged())); + getTopicsResultFuture.complete(r); } client.getCnxPool().releaseConnection(clientCnx); }); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index 1e568cf6eebdd..44ef4ac17ee75 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -24,10 +24,7 @@ import java.net.URI; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Base64; -import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.commons.lang3.StringUtils; @@ -181,15 +178,7 @@ public CompletableFuture getTopicsUnderNamespace(NamespaceName httpClient .get(String.format(format, namespace, mode.toString()), String[].class) .thenAccept(topics -> { - List result = new ArrayList<>(); - // do not keep partition part of topic name - Arrays.asList(topics).forEach(topic -> { - String filtered = TopicName.get(topic).getPartitionedTopicName(); - if (!result.contains(filtered)) { - result.add(filtered); - } - }); - future.complete(new GetTopicsResult(result, topicsHash, false, true)); + future.complete(new GetTopicsResult(topics)); }).exceptionally(ex -> { Throwable cause = FutureUtil.unwrapCompletionException(ex); log.warn("Failed to getTopicsUnderNamespace namespace {} {}.", namespace, cause.getMessage()); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java b/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java index 55fe6253ff971..80f16e6c36717 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java @@ -18,21 +18,105 @@ */ package org.apache.pulsar.common.lookup; +import java.util.ArrayList; +import java.util.Arrays; import java.util.List; -import lombok.AllArgsConstructor; import lombok.Getter; -import lombok.NoArgsConstructor; -import lombok.Setter; import lombok.ToString; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespaceResponse; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.topics.TopicList; -@Getter -@Setter -@AllArgsConstructor -@NoArgsConstructor +/*** + * A value object. + * - The response of HTTP API "admin/v2/namespaces/{domain}/topics" is a topic(non-partitioned topic or partitions) + * array. It will be wrapped to "topics: {topic array}, topicsHash: null, filtered: false, changed: true". + * - The response of binary API {@link CommandGetTopicsOfNamespace} is a {@link CommandGetTopicsOfNamespaceResponse}, + * it will be transferred to a {@link GetTopicsResult}. + * See more details https://github.com/apache/pulsar/pull/14804. + */ @ToString public class GetTopicsResult { - private List topics; - private String topicsHash; - private boolean filtered; - private boolean changed; + + /** + * Non-partitioned topics, and topic partitions of partitioned topics. + */ + @Getter + private final List nonPartitionedOrPartitionTopics; + + /** + * The topics have been filtered by Broker using a regexp. Otherwise, the client should do a client-side filter. + * There are three cases that brokers will not filter the topics: + * 1. the lookup service is typed HTTP lookup service, the HTTP API has not implemented this feature yet. + * 2. the broker does not support this feature(in other words, its version is lower than "2.11.0"). + * 3. the input param "topicPattern" is too long than the broker config "subscriptionPatternMaxLength". + */ + @Getter + private final boolean filtered; + + /** + * The topics hash that was calculated by {@link TopicList#calculateHash(List)}. The param topics that will be used + * to calculate the hash code is only contains the topics that has been filtered. + * Note: It is always "null" if broker did not filter the topics when calling the API + * "LookupService.getTopicsUnderNamespace"(in other words, {@link #filtered} is false). + */ + @Getter + private final String topicsHash; + + /** + * The topics hash has changed after compare with the input param "topicsHash" when calling + * "LookupService.getTopicsUnderNamespace". + * Note: It is always set "true" if the input param "topicsHash" that used to call + * "LookupService.getTopicsUnderNamespace" is null or the "LookupService" is "HttpLookupService". + */ + @Getter + private final boolean changed; + + /** + * Partitioned topics and non-partitioned topics. + * In other words, there is no topic partitions of partitioned topics in this list. + * Note: it is not a field of the response of "LookupService.getTopicsUnderNamespace", it is generated in + * client-side memory. + */ + private volatile List topics; + + /** + * This constructor is used for binary API. + */ + public GetTopicsResult(List nonPartitionedOrPartitionTopics, String topicsHash, boolean filtered, + boolean changed) { + this.nonPartitionedOrPartitionTopics = nonPartitionedOrPartitionTopics; + this.topicsHash = topicsHash; + this.filtered = filtered; + this.changed = changed; + } + + /** + * This constructor is used for HTTP API. + */ + public GetTopicsResult(String[] nonPartitionedOrPartitionTopics) { + this(Arrays.asList(nonPartitionedOrPartitionTopics), null, false, true); + } + + public List getTopics() { + if (topics != null) { + return topics; + } + synchronized (this) { + if (topics != null) { + return topics; + } + // Group partitioned topics. + List grouped = new ArrayList<>(); + for (String topic : nonPartitionedOrPartitionTopics) { + String partitionedTopic = TopicName.get(topic).getPartitionedTopicName(); + if (!grouped.contains(partitionedTopic)) { + grouped.add(partitionedTopic); + } + } + topics = grouped; + return topics; + } + } } From 60ebab4ca6f8a45e72d1d61e29b433dde06da192 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Thu, 30 May 2024 20:35:17 +0800 Subject: [PATCH 242/580] [cleanup] [broker] remove DeviationShedder. (#22800) --- .../loadbalance/impl/DeviationShedder.java | 153 ------------------ 1 file changed, 153 deletions(-) delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/DeviationShedder.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/DeviationShedder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/DeviationShedder.java deleted file mode 100644 index fd90a728478f4..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/DeviationShedder.java +++ /dev/null @@ -1,153 +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.broker.loadbalance.impl; - -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.Multimap; -import java.util.Map; -import java.util.TreeSet; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.loadbalance.LoadData; -import org.apache.pulsar.broker.loadbalance.LoadSheddingStrategy; -import org.apache.pulsar.policies.data.loadbalancer.BrokerData; - -/** - * An abstract class which makes a LoadSheddingStrategy which makes decisions based on standard deviation easier to - * implement. Assuming there exists some real number metric which may estimate the load on a server, this load shedding - * strategy calculates the standard deviation with respect to that metric and sheds load on brokers whose standard - * deviation is above some threshold. - */ -public abstract class DeviationShedder implements LoadSheddingStrategy { - // A Set of pairs is used in favor of a Multimap for simplicity. - protected TreeSet> metricTreeSetCache; - protected TreeSet> bundleTreeSetCache; - - /** - * Initialize this DeviationShedder. - */ - public DeviationShedder() { - bundleTreeSetCache = new TreeSet<>(); - metricTreeSetCache = new TreeSet<>(); - } - - // Measure the load incurred by a bundle. - protected abstract double bundleValue(String bundle, BrokerData brokerData, ServiceConfiguration conf); - - // Measure the load suffered by a broker. - protected abstract double brokerValue(BrokerData brokerData, ServiceConfiguration conf); - - // Get the threshold above which the standard deviation of a broker is large - // enough to warrant unloading bundles. - protected abstract double getDeviationThreshold(ServiceConfiguration conf); - - /** - * Recommend that all of the returned bundles be unloaded based on observing excessive standard deviations according - * to some metric. - * - * @param loadData - * The load data to used to make the unloading decision. - * @param conf - * The service configuration. - * @return A map from all selected bundles to the brokers on which they reside. - */ - @Override - public Multimap findBundlesForUnloading(final LoadData loadData, final ServiceConfiguration conf) { - final Multimap result = ArrayListMultimap.create(); - bundleTreeSetCache.clear(); - metricTreeSetCache.clear(); - double sum = 0; - double squareSum = 0; - final Map brokerDataMap = loadData.getBrokerData(); - - // Treating each broker as a data point, calculate the sum and squared - // sum of the evaluated broker metrics. - // These may be used to calculate the standard deviation. - for (Map.Entry entry : brokerDataMap.entrySet()) { - final double value = brokerValue(entry.getValue(), conf); - sum += value; - squareSum += value * value; - metricTreeSetCache.add(new ImmutablePair<>(value, entry.getKey())); - } - // Mean cannot change by just moving around bundles. - final double mean = sum / brokerDataMap.size(); - double standardDeviation = Math.sqrt(squareSum / brokerDataMap.size() - mean * mean); - final double deviationThreshold = getDeviationThreshold(conf); - String lastMostOverloaded = null; - // While the most loaded broker is above the standard deviation - // threshold, continue to move bundles. - while ((metricTreeSetCache.last().getKey() - mean) / standardDeviation > deviationThreshold) { - final Pair mostLoadedPair = metricTreeSetCache.last(); - final double highestValue = mostLoadedPair.getKey(); - final String mostLoaded = mostLoadedPair.getValue(); - - final Pair leastLoadedPair = metricTreeSetCache.first(); - final double leastValue = leastLoadedPair.getKey(); - final String leastLoaded = metricTreeSetCache.first().getValue(); - - if (!mostLoaded.equals(lastMostOverloaded)) { - // Reset the bundle tree set now that a different broker is - // being considered. - bundleTreeSetCache.clear(); - for (String bundle : brokerDataMap.get(mostLoaded).getLocalData().getBundles()) { - if (!result.containsKey(bundle)) { - // Don't consider bundles that are already going to be - // moved. - bundleTreeSetCache.add( - new ImmutablePair<>(bundleValue(bundle, brokerDataMap.get(mostLoaded), conf), bundle)); - } - } - lastMostOverloaded = mostLoaded; - } - boolean selected = false; - while (!(bundleTreeSetCache.isEmpty() || selected)) { - Pair mostExpensivePair = bundleTreeSetCache.pollLast(); - double loadIncurred = mostExpensivePair.getKey(); - // When the bundle is moved, we want the now least loaded server - // to have lower overall load than the - // most loaded server does not. Thus, we will only consider - // moving the bundle if this condition - // holds, and otherwise we will try the next bundle. - if (loadIncurred + leastValue < highestValue) { - // Update the standard deviation and replace the old load - // values in the broker tree set with the - // load values assuming this move took place. - final String bundleToMove = mostExpensivePair.getValue(); - result.put(bundleToMove, mostLoaded); - metricTreeSetCache.remove(mostLoadedPair); - metricTreeSetCache.remove(leastLoadedPair); - final double newHighLoad = highestValue - loadIncurred; - final double newLowLoad = leastValue - loadIncurred; - squareSum -= highestValue * highestValue + leastValue * leastValue; - squareSum += newHighLoad * newHighLoad + newLowLoad * newLowLoad; - standardDeviation = Math.sqrt(squareSum / brokerDataMap.size() - mean * mean); - metricTreeSetCache.add(new ImmutablePair<>(newLowLoad, leastLoaded)); - metricTreeSetCache.add(new ImmutablePair<>(newHighLoad, mostLoaded)); - selected = true; - } - } - if (!selected) { - // Move on to the next broker if no bundle could be moved. - metricTreeSetCache.pollLast(); - } - } - return result; - } -} From 7ad157cb9357c1ff1e98ec4a4bb157be740b60b2 Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 31 May 2024 06:04:38 +0800 Subject: [PATCH 243/580] [improve] Upgrade to Oxia client 0.3.0 (#22807) --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 84b93647d0ec4..e458200757167 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -481,8 +481,8 @@ The Apache Software License, Version 2.0 * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia - - io.streamnative.oxia-oxia-client-api-0.2.0.jar - - io.streamnative.oxia-oxia-client-0.2.0.jar + - io.streamnative.oxia-oxia-client-api-0.3.0.jar + - io.streamnative.oxia-oxia-client-0.3.0.jar * OpenHFT - net.openhft-zero-allocation-hashing-0.16.jar * Java JSON WebTokens diff --git a/pom.xml b/pom.xml index 4af94ee984a3a..347ef9e83c2c6 100644 --- a/pom.xml +++ b/pom.xml @@ -252,7 +252,7 @@ flexible messaging model and an intuitive client API. 4.5.13 4.4.15 0.7.5 - 0.2.0 + 0.3.0 2.0 1.10.12 5.3.3 From c39f9f82b425c66c899f818583714c9c98d3e213 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 31 May 2024 03:25:52 +0300 Subject: [PATCH 244/580] [fix][ml] Fix race conditions in RangeCache (#22789) --- .../bookkeeper/mledger/impl/EntryImpl.java | 7 +- .../bookkeeper/mledger/util/RangeCache.java | 278 +++++++++++++----- .../mledger/util/RangeCacheTest.java | 63 ++-- 3 files changed, 254 insertions(+), 94 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java index 803979313575a..48a79a4ac529c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java @@ -27,9 +27,10 @@ import org.apache.bookkeeper.client.api.LedgerEntry; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.util.AbstractCASReferenceCounted; +import org.apache.bookkeeper.mledger.util.RangeCache; public final class EntryImpl extends AbstractCASReferenceCounted implements Entry, Comparable, - ReferenceCounted { + RangeCache.ValueWithKeyValidation { private static final Recycler RECYCLER = new Recycler() { @Override @@ -205,4 +206,8 @@ protected void deallocate() { recyclerHandle.recycle(this); } + @Override + public boolean matchesKey(PositionImpl key) { + return key.compareTo(ledgerId, entryId) == 0; + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java index d34857e5e5177..46d03bea1b5ad 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java @@ -19,31 +19,134 @@ package org.apache.bookkeeper.mledger.util; import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.base.Predicate; +import io.netty.util.IllegalReferenceCountException; +import io.netty.util.Recycler; +import io.netty.util.Recycler.Handle; import io.netty.util.ReferenceCounted; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicLong; +import org.apache.bookkeeper.mledger.util.RangeCache.ValueWithKeyValidation; import org.apache.commons.lang3.tuple.Pair; /** * Special type of cache where get() and delete() operations can be done over a range of keys. + * The implementation avoids locks and synchronization and relies on ConcurrentSkipListMap for storing the entries. + * Since there is no locks, there is a need to have a way to ensure that a single entry in the cache is removed + * exactly once. Removing an entry multiple times would result in the entries of the cache getting released too + * while they could still be in use. * * @param * Cache key. Needs to be Comparable * @param * Cache value */ -public class RangeCache, Value extends ReferenceCounted> { +public class RangeCache, Value extends ValueWithKeyValidation> { + public interface ValueWithKeyValidation extends ReferenceCounted { + boolean matchesKey(T key); + } + // Map from key to nodes inside the linked list - private final ConcurrentNavigableMap entries; + private final ConcurrentNavigableMap> entries; private AtomicLong size; // Total size of values stored in cache private final Weighter weighter; // Weighter object used to extract the size from values private final TimestampExtractor timestampExtractor; // Extract the timestamp associated with a value + /** + * Wrapper around the value to store in Map. This is needed to ensure that a specific instance can be removed from + * the map by calling the {@link Map#remove(Object, Object)} method. Certain race conditions could result in the + * wrong value being removed from the map. The instances of this class are recycled to avoid creating new objects. + */ + private static class IdentityWrapper { + private final Handle recyclerHandle; + private static final Recycler RECYCLER = new Recycler() { + @Override + protected IdentityWrapper newObject(Handle recyclerHandle) { + return new IdentityWrapper(recyclerHandle); + } + }; + private K key; + private V value; + + private IdentityWrapper(Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + static IdentityWrapper create(K key, V value) { + IdentityWrapper identityWrapper = RECYCLER.get(); + identityWrapper.key = key; + identityWrapper.value = value; + return identityWrapper; + } + + K getKey() { + return key; + } + + V getValue() { + return value; + } + + void recycle() { + value = null; + recyclerHandle.recycle(this); + } + + @Override + public boolean equals(Object o) { + // only match exact identity of the value + return this == o; + } + + @Override + public int hashCode() { + return Objects.hashCode(key); + } + } + + /** + * Mutable object to store the number of entries and the total size removed from the cache. The instances + * are recycled to avoid creating new instances. + */ + private static class RemovalCounters { + private final Handle recyclerHandle; + private static final Recycler RECYCLER = new Recycler() { + @Override + protected RemovalCounters newObject(Handle recyclerHandle) { + return new RemovalCounters(recyclerHandle); + } + }; + int removedEntries; + long removedSize; + private RemovalCounters(Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + static RemovalCounters create() { + RemovalCounters results = RECYCLER.get(); + results.removedEntries = 0; + results.removedSize = 0; + return results; + } + + void recycle() { + removedEntries = 0; + removedSize = 0; + recyclerHandle.recycle(this); + } + + public void entryRemoved(long size) { + removedSize += size; + removedEntries++; + } + } + /** * Construct a new RangeLruCache with default Weighter. */ @@ -68,18 +171,23 @@ public RangeCache(Weighter weighter, TimestampExtractor timestampE * Insert. * * @param key - * @param value - * ref counted value with at least 1 ref to pass on the cache + * @param value ref counted value with at least 1 ref to pass on the cache * @return whether the entry was inserted in the cache */ public boolean put(Key key, Value value) { // retain value so that it's not released before we put it in the cache and calculate the weight value.retain(); try { - if (entries.putIfAbsent(key, value) == null) { + if (!value.matchesKey(key)) { + throw new IllegalArgumentException("Value '" + value + "' does not match key '" + key + "'"); + } + IdentityWrapper newWrapper = IdentityWrapper.create(key, value); + if (entries.putIfAbsent(key, newWrapper) == null) { size.addAndGet(weighter.getSize(value)); return true; } else { + // recycle the new wrapper as it was not used + newWrapper.recycle(); return false; } } finally { @@ -91,16 +199,37 @@ public boolean exists(Key key) { return key != null ? entries.containsKey(key) : true; } + /** + * Get the value associated with the key and increment the reference count of it. + * The caller is responsible for releasing the reference. + */ public Value get(Key key) { - Value value = entries.get(key); - if (value == null) { + return getValue(key, entries.get(key)); + } + + private Value getValue(Key key, IdentityWrapper valueWrapper) { + if (valueWrapper == null) { return null; } else { + if (valueWrapper.getKey() != key) { + // the wrapper has been recycled and contains another key + return null; + } + Value value = valueWrapper.getValue(); try { value.retain(); + } catch (IllegalReferenceCountException e) { + // Value was already deallocated + return null; + } + // check that the value matches the key and that there's at least 2 references to it since + // the cache should be holding one reference and a new reference was just added in this method + if (value.refCnt() > 1 && value.matchesKey(key)) { return value; - } catch (Throwable t) { - // Value was already destroyed between get() and retain() + } else { + // Value or IdentityWrapper was recycled and already contains another value + // release the reference added in this method + value.release(); return null; } } @@ -118,12 +247,10 @@ public Collection getRange(Key first, Key last) { List values = new ArrayList(); // Return the values of the entries found in cache - for (Value value : entries.subMap(first, true, last, true).values()) { - try { - value.retain(); + for (Map.Entry> entry : entries.subMap(first, true, last, true).entrySet()) { + Value value = getValue(entry.getKey(), entry.getValue()); + if (value != null) { values.add(value); - } catch (Throwable t) { - // Value was already destroyed between get() and retain() } } @@ -138,25 +265,65 @@ public Collection getRange(Key first, Key last) { * @return an pair of ints, containing the number of removed entries and the total size */ public Pair removeRange(Key first, Key last, boolean lastInclusive) { - Map subMap = entries.subMap(first, true, last, lastInclusive); + RemovalCounters counters = RemovalCounters.create(); + Map> subMap = entries.subMap(first, true, last, lastInclusive); + for (Map.Entry> entry : subMap.entrySet()) { + removeEntry(entry, counters); + } + return handleRemovalResult(counters); + } - int removedEntries = 0; - long removedSize = 0; + enum RemoveEntryResult { + ENTRY_REMOVED, + CONTINUE_LOOP, + BREAK_LOOP; + } - for (Key key : subMap.keySet()) { - Value value = entries.remove(key); - if (value == null) { - continue; - } + private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters) { + return removeEntry(entry, counters, (x) -> true); + } - removedSize += weighter.getSize(value); + private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters, + Predicate removeCondition) { + Key key = entry.getKey(); + IdentityWrapper identityWrapper = entry.getValue(); + if (identityWrapper.getKey() != key) { + // the wrapper has been recycled and contains another key + return RemoveEntryResult.CONTINUE_LOOP; + } + Value value = identityWrapper.getValue(); + try { + // add extra retain to avoid value being released while we are removing it + value.retain(); + } catch (IllegalReferenceCountException e) { + // Value was already released + return RemoveEntryResult.CONTINUE_LOOP; + } + try { + if (!removeCondition.test(value)) { + return RemoveEntryResult.BREAK_LOOP; + } + // check that the value hasn't been recycled in between + // there should be at least 2 references since this method adds one and the cache should have one + // it is valid that the value contains references even after the key has been removed from the cache + if (value.refCnt() > 1 && value.matchesKey(key) && entries.remove(key, identityWrapper)) { + identityWrapper.recycle(); + counters.entryRemoved(weighter.getSize(value)); + // remove the cache reference + value.release(); + } + } finally { + // remove the extra retain value.release(); - ++removedEntries; } + return RemoveEntryResult.ENTRY_REMOVED; + } - size.addAndGet(-removedSize); - - return Pair.of(removedEntries, removedSize); + private Pair handleRemovalResult(RemovalCounters counters) { + size.addAndGet(-counters.removedSize); + Pair result = Pair.of(counters.removedEntries, counters.removedSize); + counters.recycle(); + return result; } /** @@ -166,24 +333,15 @@ public Pair removeRange(Key first, Key last, boolean lastInclusiv */ public Pair evictLeastAccessedEntries(long minSize) { checkArgument(minSize > 0); - - long removedSize = 0; - int removedEntries = 0; - - while (removedSize < minSize) { - Map.Entry entry = entries.pollFirstEntry(); + RemovalCounters counters = RemovalCounters.create(); + while (counters.removedSize < minSize) { + Map.Entry> entry = entries.firstEntry(); if (entry == null) { break; } - - Value value = entry.getValue(); - ++removedEntries; - removedSize += weighter.getSize(value); - value.release(); + removeEntry(entry, counters); } - - size.addAndGet(-removedSize); - return Pair.of(removedEntries, removedSize); + return handleRemovalResult(counters); } /** @@ -192,27 +350,18 @@ public Pair evictLeastAccessedEntries(long minSize) { * @return the tota */ public Pair evictLEntriesBeforeTimestamp(long maxTimestamp) { - long removedSize = 0; - int removedCount = 0; - + RemovalCounters counters = RemovalCounters.create(); while (true) { - Map.Entry entry = entries.firstEntry(); - if (entry == null || timestampExtractor.getTimestamp(entry.getValue()) > maxTimestamp) { + Map.Entry> entry = entries.firstEntry(); + if (entry == null) { break; } - Value value = entry.getValue(); - boolean removeHits = entries.remove(entry.getKey(), value); - if (!removeHits) { + if (removeEntry(entry, counters, value -> timestampExtractor.getTimestamp(value) <= maxTimestamp) + == RemoveEntryResult.BREAK_LOOP) { break; } - - removedSize += weighter.getSize(value); - removedCount++; - value.release(); } - - size.addAndGet(-removedSize); - return Pair.of(removedCount, removedSize); + return handleRemovalResult(counters); } /** @@ -231,23 +380,16 @@ public long getSize() { * * @return size of removed entries */ - public synchronized Pair clear() { - long removedSize = 0; - int removedCount = 0; - + public Pair clear() { + RemovalCounters counters = RemovalCounters.create(); while (true) { - Map.Entry entry = entries.pollFirstEntry(); + Map.Entry> entry = entries.firstEntry(); if (entry == null) { break; } - Value value = entry.getValue(); - removedSize += weighter.getSize(value); - removedCount++; - value.release(); + removeEntry(entry, counters); } - - size.getAndAdd(-removedSize); - return Pair.of(removedCount, removedSize); + return handleRemovalResult(counters); } /** diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java index 8ce0db4ac4caa..01b3c67bf1113 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java @@ -23,25 +23,30 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; - import com.google.common.collect.Lists; import io.netty.util.AbstractReferenceCounted; import io.netty.util.ReferenceCounted; -import org.apache.commons.lang3.tuple.Pair; -import org.testng.annotations.Test; -import java.util.UUID; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.commons.lang3.tuple.Pair; +import org.testng.annotations.Test; public class RangeCacheTest { - class RefString extends AbstractReferenceCounted implements ReferenceCounted { + class RefString extends AbstractReferenceCounted implements RangeCache.ValueWithKeyValidation { String s; + Integer matchingKey; RefString(String s) { + this(s, null); + } + + RefString(String s, Integer matchingKey) { super(); this.s = s; + this.matchingKey = matchingKey != null ? matchingKey : Integer.parseInt(s); setRefCnt(1); } @@ -65,6 +70,11 @@ public boolean equals(Object obj) { return false; } + + @Override + public boolean matchesKey(Integer key) { + return matchingKey.equals(key); + } } @Test @@ -119,8 +129,8 @@ public void simple() { public void customWeighter() { RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - cache.put(0, new RefString("zero")); - cache.put(1, new RefString("one")); + cache.put(0, new RefString("zero", 0)); + cache.put(1, new RefString("one", 1)); assertEquals(cache.getSize(), 7); assertEquals(cache.getNumberOfEntries(), 2); @@ -132,9 +142,9 @@ public void customTimeExtraction() { RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> x.s.length()); cache.put(1, new RefString("1")); - cache.put(2, new RefString("22")); - cache.put(3, new RefString("333")); - cache.put(4, new RefString("4444")); + cache.put(22, new RefString("22")); + cache.put(333, new RefString("333")); + cache.put(4444, new RefString("4444")); assertEquals(cache.getSize(), 10); assertEquals(cache.getNumberOfEntries(), 4); @@ -151,12 +161,12 @@ public void customTimeExtraction() { public void doubleInsert() { RangeCache cache = new RangeCache<>(); - RefString s0 = new RefString("zero"); + RefString s0 = new RefString("zero", 0); assertEquals(s0.refCnt(), 1); assertTrue(cache.put(0, s0)); assertEquals(s0.refCnt(), 1); - cache.put(1, new RefString("one")); + cache.put(1, new RefString("one", 1)); assertEquals(cache.getSize(), 2); assertEquals(cache.getNumberOfEntries(), 2); @@ -164,7 +174,7 @@ public void doubleInsert() { assertEquals(s.s, "one"); assertEquals(s.refCnt(), 2); - RefString s1 = new RefString("uno"); + RefString s1 = new RefString("uno", 1); assertEquals(s1.refCnt(), 1); assertFalse(cache.put(1, s1)); assertEquals(s1.refCnt(), 1); @@ -201,10 +211,10 @@ public void getRange() { public void eviction() { RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - cache.put(0, new RefString("zero")); - cache.put(1, new RefString("one")); - cache.put(2, new RefString("two")); - cache.put(3, new RefString("three")); + cache.put(0, new RefString("zero", 0)); + cache.put(1, new RefString("one", 1)); + cache.put(2, new RefString("two", 2)); + cache.put(3, new RefString("three", 3)); // This should remove the LRU entries: 0, 1 whose combined size is 7 assertEquals(cache.evictLeastAccessedEntries(5), Pair.of(2, (long) 7)); @@ -276,20 +286,23 @@ public void evictions() { } @Test - public void testInParallel() { - RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); - executor.scheduleWithFixedDelay(cache::clear, 10, 10, TimeUnit.MILLISECONDS); - for (int i = 0; i < 1000; i++) { - cache.put(UUID.randomUUID().toString(), new RefString("zero")); + public void testPutWhileClearIsCalledConcurrently() { + RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); + int numberOfThreads = 4; + @Cleanup("shutdownNow") + ScheduledExecutorService executor = Executors.newScheduledThreadPool(numberOfThreads); + for (int i = 0; i < numberOfThreads; i++) { + executor.scheduleWithFixedDelay(cache::clear, 0, 1, TimeUnit.MILLISECONDS); + } + for (int i = 0; i < 100000; i++) { + cache.put(i, new RefString(String.valueOf(i))); } - executor.shutdown(); } @Test public void testPutSameObj() { RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - RefString s0 = new RefString("zero"); + RefString s0 = new RefString("zero", 0); assertEquals(s0.refCnt(), 1); assertTrue(cache.put(0, s0)); assertFalse(cache.put(0, s0)); From e731674f61a973e9b12eab9394f82731c8fc2384 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 31 May 2024 23:14:50 +0300 Subject: [PATCH 245/580] [improve][ml] RangeCache refactoring: test race conditions and prevent endless loops (#22814) --- .../bookkeeper/mledger/util/RangeCache.java | 172 +++++++++++------- .../mledger/impl/EntryCacheManagerTest.java | 2 +- .../mledger/util/RangeCacheTest.java | 35 +++- 3 files changed, 143 insertions(+), 66 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java index 46d03bea1b5ad..45295d7190654 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java @@ -28,32 +28,36 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicLong; +import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.util.RangeCache.ValueWithKeyValidation; import org.apache.commons.lang3.tuple.Pair; /** * Special type of cache where get() and delete() operations can be done over a range of keys. - * The implementation avoids locks and synchronization and relies on ConcurrentSkipListMap for storing the entries. - * Since there is no locks, there is a need to have a way to ensure that a single entry in the cache is removed - * exactly once. Removing an entry multiple times would result in the entries of the cache getting released too - * while they could still be in use. + * The implementation avoids locks and synchronization by relying on ConcurrentSkipListMap for storing the entries. + * Since there are no locks, it's necessary to ensure that a single entry in the cache is removed exactly once. + * Removing an entry multiple times could result in the entries of the cache being released multiple times, + * even while they are still in use. This is prevented by using a custom wrapper around the value to store in the map + * that ensures that the value is removed from the map only if the exact same instance is present in the map. + * There's also a check that ensures that the value matches the key. This is used to detect races without impacting + * consistency. * * @param * Cache key. Needs to be Comparable * @param * Cache value */ +@Slf4j public class RangeCache, Value extends ValueWithKeyValidation> { public interface ValueWithKeyValidation extends ReferenceCounted { boolean matchesKey(T key); } // Map from key to nodes inside the linked list - private final ConcurrentNavigableMap> entries; + private final ConcurrentNavigableMap> entries; private AtomicLong size; // Total size of values stored in cache private final Weighter weighter; // Weighter object used to extract the size from values private final TimestampExtractor timestampExtractor; // Extract the timestamp associated with a value @@ -63,51 +67,53 @@ public interface ValueWithKeyValidation extends ReferenceCounted { * the map by calling the {@link Map#remove(Object, Object)} method. Certain race conditions could result in the * wrong value being removed from the map. The instances of this class are recycled to avoid creating new objects. */ - private static class IdentityWrapper { - private final Handle recyclerHandle; - private static final Recycler RECYCLER = new Recycler() { + private static class EntryWrapper { + private final Handle recyclerHandle; + private static final Recycler RECYCLER = new Recycler() { @Override - protected IdentityWrapper newObject(Handle recyclerHandle) { - return new IdentityWrapper(recyclerHandle); + protected EntryWrapper newObject(Handle recyclerHandle) { + return new EntryWrapper(recyclerHandle); } }; private K key; private V value; + long size; - private IdentityWrapper(Handle recyclerHandle) { + private EntryWrapper(Handle recyclerHandle) { this.recyclerHandle = recyclerHandle; } - static IdentityWrapper create(K key, V value) { - IdentityWrapper identityWrapper = RECYCLER.get(); - identityWrapper.key = key; - identityWrapper.value = value; - return identityWrapper; + static EntryWrapper create(K key, V value, long size) { + EntryWrapper entryWrapper = RECYCLER.get(); + synchronized (entryWrapper) { + entryWrapper.key = key; + entryWrapper.value = value; + entryWrapper.size = size; + } + return entryWrapper; } - K getKey() { + synchronized K getKey() { return key; } - V getValue() { + synchronized V getValue(K key) { + if (this.key != key) { + return null; + } return value; } + synchronized long getSize() { + return size; + } + void recycle() { + key = null; value = null; + size = 0; recyclerHandle.recycle(this); } - - @Override - public boolean equals(Object o) { - // only match exact identity of the value - return this == o; - } - - @Override - public int hashCode() { - return Objects.hashCode(key); - } } /** @@ -181,9 +187,10 @@ public boolean put(Key key, Value value) { if (!value.matchesKey(key)) { throw new IllegalArgumentException("Value '" + value + "' does not match key '" + key + "'"); } - IdentityWrapper newWrapper = IdentityWrapper.create(key, value); + long entrySize = weighter.getSize(value); + EntryWrapper newWrapper = EntryWrapper.create(key, value, entrySize); if (entries.putIfAbsent(key, newWrapper) == null) { - size.addAndGet(weighter.getSize(value)); + this.size.addAndGet(entrySize); return true; } else { // recycle the new wrapper as it was not used @@ -207,15 +214,15 @@ public Value get(Key key) { return getValue(key, entries.get(key)); } - private Value getValue(Key key, IdentityWrapper valueWrapper) { + private Value getValue(Key key, EntryWrapper valueWrapper) { if (valueWrapper == null) { return null; } else { - if (valueWrapper.getKey() != key) { + Value value = valueWrapper.getValue(key); + if (value == null) { // the wrapper has been recycled and contains another key return null; } - Value value = valueWrapper.getValue(); try { value.retain(); } catch (IllegalReferenceCountException e) { @@ -247,7 +254,7 @@ public Collection getRange(Key first, Key last) { List values = new ArrayList(); // Return the values of the entries found in cache - for (Map.Entry> entry : entries.subMap(first, true, last, true).entrySet()) { + for (Map.Entry> entry : entries.subMap(first, true, last, true).entrySet()) { Value value = getValue(entry.getKey(), entry.getValue()); if (value != null) { values.add(value); @@ -266,9 +273,9 @@ public Collection getRange(Key first, Key last) { */ public Pair removeRange(Key first, Key last, boolean lastInclusive) { RemovalCounters counters = RemovalCounters.create(); - Map> subMap = entries.subMap(first, true, last, lastInclusive); - for (Map.Entry> entry : subMap.entrySet()) { - removeEntry(entry, counters); + Map> subMap = entries.subMap(first, true, last, lastInclusive); + for (Map.Entry> entry : subMap.entrySet()) { + removeEntry(entry, counters, true); } return handleRemovalResult(counters); } @@ -279,36 +286,76 @@ enum RemoveEntryResult { BREAK_LOOP; } - private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters) { - return removeEntry(entry, counters, (x) -> true); + private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters, + boolean skipInvalid) { + return removeEntry(entry, counters, skipInvalid, x -> true); } - private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters, - Predicate removeCondition) { + private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters, + boolean skipInvalid, Predicate removeCondition) { Key key = entry.getKey(); - IdentityWrapper identityWrapper = entry.getValue(); - if (identityWrapper.getKey() != key) { - // the wrapper has been recycled and contains another key + EntryWrapper entryWrapper = entry.getValue(); + Value value = entryWrapper.getValue(key); + if (value == null) { + // the wrapper has already been recycled and contains another key + if (!skipInvalid) { + EntryWrapper removed = entries.remove(key); + if (removed != null) { + // log and remove the entry without releasing the value + log.info("Key {} does not match the entry's value wrapper's key {}, removed entry by key without " + + "releasing the value", key, entryWrapper.getKey()); + counters.entryRemoved(removed.getSize()); + return RemoveEntryResult.ENTRY_REMOVED; + } + } return RemoveEntryResult.CONTINUE_LOOP; } - Value value = identityWrapper.getValue(); try { // add extra retain to avoid value being released while we are removing it value.retain(); } catch (IllegalReferenceCountException e) { // Value was already released + if (!skipInvalid) { + // remove the specific entry without releasing the value + if (entries.remove(key, entryWrapper)) { + log.info("Value was already released for key {}, removed entry without releasing the value", key); + counters.entryRemoved(entryWrapper.getSize()); + return RemoveEntryResult.ENTRY_REMOVED; + } + } return RemoveEntryResult.CONTINUE_LOOP; } + if (!value.matchesKey(key)) { + // this is unexpected since the IdentityWrapper.getValue(key) already checked that the value matches the key + log.warn("Unexpected race condition. Value {} does not match the key {}. Removing entry.", value, key); + } try { if (!removeCondition.test(value)) { return RemoveEntryResult.BREAK_LOOP; } - // check that the value hasn't been recycled in between - // there should be at least 2 references since this method adds one and the cache should have one - // it is valid that the value contains references even after the key has been removed from the cache - if (value.refCnt() > 1 && value.matchesKey(key) && entries.remove(key, identityWrapper)) { - identityWrapper.recycle(); - counters.entryRemoved(weighter.getSize(value)); + if (!skipInvalid) { + // remove the specific entry + boolean entryRemoved = entries.remove(key, entryWrapper); + if (entryRemoved) { + counters.entryRemoved(entryWrapper.getSize()); + // check that the value hasn't been recycled in between + // there should be at least 2 references since this method adds one and the cache should have + // one reference. it is valid that the value contains references even after the key has been + // removed from the cache + if (value.refCnt() > 1) { + entryWrapper.recycle(); + // remove the cache reference + value.release(); + } else { + log.info("Unexpected refCnt {} for key {}, removed entry without releasing the value", + value.refCnt(), key); + } + } + } else if (skipInvalid && value.refCnt() > 1 && entries.remove(key, entryWrapper)) { + // when skipInvalid is true, we don't remove the entry if it doesn't match matches the key + // or the refCnt is invalid + counters.entryRemoved(entryWrapper.getSize()); + entryWrapper.recycle(); // remove the cache reference value.release(); } @@ -334,12 +381,12 @@ private Pair handleRemovalResult(RemovalCounters counters) { public Pair evictLeastAccessedEntries(long minSize) { checkArgument(minSize > 0); RemovalCounters counters = RemovalCounters.create(); - while (counters.removedSize < minSize) { - Map.Entry> entry = entries.firstEntry(); + while (counters.removedSize < minSize && !Thread.currentThread().isInterrupted()) { + Map.Entry> entry = entries.firstEntry(); if (entry == null) { break; } - removeEntry(entry, counters); + removeEntry(entry, counters, false); } return handleRemovalResult(counters); } @@ -351,12 +398,12 @@ public Pair evictLeastAccessedEntries(long minSize) { */ public Pair evictLEntriesBeforeTimestamp(long maxTimestamp) { RemovalCounters counters = RemovalCounters.create(); - while (true) { - Map.Entry> entry = entries.firstEntry(); + while (!Thread.currentThread().isInterrupted()) { + Map.Entry> entry = entries.firstEntry(); if (entry == null) { break; } - if (removeEntry(entry, counters, value -> timestampExtractor.getTimestamp(value) <= maxTimestamp) + if (removeEntry(entry, counters, false, value -> timestampExtractor.getTimestamp(value) <= maxTimestamp) == RemoveEntryResult.BREAK_LOOP) { break; } @@ -382,12 +429,12 @@ public long getSize() { */ public Pair clear() { RemovalCounters counters = RemovalCounters.create(); - while (true) { - Map.Entry> entry = entries.firstEntry(); + while (!Thread.currentThread().isInterrupted()) { + Map.Entry> entry = entries.firstEntry(); if (entry == null) { break; } - removeEntry(entry, counters); + removeEntry(entry, counters, false); } return handleRemovalResult(counters); } @@ -421,5 +468,4 @@ public long getSize(Value value) { return 1; } } - } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java index 1b02cd674c567..1ab3198498ac3 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java @@ -193,9 +193,9 @@ public void cacheSizeUpdate() throws Exception { } cacheManager.removeEntryCache(ml1.getName()); - assertTrue(cacheManager.getSize() > 0); assertEquals(factory2.getMbean().getCacheInsertedEntriesCount(), 20); assertEquals(factory2.getMbean().getCacheEntriesCount(), 0); + assertEquals(0, cacheManager.getSize()); assertEquals(factory2.getMbean().getCacheEvictedEntriesCount(), 20); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java index 01b3c67bf1113..4bcf2cc6c4e35 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/RangeCacheTest.java @@ -30,11 +30,14 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import lombok.Data; import org.apache.commons.lang3.tuple.Pair; +import org.awaitility.Awaitility; import org.testng.annotations.Test; public class RangeCacheTest { + @Data class RefString extends AbstractReferenceCounted implements RangeCache.ValueWithKeyValidation { String s; Integer matchingKey; @@ -288,15 +291,21 @@ public void evictions() { @Test public void testPutWhileClearIsCalledConcurrently() { RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - int numberOfThreads = 4; + int numberOfThreads = 8; @Cleanup("shutdownNow") ScheduledExecutorService executor = Executors.newScheduledThreadPool(numberOfThreads); for (int i = 0; i < numberOfThreads; i++) { executor.scheduleWithFixedDelay(cache::clear, 0, 1, TimeUnit.MILLISECONDS); } - for (int i = 0; i < 100000; i++) { + for (int i = 0; i < 200000; i++) { cache.put(i, new RefString(String.valueOf(i))); } + executor.shutdown(); + // ensure that no clear operation got into endless loop + Awaitility.await().untilAsserted(() -> assertTrue(executor.isTerminated())); + // ensure that clear can be called and all entries are removed + cache.clear(); + assertEquals(cache.getNumberOfEntries(), 0); } @Test @@ -307,4 +316,26 @@ public void testPutSameObj() { assertTrue(cache.put(0, s0)); assertFalse(cache.put(0, s0)); } + + @Test + public void testRemoveEntryWithInvalidRefCount() { + RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); + RefString value = new RefString("1"); + cache.put(1, value); + // release the value to make the reference count invalid + value.release(); + cache.clear(); + assertEquals(cache.getNumberOfEntries(), 0); + } + + @Test + public void testRemoveEntryWithInvalidMatchingKey() { + RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); + RefString value = new RefString("1"); + cache.put(1, value); + // change the matching key to make it invalid + value.setMatchingKey(123); + cache.clear(); + assertEquals(cache.getNumberOfEntries(), 0); + } } From 7b8f4a9159cf9e985b4d45f98c20f50674c701db Mon Sep 17 00:00:00 2001 From: Lishen Yao Date: Mon, 3 Jun 2024 11:07:18 +0900 Subject: [PATCH 246/580] [fix][ci] Fix snappy-java native lib fails to load in x86 alpine (#22804) --- docker/pulsar/Dockerfile | 14 ++++++++++++++ docker/pulsar/pom.xml | 1 + 2 files changed, 15 insertions(+) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 9d46dc97374b4..f3fea0e1e9d1e 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -48,6 +48,9 @@ RUN for SUBDIRECTORY in conf data download logs instances/deps packages-storage; RUN chmod -R g+rx /pulsar/bin RUN chmod -R o+rx /pulsar +# Enable snappy-java to use system lib +RUN echo 'OPTS="$OPTS -Dorg.xerial.snappy.use.systemlib=true"' >> /pulsar/conf/bkenv.sh + ### Create one stage to include JVM distribution FROM alpine AS jvm @@ -61,7 +64,16 @@ RUN /usr/lib/jvm/default-jvm/bin/jlink --add-modules ALL-MODULE-PATH --compress RUN echo networkaddress.cache.ttl=1 >> /opt/jvm/conf/security/java.security RUN echo networkaddress.cache.negative.ttl=1 >> /opt/jvm/conf/security/java.security +## Create one stage to include snappy-java native lib +# Fix the issue when using snappy-java in x86 arch alpine +# See https://github.com/xerial/snappy-java/issues/181 https://github.com/xerial/snappy-java/issues/579 +# We need to ensure that the version of the native library matches the version of snappy-java imported via Maven +FROM alpine AS snappy-java +ARG SNAPPY_VERSION +RUN apk add git alpine-sdk util-linux cmake autoconf automake libtool openjdk17 maven curl bash tar +ENV JAVA_HOME=/usr +RUN curl -Ls https://github.com/xerial/snappy-java/archive/refs/tags/v$SNAPPY_VERSION.tar.gz | tar zxf - && cd snappy-java-$SNAPPY_VERSION && make clean-native native FROM apachepulsar/glibc-base:2.38 as glibc ## Create final stage from Alpine image @@ -115,6 +127,8 @@ RUN apk add --allow-untrusted --force-overwrite /root/packages/glibc-*.apk COPY --from=jvm /opt/jvm /opt/jvm ENV JAVA_HOME=/opt/jvm +COPY --from=snappy-java /tmp/libsnappyjava.so /usr/lib/libsnappyjava.so + # The default is /pulsat/bin and cannot be written. ENV PULSAR_PID_DIR=/pulsar/logs diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index f9393ee343d93..0cf4535b19505 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -83,6 +83,7 @@ target/pulsar-server-distribution-${project.version}-bin.tar.gz ${pulsar.client.python.version} + ${snappy.version} ${project.basedir} From 6701936939fee170d3665dc46114ca8182a0a7a1 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Mon, 3 Jun 2024 10:14:25 +0800 Subject: [PATCH 247/580] [fix] [broker] Fix doc of ThresholdShedder and remove useless method. (#22798) --- .../pulsar/broker/loadbalance/impl/ThresholdShedder.java | 2 +- .../policies/data/loadbalancer/LocalBrokerData.java | 9 +-------- 2 files changed, 2 insertions(+), 9 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ThresholdShedder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ThresholdShedder.java index 882c72a71c904..ffa16c09e9b7a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ThresholdShedder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ThresholdShedder.java @@ -41,7 +41,7 @@ * configured threshold. As a consequence, this strategy tends to distribute load among all brokers. It does this by * first computing the average resource usage per broker for the whole cluster. The resource usage for each broker is * calculated using the following method: - * {@link LocalBrokerData#getMaxResourceUsageWithWeight(double, double, double, double, double)}. The weights + * {@link LocalBrokerData#getMaxResourceUsageWithWeight(double, double, double, double)}. The weights * for each resource are configurable. Historical observations are included in the running average based on the broker's * setting for loadBalancerHistoryResourcePercentage. Once the average resource usage is calculated, a broker's * current/historical usage is compared to the average broker usage. If a broker's usage is greater than the average diff --git a/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerData.java b/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerData.java index 8c27323694598..7fd0140bab22f 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerData.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerData.java @@ -253,14 +253,7 @@ public String printResourceUsage() { cpu.percentUsage(), memory.percentUsage(), directMemory.percentUsage(), bandwidthIn.percentUsage(), bandwidthOut.percentUsage()); } - @Deprecated - public double getMaxResourceUsageWithWeight(final double cpuWeight, final double memoryWeight, - final double directMemoryWeight, final double bandwidthInWeight, - final double bandwidthOutWeight) { - return max(cpu.percentUsage() * cpuWeight, memory.percentUsage() * memoryWeight, - directMemory.percentUsage() * directMemoryWeight, bandwidthIn.percentUsage() * bandwidthInWeight, - bandwidthOut.percentUsage() * bandwidthOutWeight) / 100; - } + public double getMaxResourceUsageWithWeight(final double cpuWeight, final double directMemoryWeight, final double bandwidthInWeight, final double bandwidthOutWeight) { From 2c3909c17b0c68a39f2f6f2a50c19216ef3a6ffc Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Mon, 3 Jun 2024 10:14:31 +0800 Subject: [PATCH 248/580] [fix] [broker] maintain last active info in memory only. (#22794) --- .../mledger/impl/ManagedCursorImpl.java | 2 - .../src/main/proto/MLDataFormats.proto | 3 +- .../service/persistent/PersistentTopic.java | 31 ++++++---- .../broker/service/BrokerServiceTest.java | 62 +++++++++++++++++++ 4 files changed, 82 insertions(+), 16 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 18d9cd7cb0568..1d2065ef8e392 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -475,9 +475,7 @@ void recover(final VoidCallback callback) { ledger.getStore().asyncGetCursorInfo(ledger.getName(), name, new MetaStoreCallback() { @Override public void operationComplete(ManagedCursorInfo info, Stat stat) { - updateCursorLedgerStat(info, stat); - lastActive = info.getLastActive() != 0 ? info.getLastActive() : lastActive; if (log.isDebugEnabled()) { log.debug("[{}] [{}] Recover cursor last active to [{}]", ledger.getName(), name, lastActive); diff --git a/managed-ledger/src/main/proto/MLDataFormats.proto b/managed-ledger/src/main/proto/MLDataFormats.proto index c4e502819fa9e..fdffed6762db7 100644 --- a/managed-ledger/src/main/proto/MLDataFormats.proto +++ b/managed-ledger/src/main/proto/MLDataFormats.proto @@ -124,7 +124,8 @@ message ManagedCursorInfo { // the current cursor position repeated LongProperty properties = 5; - optional int64 lastActive = 6; + // deprecated, do not persist this field anymore + optional int64 lastActive = 6 [deprecated = true]; // Store which index in the batch message has been deleted repeated BatchedEntryDeletionIndexInfo batchedEntryDeletionIndexInfo = 7; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 69c7f404fdd57..18e69250c16b8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -3227,19 +3227,7 @@ public void checkInactiveSubscriptions() { final Integer nsExpirationTime = policies.subscription_expiration_time_minutes; final long expirationTimeMillis = TimeUnit.MINUTES .toMillis(nsExpirationTime == null ? defaultExpirationTime : nsExpirationTime); - if (expirationTimeMillis > 0) { - subscriptions.forEach((subName, sub) -> { - if (sub.dispatcher != null && sub.dispatcher.isConsumerConnected() - || sub.isReplicated() - || isCompactionSubscription(subName)) { - return; - } - if (System.currentTimeMillis() - sub.cursor.getLastActive() > expirationTimeMillis) { - sub.delete().thenAccept(v -> log.info("[{}][{}] The subscription was deleted due to expiration " - + "with last active [{}]", topic, subName, sub.cursor.getLastActive())); - } - }); - } + checkInactiveSubscriptions(expirationTimeMillis); } catch (Exception e) { if (log.isDebugEnabled()) { log.debug("[{}] Error getting policies", topic); @@ -3247,6 +3235,23 @@ public void checkInactiveSubscriptions() { } } + @VisibleForTesting + public void checkInactiveSubscriptions(long expirationTimeMillis) { + if (expirationTimeMillis > 0) { + subscriptions.forEach((subName, sub) -> { + if (sub.dispatcher != null && sub.dispatcher.isConsumerConnected() + || sub.isReplicated() + || isCompactionSubscription(subName)) { + return; + } + if (System.currentTimeMillis() - sub.cursor.getLastActive() > expirationTimeMillis) { + sub.delete().thenAccept(v -> log.info("[{}][{}] The subscription was deleted due to expiration " + + "with last active [{}]", topic, subName, sub.cursor.getLastActive())); + } + }); + } + } + @Override public void checkBackloggedCursors() { subscriptions.forEach((subName, subscription) -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index be1221b7fab41..172842b5ed3bf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -22,6 +22,7 @@ import static org.apache.pulsar.common.naming.SystemTopicNames.TRANSACTION_COORDINATOR_LOG; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -1313,6 +1314,67 @@ public void testCheckInactiveSubscriptionsShouldNotDeleteCompactionCursor() thro } + @Test + public void testCheckInactiveSubscriptionWhenNoMessageToAck() throws Exception { + String namespace = "prop/testInactiveSubscriptionWhenNoMessageToAck"; + + try { + admin.namespaces().createNamespace(namespace); + } catch (PulsarAdminException.ConflictException e) { + // Ok.. (if test fails intermittently and namespace is already created) + } + + String topic = "persistent://" + namespace + "/my-topic"; + Producer producer = pulsarClient.newProducer().topic(topic).create(); + producer.send("test".getBytes()); + producer.close(); + + // create consumer to consume all messages + Consumer consumer = pulsarClient.newConsumer().topic(topic).subscriptionName("sub1") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe(); + consumer.acknowledge(consumer.receive()); + + Optional topicOptional = pulsar.getBrokerService().getTopic(topic, true).get(); + assertTrue(topicOptional.isPresent()); + PersistentTopic persistentTopic = (PersistentTopic) topicOptional.get(); + + // wait for 1s, but consumer is still connected all the time. + // so subscription should not be deleted. + Thread.sleep(1000); + persistentTopic.checkInactiveSubscriptions(1000); + PersistentTopic finalPersistentTopic = persistentTopic; + Awaitility.await().pollDelay(3, TimeUnit.SECONDS).until(() -> + finalPersistentTopic.getSubscriptions().containsKey("sub1")); + PersistentSubscription sub = persistentTopic.getSubscription("sub1"); + + // shutdown pulsar ungracefully + // disable the updateLastActive method to simulate the ungraceful shutdown + ManagedCursorImpl cursor = (ManagedCursorImpl) sub.getCursor(); + ManagedCursorImpl spyCursor = Mockito.spy(cursor); + doNothing().when(spyCursor).updateLastActive(); + Field cursorField = PersistentSubscription.class.getDeclaredField("cursor"); + cursorField.setAccessible(true); + cursorField.set(sub, spyCursor); + + // restart pulsar + consumer.close(); + restartBroker(); + + admin.lookups().lookupTopic(topic); + topicOptional = pulsar.getBrokerService().getTopic(topic, true).get(); + assertTrue(topicOptional.isPresent()); + persistentTopic = (PersistentTopic) topicOptional.get(); + persistentTopic.checkInactiveSubscriptions(1000); + + // check if subscription is still present + PersistentTopic finalPersistentTopic1 = persistentTopic; + Awaitility.await().pollDelay(3, TimeUnit.SECONDS).until(() -> + finalPersistentTopic1.getSubscriptions().containsKey("sub1")); + sub = persistentTopic.getSubscription("sub1"); + assertNotNull(sub); + } + + /** * Verifies brokerService should not have deadlock and successfully remove topic from topicMap on topic-failure and * it should not introduce deadlock while performing it. From 274a3cb66b80f516b13cb1c5dfe87619fac29f98 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 3 Jun 2024 10:17:27 +0800 Subject: [PATCH 249/580] [improve] [pip] PIP-356: Support Geo-Replication starts at earliest position (#22806) --- pip/pip-356.md | 113 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 113 insertions(+) create mode 100644 pip/pip-356.md diff --git a/pip/pip-356.md b/pip/pip-356.md new file mode 100644 index 0000000000000..c8cf96da58802 --- /dev/null +++ b/pip/pip-356.md @@ -0,0 +1,113 @@ +# PIP-356: Support Geo-Replication starts at earliest position + +# Background knowledge + +Replication reads messages from the source cluster, and copies them to the remote cluster. +- Registers a cursor named `pulsar.repl.{remote-cluster}` on the source cluster. Replicator reads messages relies on this cursor. +- Registers a producer on the remote cluster. Replicator writes messages relies on this producer. + +# Motivation + +If you have some older messages to migrate, the steps recommended are below, which was described at [pulsar doc](https://pulsar.apache.org/docs/3.2.x/administration-geo/#migrate-data-between-clusters-using-geo-replication). +1. Create the cursor that the replicator will use manually: `pulsar-admin topics create-subscription -s pulsar.repl.{remote-cluster} -m earliest `. +2. Enable namespace-level/topic-level Geo-Replication. + +The steps recommended are difficultly to use, for example: +- Create cursor `pulsar.repl.{remote-cluster}` manually. +- The namespace/topic was unloaded due to a re-balance. + - The broker will remove the `pulsar.repl.{remote-cluster}` automatically because the Geo-Replication feature is disabled at this moment. +- Enable namespace-level/topic-level Geo-Replication, but the cursor that was created manually has been deleted, the broker will create a new one with latest position, which is not expected. + + +# Goals +Add an optional config(broker level, namespace level, and topic level) to support Geo-Replication starting at the earliest position. + +### Configuration + +**broker.conf** +```properties +# The position that replication task start at, it can be set to "earliest" or "latest (default)". +replicationStartAt=latest +``` + +**ServiceConfiguration** +```java +@FieldContext( + category = CATEGORY_REPLICATION, + dynamic = true, + doc = "The position that replication task start at, it can be set to earliest or latest (default)." +) +String replicationStartAt = "latest"; +``` + +### Public API + +**V2/Namespaces.java** +```java +@POST +@Path("/{tenant}/{namespace}/replicationStartAt") +public void setNamespaceLevelReplicationStartAt( + @PathParam("tenant") String tenant, + @PathParam("namespace") String namespace, + @QueryParam("replicationStartAt") String replicationStartAt) { + ... + ... +} +``` + +**V2/PersistentTopics.java** +```java +@POST +@Path("/{tenant}/{namespace}/{topic}/replicationStartAt") +public void setNamespaceLevelReplicationStartAt( + @PathParam("tenant") String tenant, + @PathParam("namespace") String namespace, + @PathParam("topic") @Encoded String encodedTopic, + @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal, + @QueryParam("replicationStartAt") String replicationStartAt) { + ... + ... +} +``` + +### CLI + +**Namespaces command** +```shell +pulsar-admin namespaces set-replication-start-at {earliest|latest} +``` + +**Topics command** +```shell +pulsar-admin topics set-replication-start-at {earliest|latest} +``` + +### Binary protocol + +Nothing. + +### Metrics + +Nothing. + +# Monitoring + +Nothing. + +# Security Considerations + +Nothing. + +# Backward & Forward Compatibility + +You can do upgrading or reverting normally, no specified steps are needed to do. + +# Alternatives + +Nothing. + +# General Notes + +# Links +* Mailing List discussion thread: https://lists.apache.org/thread/8tp0rl05mjmqrxbp8m8nxx77d9x42chz +* Mailing List voting thread: https://lists.apache.org/thread/36jwdtdqspl4cq3m1cgz7xjk3gdpj45j From a6cee2b4f331a57429dfdbbfbec9777955855edb Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Mon, 3 Jun 2024 02:20:01 -0700 Subject: [PATCH 250/580] [feat][broker] PIP-264: Add schema registry metrics (#22624) --- .../apache/pulsar/broker/PulsarService.java | 2 +- .../service/schema/SchemaRegistryService.java | 6 +- .../schema/SchemaRegistryServiceImpl.java | 37 ++-- .../service/schema/SchemaRegistryStats.java | 198 ++++++++++++------ .../apache/pulsar/TestNGInstanceOrder.java | 38 ++++ .../service/schema/SchemaServiceTest.java | 118 ++++++++--- .../pulsar/client/api/SimpleSchemaTest.java | 3 + 7 files changed, 291 insertions(+), 111 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/TestNGInstanceOrder.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 722bfda426dd7..2e9f9dc6b0105 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -869,7 +869,7 @@ public void start() throws PulsarServerException { schemaStorage = createAndStartSchemaStorage(); schemaRegistryService = SchemaRegistryService.create( - schemaStorage, config.getSchemaRegistryCompatibilityCheckers(), this.executor); + schemaStorage, config.getSchemaRegistryCompatibilityCheckers(), this); OffloadPoliciesImpl defaultOffloadPolicies = OffloadPoliciesImpl.create(this.getConfiguration().getProperties()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryService.java index 3c5e3aae7ff5d..2a2467d3947ee 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryService.java @@ -21,7 +21,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; -import java.util.concurrent.ScheduledExecutorService; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.schema.validator.SchemaRegistryServiceWithSchemaDataValidator; import org.apache.pulsar.common.protocol.schema.SchemaStorage; import org.apache.pulsar.common.schema.SchemaType; @@ -44,13 +44,13 @@ static Map getCheckers(Set checker } static SchemaRegistryService create(SchemaStorage schemaStorage, Set schemaRegistryCompatibilityCheckers, - ScheduledExecutorService scheduler) { + PulsarService pulsarService) { if (schemaStorage != null) { try { Map checkers = getCheckers(schemaRegistryCompatibilityCheckers); checkers.put(SchemaType.KEY_VALUE, new KeyValueSchemaCompatibilityCheck(checkers)); return SchemaRegistryServiceWithSchemaDataValidator.of( - new SchemaRegistryServiceImpl(schemaStorage, checkers, scheduler)); + new SchemaRegistryServiceImpl(schemaStorage, checkers, pulsarService)); } catch (Exception e) { LOG.warn("Unable to create schema registry storage, defaulting to empty storage", e); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java index 903f57cb7803a..3e9e13b14fe46 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java @@ -38,7 +38,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; import javax.validation.constraints.NotNull; import lombok.extern.slf4j.Slf4j; @@ -47,6 +46,7 @@ import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.mutable.MutableLong; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; import org.apache.pulsar.broker.service.schema.exceptions.SchemaException; import org.apache.pulsar.broker.service.schema.proto.SchemaRegistryFormat; @@ -70,19 +70,19 @@ public class SchemaRegistryServiceImpl implements SchemaRegistryService { @VisibleForTesting SchemaRegistryServiceImpl(SchemaStorage schemaStorage, - Map compatibilityChecks, Clock clock, - ScheduledExecutorService scheduler) { + Map compatibilityChecks, + Clock clock, + PulsarService pulsarService) { this.schemaStorage = schemaStorage; this.compatibilityChecks = compatibilityChecks; this.clock = clock; - this.stats = SchemaRegistryStats.getInstance(scheduler); + this.stats = new SchemaRegistryStats(pulsarService); } - @VisibleForTesting SchemaRegistryServiceImpl(SchemaStorage schemaStorage, Map compatibilityChecks, - ScheduledExecutorService scheduler) { - this(schemaStorage, compatibilityChecks, Clock.systemUTC(), scheduler); + PulsarService pulsarService) { + this(schemaStorage, compatibilityChecks, Clock.systemUTC(), pulsarService); } @Override @@ -136,16 +136,17 @@ public CompletableFuture getSchema(String schemaId, SchemaVer } }) .whenComplete((v, t) -> { + var latencyMs = this.clock.millis() - start; if (t != null) { if (log.isDebugEnabled()) { log.debug("[{}] Get schema failed", schemaId); } - this.stats.recordGetFailed(schemaId); + this.stats.recordGetFailed(schemaId, latencyMs); } else { if (log.isDebugEnabled()) { log.debug(null == v ? "[{}] Schema not found" : "[{}] Schema is present", schemaId); } - this.stats.recordGetLatency(schemaId, this.clock.millis() - start); + this.stats.recordGetLatency(schemaId, latencyMs); } }); } @@ -157,10 +158,11 @@ public CompletableFuture>> getAllSchem return schemaStorage.getAll(schemaId) .thenCompose(schemas -> convertToSchemaAndMetadata(schemaId, schemas)) .whenComplete((v, t) -> { + var latencyMs = this.clock.millis() - start; if (t != null) { - this.stats.recordGetFailed(schemaId); + this.stats.recordListFailed(schemaId, latencyMs); } else { - this.stats.recordGetLatency(schemaId, this.clock.millis() - start); + this.stats.recordListLatency(schemaId, latencyMs); } }); } @@ -228,10 +230,11 @@ public CompletableFuture putSchemaIfAbsent(String schemaId, Schem return CompletableFuture.completedFuture(Pair.of(info.toByteArray(), context)); }); }))).whenComplete((v, ex) -> { + var latencyMs = this.clock.millis() - start.getValue(); if (ex != null) { log.error("[{}] Put schema failed", schemaId, ex); if (start.getValue() != 0) { - this.stats.recordPutFailed(schemaId); + this.stats.recordPutFailed(schemaId, latencyMs); } promise.completeExceptionally(ex); } else { @@ -261,14 +264,15 @@ public CompletableFuture deleteSchema(String schemaId, String use return schemaStorage .put(schemaId, deletedEntry, new byte[]{}) .whenComplete((v, t) -> { + var latencyMs = this.clock.millis() - start; if (t != null) { log.error("[{}] User {} delete schema failed", schemaId, user); - this.stats.recordDelFailed(schemaId); + this.stats.recordDelFailed(schemaId, latencyMs); } else { if (log.isDebugEnabled()) { log.debug("[{}] User {} delete schema finished", schemaId, user); } - this.stats.recordDelLatency(schemaId, this.clock.millis() - start); + this.stats.recordDelLatency(schemaId, latencyMs); } }); } @@ -284,11 +288,12 @@ public CompletableFuture deleteSchemaStorage(String schemaId, boo return schemaStorage.delete(schemaId, forcefully) .whenComplete((v, t) -> { + var latencyMs = this.clock.millis() - start; if (t != null) { - this.stats.recordDelFailed(schemaId); + this.stats.recordDelFailed(schemaId, latencyMs); log.error("[{}] Delete schema storage failed", schemaId); } else { - this.stats.recordDelLatency(schemaId, this.clock.millis() - start); + this.stats.recordDelLatency(schemaId, latencyMs); if (log.isDebugEnabled()) { log.debug("[{}] Delete schema storage finished", schemaId); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryStats.java index 32e9e36853026..b1a7dc2a54133 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryStats.java @@ -18,69 +18,111 @@ */ package org.apache.pulsar.broker.service.schema; -import io.prometheus.client.CollectorRegistry; +import com.google.common.annotations.VisibleForTesting; +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.DoubleHistogram; +import io.opentelemetry.api.metrics.LongCounter; import io.prometheus.client.Counter; import io.prometheus.client.Summary; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.stats.MetricsUtil; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; +import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric; class SchemaRegistryStats implements AutoCloseable, Runnable { private static final String NAMESPACE = "namespace"; private static final double[] QUANTILES = {0.50, 0.75, 0.95, 0.99, 0.999, 0.9999, 1}; - private static final AtomicBoolean CLOSED = new AtomicBoolean(false); - private final Counter getOpsFailedCounter; - private final Counter putOpsFailedCounter; - private final Counter deleteOpsFailedCounter; + public static final AttributeKey REQUEST_TYPE_KEY = + AttributeKey.stringKey("pulsar.schema_registry.request"); + @VisibleForTesting + enum RequestType { + GET, + LIST, + PUT, + DELETE; - private final Counter compatibleCounter; - private final Counter incompatibleCounter; - - private final Summary deleteOpsLatency; - private final Summary getOpsLatency; - private final Summary putOpsLatency; + public final Attributes attributes = Attributes.of(REQUEST_TYPE_KEY, name().toLowerCase()); + } - private final Map namespaceAccess = new ConcurrentHashMap<>(); - private ScheduledFuture future; + public static final AttributeKey RESPONSE_TYPE_KEY = + AttributeKey.stringKey("pulsar.schema_registry.response"); + @VisibleForTesting + enum ResponseType { + SUCCESS, + FAILURE; - private static volatile SchemaRegistryStats instance; + public final Attributes attributes = Attributes.of(RESPONSE_TYPE_KEY, name().toLowerCase()); + } - static synchronized SchemaRegistryStats getInstance(ScheduledExecutorService scheduler) { - if (null == instance) { - instance = new SchemaRegistryStats(scheduler); - } + public static final AttributeKey COMPATIBILITY_CHECK_RESPONSE_KEY = + AttributeKey.stringKey("pulsar.schema_registry.compatibility_check.response"); + @VisibleForTesting + enum CompatibilityCheckResponse { + COMPATIBLE, + INCOMPATIBLE; - return instance; + public final Attributes attributes = Attributes.of(COMPATIBILITY_CHECK_RESPONSE_KEY, name().toLowerCase()); } - private SchemaRegistryStats(ScheduledExecutorService scheduler) { - this.deleteOpsFailedCounter = Counter.build("pulsar_schema_del_ops_failed_total", "-") - .labelNames(NAMESPACE).create().register(); - this.getOpsFailedCounter = Counter.build("pulsar_schema_get_ops_failed_total", "-") - .labelNames(NAMESPACE).create().register(); - this.putOpsFailedCounter = Counter.build("pulsar_schema_put_ops_failed_total", "-") - .labelNames(NAMESPACE).create().register(); + public static final String SCHEMA_REGISTRY_REQUEST_DURATION_METRIC_NAME = + "pulsar.broker.request.schema_registry.duration"; + private final DoubleHistogram latencyHistogram; - this.compatibleCounter = Counter.build("pulsar_schema_compatible_total", "-") - .labelNames(NAMESPACE).create().register(); - this.incompatibleCounter = Counter.build("pulsar_schema_incompatible_total", "-") - .labelNames(NAMESPACE).create().register(); + public static final String COMPATIBLE_COUNTER_METRIC_NAME = + "pulsar.broker.operation.schema_registry.compatibility_check.count"; + private final LongCounter schemaCompatibilityCounter; - this.deleteOpsLatency = this.buildSummary("pulsar_schema_del_ops_latency", "-"); - this.getOpsLatency = this.buildSummary("pulsar_schema_get_ops_latency", "-"); - this.putOpsLatency = this.buildSummary("pulsar_schema_put_ops_latency", "-"); + @PulsarDeprecatedMetric(newMetricName = SCHEMA_REGISTRY_REQUEST_DURATION_METRIC_NAME) + private static final Counter getOpsFailedCounter = + Counter.build("pulsar_schema_get_ops_failed_total", "-").labelNames(NAMESPACE).create().register(); + @PulsarDeprecatedMetric(newMetricName = SCHEMA_REGISTRY_REQUEST_DURATION_METRIC_NAME) + private static final Counter putOpsFailedCounter = + Counter.build("pulsar_schema_put_ops_failed_total", "-").labelNames(NAMESPACE).create().register(); + @PulsarDeprecatedMetric(newMetricName = SCHEMA_REGISTRY_REQUEST_DURATION_METRIC_NAME) + private static final Counter deleteOpsFailedCounter = + Counter.build("pulsar_schema_del_ops_failed_total", "-").labelNames(NAMESPACE).create().register(); - if (null != scheduler) { - this.future = scheduler.scheduleAtFixedRate(this, 1, 1, TimeUnit.MINUTES); - } + @PulsarDeprecatedMetric(newMetricName = COMPATIBLE_COUNTER_METRIC_NAME) + private static final Counter compatibleCounter = + Counter.build("pulsar_schema_compatible_total", "-").labelNames(NAMESPACE).create().register(); + @PulsarDeprecatedMetric(newMetricName = COMPATIBLE_COUNTER_METRIC_NAME) + private static final Counter incompatibleCounter = + Counter.build("pulsar_schema_incompatible_total", "-").labelNames(NAMESPACE).create().register(); + + @PulsarDeprecatedMetric(newMetricName = SCHEMA_REGISTRY_REQUEST_DURATION_METRIC_NAME) + private static final Summary deleteOpsLatency = buildSummary("pulsar_schema_del_ops_latency", "-"); + + @PulsarDeprecatedMetric(newMetricName = SCHEMA_REGISTRY_REQUEST_DURATION_METRIC_NAME) + private static final Summary getOpsLatency = buildSummary("pulsar_schema_get_ops_latency", "-"); + + @PulsarDeprecatedMetric(newMetricName = SCHEMA_REGISTRY_REQUEST_DURATION_METRIC_NAME) + private static final Summary putOpsLatency = buildSummary("pulsar_schema_put_ops_latency", "-"); + + private final Map namespaceAccess = new ConcurrentHashMap<>(); + private final ScheduledFuture future; + + public SchemaRegistryStats(PulsarService pulsarService) { + this.future = pulsarService.getExecutor().scheduleAtFixedRate(this, 1, 1, TimeUnit.MINUTES); + + var meter = pulsarService.getOpenTelemetry().getMeter(); + latencyHistogram = meter.histogramBuilder(SCHEMA_REGISTRY_REQUEST_DURATION_METRIC_NAME) + .setDescription("The duration of Schema Registry requests.") + .setUnit("s") + .build(); + schemaCompatibilityCounter = meter.counterBuilder(COMPATIBLE_COUNTER_METRIC_NAME) + .setDescription("The number of Schema Registry compatibility check operations performed by the broker.") + .setUnit("{operation}") + .build(); } - private Summary buildSummary(String name, String help) { + private static Summary buildSummary(String name, String help) { Summary.Builder builder = Summary.build(name, help).labelNames(NAMESPACE); for (double quantile : QUANTILES) { @@ -90,38 +132,77 @@ private Summary buildSummary(String name, String help) { return builder.create().register(); } - void recordDelFailed(String schemaId) { - this.deleteOpsFailedCounter.labels(getNamespace(schemaId)).inc(); + void recordDelFailed(String schemaId, long millis) { + deleteOpsFailedCounter.labels(getNamespace(schemaId)).inc(); + recordOperationLatency(schemaId, millis, RequestType.DELETE, ResponseType.FAILURE); + } + + void recordGetFailed(String schemaId, long millis) { + getOpsFailedCounter.labels(getNamespace(schemaId)).inc(); + recordOperationLatency(schemaId, millis, RequestType.GET, ResponseType.FAILURE); } - void recordGetFailed(String schemaId) { - this.getOpsFailedCounter.labels(getNamespace(schemaId)).inc(); + void recordListFailed(String schemaId, long millis) { + getOpsFailedCounter.labels(getNamespace(schemaId)).inc(); + recordOperationLatency(schemaId, millis, RequestType.LIST, ResponseType.FAILURE); } - void recordPutFailed(String schemaId) { - this.putOpsFailedCounter.labels(getNamespace(schemaId)).inc(); + void recordPutFailed(String schemaId, long millis) { + putOpsFailedCounter.labels(getNamespace(schemaId)).inc(); + recordOperationLatency(schemaId, millis, RequestType.PUT, ResponseType.FAILURE); } void recordDelLatency(String schemaId, long millis) { - this.deleteOpsLatency.labels(getNamespace(schemaId)).observe(millis); + deleteOpsLatency.labels(getNamespace(schemaId)).observe(millis); + recordOperationLatency(schemaId, millis, RequestType.DELETE, ResponseType.SUCCESS); } void recordGetLatency(String schemaId, long millis) { - this.getOpsLatency.labels(getNamespace(schemaId)).observe(millis); + getOpsLatency.labels(getNamespace(schemaId)).observe(millis); + recordOperationLatency(schemaId, millis, RequestType.GET, ResponseType.SUCCESS); + } + + void recordListLatency(String schemaId, long millis) { + getOpsLatency.labels(getNamespace(schemaId)).observe(millis); + recordOperationLatency(schemaId, millis, RequestType.LIST, ResponseType.SUCCESS); } void recordPutLatency(String schemaId, long millis) { - this.putOpsLatency.labels(getNamespace(schemaId)).observe(millis); + putOpsLatency.labels(getNamespace(schemaId)).observe(millis); + recordOperationLatency(schemaId, millis, RequestType.PUT, ResponseType.SUCCESS); + } + + private void recordOperationLatency(String schemaId, long millis, + RequestType requestType, ResponseType responseType) { + var duration = MetricsUtil.convertToSeconds(millis, TimeUnit.MILLISECONDS); + var namespace = getNamespace(schemaId); + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, namespace) + .putAll(requestType.attributes) + .putAll(responseType.attributes) + .build(); + latencyHistogram.record(duration, attributes); } void recordSchemaIncompatible(String schemaId) { - this.incompatibleCounter.labels(getNamespace(schemaId)).inc(); + var namespace = getNamespace(schemaId); + incompatibleCounter.labels(namespace).inc(); + recordSchemaCompabilityResult(namespace, CompatibilityCheckResponse.INCOMPATIBLE); } void recordSchemaCompatible(String schemaId) { - this.compatibleCounter.labels(getNamespace(schemaId)).inc(); + var namespace = getNamespace(schemaId); + compatibleCounter.labels(namespace).inc(); + recordSchemaCompabilityResult(namespace, CompatibilityCheckResponse.COMPATIBLE); } + private void recordSchemaCompabilityResult(String namespace, CompatibilityCheckResponse result) { + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, namespace) + .putAll(result.attributes) + .build(); + schemaCompatibilityCounter.add(1, attributes); + } private String getNamespace(String schemaId) { String namespace; @@ -148,20 +229,9 @@ private void removeChild(String namespace) { } @Override - public void close() throws Exception { - if (CLOSED.compareAndSet(false, true)) { - CollectorRegistry.defaultRegistry.unregister(this.deleteOpsFailedCounter); - CollectorRegistry.defaultRegistry.unregister(this.getOpsFailedCounter); - CollectorRegistry.defaultRegistry.unregister(this.putOpsFailedCounter); - CollectorRegistry.defaultRegistry.unregister(this.compatibleCounter); - CollectorRegistry.defaultRegistry.unregister(this.incompatibleCounter); - CollectorRegistry.defaultRegistry.unregister(this.deleteOpsLatency); - CollectorRegistry.defaultRegistry.unregister(this.getOpsLatency); - CollectorRegistry.defaultRegistry.unregister(this.putOpsLatency); - if (null != this.future) { - this.future.cancel(false); - } - } + public synchronized void close() throws Exception { + namespaceAccess.keySet().forEach(this::removeChild); + future.cancel(false); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/TestNGInstanceOrder.java b/pulsar-broker/src/test/java/org/apache/pulsar/TestNGInstanceOrder.java new file mode 100644 index 0000000000000..50c9863d586ec --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/TestNGInstanceOrder.java @@ -0,0 +1,38 @@ +/* + * 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; + +import java.util.Comparator; +import java.util.List; +import org.testng.IMethodInstance; +import org.testng.IMethodInterceptor; +import org.testng.ITestContext; + +// Sorts the test methods by test object instance hashcode, then priority, then method name. Useful when Factory +// generated tests interfere with each other. +public class TestNGInstanceOrder implements IMethodInterceptor { + @Override + public List intercept(List methods, ITestContext context) { + return methods.stream().sorted(Comparator.comparingInt(o -> o.getInstance().hashCode()) + .thenComparingInt(o -> o.getMethod().getInterceptedPriority()) + .thenComparingInt(o -> o.getMethod().getPriority()) + .thenComparing(o -> o.getMethod().getMethodName())) + .toList(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java index fbf8c5cc15444..658ea268c644c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/SchemaServiceTest.java @@ -18,19 +18,23 @@ */ package org.apache.pulsar.broker.service.schema; +import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; +import static org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy.BACKWARD; +import static org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy.BACKWARD_TRANSITIVE; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.testng.Assert.fail; import static org.testng.AssertJUnit.assertEquals; -import static org.testng.AssertJUnit.assertFalse; import static org.testng.AssertJUnit.assertNull; -import static org.testng.AssertJUnit.assertTrue; import com.google.common.collect.Multimap; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; +import io.opentelemetry.api.common.Attributes; import java.io.ByteArrayOutputStream; import java.nio.charset.StandardCharsets; import java.time.Clock; +import java.time.Duration; import java.time.Instant; import java.time.ZoneId; import java.util.ArrayList; @@ -38,15 +42,17 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; +import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; @@ -58,6 +64,9 @@ import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaInfoWithVersion; import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; +import org.assertj.core.api.InstanceOfAssertFactories; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -66,7 +75,7 @@ @Test(groups = "broker") public class SchemaServiceTest extends MockedPulsarServiceBaseTest { - private static final Clock MockClock = Clock.fixed(Instant.EPOCH, ZoneId.systemDefault()); + private static final Clock MockClock = Clock.fixed(Instant.now(), ZoneId.systemDefault()); private final String schemaId1 = "1/2/3/4"; private static final String userId = "user"; @@ -99,10 +108,23 @@ protected void setup() throws Exception { storage.start(); Map checkMap = new HashMap<>(); checkMap.put(SchemaType.AVRO, new AvroSchemaCompatibilityCheck()); - schemaRegistryService = new SchemaRegistryServiceImpl(storage, checkMap, MockClock, null); + schemaRegistryService = new SchemaRegistryServiceImpl(storage, checkMap, MockClock, pulsar); + + var schemaRegistryStats = + Mockito.spy((SchemaRegistryStats) FieldUtils.readField(schemaRegistryService, "stats", true)); + // Disable periodic cleanup of Prometheus entries. + Mockito.doNothing().when(schemaRegistryStats).run(); + FieldUtils.writeField(schemaRegistryService, "stats", schemaRegistryStats, true); + setupDefaultTenantAndNamespace(); } + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder pulsarTestContextBuilder) { + super.customizeMainPulsarTestContextBuilder(pulsarTestContextBuilder); + pulsarTestContextBuilder.enableOpenTelemetry(true); + } + @AfterMethod(alwaysRun = true) @Override protected void cleanup() throws Exception { @@ -118,6 +140,32 @@ public void testSchemaRegistryMetrics() throws Exception { getSchema(schemaId, version(0)); deleteSchema(schemaId, version(1)); + var otelMetrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertThat(otelMetrics).anySatisfy(metric -> assertThat(metric) + .hasName(SchemaRegistryStats.SCHEMA_REGISTRY_REQUEST_DURATION_METRIC_NAME) + .hasHistogramSatisfying(histogram -> histogram.hasPointsSatisfying( + point -> point + .hasAttributes(Attributes.of(OpenTelemetryAttributes.PULSAR_NAMESPACE, "tenant/ns", + SchemaRegistryStats.REQUEST_TYPE_KEY, "delete", + SchemaRegistryStats.RESPONSE_TYPE_KEY, "success")) + .hasCount(1), + point -> point + .hasAttributes(Attributes.of(OpenTelemetryAttributes.PULSAR_NAMESPACE, "tenant/ns", + SchemaRegistryStats.REQUEST_TYPE_KEY, "put", + SchemaRegistryStats.RESPONSE_TYPE_KEY, "success")) + .hasCount(1), + point -> point + .hasAttributes(Attributes.of(OpenTelemetryAttributes.PULSAR_NAMESPACE, "tenant/ns", + SchemaRegistryStats.REQUEST_TYPE_KEY, "list", + SchemaRegistryStats.RESPONSE_TYPE_KEY, "success")) + .hasCount(1), + point -> point + .hasAttributes(Attributes.of(OpenTelemetryAttributes.PULSAR_NAMESPACE, "tenant/ns", + SchemaRegistryStats.REQUEST_TYPE_KEY, "get", + SchemaRegistryStats.RESPONSE_TYPE_KEY, "success")) + .hasCount(1) + ))); + ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, false, false, false, output); output.flush(); @@ -309,16 +357,39 @@ public void dontReAddExistingSchemaInMiddle() throws Exception { putSchema(schemaId1, schemaData2, version(1)); } - @Test(expectedExceptions = ExecutionException.class) + @Test public void checkIsCompatible() throws Exception { - putSchema(schemaId1, schemaData1, version(0), SchemaCompatibilityStrategy.BACKWARD_TRANSITIVE); - putSchema(schemaId1, schemaData2, version(1), SchemaCompatibilityStrategy.BACKWARD_TRANSITIVE); - - assertTrue(schemaRegistryService.isCompatible(schemaId1, schemaData3, - SchemaCompatibilityStrategy.BACKWARD).get()); - assertFalse(schemaRegistryService.isCompatible(schemaId1, schemaData3, - SchemaCompatibilityStrategy.BACKWARD_TRANSITIVE).get()); - putSchema(schemaId1, schemaData3, version(2), SchemaCompatibilityStrategy.BACKWARD_TRANSITIVE); + var schemaId = BrokerTestUtil.newUniqueName("tenant/ns/topic"); + putSchema(schemaId, schemaData1, version(0), BACKWARD_TRANSITIVE); + putSchema(schemaId, schemaData2, version(1), BACKWARD_TRANSITIVE); + + var timeout = Duration.ofSeconds(1); + assertThat(schemaRegistryService.isCompatible(schemaId, schemaData3, BACKWARD)) + .succeedsWithin(timeout, InstanceOfAssertFactories.BOOLEAN) + .isTrue(); + assertThat(schemaRegistryService.isCompatible(schemaId, schemaData3, BACKWARD_TRANSITIVE)) + .failsWithin(timeout) + .withThrowableOfType(ExecutionException.class) + .withCauseInstanceOf(IncompatibleSchemaException.class); + assertThatThrownBy(() -> putSchema(schemaId, schemaData3, version(2), BACKWARD_TRANSITIVE)) + .isInstanceOf(ExecutionException.class) + .hasCauseInstanceOf(IncompatibleSchemaException.class); + + assertThat(pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics()) + .anySatisfy(metric -> assertThat(metric) + .hasName(SchemaRegistryStats.COMPATIBLE_COUNTER_METRIC_NAME) + .hasLongSumSatisfying( + sum -> sum.hasPointsSatisfying( + point -> point + .hasAttributes(Attributes.of( + OpenTelemetryAttributes.PULSAR_NAMESPACE, "tenant/ns", + SchemaRegistryStats.COMPATIBILITY_CHECK_RESPONSE_KEY, "compatible")) + .hasValue(2), + point -> point + .hasAttributes(Attributes.of( + OpenTelemetryAttributes.PULSAR_NAMESPACE, "tenant/ns", + SchemaRegistryStats.COMPATIBILITY_CHECK_RESPONSE_KEY, "incompatible")) + .hasValue(2)))); } @Test @@ -374,20 +445,13 @@ private void deleteSchema(String schemaId, SchemaVersion expectedVersion) throws assertEquals(expectedVersion, version); } - private SchemaData randomSchema() { - UUID randomString = UUID.randomUUID(); - return SchemaData.builder() - .user(userId) - .type(SchemaType.JSON) - .timestamp(MockClock.millis()) - .isDeleted(false) - .data(randomString.toString().getBytes()) - .props(new TreeMap<>()) - .build(); - } - private static SchemaData getSchemaData(String schemaJson) { - return SchemaData.builder().data(schemaJson.getBytes()).type(SchemaType.AVRO).user(userId).build(); + return SchemaData.builder() + .data(schemaJson.getBytes()) + .type(SchemaType.AVRO) + .user(userId) + .timestamp(MockClock.millis()) + .build(); } private SchemaVersion version(long version) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleSchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleSchemaTest.java index c8c7c3b2ccc38..e006b72fad279 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleSchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleSchemaTest.java @@ -41,6 +41,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema.Parser; import org.apache.avro.reflect.ReflectData; +import org.apache.pulsar.TestNGInstanceOrder; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.PulsarClientException.IncompatibleSchemaException; import org.apache.pulsar.client.api.PulsarClientException.InvalidMessageException; @@ -66,10 +67,12 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; import org.testng.annotations.Factory; +import org.testng.annotations.Listeners; import org.testng.annotations.Test; @Test(groups = "broker-api") @Slf4j +@Listeners({ TestNGInstanceOrder.class }) public class SimpleSchemaTest extends ProducerConsumerBase { private static final String NAMESPACE = "my-property/my-ns"; From 245c3e8bee2c1db2b61f00bafb6210ec8a2a612a Mon Sep 17 00:00:00 2001 From: entvex <1580435+entvex@users.noreply.github.com> Date: Mon, 3 Jun 2024 18:33:44 +0200 Subject: [PATCH 251/580] [fix][cli] Fix expiration of tokens created with "pulsar tokens create" (#22815) Co-authored-by: David Jensen --- .../utils/auth/tokens/TokensCliUtils.java | 4 +- .../utils/auth/tokens/TokensCliUtilsTest.java | 57 +++++++++++++++++++ 2 files changed, 59 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java index 78268a6295c28..82f0178c9ca82 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java @@ -40,7 +40,7 @@ import javax.crypto.SecretKey; import lombok.Cleanup; import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; -import org.apache.pulsar.cli.converters.picocli.TimeUnitToSecondsConverter; +import org.apache.pulsar.cli.converters.picocli.TimeUnitToMillisConverter; import org.apache.pulsar.docs.tools.CmdGenerateDocs; import picocli.CommandLine; import picocli.CommandLine.Command; @@ -128,7 +128,7 @@ public static class CommandCreateToken implements Callable { "--expiry-time"}, description = "Relative expiry time for the token (eg: 1h, 3d, 10y)." + " (m=minutes) Default: no expiration", - converter = TimeUnitToSecondsConverter.class) + converter = TimeUnitToMillisConverter.class) private Long expiryTime = null; @Option(names = {"-sk", diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtilsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtilsTest.java index 65c5d9981bfd2..eec568c64e313 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtilsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtilsTest.java @@ -20,6 +20,8 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; + +import io.jsonwebtoken.Claims; import io.jsonwebtoken.JwsHeader; import io.jsonwebtoken.Jwt; import io.jsonwebtoken.Jwts; @@ -27,7 +29,12 @@ import java.io.ByteArrayOutputStream; import java.io.PrintStream; import java.lang.reflect.Field; +import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.Arrays; +import java.util.Date; + +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import picocli.CommandLine.Option; @@ -36,6 +43,18 @@ */ public class TokensCliUtilsTest { + @DataProvider(name = "desiredExpireTime") + public Object[][] desiredExpireTime() { + return new Object[][] { + {"600", 600}, //10m + {"5m", 300}, + {"1h", 3600}, + {"1d", 86400}, + {"1w", 604800}, + {"1y", 31536000} + }; + } + @Test public void testCreateToken() { PrintStream oldStream = System.out; @@ -75,6 +94,44 @@ public void testCreateToken() { } } + @Test(dataProvider = "desiredExpireTime") + public void commandCreateToken_WhenCreatingATokenWithExpiryTime_ShouldHaveTheDesiredExpireTime(String expireTime, int expireAsSec) throws Exception { + PrintStream oldStream = System.out; + try { + //Arrange + ByteArrayOutputStream baoStream = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baoStream)); + + String[] command = {"create", "--secret-key", + "data:;base64,u+FxaxYWpsTfxeEmMh8fQeS3g2jfXw4+sGIv+PTY+BY=", + "--subject", "test", + "--expiry-time", expireTime, + }; + + new TokensCliUtils().execute(command); + String token = baoStream.toString(); + + Instant start = (new Date().toInstant().plus(expireAsSec - 5, ChronoUnit.SECONDS)); + Instant stop = (new Date().toInstant().plus(expireAsSec + 5, ChronoUnit.SECONDS)); + + //Act + Claims jwt = Jwts.parserBuilder() + .setSigningKey(Decoders.BASE64.decode("u+FxaxYWpsTfxeEmMh8fQeS3g2jfXw4+sGIv+PTY+BY=")) + .build() + .parseClaimsJws(token) + .getBody(); + + //Assert + //Checks if the token expires within +-5 sec. + assertTrue(( ! jwt.getExpiration().toInstant().isBefore( start ) ) && ( jwt.getExpiration().toInstant().isBefore( stop ) )); + + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + System.setOut(oldStream); + } + } + /** * Test tokens generate docs. * From 208d8a53be20a95613703f0a11527ca9735e7bf8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 3 Jun 2024 19:40:50 +0300 Subject: [PATCH 252/580] [improve][ml] RangeCache refactoring follow-up: use StampedLock instead of synchronized (#22818) --- .../bookkeeper/mledger/util/RangeCache.java | 49 ++++++++++++++----- 1 file changed, 37 insertions(+), 12 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java index 45295d7190654..2f2b161a30684 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java @@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.StampedLock; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.util.RangeCache.ValueWithKeyValidation; import org.apache.commons.lang3.tuple.Pair; @@ -75,6 +76,7 @@ protected EntryWrapper newObject(Handle recyclerHandle) { return new EntryWrapper(recyclerHandle); } }; + private final StampedLock lock = new StampedLock(); private K key; private V value; long size; @@ -85,27 +87,50 @@ private EntryWrapper(Handle recyclerHandle) { static EntryWrapper create(K key, V value, long size) { EntryWrapper entryWrapper = RECYCLER.get(); - synchronized (entryWrapper) { - entryWrapper.key = key; - entryWrapper.value = value; - entryWrapper.size = size; - } + long stamp = entryWrapper.lock.writeLock(); + entryWrapper.key = key; + entryWrapper.value = value; + entryWrapper.size = size; + entryWrapper.lock.unlockWrite(stamp); return entryWrapper; } - synchronized K getKey() { - return key; + K getKey() { + long stamp = lock.tryOptimisticRead(); + K localKey = key; + if (!lock.validate(stamp)) { + stamp = lock.readLock(); + localKey = key; + lock.unlockRead(stamp); + } + return localKey; } - synchronized V getValue(K key) { - if (this.key != key) { + V getValue(K key) { + long stamp = lock.tryOptimisticRead(); + K localKey = this.key; + V localValue = this.value; + if (!lock.validate(stamp)) { + stamp = lock.readLock(); + localKey = this.key; + localValue = this.value; + lock.unlockRead(stamp); + } + if (localKey != key) { return null; } - return value; + return localValue; } - synchronized long getSize() { - return size; + long getSize() { + long stamp = lock.tryOptimisticRead(); + long localSize = size; + if (!lock.validate(stamp)) { + stamp = lock.readLock(); + localSize = size; + lock.unlockRead(stamp); + } + return localSize; } void recycle() { From 05d98f7b07b6e3ac249845f042bfa937d1744f42 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 3 Jun 2024 19:49:02 +0300 Subject: [PATCH 253/580] [fix][sec] Upgrade Bouncycastle libraries to address CVEs (#22826) --- bouncy-castle/bc/LICENSE | 5 ++--- distribution/server/src/assemble/LICENSE.bin.txt | 7 +++---- distribution/shell/src/assemble/LICENSE.bin.txt | 7 +++---- pom.xml | 6 +++--- 4 files changed, 11 insertions(+), 14 deletions(-) diff --git a/bouncy-castle/bc/LICENSE b/bouncy-castle/bc/LICENSE index 14f4e76e921d3..c95d33d3d1ffb 100644 --- a/bouncy-castle/bc/LICENSE +++ b/bouncy-castle/bc/LICENSE @@ -205,6 +205,5 @@ This projects includes binary packages with the following licenses: Bouncy Castle License * Bouncy Castle -- licenses/LICENSE-bouncycastle.txt - - org.bouncycastle-bcpkix-jdk18on-1.78.jar - - org.bouncycastle-bcprov-jdk18on-1.78.jar - - org.bouncycastle-bcprov-ext-jdk18on-1.78.jar + - org.bouncycastle-bcpkix-jdk18on-1.78.1.jar + - org.bouncycastle-bcprov-jdk18on-1.78.1.jar diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index e458200757167..cfee51da21d51 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -617,10 +617,9 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- ../licenses/LICENSE-bouncycastle.txt - - org.bouncycastle-bcpkix-jdk18on-1.78.jar - - org.bouncycastle-bcprov-ext-jdk18on-1.78.jar - - org.bouncycastle-bcprov-jdk18on-1.78.jar - - org.bouncycastle-bcutil-jdk18on-1.78.jar + - org.bouncycastle-bcpkix-jdk18on-1.78.1.jar + - org.bouncycastle-bcprov-jdk18on-1.78.1.jar + - org.bouncycastle-bcutil-jdk18on-1.78.1.jar ------------------------ diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index be1f7db63134c..d3e58b999c5f2 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -474,10 +474,9 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- ../licenses/LICENSE-bouncycastle.txt - - bcpkix-jdk18on-1.78.jar - - bcprov-ext-jdk18on-1.78.jar - - bcprov-jdk18on-1.78.jar - - bcutil-jdk18on-1.78.jar + - bcpkix-jdk18on-1.78.1.jar + - bcprov-jdk18on-1.78.1.jar + - bcutil-jdk18on-1.78.1.jar ------------------------ diff --git a/pom.xml b/pom.xml index 347ef9e83c2c6..69d23f1793945 100644 --- a/pom.xml +++ b/pom.xml @@ -160,9 +160,9 @@ flexible messaging model and an intuitive client API. 2.0.13 4.4 2.23.1 - 1.78 - 1.0.6 - 1.0.2.4 + 1.78.1 + 1.0.7 + 1.0.2.5 2.14.2 0.10.2 1.6.2 From 2c2ecabfcdcda07b14d46d91c45fe46550b0be32 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 3 Jun 2024 20:03:03 +0300 Subject: [PATCH 254/580] [improve][misc] Upgrade OTel library to 1.38.0 version (#22825) --- .../server/src/assemble/LICENSE.bin.txt | 40 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 6 +-- pom.xml | 4 +- 3 files changed, 25 insertions(+), 25 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index cfee51da21d51..32d05b28d1322 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -522,26 +522,26 @@ The Apache Software License, Version 2.0 - org.roaringbitmap-RoaringBitmap-0.9.44.jar - org.roaringbitmap-shims-0.9.44.jar * OpenTelemetry - - io.opentelemetry-opentelemetry-api-1.37.0.jar - - io.opentelemetry-opentelemetry-api-incubator-1.37.0-alpha.jar - - io.opentelemetry-opentelemetry-context-1.37.0.jar - - io.opentelemetry-opentelemetry-exporter-common-1.37.0.jar - - io.opentelemetry-opentelemetry-exporter-otlp-1.37.0.jar - - io.opentelemetry-opentelemetry-exporter-otlp-common-1.37.0.jar - - io.opentelemetry-opentelemetry-exporter-prometheus-1.37.0-alpha.jar - - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.37.0.jar - - io.opentelemetry-opentelemetry-sdk-1.37.0.jar - - io.opentelemetry-opentelemetry-sdk-common-1.37.0.jar - - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.37.0.jar - - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.37.0.jar - - io.opentelemetry-opentelemetry-sdk-logs-1.37.0.jar - - io.opentelemetry-opentelemetry-sdk-metrics-1.37.0.jar - - io.opentelemetry-opentelemetry-sdk-trace-1.37.0.jar - - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-1.33.2.jar - - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-semconv-1.33.2-alpha.jar - - io.opentelemetry.instrumentation-opentelemetry-resources-1.33.2-alpha.jar - - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-1.33.2-alpha.jar - - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-1.33.2-alpha.jar + - io.opentelemetry-opentelemetry-api-1.38.0.jar + - io.opentelemetry-opentelemetry-api-incubator-1.38.0-alpha.jar + - io.opentelemetry-opentelemetry-context-1.38.0.jar + - io.opentelemetry-opentelemetry-exporter-common-1.38.0.jar + - io.opentelemetry-opentelemetry-exporter-otlp-1.38.0.jar + - io.opentelemetry-opentelemetry-exporter-otlp-common-1.38.0.jar + - io.opentelemetry-opentelemetry-exporter-prometheus-1.38.0-alpha.jar + - io.opentelemetry-opentelemetry-exporter-sender-okhttp-1.38.0.jar + - io.opentelemetry-opentelemetry-sdk-1.38.0.jar + - io.opentelemetry-opentelemetry-sdk-common-1.38.0.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-1.38.0.jar + - io.opentelemetry-opentelemetry-sdk-extension-autoconfigure-spi-1.38.0.jar + - io.opentelemetry-opentelemetry-sdk-logs-1.38.0.jar + - io.opentelemetry-opentelemetry-sdk-metrics-1.38.0.jar + - io.opentelemetry-opentelemetry-sdk-trace-1.38.0.jar + - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-1.33.3.jar + - io.opentelemetry.instrumentation-opentelemetry-instrumentation-api-semconv-1.33.3-alpha.jar + - io.opentelemetry.instrumentation-opentelemetry-resources-1.33.3-alpha.jar + - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-1.33.3-alpha.jar + - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-1.33.3-alpha.jar - io.opentelemetry.semconv-opentelemetry-semconv-1.25.0-alpha.jar BSD 3-clause "New" or "Revised" License diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index d3e58b999c5f2..dfb54f739bf74 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -389,9 +389,9 @@ The Apache Software License, Version 2.0 - log4j-slf4j2-impl-2.23.1.jar - log4j-web-2.23.1.jar * OpenTelemetry - - opentelemetry-api-1.37.0.jar - - opentelemetry-api-incubator-1.37.0-alpha.jar - - opentelemetry-context-1.37.0.jar + - opentelemetry-api-1.38.0.jar + - opentelemetry-api-incubator-1.38.0-alpha.jar + - opentelemetry-context-1.38.0.jar * BookKeeper - bookkeeper-common-allocator-4.17.0.jar diff --git a/pom.xml b/pom.xml index 69d23f1793945..49b74d278ed69 100644 --- a/pom.xml +++ b/pom.xml @@ -259,9 +259,9 @@ flexible messaging model and an intuitive client API. 3.4.3 1.5.2-3 2.0.6 - 1.37.0 + 1.38.0 ${opentelemetry.version}-alpha - 1.33.2 + 1.33.3 ${opentelemetry.instrumentation.version}-alpha 1.25.0-alpha 4.7.5 From 91781d5b57363ba1b98515f730af98e5ea116eec Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 3 Jun 2024 11:07:24 -0700 Subject: [PATCH 255/580] [fix] Bump io.airlift:aircompressor from 0.20 to 0.27 (#22819) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Matteo Merli --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- distribution/shell/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 32d05b28d1322..dad4ea90e97af 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -388,7 +388,7 @@ The Apache Software License, Version 2.0 - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar * AirCompressor - - io.airlift-aircompressor-0.20.jar + - io.airlift-aircompressor-0.27.jar * AsyncHttpClient - org.asynchttpclient-async-http-client-2.12.1.jar - org.asynchttpclient-async-http-client-netty-utils-2.12.1.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index dfb54f739bf74..0049f7f8ef38e 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -398,7 +398,7 @@ The Apache Software License, Version 2.0 - cpu-affinity-4.17.0.jar - circe-checksum-4.17.0.jar * AirCompressor - - aircompressor-0.20.jar + - aircompressor-0.27.jar * AsyncHttpClient - async-http-client-2.12.1.jar - async-http-client-netty-utils-2.12.1.jar diff --git a/pom.xml b/pom.xml index 49b74d278ed69..ad5b0567f7194 100644 --- a/pom.xml +++ b/pom.xml @@ -215,7 +215,7 @@ flexible messaging model and an intuitive client API. 1.0 0.16.1 6.2.8 - 0.20 + 0.27 2.12.1 3.11 1.10 From b0910812b7e9d460c30580e57565344a150f02f1 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 3 Jun 2024 17:29:43 -0700 Subject: [PATCH 256/580] [fix] Removing out of the box option for Java serde in functions (#22832) --- .../pulsar/functions/api/utils/JavaSerDe.java | 69 ------------------- .../functions/api/utils/JavaSerDeTest.java | 51 -------------- 2 files changed, 120 deletions(-) delete mode 100644 pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/utils/JavaSerDe.java delete mode 100644 pulsar-functions/api-java/src/test/java/org/apache/pulsar/functions/api/utils/JavaSerDeTest.java diff --git a/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/utils/JavaSerDe.java b/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/utils/JavaSerDe.java deleted file mode 100644 index c145179abb42b..0000000000000 --- a/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/utils/JavaSerDe.java +++ /dev/null @@ -1,69 +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.functions.api.utils; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.ObjectInputStream; -import java.io.ObjectOutput; -import java.io.ObjectOutputStream; -import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.common.classification.InterfaceAudience; -import org.apache.pulsar.common.classification.InterfaceStability; -import org.apache.pulsar.functions.api.SerDe; - -/** - * Java Serialization based SerDe. - */ -@InterfaceAudience.Public -@InterfaceStability.Stable -@Slf4j -public class JavaSerDe implements SerDe { - - private static final JavaSerDe INSTANCE = new JavaSerDe(); - - public static JavaSerDe of() { - return INSTANCE; - } - - @Override - public byte[] serialize(Object resultValue) { - try (ByteArrayOutputStream bos = new ByteArrayOutputStream(); - ObjectOutput out = new ObjectOutputStream(bos)) { - out.writeObject(resultValue); - out.flush(); - return bos.toByteArray(); - } catch (Exception ex) { - log.info("Exception during serialization", ex); - } - return null; - } - - @Override - public Object deserialize(byte[] data) { - Object obj = null; - try (ByteArrayInputStream bis = new ByteArrayInputStream(data); - ObjectInputStream ois = new ObjectInputStream(bis)) { - obj = ois.readObject(); - } catch (Exception ex) { - log.info("Exception during deserialization", ex); - } - return obj; - } -} diff --git a/pulsar-functions/api-java/src/test/java/org/apache/pulsar/functions/api/utils/JavaSerDeTest.java b/pulsar-functions/api-java/src/test/java/org/apache/pulsar/functions/api/utils/JavaSerDeTest.java deleted file mode 100644 index 164709869b7ba..0000000000000 --- a/pulsar-functions/api-java/src/test/java/org/apache/pulsar/functions/api/utils/JavaSerDeTest.java +++ /dev/null @@ -1,51 +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.functions.api.utils; - -import static org.testng.Assert.assertEquals; -import java.io.Serializable; -import lombok.AllArgsConstructor; -import lombok.Data; -import org.testng.annotations.Test; - -/** - * Unit test of {@link JavaSerDe}. - */ -public class JavaSerDeTest { - - @Data - @AllArgsConstructor - private static class TestObject implements Serializable { - - private int intField; - private String stringField; - - } - - @Test - public void testSerDe() { - TestObject to = new TestObject(1234, "test-serde-java-object"); - - byte[] data = JavaSerDe.of().serialize(to); - TestObject deserializeTo = (TestObject) JavaSerDe.of().deserialize(data); - - assertEquals(to, deserializeTo); - } - -} From ca8b465897fd6176b614e2b3f2a841b349037aad Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 3 Jun 2024 19:31:15 -0700 Subject: [PATCH 257/580] [improve] Validate user paths in Functions utils (#22833) --- .../pulsar/broker/web/ExceptionHandler.java | 2 + .../functions/utils/FunctionConfigUtils.java | 14 ++++++- .../filesystem/FileSystemPackagesStorage.java | 42 +++++++++++++------ 3 files changed, 43 insertions(+), 15 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/ExceptionHandler.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/ExceptionHandler.java index b11ec3a8a98db..205e02ed75a2e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/ExceptionHandler.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/ExceptionHandler.java @@ -24,6 +24,7 @@ import javax.servlet.ServletResponse; import javax.servlet.http.HttpServletResponse; import javax.ws.rs.core.Response; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.intercept.InterceptException; import org.apache.pulsar.common.policies.data.ErrorData; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -36,6 +37,7 @@ /** * Exception handler for handle exception. */ +@Slf4j public class ExceptionHandler { public void handle(ServletResponse response, Exception ex) throws IOException { diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java index ee59317daf755..9dc9d5428eda3 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java @@ -853,14 +853,24 @@ public static void doCommonChecks(FunctionConfig functionConfig) { if (!isEmpty(functionConfig.getPy()) && !org.apache.pulsar.common.functions.Utils .isFunctionPackageUrlSupported(functionConfig.getPy()) && functionConfig.getPy().startsWith(BUILTIN)) { - if (!new File(functionConfig.getPy()).exists()) { + String filename = functionConfig.getPy(); + if (filename.contains("..")) { + throw new IllegalArgumentException("Invalid filename: " + filename); + } + + if (!new File(filename).exists()) { throw new IllegalArgumentException("The supplied python file does not exist"); } } if (!isEmpty(functionConfig.getGo()) && !org.apache.pulsar.common.functions.Utils .isFunctionPackageUrlSupported(functionConfig.getGo()) && functionConfig.getGo().startsWith(BUILTIN)) { - if (!new File(functionConfig.getGo()).exists()) { + String filename = functionConfig.getGo(); + if (filename.contains("..")) { + throw new IllegalArgumentException("Invalid filename: " + filename); + } + + if (!new File(filename).exists()) { throw new IllegalArgumentException("The supplied go file does not exist"); } } diff --git a/pulsar-package-management/filesystem-storage/src/main/java/org/apache/pulsar/packages/management/storage/filesystem/FileSystemPackagesStorage.java b/pulsar-package-management/filesystem-storage/src/main/java/org/apache/pulsar/packages/management/storage/filesystem/FileSystemPackagesStorage.java index 47d825ea928f4..2bb43bb207203 100644 --- a/pulsar-package-management/filesystem-storage/src/main/java/org/apache/pulsar/packages/management/storage/filesystem/FileSystemPackagesStorage.java +++ b/pulsar-package-management/filesystem-storage/src/main/java/org/apache/pulsar/packages/management/storage/filesystem/FileSystemPackagesStorage.java @@ -58,7 +58,11 @@ public class FileSystemPackagesStorage implements PackagesStorage { } } - private File getPath(String path) { + private File getPath(String path) throws IOException { + if (path.contains("..")) { + throw new IOException("Invalid path: " + path); + } + File f = Paths.get(storagePath.toString(), path).toFile(); if (!f.getParentFile().exists()) { if (!f.getParentFile().mkdirs()) { @@ -119,28 +123,40 @@ public CompletableFuture readAsync(String path, OutputStream outputStream) @Override public CompletableFuture deleteAsync(String path) { - if (getPath(path).delete()) { - return CompletableFuture.completedFuture(null); - } else { - CompletableFuture f = new CompletableFuture<>(); - f.completeExceptionally(new IOException("Failed to delete file at " + path)); - return f; + try { + if (getPath(path).delete()) { + return CompletableFuture.completedFuture(null); + } else { + CompletableFuture f = new CompletableFuture<>(); + f.completeExceptionally(new IOException("Failed to delete file at " + path)); + return f; + } + } catch (IOException e) { + return CompletableFuture.failedFuture(e); } } @Override public CompletableFuture> listAsync(String path) { - String[] files = getPath(path).list(); - if (files == null) { - return CompletableFuture.completedFuture(Collections.emptyList()); - } else { - return CompletableFuture.completedFuture(Arrays.asList(files)); + try { + String[] files = getPath(path).list(); + if (files == null) { + return CompletableFuture.completedFuture(Collections.emptyList()); + } else { + return CompletableFuture.completedFuture(Arrays.asList(files)); + } + } catch (IOException e) { + return CompletableFuture.failedFuture(e); } } @Override public CompletableFuture existAsync(String path) { - return CompletableFuture.completedFuture(getPath(path).exists()); + try { + return CompletableFuture.completedFuture(getPath(path).exists()); + } catch (IOException e) { + return CompletableFuture.failedFuture(e); + } } @Override From 2532fbd5ef0b718695b2a4a76d63669bb5097b9e Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 3 Jun 2024 21:01:13 -0700 Subject: [PATCH 258/580] [fix] JWT CLI util should force the token validation (#22831) --- .../org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java index 82f0178c9ca82..6f71860164638 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java @@ -291,11 +291,10 @@ public Integer call() throws Exception { } // Validate the token - @SuppressWarnings("unchecked") Jwt jwt = Jwts.parserBuilder() .setSigningKey(validationKey) .build() - .parse(token); + .parseClaimsJws(token); System.out.println(jwt.getBody()); return 0; From 02fd1eed0924fa09c723fcab23129cb31cbf957f Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Tue, 4 Jun 2024 17:11:34 +0800 Subject: [PATCH 259/580] [fix] [broker] disable loadBalancerDirectMemoryResourceWeight by default (#22821) --- conf/standalone.conf | 2 +- deployment/terraform-ansible/templates/broker.conf | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/conf/standalone.conf b/conf/standalone.conf index 7c6aeb6815d6b..1a0c501899d1d 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -928,7 +928,7 @@ loadBalancerMemoryResourceWeight=1.0 # The direct memory usage weight when calculating new resource usage. # It only takes effect in the ThresholdShedder strategy. -loadBalancerDirectMemoryResourceWeight=1.0 +loadBalancerDirectMemoryResourceWeight=0 # Bundle unload minimum throughput threshold (MB), avoiding bundle unload frequently. # It only takes effect in the ThresholdShedder strategy. diff --git a/deployment/terraform-ansible/templates/broker.conf b/deployment/terraform-ansible/templates/broker.conf index fe3bae6bb153b..840455bb3c1c0 100644 --- a/deployment/terraform-ansible/templates/broker.conf +++ b/deployment/terraform-ansible/templates/broker.conf @@ -971,7 +971,7 @@ loadBalancerMemoryResourceWeight=1.0 # The direct memory usage weight when calculating new resourde usage. # It only take effect in ThresholdShedder strategy. -loadBalancerDirectMemoryResourceWeight=1.0 +loadBalancerDirectMemoryResourceWeight=0 # Bundle unload minimum throughput threshold (MB), avoding bundle unload frequently. # It only take effect in ThresholdShedder strategy. From 94549856364656cbde2d26e4907bc8f2d4c60e07 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Tue, 4 Jun 2024 17:12:31 +0800 Subject: [PATCH 260/580] [fix] [conf] fix configuration name and typo. (#22822) --- .../terraform-ansible/templates/broker.conf | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/deployment/terraform-ansible/templates/broker.conf b/deployment/terraform-ansible/templates/broker.conf index 840455bb3c1c0..43bbdc0d52d3e 100644 --- a/deployment/terraform-ansible/templates/broker.conf +++ b/deployment/terraform-ansible/templates/broker.conf @@ -320,7 +320,7 @@ dispatcherMinReadBatchSize=1 # Max number of entries to dispatch for a shared subscription. By default it is 20 entries. dispatcherMaxRoundRobinBatchSize=20 -# Precise dispathcer flow control according to history message number of each entry +# Precise dispatcher flow control according to history message number of each entry preciseDispatcherFlowControl=false # Max number of concurrent lookup request broker allows to throttle heavy incoming lookup traffic @@ -638,7 +638,7 @@ bookkeeperMetadataServiceUri= # Authentication plugin to use when connecting to bookies bookkeeperClientAuthenticationPlugin= -# BookKeeper auth plugin implementatation specifics parameters name and values +# BookKeeper auth plugin implementation specifics parameters name and values bookkeeperClientAuthenticationParametersName= bookkeeperClientAuthenticationParameters= @@ -944,7 +944,7 @@ defaultNamespaceBundleSplitAlgorithm=range_equally_divide loadBalancerLoadSheddingStrategy=org.apache.pulsar.broker.loadbalance.impl.ThresholdShedder # The broker resource usage threshold. -# When the broker resource usage is gratter than the pulsar cluster average resource usge, +# When the broker resource usage is greater than the pulsar cluster average resource usge, # the threshold shedder will be triggered to offload bundles from the broker. # It only take effect in ThresholdShedder strategy. loadBalancerBrokerThresholdShedderPercentage=10 @@ -953,27 +953,27 @@ loadBalancerBrokerThresholdShedderPercentage=10 # It only take effect in ThresholdShedder strategy. loadBalancerHistoryResourcePercentage=0.9 -# The BandWithIn usage weight when calculating new resourde usage. +# The BandWithIn usage weight when calculating new resource usage. # It only take effect in ThresholdShedder strategy. loadBalancerBandwithInResourceWeight=1.0 -# The BandWithOut usage weight when calculating new resourde usage. +# The BandWithOut usage weight when calculating new resource usage. # It only take effect in ThresholdShedder strategy. loadBalancerBandwithOutResourceWeight=1.0 -# The CPU usage weight when calculating new resourde usage. +# The CPU usage weight when calculating new resource usage. # It only take effect in ThresholdShedder strategy. loadBalancerCPUResourceWeight=1.0 -# The heap memory usage weight when calculating new resourde usage. +# The heap memory usage weight when calculating new resource usage. # It only take effect in ThresholdShedder strategy. loadBalancerMemoryResourceWeight=1.0 -# The direct memory usage weight when calculating new resourde usage. +# The direct memory usage weight when calculating new resource usage. # It only take effect in ThresholdShedder strategy. loadBalancerDirectMemoryResourceWeight=0 -# Bundle unload minimum throughput threshold (MB), avoding bundle unload frequently. +# Bundle unload minimum throughput threshold (MB), avoiding bundle unload frequently. # It only take effect in ThresholdShedder strategy. loadBalancerBundleUnloadMinThroughputThreshold=10 @@ -995,7 +995,7 @@ replicatorPrefix=pulsar.repl # Duration to check replication policy to avoid replicator inconsistency # due to missing ZooKeeper watch (disable with value 0) -replicatioPolicyCheckDurationSeconds=600 +replicationPolicyCheckDurationSeconds=600 # Default message retention time. 0 means retention is disabled. -1 means data is not removed by time quota defaultRetentionTimeInMinutes=0 From 75293574665809ac6b439e0f20693fc607797f7a Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 4 Jun 2024 03:12:33 -0700 Subject: [PATCH 261/580] [improve] Validate range of argument before long -> int conversion (#22830) --- .../pulsar/broker/admin/impl/TransactionsBase.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TransactionsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TransactionsBase.java index 1014c9fe8e372..4fef0802ed413 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TransactionsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TransactionsBase.java @@ -564,8 +564,15 @@ protected CompletableFuture internalGetPositionStatsP protected CompletableFuture internalAbortTransaction(boolean authoritative, long mostSigBits, long leastSigBits) { + + if (mostSigBits < 0 || mostSigBits > Integer.MAX_VALUE) { + return CompletableFuture.failedFuture(new IllegalArgumentException("mostSigBits out of bounds")); + } + + int partitionIdx = (int) mostSigBits; + return validateTopicOwnershipAsync( - SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN.getPartition((int) mostSigBits), authoritative) + SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN.getPartition(partitionIdx), authoritative) .thenCompose(__ -> validateSuperUserAccessAsync()) .thenCompose(__ -> pulsar().getTransactionMetadataStoreService() .endTransaction(new TxnID(mostSigBits, leastSigBits), TxnAction.ABORT_VALUE, false)); From be5eb919f8c9fb4612fea74054eee8c1412b954b Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 4 Jun 2024 03:18:39 -0700 Subject: [PATCH 262/580] [improve] Upgrade Jetcd to 0.7.7 and VertX to 4.5.8 (#22835) --- .../server/src/assemble/LICENSE.bin.txt | 23 ++++++++++--------- pom.xml | 23 +++++++++++++++++-- tests/integration/pom.xml | 6 +++++ .../integration/io/PulsarIOTestRunner.java | 7 +++--- .../io/sinks/PulsarIOSinkRunner.java | 2 +- .../io/sources/PulsarIOSourceRunner.java | 2 +- .../PulsarIODebeziumSourceRunner.java | 2 +- 7 files changed, 46 insertions(+), 19 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index dad4ea90e97af..25b6787d420df 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -447,6 +447,7 @@ The Apache Software License, Version 2.0 - io.grpc-grpc-rls-1.56.0.jar - io.grpc-grpc-servlet-1.56.0.jar - io.grpc-grpc-servlet-jakarta-1.56.0.jar + - io.grpc-grpc-util-1.60.0.jar * Perfmark - io.perfmark-perfmark-api-0.26.0.jar * OpenCensus @@ -455,7 +456,7 @@ The Apache Software License, Version 2.0 - io.opencensus-opencensus-proto-0.2.0.jar * Jodah - net.jodah-typetools-0.5.0.jar - - net.jodah-failsafe-2.4.4.jar + - dev.failsafe-failsafe-3.3.2.jar * Byte Buddy - net.bytebuddy-byte-buddy-1.14.12.jar * zt-zip @@ -492,12 +493,12 @@ The Apache Software License, Version 2.0 * JCTools - Java Concurrency Tools for the JVM - org.jctools-jctools-core-2.1.2.jar * Vertx - - io.vertx-vertx-auth-common-4.3.8.jar - - io.vertx-vertx-bridge-common-4.3.8.jar - - io.vertx-vertx-core-4.3.8.jar - - io.vertx-vertx-web-4.3.8.jar - - io.vertx-vertx-web-common-4.3.8.jar - - io.vertx-vertx-grpc-4.3.5.jar + - io.vertx-vertx-auth-common-4.5.8.jar + - io.vertx-vertx-bridge-common-4.5.8.jar + - io.vertx-vertx-core-4.5.8.jar + - io.vertx-vertx-web-4.5.8.jar + - io.vertx-vertx-web-common-4.5.8.jar + - io.vertx-vertx-grpc-4.5.8.jar * Apache ZooKeeper - org.apache.zookeeper-zookeeper-3.9.2.jar - org.apache.zookeeper-zookeeper-jute-3.9.2.jar @@ -510,10 +511,10 @@ The Apache Software License, Version 2.0 - com.google.auto.value-auto-value-annotations-1.10.1.jar - com.google.re2j-re2j-1.7.jar * Jetcd - - io.etcd-jetcd-api-0.7.5.jar - - io.etcd-jetcd-common-0.7.5.jar - - io.etcd-jetcd-core-0.7.5.jar - - io.etcd-jetcd-grpc-0.7.5.jar + - io.etcd-jetcd-api-0.7.7.jar + - io.etcd-jetcd-common-0.7.7.jar + - io.etcd-jetcd-core-0.7.7.jar + - io.etcd-jetcd-grpc-0.7.7.jar * IPAddress - com.github.seancfoley-ipaddress-5.3.3.jar * RxJava diff --git a/pom.xml b/pom.xml index ad5b0567f7194..79b6a40804aa0 100644 --- a/pom.xml +++ b/pom.xml @@ -155,7 +155,7 @@ flexible messaging model and an intuitive client API. 2.41 1.10.50 0.16.0 - 4.3.8 + 4.5.8 7.9.2 2.0.13 4.4 @@ -251,7 +251,7 @@ flexible messaging model and an intuitive client API. 5.3.27 4.5.13 4.4.15 - 0.7.5 + 0.7.7 0.3.0 2.0 1.10.12 @@ -265,6 +265,7 @@ flexible messaging model and an intuitive client API. ${opentelemetry.instrumentation.version}-alpha 1.25.0-alpha 4.7.5 + 3.3.2 1.18.3 @@ -384,6 +385,12 @@ flexible messaging model and an intuitive client API. ${mockito.version} + + dev.failsafe + failsafe + ${failsafe.version} + + org.apache.zookeeper zookeeper @@ -509,6 +516,11 @@ flexible messaging model and an intuitive client API. vertx-web ${vertx.version} + + io.vertx + vertx-grpc + ${vertx.version} + org.apache.curator @@ -607,6 +619,13 @@ flexible messaging model and an intuitive client API. + + io.grpc + grpc-util + + 1.60.0 + + org.apache.bookkeeper bookkeeper-common diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index af564e2fed7db..b23395b0ad2e5 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -115,6 +115,12 @@ test + + dev.failsafe + failsafe + test + + org.testcontainers mysql diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/PulsarIOTestRunner.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/PulsarIOTestRunner.java index 4492f6a407520..7c47a0dcff89b 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/PulsarIOTestRunner.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/PulsarIOTestRunner.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.tests.integration.io; +import dev.failsafe.RetryPolicy; import java.time.Duration; import java.util.LinkedHashMap; import java.util.Map; @@ -33,7 +34,6 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; -import net.jodah.failsafe.RetryPolicy; @Slf4j public abstract class PulsarIOTestRunner { @@ -42,10 +42,11 @@ public abstract class PulsarIOTestRunner { final Duration ONE_MINUTE = Duration.ofMinutes(1); final Duration TEN_SECONDS = Duration.ofSeconds(10); - protected final RetryPolicy statusRetryPolicy = new RetryPolicy() + protected final RetryPolicy statusRetryPolicy = RetryPolicy.builder() .withMaxDuration(ONE_MINUTE) .withDelay(TEN_SECONDS) - .onRetry(e -> log.error("Retry ... ")); + .onRetry(e -> log.error("Retry ... ")) + .build(); protected PulsarCluster pulsarCluster; protected String functionRuntimeType; diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarIOSinkRunner.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarIOSinkRunner.java index e5b524ebbef8b..3736bd0155343 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarIOSinkRunner.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarIOSinkRunner.java @@ -22,6 +22,7 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import dev.failsafe.Failsafe; import java.util.LinkedHashMap; import java.util.Map; @@ -46,7 +47,6 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; -import net.jodah.failsafe.Failsafe; @Slf4j public class PulsarIOSinkRunner extends PulsarIOTestRunner { diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarIOSourceRunner.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarIOSourceRunner.java index b843e146e2985..daf645020ce5a 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarIOSourceRunner.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarIOSourceRunner.java @@ -22,6 +22,7 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import dev.failsafe.Failsafe; import java.util.Map; import org.apache.commons.lang3.StringUtils; @@ -45,7 +46,6 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; -import net.jodah.failsafe.Failsafe; @Slf4j public class PulsarIOSourceRunner extends PulsarIOTestRunner { diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarIODebeziumSourceRunner.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarIODebeziumSourceRunner.java index 762dd34e17c91..8f45f0604e378 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarIODebeziumSourceRunner.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarIODebeziumSourceRunner.java @@ -19,9 +19,9 @@ package org.apache.pulsar.tests.integration.io.sources.debezium; import com.google.common.base.Preconditions; +import dev.failsafe.Failsafe; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; -import net.jodah.failsafe.Failsafe; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.SubscriptionInitialPosition; From 30069db47bc84494a1dd62abc0b5fc0d416c856e Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 4 Jun 2024 07:18:23 -0700 Subject: [PATCH 263/580] [improve] Use Google re2/j library for user provided regexes (#22829) Co-authored-by: Lari Hotari --- .../shell/src/assemble/LICENSE.bin.txt | 1 + pom.xml | 7 ++++++ .../pulsar/broker/service/ServerCnx.java | 2 +- .../broker/service/TopicListService.java | 2 +- .../broker/service/TopicListServiceTest.java | 2 +- .../broker/service/TopicListWatcherTest.java | 2 +- .../PatternTopicsConsumerImplAuthTest.java | 2 +- .../impl/PatternTopicsConsumerImplTest.java | 24 +++++++++---------- .../impl/PatternMultiTopicsConsumerImpl.java | 2 +- .../pulsar/client/impl/PulsarClientImpl.java | 6 +++-- .../pulsar/client/impl/TopicListWatcher.java | 2 +- .../PatternMultiTopicsConsumerImplTest.java | 2 +- .../client/impl/TopicListWatcherTest.java | 2 +- pulsar-common/pom.xml | 5 ++++ .../org/apache/pulsar/PulsarVersion.java | 4 ++-- .../pulsar/common/topics/TopicList.java | 3 ++- .../pulsar/common/topics/TopicListTest.java | 2 +- .../testclient/LoadSimulationClient.java | 12 ++++++---- 18 files changed, 51 insertions(+), 31 deletions(-) diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 0049f7f8ef38e..5c3b051cfdd70 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -417,6 +417,7 @@ The Apache Software License, Version 2.0 * Apache Avro - avro-1.11.3.jar - avro-protobuf-1.11.3.jar + * RE2j -- re2j-1.7.jar BSD 3-clause "New" or "Revised" License * JSR305 -- jsr305-3.0.2.jar -- ../licenses/LICENSE-JSR305.txt diff --git a/pom.xml b/pom.xml index 79b6a40804aa0..de385c9705932 100644 --- a/pom.xml +++ b/pom.xml @@ -265,6 +265,7 @@ flexible messaging model and an intuitive client API. ${opentelemetry.instrumentation.version}-alpha 1.25.0-alpha 4.7.5 + 1.7 3.3.2 @@ -656,6 +657,12 @@ flexible messaging model and an intuitive client API. ${bookkeeper.version} + + com.google.re2j + re2j + ${re2j.version} + + org.rocksdb rocksdbjni 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 926ca13c05a20..26a00c00b5a6a 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 @@ -33,6 +33,7 @@ import static org.apache.pulsar.common.protocol.Commands.newLookupErrorResponse; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; +import com.google.re2j.Pattern; import io.netty.buffer.ByteBuf; import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandlerContext; @@ -59,7 +60,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import java.util.regex.Pattern; import java.util.stream.Collectors; import javax.naming.AuthenticationException; import javax.net.ssl.SSLSession; 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 b18286ee06259..e04d07460a2cb 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,13 +18,13 @@ */ package org.apache.pulsar.broker.service; +import com.google.re2j.Pattern; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Semaphore; import java.util.function.BiConsumer; -import java.util.regex.Pattern; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.resources.TopicResources; 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 2b0b852a27375..069794ec504dd 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,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import com.google.re2j.Pattern; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.namespace.NamespaceService; @@ -43,7 +44,6 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Semaphore; -import java.util.regex.Pattern; public class TopicListServiceTest { 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 c232675779fca..641b1bd4e74b3 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,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import com.google.re2j.Pattern; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.metadata.api.NotificationType; import static org.mockito.Mockito.mock; @@ -29,7 +30,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.regex.Pattern; public class TopicListWatcherTest { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplAuthTest.java index a3759c5682165..15cfb2f5654de 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplAuthTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplAuthTest.java @@ -204,7 +204,7 @@ public void testBinaryProtoToGetTopicsOfNamespace() throws Exception { assertTrue(consumer.getTopic().startsWith(PatternMultiTopicsConsumerImpl.DUMMY_TOPIC_NAME_PREFIX)); // 4. verify consumer - assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); + assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().pattern()); List topics = ((PatternMultiTopicsConsumerImpl) consumer).getPartitions(); List> consumers = ((PatternMultiTopicsConsumerImpl) consumer).getConsumers(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java index 94d78e418ab87..c5504a0c02a0c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java @@ -227,7 +227,7 @@ public void testBinaryProtoToGetTopicsOfNamespacePersistent() throws Exception { }); // 4. verify consumer get methods, to get right number of partitions and topics. - assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); + assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().pattern()); List topics = ((PatternMultiTopicsConsumerImpl) consumer).getPartitions(); List> consumers = ((PatternMultiTopicsConsumerImpl) consumer).getConsumers(); @@ -310,7 +310,7 @@ public void testBinaryProtoSubscribeAllTopicOfNamespace() throws Exception { }); // 4. verify consumer get methods, to get right number of partitions and topics. - assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); + assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().pattern()); List topics = ((PatternMultiTopicsConsumerImpl) consumer).getPartitions(); List> consumers = ((PatternMultiTopicsConsumerImpl) consumer).getConsumers(); @@ -393,7 +393,7 @@ public void testBinaryProtoToGetTopicsOfNamespaceNonPersistent() throws Exceptio }); // 4. verify consumer get methods, to get right number of partitions and topics. - assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); + assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().pattern()); List topics = ((PatternMultiTopicsConsumerImpl) consumer).getPartitions(); List> consumers = ((PatternMultiTopicsConsumerImpl) consumer).getConsumers(); @@ -490,7 +490,7 @@ public void testBinaryProtoToGetTopicsOfNamespaceAll() throws Exception { }); // 4. verify consumer get methods, to get right number of partitions and topics. - assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); + assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().pattern()); List topics = ((PatternMultiTopicsConsumerImpl) consumer).getPartitions(); List> consumers = ((PatternMultiTopicsConsumerImpl) consumer).getConsumers(); @@ -566,7 +566,7 @@ public void testStartEmptyPatternConsumer() throws Exception { }); // 3. verify consumer get methods, to get 5 number of partitions and topics. - assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); + assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().pattern()); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 5); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 5); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitionedTopics().size(), 2); @@ -595,7 +595,7 @@ public void testStartEmptyPatternConsumer() throws Exception { // 6. verify consumer get methods, to get number of partitions and topics, value 6=1+2+3. Awaitility.await().untilAsserted(() -> { - assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); + assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().pattern()); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 6); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 6); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitionedTopics().size(), 2); @@ -667,7 +667,7 @@ public void testAutoSubscribePatterConsumerFromBrokerWatcher(boolean delayWatchi // 2. verify consumer get methods. There is no need to trigger discovery, because the broker will push the // changes to update(CommandWatchTopicUpdate). - assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); + assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().pattern()); Awaitility.await().untilAsserted(() -> { assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 4); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 4); @@ -728,7 +728,7 @@ public void testPreciseRegexpSubscribe(boolean partitioned, boolean createTopicA } // 2. verify consumer can subscribe the topic. - assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); + assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().pattern()); Awaitility.await().untilAsserted(() -> { assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 1); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 1); @@ -786,7 +786,7 @@ public void testPreciseRegexpSubscribeDisabledTopicWatcher(boolean partitioned) // 2. verify consumer can subscribe the topic. // Since the minimum value of `patternAutoDiscoveryPeriod` is 60s, we set the test timeout to a triple value. - assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); + assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().pattern()); Awaitility.await().atMost(Duration.ofMinutes(3)).untilAsserted(() -> { assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 1); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 1); @@ -883,7 +883,7 @@ public void testAutoSubscribePatternConsumer() throws Exception { assertTrue(consumer instanceof PatternMultiTopicsConsumerImpl); // 4. verify consumer get methods, to get 6 number of partitions and topics: 6=1+2+3 - assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); + assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().pattern()); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 6); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 6); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitionedTopics().size(), 2); @@ -999,7 +999,7 @@ public void testAutoUnsubscribePatternConsumer() throws Exception { assertTrue(consumer instanceof PatternMultiTopicsConsumerImpl); // 4. verify consumer get methods, to get 0 number of partitions and topics: 6=1+2+3 - assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); + assertSame(pattern.pattern(), ((PatternMultiTopicsConsumerImpl) consumer).getPattern().pattern()); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 6); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 6); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitionedTopics().size(), 2); @@ -1092,7 +1092,7 @@ public void testTopicDeletion() throws Exception { PatternMultiTopicsConsumerImpl consumerImpl = (PatternMultiTopicsConsumerImpl) consumer; // 4. verify consumer get methods - assertSame(consumerImpl.getPattern(), pattern); + assertSame(consumerImpl.getPattern().pattern(), pattern.pattern()); assertEquals(consumerImpl.getPartitionedTopics().size(), 0); producer1.send("msg-1"); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index ec7ff7930c0ac..ffca79dfa4342 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; +import com.google.re2j.Pattern; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.netty.util.Timeout; import io.netty.util.TimerTask; @@ -32,7 +33,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.regex.Pattern; import java.util.stream.Collectors; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Schema; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 8aa16ef0e5f5f..e8107efe98ec0 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -23,6 +23,7 @@ import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import com.google.re2j.Pattern; import io.netty.channel.EventLoopGroup; import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; @@ -577,6 +578,7 @@ private CompletableFuture> patternTopicSubscribeAsync(ConsumerCo Mode subscriptionMode = convertRegexSubscriptionMode(conf.getRegexSubscriptionMode()); TopicName destination = TopicName.get(regex); NamespaceName namespaceName = destination.getNamespaceObject(); + Pattern pattern = Pattern.compile(conf.getTopicsPattern().pattern()); CompletableFuture> consumerSubscribedFuture = new CompletableFuture<>(); lookup.getTopicsUnderNamespace(namespaceName, subscriptionMode, regex, null) @@ -592,10 +594,10 @@ private CompletableFuture> patternTopicSubscribeAsync(ConsumerCo List topicsList = getTopicsResult.getTopics(); if (!getTopicsResult.isFiltered()) { - topicsList = TopicList.filterTopics(getTopicsResult.getTopics(), conf.getTopicsPattern()); + topicsList = TopicList.filterTopics(getTopicsResult.getTopics(), pattern); } conf.getTopicNames().addAll(topicsList); - ConsumerBase consumer = new PatternMultiTopicsConsumerImpl<>(conf.getTopicsPattern(), + ConsumerBase consumer = new PatternMultiTopicsConsumerImpl<>(pattern, getTopicsResult.getTopicsHash(), PulsarClientImpl.this, conf, diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java index 4e635e0d2e8d2..15922d1180ce0 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.impl; +import com.google.re2j.Pattern; import io.netty.channel.ChannelHandlerContext; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -25,7 +26,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.AtomicReference; -import java.util.regex.Pattern; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.api.proto.BaseCommand; import org.apache.pulsar.common.api.proto.CommandWatchTopicUpdate; diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java index 5baca24cf8aa1..116a69b63e4ec 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java @@ -24,6 +24,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import com.google.common.collect.Sets; +import com.google.re2j.Pattern; import org.apache.pulsar.common.lookup.GetTopicsResult; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -32,7 +33,6 @@ import java.util.Collections; import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; -import java.util.regex.Pattern; public class PatternMultiTopicsConsumerImplTest { diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java index 7e9fd601d4f67..74a71f3da850d 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.impl; +import com.google.re2j.Pattern; import io.netty.channel.ChannelHandlerContext; import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; @@ -41,7 +42,6 @@ import org.testng.annotations.Test; import java.util.Collections; import java.util.concurrent.CompletableFuture; -import java.util.regex.Pattern; public class TopicListWatcherTest { diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 31f425e8b4181..cdc30dac2897d 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -238,6 +238,11 @@ gson + + com.google.re2j + re2j + + org.awaitility awaitility diff --git a/pulsar-common/src/main/java-templates/org/apache/pulsar/PulsarVersion.java b/pulsar-common/src/main/java-templates/org/apache/pulsar/PulsarVersion.java index 119e46b9536f3..c597dd327f672 100644 --- a/pulsar-common/src/main/java-templates/org/apache/pulsar/PulsarVersion.java +++ b/pulsar-common/src/main/java-templates/org/apache/pulsar/PulsarVersion.java @@ -18,8 +18,8 @@ */ package org.apache.pulsar; -import java.util.regex.Matcher; -import java.util.regex.Pattern; +import com.google.re2j.Matcher; +import com.google.re2j.Pattern; public class PulsarVersion { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/topics/TopicList.java b/pulsar-common/src/main/java/org/apache/pulsar/common/topics/TopicList.java index 4c0a8d500b703..e8a485b844df5 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/topics/TopicList.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/topics/TopicList.java @@ -19,12 +19,12 @@ package org.apache.pulsar.common.topics; import com.google.common.hash.Hashing; +import com.google.re2j.Pattern; import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Set; -import java.util.regex.Pattern; import java.util.stream.Collectors; import lombok.experimental.UtilityClass; import org.apache.pulsar.common.naming.SystemTopicNames; @@ -47,6 +47,7 @@ public static List filterTopics(List original, String regex) { } public static List filterTopics(List original, Pattern topicsPattern) { + final Pattern shortenedTopicsPattern = Pattern.compile(removeTopicDomainScheme(topicsPattern.toString())); return original.stream() diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/topics/TopicListTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/topics/TopicListTest.java index 9069dd6dcc7b9..a83ef2ac8c719 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/topics/TopicListTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/topics/TopicListTest.java @@ -19,12 +19,12 @@ package org.apache.pulsar.common.topics; import com.google.common.collect.Lists; +import com.google.re2j.Pattern; import org.testng.annotations.Test; import java.util.Arrays; import java.util.List; import java.util.Set; -import java.util.regex.Pattern; import java.util.stream.Stream; import static org.testng.Assert.assertEquals; diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java index c58de64056a66..115733d5ecd41 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationClient.java @@ -19,6 +19,7 @@ package org.apache.pulsar.testclient; import com.google.common.util.concurrent.RateLimiter; +import com.google.re2j.Pattern; import io.netty.util.concurrent.DefaultThreadFactory; import java.io.DataInputStream; import java.io.DataOutputStream; @@ -269,11 +270,14 @@ private void handle(final byte command, final DataInputStream inputStream, final tradeConf.size = inputStream.readInt(); tradeConf.rate = inputStream.readDouble(); // See if a topic belongs to this tenant and group using this regex. - final String groupRegex = ".*://" + tradeConf.tenant + "/.*/" + tradeConf.group + "-.*/.*"; + final Pattern groupRegex = + Pattern.compile(".*://" + tradeConf.tenant + "/.*/" + tradeConf.group + "-.*/.*"); + for (Map.Entry entry : topicsToTradeUnits.entrySet()) { final String topic = entry.getKey(); final TradeUnit unit = entry.getValue(); - if (topic.matches(groupRegex)) { + + if (groupRegex.matcher(topic).matches()) { unit.change(tradeConf); } } @@ -282,11 +286,11 @@ private void handle(final byte command, final DataInputStream inputStream, final // Stop all topics belonging to a group. decodeGroupOptions(tradeConf, inputStream); // See if a topic belongs to this tenant and group using this regex. - final String regex = ".*://" + tradeConf.tenant + "/.*/" + tradeConf.group + "-.*/.*"; + final Pattern regex = Pattern.compile(".*://" + tradeConf.tenant + "/.*/" + tradeConf.group + "-.*/.*"); for (Map.Entry entry : topicsToTradeUnits.entrySet()) { final String topic = entry.getKey(); final TradeUnit unit = entry.getValue(); - if (topic.matches(regex)) { + if (regex.matcher(topic).matches()) { unit.stop.set(true); } } From bb95b85b3ed650182b050e65c3618072619dbd50 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 4 Jun 2024 09:55:41 -0700 Subject: [PATCH 264/580] [fix] Bump google.golang.org/protobuf from 1.32.0 to 1.33.0 in /pulsar-function-go (#22261) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Matteo Merli --- pulsar-function-go/examples/go.mod | 2 +- pulsar-function-go/examples/go.sum | 4 +- pulsar-function-go/go.mod | 2 +- pulsar-function-go/go.sum | 4 +- pulsar-function-go/pf/stats_test.go | 73 +---------------------------- 5 files changed, 7 insertions(+), 78 deletions(-) diff --git a/pulsar-function-go/examples/go.mod b/pulsar-function-go/examples/go.mod index 59e695f5a33eb..0c2c6235b0fb6 100644 --- a/pulsar-function-go/examples/go.mod +++ b/pulsar-function-go/examples/go.mod @@ -51,7 +51,7 @@ require ( google.golang.org/appengine v1.6.8 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20231002182017-d307bd883b97 // indirect google.golang.org/grpc v1.60.0 // indirect - google.golang.org/protobuf v1.33.0 // indirect + google.golang.org/protobuf v1.34.1 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/pulsar-function-go/examples/go.sum b/pulsar-function-go/examples/go.sum index 85390cf32e59a..37c84e71c8b26 100644 --- a/pulsar-function-go/examples/go.sum +++ b/pulsar-function-go/examples/go.sum @@ -745,8 +745,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= -google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/pulsar-function-go/go.mod b/pulsar-function-go/go.mod index bb5c18a4499e2..8dd3f4ef55473 100644 --- a/pulsar-function-go/go.mod +++ b/pulsar-function-go/go.mod @@ -10,7 +10,7 @@ require ( github.com/sirupsen/logrus v1.6.0 github.com/stretchr/testify v1.8.4 google.golang.org/grpc v1.60.0 - google.golang.org/protobuf v1.32.0 + google.golang.org/protobuf v1.34.1 gopkg.in/yaml.v2 v2.4.0 ) diff --git a/pulsar-function-go/go.sum b/pulsar-function-go/go.sum index d840906772c56..0acd26248a8fd 100644 --- a/pulsar-function-go/go.sum +++ b/pulsar-function-go/go.sum @@ -745,8 +745,8 @@ google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGj google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.32.0 h1:pPC6BG5ex8PDFnkbrGU3EixyhKcQ2aDuBS36lqK/C7I= -google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +google.golang.org/protobuf v1.34.1 h1:9ddQBjfCyZPOHPUiPxpYESBLc+T8P3E+Vo4IbKZgFWg= +google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/pulsar-function-go/pf/stats_test.go b/pulsar-function-go/pf/stats_test.go index 0921038bba835..138dc91cd9cd3 100644 --- a/pulsar-function-go/pf/stats_test.go +++ b/pulsar-function-go/pf/stats_test.go @@ -73,80 +73,9 @@ func TestExampleSummaryVec(t *testing.T) { if len(filteredMetricFamilies) > 1 { t.Fatal("Too many metric families") } - // Then, we need to filter the metrics in the family to one that matches our label. - expectedValue := "name: \"pond_temperature_celsius\"\n" + - "help: \"The temperature of the frog pond.\"\n" + - "type: SUMMARY\n" + - "metric: {\n" + - " label: {\n" + - " name: \"species\"\n" + - " value: \"leiopelma-hochstetteri\"\n" + - " }\n" + - " summary: {\n" + - " sample_count: 0\n" + - " sample_sum: 0\n" + - " quantile: {\n" + - " quantile: 0.5\n" + - " value: nan\n" + - " }\n" + - " quantile: {\n" + - " quantile: 0.9\n" + - " value: nan\n" + - " }\n" + - " quantile: {\n" + - " quantile: 0.99\n" + - " value: nan\n" + - " }\n" + - " }\n" + - "}\n" + - "metric: {\n" + - " label: {\n" + - " name: \"species\"\n" + - " value: \"lithobates-catesbeianus\"\n" + - " }\n" + - " summary: {\n" + - " sample_count: 1000\n" + - " sample_sum: 31956.100000000017\n" + - " quantile: {\n" + - " quantile: 0.5\n" + - " value: 32.4\n" + - " }\n" + - " quantile: {\n" + - " quantile: 0.9\n" + - " value: 41.4\n" + - " }\n" + - " quantile: {\n" + - " quantile: 0.99\n" + - " value: 41.9\n" + - " }\n" + - " }\n" + - "}\n" + - "metric: {\n" + - " label: {\n" + - " name: \"species\"\n" + - " value: \"litoria-caerulea\"\n" + - " }\n" + - " summary: {\n" + - " sample_count: 1000\n" + - " sample_sum: 29969.50000000001\n" + - " quantile: {\n" + - " quantile: 0.5\n" + - " value: 31.1\n" + - " }\n" + - " quantile: {\n" + - " quantile: 0.9\n" + - " value: 41.3\n" + - " }\n" + - " quantile: {\n" + - " quantile: 0.99\n" + - " value: 41.9\n" + - " }\n" + - " }\n" + - "}\n" - r, err := prototext.MarshalOptions{Indent: " "}.Marshal(metricFamilies[0]) + _, err = prototext.MarshalOptions{Indent: " "}.Marshal(metricFamilies[0]) assert.NoError(t, err) - assert.Equal(t, expectedValue, string(r)) } func TestExampleSummaryVec_Pulsar(t *testing.T) { _statProcessLatencyMs1 := prometheus.NewSummaryVec( From 8276f218f576e81c212cedf8b3691f7c1a654e0e Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 4 Jun 2024 12:57:21 -0700 Subject: [PATCH 265/580] [improve][broker] Reduce number of OpenTelemetry consumer attributes (#22837) --- .../pulsar/broker/service/Consumer.java | 36 +++++++++++++ .../stats/OpenTelemetryConsumerStats.java | 54 +++++-------------- .../stats/OpenTelemetryConsumerStatsTest.java | 34 +----------- .../OpenTelemetryAttributes.java | 5 -- 4 files changed, 51 insertions(+), 78 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index c9f417c4bc4f7..19711bfa718f4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -25,6 +25,7 @@ import com.google.common.util.concurrent.AtomicDouble; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.Promise; +import io.opentelemetry.api.common.Attributes; import java.time.Instant; import java.util.ArrayList; import java.util.BitSet; @@ -35,6 +36,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.concurrent.atomic.LongAdder; import java.util.stream.Collectors; import lombok.Getter; @@ -69,6 +71,7 @@ import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap; import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap.LongPair; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.apache.pulsar.transaction.common.exception.TransactionConflictException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -158,6 +161,10 @@ public class Consumer { @Getter private final Instant connectedSince = Instant.now(); + private volatile Attributes openTelemetryAttributes; + private static final AtomicReferenceFieldUpdater OPEN_TELEMETRY_ATTRIBUTES_FIELD_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(Consumer.class, Attributes.class, "openTelemetryAttributes"); + public Consumer(Subscription subscription, SubType subType, String topicName, long consumerId, int priorityLevel, String consumerName, boolean isDurable, TransportCnx cnx, String appId, @@ -231,6 +238,8 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo .getPulsar().getConfiguration().isAcknowledgmentAtBatchIndexLevelEnabled(); this.schemaType = schemaType; + + OPEN_TELEMETRY_ATTRIBUTES_FIELD_UPDATER.set(this, null); } @VisibleForTesting @@ -263,6 +272,7 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo this.isAcknowledgmentAtBatchIndexLevelEnabled = false; this.schemaType = null; MESSAGE_PERMITS_UPDATER.set(this, availablePermits); + OPEN_TELEMETRY_ATTRIBUTES_FIELD_UPDATER.set(this, null); } public SubType subType() { @@ -1203,4 +1213,30 @@ private int getStickyKeyHash(Entry entry) { } private static final Logger log = LoggerFactory.getLogger(Consumer.class); + + public Attributes getOpenTelemetryAttributes() { + if (openTelemetryAttributes != null) { + return openTelemetryAttributes; + } + return OPEN_TELEMETRY_ATTRIBUTES_FIELD_UPDATER.updateAndGet(this, oldValue -> { + if (oldValue != null) { + return oldValue; + } + var topicName = TopicName.get(subscription.getTopic().getName()); + + var builder = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_CONSUMER_NAME, consumerName) + .put(OpenTelemetryAttributes.PULSAR_CONSUMER_ID, consumerId) + .put(OpenTelemetryAttributes.PULSAR_SUBSCRIPTION_NAME, subscription.getName()) + .put(OpenTelemetryAttributes.PULSAR_SUBSCRIPTION_TYPE, subType.toString()) + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, topicName.getDomain().toString()) + .put(OpenTelemetryAttributes.PULSAR_TENANT, topicName.getTenant()) + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, topicName.getNamespace()) + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName.getPartitionedTopicName()); + if (topicName.isPartitioned()) { + builder.put(OpenTelemetryAttributes.PULSAR_PARTITION_INDEX, topicName.getPartitionIndex()); + } + return builder.build(); + }); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStats.java index 25af3959db32d..09b487a8fa2c3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStats.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.stats; -import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.BatchCallback; import io.opentelemetry.api.metrics.ObservableLongMeasurement; import java.util.Collection; @@ -27,8 +26,6 @@ import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; -import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; public class OpenTelemetryConsumerStats implements AutoCloseable { @@ -52,6 +49,9 @@ public class OpenTelemetryConsumerStats implements AutoCloseable { public static final String MESSAGE_UNACKNOWLEDGED_COUNTER = "pulsar.broker.consumer.message.unack.count"; private final ObservableLongMeasurement messageUnacknowledgedCounter; + public static final String CONSUMER_BLOCKED_COUNTER = "pulsar.broker.consumer.blocked"; + private final ObservableLongMeasurement consumerBlockedCounter; + // Replaces pulsar_consumer_available_permits public static final String MESSAGE_PERMITS_COUNTER = "pulsar.broker.consumer.permit.count"; private final ObservableLongMeasurement messagePermitsCounter; @@ -91,6 +91,12 @@ public OpenTelemetryConsumerStats(PulsarService pulsar) { .setDescription("The total number of messages unacknowledged by this consumer.") .buildObserver(); + consumerBlockedCounter = meter + .upDownCounterBuilder(CONSUMER_BLOCKED_COUNTER) + .setUnit("1") + .setDescription("Indicates whether the consumer is currently blocked due to unacknowledged messages.") + .buildObserver(); + messagePermitsCounter = meter .upDownCounterBuilder(MESSAGE_PERMITS_COUNTER) .setUnit("{permit}") @@ -114,6 +120,7 @@ public OpenTelemetryConsumerStats(PulsarService pulsar) { messageAckCounter, messageRedeliverCounter, messageUnacknowledgedCounter, + consumerBlockedCounter, messagePermitsCounter); } @@ -123,48 +130,13 @@ public void close() { } private void recordMetricsForConsumer(Consumer consumer) { - var subscription = consumer.getSubscription(); - var topicName = TopicName.get(subscription.getTopic().getName()); - - var builder = Attributes.builder() - .put(OpenTelemetryAttributes.PULSAR_CONSUMER_NAME, consumer.consumerName()) - .put(OpenTelemetryAttributes.PULSAR_CONSUMER_ID, consumer.consumerId()) - .put(OpenTelemetryAttributes.PULSAR_CONSUMER_CONNECTED_SINCE, - consumer.getConnectedSince().getEpochSecond()) - .put(OpenTelemetryAttributes.PULSAR_SUBSCRIPTION_NAME, subscription.getName()) - .put(OpenTelemetryAttributes.PULSAR_SUBSCRIPTION_TYPE, consumer.subType().toString()) - .put(OpenTelemetryAttributes.PULSAR_DOMAIN, topicName.getDomain().toString()) - .put(OpenTelemetryAttributes.PULSAR_TENANT, topicName.getTenant()) - .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, topicName.getNamespace()) - .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName.getPartitionedTopicName()); - if (topicName.isPartitioned()) { - builder.put(OpenTelemetryAttributes.PULSAR_PARTITION_INDEX, topicName.getPartitionIndex()); - } - var clientAddress = consumer.getClientAddressAndPort(); - if (clientAddress != null) { - builder.put(OpenTelemetryAttributes.PULSAR_CLIENT_ADDRESS, clientAddress); - } - var clientVersion = consumer.getClientVersion(); - if (clientVersion != null) { - builder.put(OpenTelemetryAttributes.PULSAR_CLIENT_VERSION, clientVersion); - } - var metadataList = consumer.getMetadata() - .entrySet() - .stream() - .map(e -> String.format("%s:%s", e.getKey(), e.getValue())) - .toList(); - builder.put(OpenTelemetryAttributes.PULSAR_CONSUMER_METADATA, metadataList); - var attributes = builder.build(); - + var attributes = consumer.getOpenTelemetryAttributes(); messageOutCounter.record(consumer.getMsgOutCounter(), attributes); bytesOutCounter.record(consumer.getBytesOutCounter(), attributes); messageAckCounter.record(consumer.getMessageAckCounter(), attributes); messageRedeliverCounter.record(consumer.getMessageRedeliverCounter(), attributes); - messageUnacknowledgedCounter.record(consumer.getUnackedMessages(), - Attributes.builder() - .putAll(attributes) - .put(OpenTelemetryAttributes.PULSAR_CONSUMER_BLOCKED, consumer.isBlocked()) - .build()); + messageUnacknowledgedCounter.record(consumer.getUnackedMessages(), attributes); + consumerBlockedCounter.record(consumer.isBlocked() ? 1 : 0, attributes); messagePermitsCounter.record(consumer.getAvailablePermits(), attributes); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStatsTest.java index 5fcc6754b08fd..a05d7075cf3d7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryConsumerStatsTest.java @@ -20,37 +20,25 @@ import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.argThat; -import static org.mockito.Mockito.doAnswer; import io.opentelemetry.api.common.Attributes; -import java.util.List; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import org.apache.pulsar.broker.BrokerTestUtil; -import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.service.BrokerTestBase; -import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.awaitility.Awaitility; -import org.mockito.Mockito; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; public class OpenTelemetryConsumerStatsTest extends BrokerTestBase { - private BrokerInterceptor brokerInterceptor; - @BeforeMethod(alwaysRun = true) @Override protected void setup() throws Exception { - brokerInterceptor = - Mockito.mock(BrokerInterceptor.class, Mockito.withSettings().defaultAnswer(Mockito.CALLS_REAL_METHODS)); super.baseSetup(); } @@ -64,7 +52,6 @@ protected void cleanup() throws Exception { protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder builder) { super.customizeMainPulsarTestContextBuilder(builder); builder.enableOpenTelemetry(true); - builder.brokerInterceptor(brokerInterceptor); } @Test(timeOut = 30_000) @@ -78,14 +65,6 @@ public void testMessagingMetrics() throws Exception { var subscriptionName = BrokerTestUtil.newUniqueName("test"); var receiverQueueSize = 100; - // Intercept calls to create consumer, in order to fetch client information. - var consumerRef = new AtomicReference(); - doAnswer(invocation -> { - consumerRef.compareAndSet(null, invocation.getArgument(1)); - return null; - }).when(brokerInterceptor) - .consumerCreated(any(), argThat(arg -> arg.getSubscription().getName().equals(subscriptionName)), any()); - @Cleanup var consumer = pulsarClient.newConsumer() .topic(topicName) @@ -94,12 +73,8 @@ public void testMessagingMetrics() throws Exception { .subscriptionType(SubscriptionType.Shared) .ackTimeout(1, TimeUnit.SECONDS) .receiverQueueSize(receiverQueueSize) - .property("prop1", "value1") .subscribe(); - Awaitility.await().until(() -> consumerRef.get() != null); - var serverConsumer = consumerRef.get(); - @Cleanup var producer = pulsarClient.newProducer() .topic(topicName) @@ -121,11 +96,6 @@ public void testMessagingMetrics() throws Exception { .put(OpenTelemetryAttributes.PULSAR_SUBSCRIPTION_TYPE, SubscriptionType.Shared.toString()) .put(OpenTelemetryAttributes.PULSAR_CONSUMER_NAME, consumer.getConsumerName()) .put(OpenTelemetryAttributes.PULSAR_CONSUMER_ID, 0) - .put(OpenTelemetryAttributes.PULSAR_CONSUMER_CONNECTED_SINCE, - serverConsumer.getConnectedSince().getEpochSecond()) - .put(OpenTelemetryAttributes.PULSAR_CLIENT_ADDRESS, serverConsumer.getClientAddressAndPort()) - .put(OpenTelemetryAttributes.PULSAR_CLIENT_VERSION, serverConsumer.getClientVersion()) - .put(OpenTelemetryAttributes.PULSAR_CONSUMER_METADATA, List.of("prop1:value1")) .build(); Awaitility.await().untilAsserted(() -> { @@ -141,9 +111,9 @@ public void testMessagingMetrics() throws Exception { actual -> assertThat(actual).isGreaterThanOrEqualTo(receiverQueueSize - messageCount - ackCount)); var unAckCount = messageCount - ackCount; - assertMetricLongSumValue(metrics, OpenTelemetryConsumerStats.MESSAGE_UNACKNOWLEDGED_COUNTER, - attributes.toBuilder().put(OpenTelemetryAttributes.PULSAR_CONSUMER_BLOCKED, false).build(), + assertMetricLongSumValue(metrics, OpenTelemetryConsumerStats.MESSAGE_UNACKNOWLEDGED_COUNTER, attributes, unAckCount); + assertMetricLongSumValue(metrics, OpenTelemetryConsumerStats.CONSUMER_BLOCKED_COUNTER, attributes, 0); assertMetricLongSumValue(metrics, OpenTelemetryConsumerStats.MESSAGE_REDELIVER_COUNTER, attributes, actual -> assertThat(actual).isGreaterThanOrEqualTo(unAckCount)); }); diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index 4f898b382e633..a3e8a0c1e725c 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -76,11 +76,6 @@ public interface OpenTelemetryAttributes { */ AttributeKey PULSAR_CONSUMER_ID = AttributeKey.longKey("pulsar.consumer.id"); - /** - * Indicates whether the consumer is currently blocked on unacknowledged messages or not. - */ - AttributeKey PULSAR_CONSUMER_BLOCKED = AttributeKey.booleanKey("pulsar.consumer.blocked"); - /** * The consumer metadata properties, as a list of "key:value" pairs. */ From 342d88dd193bb85c0af91c5193b1422808a9c821 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Wed, 5 Jun 2024 09:51:33 +0800 Subject: [PATCH 266/580] [fix] [broker] disable loadBalancerMemoryResourceWeight by default (#22820) --- conf/broker.conf | 2 +- conf/standalone.conf | 2 +- deployment/terraform-ansible/templates/broker.conf | 2 +- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index d68b6c6ca61de..02e294029893a 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1816,7 +1816,7 @@ strictBookieAffinityEnabled=false # The heap memory usage weight when calculating new resource usage. # It only takes effect in the ThresholdShedder strategy. # Deprecated: Memory is no longer used as a load balancing item -loadBalancerMemoryResourceWeight=1.0 +loadBalancerMemoryResourceWeight=0 # Zookeeper quorum connection string # Deprecated: use metadataStoreUrl instead diff --git a/conf/standalone.conf b/conf/standalone.conf index 1a0c501899d1d..07d19c7bee929 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -924,7 +924,7 @@ loadBalancerCPUResourceWeight=1.0 # The heap memory usage weight when calculating new resource usage. # It only takes effect in the ThresholdShedder strategy. -loadBalancerMemoryResourceWeight=1.0 +loadBalancerMemoryResourceWeight=0 # The direct memory usage weight when calculating new resource usage. # It only takes effect in the ThresholdShedder strategy. diff --git a/deployment/terraform-ansible/templates/broker.conf b/deployment/terraform-ansible/templates/broker.conf index 43bbdc0d52d3e..084d7f46ce1ce 100644 --- a/deployment/terraform-ansible/templates/broker.conf +++ b/deployment/terraform-ansible/templates/broker.conf @@ -967,7 +967,7 @@ loadBalancerCPUResourceWeight=1.0 # The heap memory usage weight when calculating new resource usage. # It only take effect in ThresholdShedder strategy. -loadBalancerMemoryResourceWeight=1.0 +loadBalancerMemoryResourceWeight=0 # The direct memory usage weight when calculating new resource usage. # It only take effect in ThresholdShedder strategy. 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 6e8820db27ca7..204ea453bae59 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 @@ -2469,7 +2469,7 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "Memory Resource Usage Weight. Deprecated: Memory is no longer used as a load balancing item.", deprecated = true ) - private double loadBalancerMemoryResourceWeight = 1.0; + private double loadBalancerMemoryResourceWeight = 0; @FieldContext( dynamic = true, From aece67e35ecec4a9d90a951b78cfc89ca6395054 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 5 Jun 2024 10:49:00 -0700 Subject: [PATCH 267/580] [fix] Remove blocking calls from BookieRackAffinityMapping (#22846) --- .../BookieRackAffinityMapping.java | 44 ++++++++++++------- ...IsolatedBookieEnsemblePlacementPolicy.java | 2 +- 2 files changed, 28 insertions(+), 18 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java index 983822f22941b..4a5ff746f4039 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java @@ -70,7 +70,7 @@ public class BookieRackAffinityMapping extends AbstractDNSToSwitchMapping private BookiesRackConfiguration racksWithHost = new BookiesRackConfiguration(); private Map bookieInfoMap = new HashMap<>(); - public static MetadataStore createMetadataStore(Configuration conf) throws MetadataException { + static MetadataStore getMetadataStore(Configuration conf) throws MetadataException { MetadataStore store; Object storeProperty = conf.getProperty(METADATA_STORE_INSTANCE); if (storeProperty != null) { @@ -116,12 +116,20 @@ public synchronized void setConf(Configuration conf) { super.setConf(conf); MetadataStore store; try { - store = createMetadataStore(conf); - bookieMappingCache = store.getMetadataCache(BookiesRackConfiguration.class); - store.registerListener(this::handleUpdates); - racksWithHost = bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH).get() - .orElseGet(BookiesRackConfiguration::new); - for (Map bookieMapping : racksWithHost.values()) { + store = getMetadataStore(conf); + } catch (MetadataException e) { + throw new RuntimeException(METADATA_STORE_INSTANCE + " failed to init BookieId list"); + } + + bookieMappingCache = store.getMetadataCache(BookiesRackConfiguration.class); + store.registerListener(this::handleUpdates); + + try { + var racksWithHost = bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH) + .thenApply(optRes -> optRes.orElseGet(BookiesRackConfiguration::new)) + .get(); + + for (var bookieMapping : racksWithHost.values()) { for (String address : bookieMapping.keySet()) { bookieAddressListLastTime.add(BookieId.parse(address)); } @@ -131,10 +139,12 @@ public synchronized void setConf(Configuration conf) { } } updateRacksWithHost(racksWithHost); - watchAvailableBookies(); - } catch (InterruptedException | ExecutionException | MetadataException e) { - throw new RuntimeException(METADATA_STORE_INSTANCE + " failed to init BookieId list"); + } catch (ExecutionException | InterruptedException e) { + LOG.error("Failed to update rack info. ", e); + throw new RuntimeException(e); } + + watchAvailableBookies(); } private void watchAvailableBookies() { @@ -145,13 +155,13 @@ private void watchAvailableBookies() { field.setAccessible(true); RegistrationClient registrationClient = (RegistrationClient) field.get(bookieAddressResolver); registrationClient.watchWritableBookies(versioned -> { - try { - racksWithHost = bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH).get() - .orElseGet(BookiesRackConfiguration::new); - updateRacksWithHost(racksWithHost); - } catch (InterruptedException | ExecutionException e) { - LOG.error("Failed to update rack info. ", e); - } + bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH) + .thenApply(optRes -> optRes.orElseGet(BookiesRackConfiguration::new)) + .thenAccept(this::updateRacksWithHost) + .exceptionally(ex -> { + LOG.error("Failed to update rack info. ", ex); + return null; + }); }); } catch (NoSuchFieldException | IllegalAccessException e) { LOG.error("Failed watch available bookies.", e); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java index 8839e6e2d26c8..62b7ffa1e29da 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java @@ -73,7 +73,7 @@ public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration conf, StatsLogger statsLogger, BookieAddressResolver bookieAddressResolver) { MetadataStore store; try { - store = BookieRackAffinityMapping.createMetadataStore(conf); + store = BookieRackAffinityMapping.getMetadataStore(conf); } catch (MetadataException e) { throw new RuntimeException(METADATA_STORE_INSTANCE + " failed initialized"); } From 326e9fa731ae17304621ab915e36d52a9b28a7a0 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Wed, 5 Jun 2024 11:19:12 -0700 Subject: [PATCH 268/580] [fix] [broker] Fix Broker was failing to load stats-internal with broken schema ledger (#22845) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 18e69250c16b8..2165247b1619e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -58,6 +58,8 @@ import javax.annotation.Nonnull; import lombok.Getter; import lombok.Value; +import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException; +import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsOnMetadataServerException; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; @@ -2829,6 +2831,11 @@ public CompletableFuture getInternalStats(boolean }).exceptionally(e -> { log.error("[{}] Failed to get ledger metadata for the schema ledger {}", topic, ledgerId, e); + if ((e.getCause() instanceof BKNoSuchLedgerExistsOnMetadataServerException) + || (e.getCause() instanceof BKNoSuchLedgerExistsException)) { + completableFuture.complete(null); + return null; + } completableFuture.completeExceptionally(e); return null; }); From 74192871ed00870e5181a5bd4018ba196fd8f698 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Jun 2024 22:02:43 +0300 Subject: [PATCH 269/580] [fix][meta] Check if metadata store is closed in RocksdbMetadataStore (#22852) --- .../metadata/impl/AbstractMetadataStore.java | 25 +++++++++---------- .../metadata/impl/RocksdbMetadataStore.java | 15 +++++++++++ 2 files changed, 27 insertions(+), 13 deletions(-) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java index fa827bb40e706..7315e6a04a230 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java @@ -257,8 +257,7 @@ public MetadataCache getMetadataCache(MetadataSerde serde, MetadataCac @Override public CompletableFuture> get(String path) { if (isClosed()) { - return FutureUtil.failedFuture( - new MetadataStoreException.AlreadyClosedException()); + return alreadyClosedFailedFuture(); } long start = System.currentTimeMillis(); if (!isValidPath(path)) { @@ -286,8 +285,7 @@ public CompletableFuture put(String path, byte[] value, Optional exp @Override public final CompletableFuture> getChildren(String path) { if (isClosed()) { - return FutureUtil.failedFuture( - new MetadataStoreException.AlreadyClosedException()); + return alreadyClosedFailedFuture(); } if (!isValidPath(path)) { return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path)); @@ -298,8 +296,7 @@ public final CompletableFuture> getChildren(String path) { @Override public final CompletableFuture exists(String path) { if (isClosed()) { - return FutureUtil.failedFuture( - new MetadataStoreException.AlreadyClosedException()); + return alreadyClosedFailedFuture(); } if (!isValidPath(path)) { return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path)); @@ -362,8 +359,7 @@ public void accept(Notification n) { public final CompletableFuture delete(String path, Optional expectedVersion) { log.info("Deleting path: {} (v. {})", path, expectedVersion); if (isClosed()) { - return FutureUtil.failedFuture( - new MetadataStoreException.AlreadyClosedException()); + return alreadyClosedFailedFuture(); } long start = System.currentTimeMillis(); if (!isValidPath(path)) { @@ -414,8 +410,7 @@ private CompletableFuture deleteInternal(String path, Optional expec public CompletableFuture deleteRecursive(String path) { log.info("Deleting recursively path: {}", path); if (isClosed()) { - return FutureUtil.failedFuture( - new MetadataStoreException.AlreadyClosedException()); + return alreadyClosedFailedFuture(); } return getChildren(path) .thenCompose(children -> FutureUtil.waitForAll( @@ -435,8 +430,7 @@ protected abstract CompletableFuture storePut(String path, byte[] data, Op public final CompletableFuture put(String path, byte[] data, Optional optExpectedVersion, EnumSet options) { if (isClosed()) { - return FutureUtil.failedFuture( - new MetadataStoreException.AlreadyClosedException()); + return alreadyClosedFailedFuture(); } long start = System.currentTimeMillis(); if (!isValidPath(path)) { @@ -516,10 +510,15 @@ protected void receivedSessionEvent(SessionEvent event) { } } - private boolean isClosed() { + protected boolean isClosed() { return isClosed.get(); } + protected static CompletableFuture alreadyClosedFailedFuture() { + return FutureUtil.failedFuture( + new MetadataStoreException.AlreadyClosedException()); + } + @Override public void close() throws Exception { executor.shutdownNow(); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java index 39f7edd5ceed5..06f7b26053693 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java @@ -375,6 +375,9 @@ public CompletableFuture> storeGet(String path) { } try { dbStateLock.readLock().lock(); + if (isClosed()) { + return alreadyClosedFailedFuture(); + } byte[] value = db.get(optionCache, toBytes(path)); if (value == null) { return CompletableFuture.completedFuture(Optional.empty()); @@ -407,6 +410,9 @@ protected CompletableFuture> getChildrenFromStore(String path) { } try { dbStateLock.readLock().lock(); + if (isClosed()) { + return alreadyClosedFailedFuture(); + } try (RocksIterator iterator = db.newIterator(optionDontCache)) { Set result = new HashSet<>(); String firstKey = path.equals("/") ? path : path + "/"; @@ -449,6 +455,9 @@ protected CompletableFuture existsFromStore(String path) { } try { dbStateLock.readLock().lock(); + if (isClosed()) { + return alreadyClosedFailedFuture(); + } byte[] value = db.get(optionDontCache, toBytes(path)); if (log.isDebugEnabled()) { if (value != null) { @@ -471,6 +480,9 @@ protected CompletableFuture storeDelete(String path, Optional expect } try { dbStateLock.readLock().lock(); + if (isClosed()) { + return alreadyClosedFailedFuture(); + } try (Transaction transaction = db.beginTransaction(writeOptions)) { byte[] pathBytes = toBytes(path); byte[] oldValueData = transaction.getForUpdate(optionDontCache, pathBytes, true); @@ -507,6 +519,9 @@ protected CompletableFuture storePut(String path, byte[] data, Optional Date: Thu, 6 Jun 2024 00:41:15 +0300 Subject: [PATCH 270/580] [improve][build] Support git worktree working directory while building docker images (#22851) --- docker/pulsar-all/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index f54e95fd8857c..6a957d6f4623c 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -70,7 +70,7 @@ git-commit-id-no-git - ${basedir}/../../.git/index + ${basedir}/../../.git diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 0cf4535b19505..228c2b810313d 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -52,7 +52,7 @@ git-commit-id-no-git - ${basedir}/../../.git/index + ${basedir}/../../.git From 4341f0f301e0da344bb5ce07bc62c373e7ce48ef Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 5 Jun 2024 16:34:56 -0700 Subject: [PATCH 271/580] [feat][broker] PIP-264: Add broker web executor metrics (#22816) --- .../web/WebExecutorThreadPoolStats.java | 83 +++++++++++++++++++ .../pulsar/broker/web/WebExecutorStats.java | 7 ++ .../apache/pulsar/broker/web/WebService.java | 5 ++ .../pulsar/broker/web/WebServiceTest.java | 18 ++++ 4 files changed, 113 insertions(+) create mode 100644 pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/WebExecutorThreadPoolStats.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/WebExecutorThreadPoolStats.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/WebExecutorThreadPoolStats.java new file mode 100644 index 0000000000000..6bfe4e33b8e5b --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/WebExecutorThreadPoolStats.java @@ -0,0 +1,83 @@ +/* + * 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.web; + +import com.google.common.annotations.VisibleForTesting; +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.api.metrics.ObservableLongUpDownCounter; + +public class WebExecutorThreadPoolStats implements AutoCloseable { + // Replaces ['pulsar_web_executor_max_threads', 'pulsar_web_executor_min_threads'] + public static final String LIMIT_COUNTER = "pulsar.web.executor.thread.limit"; + private final ObservableLongUpDownCounter limitCounter; + + // Replaces + // ['pulsar_web_executor_active_threads', 'pulsar_web_executor_current_threads', 'pulsar_web_executor_idle_threads'] + public static final String USAGE_COUNTER = "pulsar.web.executor.thread.usage"; + private final ObservableLongUpDownCounter usageCounter; + + public static final AttributeKey LIMIT_TYPE_KEY = + AttributeKey.stringKey("pulsar.web.executor.thread.limit.type"); + @VisibleForTesting + enum LimitType { + MAX, + MIN; + public final Attributes attributes = Attributes.of(LIMIT_TYPE_KEY, name().toLowerCase()); + } + + public static final AttributeKey USAGE_TYPE_KEY = + AttributeKey.stringKey("pulsar.web.executor.thread.usage.type"); + @VisibleForTesting + enum UsageType { + ACTIVE, + CURRENT, + IDLE; + public final Attributes attributes = Attributes.of(USAGE_TYPE_KEY, name().toLowerCase()); + } + + public WebExecutorThreadPoolStats(Meter meter, WebExecutorThreadPool executor) { + limitCounter = meter + .upDownCounterBuilder(LIMIT_COUNTER) + .setUnit("{thread}") + .setDescription("The thread limits for the pulsar-web executor pool.") + .buildWithCallback(measurement -> { + measurement.record(executor.getMaxThreads(), LimitType.MAX.attributes); + measurement.record(executor.getMinThreads(), LimitType.MIN.attributes); + }); + usageCounter = meter + .upDownCounterBuilder(USAGE_COUNTER) + .setUnit("{thread}") + .setDescription("The current usage of threads in the pulsar-web executor pool.") + .buildWithCallback(measurement -> { + var idleThreads = executor.getIdleThreads(); + var currentThreads = executor.getThreads(); + measurement.record(idleThreads, UsageType.IDLE.attributes); + measurement.record(currentThreads, UsageType.CURRENT.attributes); + measurement.record(currentThreads - idleThreads, UsageType.ACTIVE.attributes); + }); + } + + @Override + public synchronized void close() { + limitCounter.close(); + usageCounter.close(); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebExecutorStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebExecutorStats.java index 585df813027d7..28cfa7430cbe6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebExecutorStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebExecutorStats.java @@ -21,14 +21,21 @@ import io.prometheus.client.CollectorRegistry; import io.prometheus.client.Gauge; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric; +@Deprecated class WebExecutorStats implements AutoCloseable { private static final AtomicBoolean CLOSED = new AtomicBoolean(false); + @PulsarDeprecatedMetric(newMetricName = WebExecutorThreadPoolStats.LIMIT_COUNTER) private final Gauge maxThreads; + @PulsarDeprecatedMetric(newMetricName = WebExecutorThreadPoolStats.LIMIT_COUNTER) private final Gauge minThreads; + @PulsarDeprecatedMetric(newMetricName = WebExecutorThreadPoolStats.USAGE_COUNTER) private final Gauge idleThreads; + @PulsarDeprecatedMetric(newMetricName = WebExecutorThreadPoolStats.USAGE_COUNTER) private final Gauge activeThreads; + @PulsarDeprecatedMetric(newMetricName = WebExecutorThreadPoolStats.USAGE_COUNTER) private final Gauge currentThreads; private final WebExecutorThreadPool executor; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java index 9a439268a8b4f..bf484d4f41f65 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java @@ -75,7 +75,9 @@ public class WebService implements AutoCloseable { private final PulsarService pulsar; private final Server server; private final List handlers; + @Deprecated private final WebExecutorStats executorStats; + private final WebExecutorThreadPoolStats webExecutorThreadPoolStats; private final WebExecutorThreadPool webServiceExecutor; private final ServerConnector httpConnector; @@ -101,6 +103,8 @@ public WebService(PulsarService pulsar) throws PulsarServerException { "pulsar-web", config.getHttpServerThreadPoolQueueSize()); this.executorStats = WebExecutorStats.getStats(webServiceExecutor); + this.webExecutorThreadPoolStats = + new WebExecutorThreadPoolStats(pulsar.getOpenTelemetry().getMeter(), webServiceExecutor); this.server = new Server(webServiceExecutor); if (config.getMaxHttpServerConnections() > 0) { server.addBean(new ConnectionLimit(config.getMaxHttpServerConnections(), server)); @@ -376,6 +380,7 @@ public void close() throws PulsarServerException { jettyStatisticsCollector = null; } webServiceExecutor.join(); + webExecutorThreadPoolStats.close(); this.executorStats.close(); log.info("Web service closed"); } catch (Exception e) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index 17588a7ecac8b..30644237a7405 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -18,8 +18,10 @@ */ package org.apache.pulsar.broker.web; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -59,6 +61,8 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.broker.web.WebExecutorThreadPoolStats.LimitType; +import org.apache.pulsar.broker.web.WebExecutorThreadPoolStats.UsageType; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; import org.apache.pulsar.client.admin.PulsarAdminException.ConflictException; @@ -106,6 +110,19 @@ public class WebServiceTest { @Test public void testWebExecutorMetrics() throws Exception { setupEnv(true, false, false, false, -1, false); + + var otelMetrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(otelMetrics, WebExecutorThreadPoolStats.LIMIT_COUNTER, LimitType.MAX.attributes, + value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, WebExecutorThreadPoolStats.LIMIT_COUNTER, LimitType.MIN.attributes, + value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, WebExecutorThreadPoolStats.USAGE_COUNTER, UsageType.ACTIVE.attributes, + value -> assertThat(value).isNotNegative()); + assertMetricLongSumValue(otelMetrics, WebExecutorThreadPoolStats.USAGE_COUNTER, UsageType.CURRENT.attributes, + value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, WebExecutorThreadPoolStats.USAGE_COUNTER, UsageType.IDLE.attributes, + value -> assertThat(value).isNotNegative()); + ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); @@ -498,6 +515,7 @@ private void setupEnv(boolean enableFilter, boolean enableTls, boolean enableAut pulsarTestContext = PulsarTestContext.builder() .spyByDefault() .config(config) + .enableOpenTelemetry(true) .build(); pulsar = pulsarTestContext.getPulsarService(); From d74010c271abfb0a77a4dacf0ab072a957afeb5a Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 5 Jun 2024 17:09:32 -0700 Subject: [PATCH 272/580] [improve] Refactored BK ClientFactory to return futures (#22853) --- .../impl/ManagedLedgerFactoryImpl.java | 223 +++++++++--------- .../impl/ManagedLedgerOfflineBacklog.java | 20 +- .../broker/BookKeeperClientFactory.java | 19 +- .../broker/BookKeeperClientFactoryImpl.java | 28 ++- .../broker/ManagedLedgerClientFactory.java | 39 ++- .../BookkeeperBucketSnapshotStorage.java | 2 +- .../schema/BookkeeperSchemaStorage.java | 2 +- .../pulsar/compaction/CompactorTool.java | 2 +- .../broker/MockedBookKeeperClientFactory.java | 18 +- .../MockBookKeeperClientFactory.java | 15 +- .../pulsar/compaction/CompactedTopicTest.java | 6 +- .../compaction/CompactionRetentionTest.java | 2 +- .../pulsar/compaction/CompactionTest.java | 2 +- .../pulsar/compaction/CompactorTest.java | 2 +- .../ServiceUnitStateCompactionTest.java | 2 +- .../TopicCompactionServiceTest.java | 2 +- 16 files changed, 193 insertions(+), 191 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index d867f2f4c0221..ed803a81462e1 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -161,7 +161,7 @@ public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, BookKeeper public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, BookKeeper bookKeeper, ManagedLedgerFactoryConfig config) throws Exception { - this(metadataStore, (policyConfig) -> bookKeeper, config); + this(metadataStore, (policyConfig) -> CompletableFuture.completedFuture(bookKeeper), config); } public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, @@ -233,8 +233,8 @@ public DefaultBkFactory(ClientConfiguration bkClientConfiguration) } @Override - public BookKeeper get(EnsemblePlacementPolicyConfig policy) { - return bkClient; + public CompletableFuture get(EnsemblePlacementPolicyConfig policy) { + return CompletableFuture.completedFuture(bkClient); } } @@ -378,56 +378,63 @@ public void asyncOpen(final String name, final ManagedLedgerConfig config, final ledgers.computeIfAbsent(name, (mlName) -> { // Create the managed ledger CompletableFuture future = new CompletableFuture<>(); - BookKeeper bk = bookkeeperFactory.get( - new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), - config.getBookKeeperEnsemblePlacementPolicyProperties())); - final ManagedLedgerImpl newledger = config.getShadowSource() == null - ? new ManagedLedgerImpl(this, bk, store, config, scheduledExecutor, name, mlOwnershipChecker) - : new ShadowManagedLedgerImpl(this, bk, store, config, scheduledExecutor, name, - mlOwnershipChecker); - PendingInitializeManagedLedger pendingLedger = new PendingInitializeManagedLedger(newledger); - pendingInitializeLedgers.put(name, pendingLedger); - newledger.initialize(new ManagedLedgerInitializeLedgerCallback() { - @Override - public void initializeComplete() { - log.info("[{}] Successfully initialize managed ledger", name); - pendingInitializeLedgers.remove(name, pendingLedger); - future.complete(newledger); - - // May need to update the cursor position - newledger.maybeUpdateCursorBeforeTrimmingConsumedLedger(); - // May need to trigger offloading - if (config.isTriggerOffloadOnTopicLoad()) { - newledger.maybeOffloadInBackground(NULL_OFFLOAD_PROMISE); - } - } - - @Override - public void initializeFailed(ManagedLedgerException e) { - if (config.isCreateIfMissing()) { - log.error("[{}] Failed to initialize managed ledger: {}", name, e.getMessage()); - } - - // Clean the map if initialization fails - ledgers.remove(name, future); - - if (pendingInitializeLedgers.remove(name, pendingLedger)) { - pendingLedger.ledger.asyncClose(new CloseCallback() { + bookkeeperFactory.get( + new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), + config.getBookKeeperEnsemblePlacementPolicyProperties())) + .thenAccept(bk -> { + final ManagedLedgerImpl newledger = config.getShadowSource() == null + ? new ManagedLedgerImpl(this, bk, store, config, scheduledExecutor, name, + mlOwnershipChecker) + : new ShadowManagedLedgerImpl(this, bk, store, config, scheduledExecutor, name, + mlOwnershipChecker); + PendingInitializeManagedLedger pendingLedger = new PendingInitializeManagedLedger(newledger); + pendingInitializeLedgers.put(name, pendingLedger); + newledger.initialize(new ManagedLedgerInitializeLedgerCallback() { @Override - public void closeComplete(Object ctx) { - // no-op + public void initializeComplete() { + log.info("[{}] Successfully initialize managed ledger", name); + pendingInitializeLedgers.remove(name, pendingLedger); + future.complete(newledger); + + // May need to update the cursor position + newledger.maybeUpdateCursorBeforeTrimmingConsumedLedger(); + // May need to trigger offloading + if (config.isTriggerOffloadOnTopicLoad()) { + newledger.maybeOffloadInBackground(NULL_OFFLOAD_PROMISE); + } } @Override - public void closeFailed(ManagedLedgerException exception, Object ctx) { - log.warn("[{}] Failed to a pending initialization managed ledger", name, exception); + public void initializeFailed(ManagedLedgerException e) { + if (config.isCreateIfMissing()) { + log.error("[{}] Failed to initialize managed ledger: {}", name, e.getMessage()); + } + + // Clean the map if initialization fails + ledgers.remove(name, future); + + if (pendingInitializeLedgers.remove(name, pendingLedger)) { + pendingLedger.ledger.asyncClose(new CloseCallback() { + @Override + public void closeComplete(Object ctx) { + // no-op + } + + @Override + public void closeFailed(ManagedLedgerException exception, Object ctx) { + log.warn("[{}] Failed to a pending initialization managed ledger", name, + exception); + } + }, null); + } + + future.completeExceptionally(e); } }, null); - } - - future.completeExceptionally(e); - } - }, null); + }).exceptionally(ex -> { + future.completeExceptionally(ex); + return null; + }); return future; }).thenAccept(ml -> callback.openLedgerComplete(ml, ctx)).exceptionally(exception -> { callback.openLedgerFailed((ManagedLedgerException) exception.getCause(), ctx); @@ -443,20 +450,22 @@ public void asyncOpenReadOnlyManagedLedger(String managedLedgerName, callback.openReadOnlyManagedLedgerFailed( new ManagedLedgerException.ManagedLedgerFactoryClosedException(), ctx); } - ReadOnlyManagedLedgerImpl roManagedLedger = new ReadOnlyManagedLedgerImpl(this, - bookkeeperFactory - .get(new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), - config.getBookKeeperEnsemblePlacementPolicyProperties())), - store, config, scheduledExecutor, managedLedgerName); - roManagedLedger.initialize().thenRun(() -> { - log.info("[{}] Successfully initialize Read-only managed ledger", managedLedgerName); - callback.openReadOnlyManagedLedgerComplete(roManagedLedger, ctx); - - }).exceptionally(e -> { - log.error("[{}] Failed to initialize Read-only managed ledger", managedLedgerName, e); - callback.openReadOnlyManagedLedgerFailed((ManagedLedgerException) e.getCause(), ctx); - return null; - }); + + bookkeeperFactory + .get(new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), + config.getBookKeeperEnsemblePlacementPolicyProperties())) + .thenCompose(bk -> { + ReadOnlyManagedLedgerImpl roManagedLedger = new ReadOnlyManagedLedgerImpl(this, bk, + store, config, scheduledExecutor, managedLedgerName); + return roManagedLedger.initialize().thenApply(v -> roManagedLedger); + }).thenAccept(roManagedLedger -> { + log.info("[{}] Successfully initialize Read-only managed ledger", managedLedgerName); + callback.openReadOnlyManagedLedgerComplete(roManagedLedger, ctx); + }).exceptionally(e -> { + log.error("[{}] Failed to initialize Read-only managed ledger", managedLedgerName, e); + callback.openReadOnlyManagedLedgerFailed((ManagedLedgerException) e.getCause(), ctx); + return null; + }); } @Override @@ -578,49 +587,35 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { ledgerFuture.completeExceptionally(new ManagedLedgerException.ManagedLedgerFactoryClosedException()); } } - CompletableFuture bookkeeperFuture = new CompletableFuture<>(); - futures.add(bookkeeperFuture); - futures.add(CompletableFuture.runAsync(() -> { - if (isBookkeeperManaged) { - try { - BookKeeper bookkeeper = bookkeeperFactory.get(); - if (bookkeeper != null) { - bookkeeper.close(); - } - bookkeeperFuture.complete(null); - } catch (Throwable throwable) { - bookkeeperFuture.completeExceptionally(throwable); - } - } else { - bookkeeperFuture.complete(null); - } - if (!ledgers.isEmpty()) { - log.info("Force closing {} ledgers.", ledgers.size()); - //make sure all callbacks is called. - ledgers.forEach(((ledgerName, ledgerFuture) -> { - if (!ledgerFuture.isDone()) { - ledgerFuture.completeExceptionally( - new ManagedLedgerException.ManagedLedgerFactoryClosedException()); - } else { - ManagedLedgerImpl managedLedger = ledgerFuture.getNow(null); - if (managedLedger == null) { - return; - } - try { - managedLedger.close(); - } catch (Throwable throwable) { - log.warn("[{}] Got exception when closing managed ledger: {}", managedLedger.getName(), - throwable); + CompletableFuture bookkeeperFuture = isBookkeeperManaged + ? bookkeeperFactory.get() + : CompletableFuture.completedFuture(null); + return bookkeeperFuture + .thenRun(() -> { + log.info("Closing {} ledgers.", ledgers.size()); + //make sure all callbacks is called. + ledgers.forEach(((ledgerName, ledgerFuture) -> { + if (!ledgerFuture.isDone()) { + ledgerFuture.completeExceptionally( + new ManagedLedgerException.ManagedLedgerFactoryClosedException()); + } else { + ManagedLedgerImpl managedLedger = ledgerFuture.getNow(null); + if (managedLedger == null) { + return; + } + try { + managedLedger.close(); + } catch (Throwable throwable) { + log.warn("[{}] Got exception when closing managed ledger: {}", managedLedger.getName(), + throwable); + } } - } - })); - } - })); - return FutureUtil.waitForAll(futures).thenAcceptAsync(__ -> { - //wait for tasks in scheduledExecutor executed. - scheduledExecutor.shutdownNow(); - entryCacheManager.clear(); - }); + })); + }).thenAcceptAsync(__ -> { + //wait for tasks in scheduledExecutor executed. + scheduledExecutor.shutdownNow(); + entryCacheManager.clear(); + }); } @Override @@ -861,14 +856,14 @@ void deleteManagedLedger(String managedLedgerName, CompletableFuture> futures = info.cursors.entrySet().stream() - .map(e -> deleteCursor(bkc, managedLedgerName, e.getKey(), e.getValue())) - .collect(Collectors.toList()); - Futures.waitForAll(futures).thenRun(() -> { - deleteManagedLedgerData(bkc, managedLedgerName, info, mlConfigFuture, callback, ctx); + getBookKeeper().thenCompose(bk -> { + // First delete all cursors resources + List> futures = info.cursors.entrySet().stream() + .map(e -> deleteCursor(bk, managedLedgerName, e.getKey(), e.getValue())) + .collect(Collectors.toList()); + return Futures.waitForAll(futures).thenApply(v -> bk); + }).thenAccept(bk -> { + deleteManagedLedgerData(bk, managedLedgerName, info, mlConfigFuture, callback, ctx); }).exceptionally(ex -> { callback.deleteLedgerFailed(new ManagedLedgerException(ex), ctx); return null; @@ -1053,7 +1048,7 @@ public ManagedLedgerFactoryMXBean getCacheStats() { return this.mbean; } - public BookKeeper getBookKeeper() { + public CompletableFuture getBookKeeper() { return bookkeeperFactory.get(); } @@ -1062,7 +1057,7 @@ public BookKeeper getBookKeeper() { * */ public interface BookkeeperFactoryForCustomEnsemblePlacementPolicy { - default BookKeeper get() { + default CompletableFuture get() { return get(null); } @@ -1073,7 +1068,7 @@ default BookKeeper get() { * @param ensemblePlacementPolicyMetadata * @return */ - BookKeeper get(EnsemblePlacementPolicyConfig ensemblePlacementPolicyMetadata); + CompletableFuture get(EnsemblePlacementPolicyConfig ensemblePlacementPolicyMetadata); } private static final Logger log = LoggerFactory.getLogger(ManagedLedgerFactoryImpl.class); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java index a271d439e0609..81cd94e5bf96c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java @@ -140,7 +140,7 @@ private void readLedgerMeta(final ManagedLedgerFactoryImpl factory, final TopicN final NavigableMap ledgers) throws Exception { String managedLedgerName = topicName.getPersistenceNamingEncoding(); MetaStore store = factory.getMetaStore(); - BookKeeper bk = factory.getBookKeeper(); + final CountDownLatch mlMetaCounter = new CountDownLatch(1); store.getManagedLedgerInfo(managedLedgerName, false /* createIfMissing */, @@ -180,12 +180,16 @@ public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) if (log.isDebugEnabled()) { log.debug("[{}] Opening ledger {}", managedLedgerName, id); } - try { - bk.asyncOpenLedgerNoRecovery(id, digestType, password, opencb, null); - } catch (Exception e) { - log.warn("[{}] Failed to open ledger {}: {}", managedLedgerName, id, e); - mlMetaCounter.countDown(); - } + + factory.getBookKeeper() + .thenAccept(bk -> { + bk.asyncOpenLedgerNoRecovery(id, digestType, password, opencb, null); + }).exceptionally(ex -> { + log.warn("[{}] Failed to open ledger {}: {}", managedLedgerName, id, ex); + opencb.openComplete(-1, null, null); + mlMetaCounter.countDown(); + return null; + }); } else { log.warn("[{}] Ledger list empty", managedLedgerName); mlMetaCounter.countDown(); @@ -217,7 +221,7 @@ private void calculateCursorBacklogs(final ManagedLedgerFactoryImpl factory, fin } String managedLedgerName = topicName.getPersistenceNamingEncoding(); MetaStore store = factory.getMetaStore(); - BookKeeper bk = factory.getBookKeeper(); + BookKeeper bk = factory.getBookKeeper().get(); final CountDownLatch allCursorsCounter = new CountDownLatch(1); final long errorInReadingCursor = -1; ConcurrentOpenHashMap ledgerRetryMap = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactory.java index 95923baac0294..5ab1a01838df7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactory.java @@ -19,9 +19,9 @@ package org.apache.pulsar.broker; import io.netty.channel.EventLoopGroup; -import java.io.IOException; import java.util.Map; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.EnsemblePlacementPolicy; import org.apache.bookkeeper.stats.StatsLogger; @@ -31,13 +31,16 @@ * Provider of a new BookKeeper client instance. */ public interface BookKeeperClientFactory { - BookKeeper create(ServiceConfiguration conf, MetadataStoreExtended store, EventLoopGroup eventLoopGroup, - Optional> ensemblePlacementPolicyClass, - Map ensemblePlacementPolicyProperties) throws IOException; + CompletableFuture create(ServiceConfiguration conf, MetadataStoreExtended store, + EventLoopGroup eventLoopGroup, + Optional> policyClass, + Map ensemblePlacementPolicyProperties); + + CompletableFuture create(ServiceConfiguration conf, MetadataStoreExtended store, + EventLoopGroup eventLoopGroup, + Optional> policyClass, + Map ensemblePlacementPolicyProperties, + StatsLogger statsLogger); - BookKeeper create(ServiceConfiguration conf, MetadataStoreExtended store, EventLoopGroup eventLoopGroup, - Optional> ensemblePlacementPolicyClass, - Map ensemblePlacementPolicyProperties, - StatsLogger statsLogger) throws IOException; void close(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java index e5293cee24e4a..45299d9ed05d5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java @@ -29,6 +29,7 @@ import java.io.IOException; import java.util.Map; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BKException; @@ -53,19 +54,19 @@ public class BookKeeperClientFactoryImpl implements BookKeeperClientFactory { @Override - public BookKeeper create(ServiceConfiguration conf, MetadataStoreExtended store, - EventLoopGroup eventLoopGroup, - Optional> ensemblePlacementPolicyClass, - Map properties) throws IOException { - return create(conf, store, eventLoopGroup, ensemblePlacementPolicyClass, properties, + public CompletableFuture create(ServiceConfiguration conf, MetadataStoreExtended store, + EventLoopGroup eventLoopGroup, + Optional> policyClass, + Map properties) { + return create(conf, store, eventLoopGroup, policyClass, properties, NullStatsLogger.INSTANCE); } @Override - public BookKeeper create(ServiceConfiguration conf, MetadataStoreExtended store, + public CompletableFuture create(ServiceConfiguration conf, MetadataStoreExtended store, EventLoopGroup eventLoopGroup, Optional> ensemblePlacementPolicyClass, - Map properties, StatsLogger statsLogger) throws IOException { + Map properties, StatsLogger statsLogger) { PulsarMetadataClientDriver.init(); ClientConfiguration bkConf = createBkClientConfiguration(store, conf); @@ -77,11 +78,14 @@ public BookKeeper create(ServiceConfiguration conf, MetadataStoreExtended store, } else { setDefaultEnsemblePlacementPolicy(bkConf, conf, store); } - try { - return getBookKeeperBuilder(conf, eventLoopGroup, statsLogger, bkConf).build(); - } catch (InterruptedException | BKException e) { - throw new IOException(e); - } + + return CompletableFuture.supplyAsync(() -> { + try { + return getBookKeeperBuilder(conf, eventLoopGroup, statsLogger, bkConf).build(); + } catch (InterruptedException | BKException | IOException e) { + throw new RuntimeException(e); + } + }); } @VisibleForTesting diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java index 8861b12f0c113..6ed95f167a15a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java @@ -18,12 +18,14 @@ */ package org.apache.pulsar.broker; +import com.github.benmanes.caffeine.cache.AsyncCache; +import com.github.benmanes.caffeine.cache.Caffeine; import com.google.common.annotations.VisibleForTesting; import io.netty.channel.EventLoopGroup; import java.io.IOException; import java.util.Map; import java.util.Optional; -import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.RejectedExecutionException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.conf.ClientConfiguration; @@ -48,8 +50,8 @@ public class ManagedLedgerClientFactory implements ManagedLedgerStorage { private ManagedLedgerFactory managedLedgerFactory; private BookKeeper defaultBkClient; - private final Map - bkEnsemblePolicyToBkClientMap = new ConcurrentHashMap<>(); + private final AsyncCache + bkEnsemblePolicyToBkClientMap = Caffeine.newBuilder().buildAsync(); private StatsProvider statsProvider = new NullStatsProvider(); public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadataStore, @@ -89,27 +91,20 @@ public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadata StatsLogger statsLogger = statsProvider.getStatsLogger("pulsar_managedLedger_client"); this.defaultBkClient = - bookkeeperProvider.create(conf, metadataStore, eventLoopGroup, Optional.empty(), null, statsLogger); + bookkeeperProvider.create(conf, metadataStore, eventLoopGroup, Optional.empty(), null, statsLogger) + .get(); BookkeeperFactoryForCustomEnsemblePlacementPolicy bkFactory = ( EnsemblePlacementPolicyConfig ensemblePlacementPolicyConfig) -> { - BookKeeper bkClient = null; - // find or create bk-client in cache for a specific ensemblePlacementPolicy - if (ensemblePlacementPolicyConfig != null && ensemblePlacementPolicyConfig.getPolicyClass() != null) { - bkClient = bkEnsemblePolicyToBkClientMap.computeIfAbsent(ensemblePlacementPolicyConfig, (key) -> { - try { - return bookkeeperProvider.create(conf, metadataStore, eventLoopGroup, - Optional.ofNullable(ensemblePlacementPolicyConfig.getPolicyClass()), - ensemblePlacementPolicyConfig.getProperties(), statsLogger); - } catch (Exception e) { - log.error("Failed to initialize bk-client for policy {}, properties {}", - ensemblePlacementPolicyConfig.getPolicyClass(), - ensemblePlacementPolicyConfig.getProperties(), e); - } - return this.defaultBkClient; - }); + if (ensemblePlacementPolicyConfig == null || ensemblePlacementPolicyConfig.getPolicyClass() == null) { + return CompletableFuture.completedFuture(defaultBkClient); } - return bkClient != null ? bkClient : defaultBkClient; + + // find or create bk-client in cache for a specific ensemblePlacementPolicy + return bkEnsemblePolicyToBkClientMap.get(ensemblePlacementPolicyConfig, + (config, executor) -> bookkeeperProvider.create(conf, metadataStore, eventLoopGroup, + Optional.ofNullable(ensemblePlacementPolicyConfig.getPolicyClass()), + ensemblePlacementPolicyConfig.getProperties(), statsLogger)); }; try { @@ -136,7 +131,7 @@ public StatsProvider getStatsProvider() { @VisibleForTesting public Map getBkEnsemblePolicyToBookKeeperMap() { - return bkEnsemblePolicyToBkClientMap; + return bkEnsemblePolicyToBkClientMap.synchronous().asMap(); } @Override @@ -164,7 +159,7 @@ public void close() throws IOException { // factory, however that might be introducing more unknowns. log.warn("Encountered exceptions on closing bookkeeper client", ree); } - bkEnsemblePolicyToBkClientMap.forEach((policy, bk) -> { + bkEnsemblePolicyToBkClientMap.synchronous().asMap().forEach((policy, bk) -> { try { if (bk != null) { bk.close(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BookkeeperBucketSnapshotStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BookkeeperBucketSnapshotStorage.java index e99f39b382f56..8dcfe8d39a8b4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BookkeeperBucketSnapshotStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BookkeeperBucketSnapshotStorage.java @@ -107,7 +107,7 @@ public void start() throws Exception { pulsar.getIoEventLoopGroup(), Optional.empty(), null - ); + ).get(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java index acdd906f6b8af..99f0249b304b3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java @@ -110,7 +110,7 @@ public void start() throws IOException { pulsar.getIoEventLoopGroup(), Optional.empty(), null - ); + ).join(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java index 3225f7294d5a0..7d35c2c0f7b9e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java @@ -165,7 +165,7 @@ public static void main(String[] args) throws Exception { new DefaultThreadFactory("compactor-io")); @Cleanup - BookKeeper bk = bkClientFactory.create(brokerConfig, store, eventLoopGroup, Optional.empty(), null); + BookKeeper bk = bkClientFactory.create(brokerConfig, store, eventLoopGroup, Optional.empty(), null).get(); @Cleanup PulsarClient pulsar = createClient(brokerConfig); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/MockedBookKeeperClientFactory.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/MockedBookKeeperClientFactory.java index 6d65687a501df..887e35e2774f7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/MockedBookKeeperClientFactory.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/MockedBookKeeperClientFactory.java @@ -19,9 +19,9 @@ package org.apache.pulsar.broker; import io.netty.channel.EventLoopGroup; -import java.io.IOException; import java.util.Map; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.EnsemblePlacementPolicy; @@ -51,19 +51,19 @@ public MockedBookKeeperClientFactory() { } @Override - public BookKeeper create(ServiceConfiguration conf, MetadataStoreExtended store, - EventLoopGroup eventLoopGroup, - Optional> ensemblePlacementPolicyClass, - Map properties) throws IOException { - return mockedBk; + public CompletableFuture create(ServiceConfiguration conf, MetadataStoreExtended store, + EventLoopGroup eventLoopGroup, + Optional> ensemblePlacementPolicyClass, + Map properties) { + return CompletableFuture.completedFuture(mockedBk); } @Override - public BookKeeper create(ServiceConfiguration conf, MetadataStoreExtended store, + public CompletableFuture create(ServiceConfiguration conf, MetadataStoreExtended store, EventLoopGroup eventLoopGroup, Optional> ensemblePlacementPolicyClass, - Map properties, StatsLogger statsLogger) throws IOException { - return mockedBk; + Map properties, StatsLogger statsLogger) { + return CompletableFuture.completedFuture(mockedBk); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/MockBookKeeperClientFactory.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/MockBookKeeperClientFactory.java index fd457687323bf..5f02fd7af48f0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/MockBookKeeperClientFactory.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/MockBookKeeperClientFactory.java @@ -21,6 +21,7 @@ import io.netty.channel.EventLoopGroup; import java.util.Map; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.EnsemblePlacementPolicy; import org.apache.bookkeeper.stats.StatsLogger; @@ -39,21 +40,21 @@ class MockBookKeeperClientFactory implements BookKeeperClientFactory { } @Override - public BookKeeper create(ServiceConfiguration conf, MetadataStoreExtended store, - EventLoopGroup eventLoopGroup, - Optional> ensemblePlacementPolicyClass, - Map properties) { + public CompletableFuture create(ServiceConfiguration conf, MetadataStoreExtended store, + EventLoopGroup eventLoopGroup, + Optional> ensemblePlacementPolicyClass, + Map properties) { // Always return the same instance (so that we don't loose the mock BK content on broker restart - return mockBookKeeper; + return CompletableFuture.completedFuture(mockBookKeeper); } @Override - public BookKeeper create(ServiceConfiguration conf, MetadataStoreExtended store, + public CompletableFuture create(ServiceConfiguration conf, MetadataStoreExtended store, EventLoopGroup eventLoopGroup, Optional> ensemblePlacementPolicyClass, Map properties, StatsLogger statsLogger) { // Always return the same instance (so that we don't loose the mock BK content on broker restart - return mockBookKeeper; + return CompletableFuture.completedFuture(mockBookKeeper); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java index e955a433ad5e1..3cca85aa2f1b6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java @@ -163,7 +163,7 @@ public void cleanup() throws Exception { public void testEntryLookup() throws Exception { @Cleanup BookKeeper bk = pulsar.getBookKeeperClientFactory().create( - this.conf, null, null, Optional.empty(), null); + this.conf, null, null, Optional.empty(), null).get(); Triple>, List>> compactedLedgerData = buildCompactedLedger(bk, 500); @@ -219,7 +219,7 @@ public void testEntryLookup() throws Exception { public void testCleanupOldCompactedTopicLedger() throws Exception { @Cleanup BookKeeper bk = pulsar.getBookKeeperClientFactory().create( - this.conf, null, null, Optional.empty(), null); + this.conf, null, null, Optional.empty(), null).get(); LedgerHandle oldCompactedLedger = bk.createLedger(1, 1, Compactor.COMPACTED_TOPIC_LEDGER_DIGEST_TYPE, @@ -849,7 +849,7 @@ public void testReadCompactedLatestMessageWithInclusive() throws Exception { public void testCompactWithConcurrentGetCompactionHorizonAndCompactedTopicContext() throws Exception { @Cleanup BookKeeper bk = pulsar.getBookKeeperClientFactory().create( - this.conf, null, null, Optional.empty(), null); + this.conf, null, null, Optional.empty(), null).get(); Mockito.doAnswer(invocation -> { Thread.sleep(1500); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java index 98bf2b819c2ba..ac1ba6bc814b1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java @@ -78,7 +78,7 @@ public void setup() throws Exception { compactionScheduler = Executors.newSingleThreadScheduledExecutor( new ThreadFactoryBuilder().setNameFormat("compaction-%d").setDaemon(true).build()); - bk = pulsar.getBookKeeperClientFactory().create(this.conf, null, null, Optional.empty(), null); + bk = pulsar.getBookKeeperClientFactory().create(this.conf, null, null, Optional.empty(), null).get(); compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java index f0010096b1e52..081831b0300e0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java @@ -123,7 +123,7 @@ public void setup() throws Exception { compactionScheduler = Executors.newSingleThreadScheduledExecutor( new ThreadFactoryBuilder().setNameFormat("compaction-%d").setDaemon(true).build()); - bk = pulsar.getBookKeeperClientFactory().create(this.conf, null, null, Optional.empty(), null); + bk = pulsar.getBookKeeperClientFactory().create(this.conf, null, null, Optional.empty(), null).get(); compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java index debc3dd5e3f98..16945a60f5d47 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java @@ -100,7 +100,7 @@ public void setup() throws Exception { compactionScheduler = Executors.newSingleThreadScheduledExecutor( new ThreadFactoryBuilder().setNameFormat("compactor").setDaemon(true).build()); bk = pulsar.getBookKeeperClientFactory().create( - this.conf, null, null, Optional.empty(), null); + this.conf, null, null, Optional.empty(), null).get(); compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java index b3a48f405474c..9140216810826 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java @@ -155,7 +155,7 @@ public void setup() throws Exception { compactionScheduler = Executors.newSingleThreadScheduledExecutor( new ThreadFactoryBuilder().setNameFormat("compaction-%d").setDaemon(true).build()); - bk = pulsar.getBookKeeperClientFactory().create(this.conf, null, null, Optional.empty(), null); + bk = pulsar.getBookKeeperClientFactory().create(this.conf, null, null, Optional.empty(), null).get(); schema = Schema.JSON(ServiceUnitStateData.class); strategy = new ServiceUnitStateCompactionStrategy(); strategy.checkBrokers(false); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/TopicCompactionServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/TopicCompactionServiceTest.java index d84d1ccc9ea45..ba77ce5bd9d29 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/TopicCompactionServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/TopicCompactionServiceTest.java @@ -72,7 +72,7 @@ public void setup() throws Exception { compactionScheduler = Executors.newSingleThreadScheduledExecutor( new ThreadFactoryBuilder().setNameFormat("compactor").setDaemon(true).build()); bk = pulsar.getBookKeeperClientFactory().create( - this.conf, null, null, Optional.empty(), null); + this.conf, null, null, Optional.empty(), null).get(); compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); } From 6236116754472c61b2166da6d4797fc63c83f364 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 6 Jun 2024 16:09:38 +0800 Subject: [PATCH 273/580] [improve] [client] PIP-344 support feature flag supportsGetPartitionedMetadataWithoutAutoCreation (#22773) --- .../pulsar/client/impl/ClientCnxTest.java | 44 +++++++++++++++++++ .../client/impl/BinaryProtoLookupService.java | 6 +++ .../apache/pulsar/client/impl/ClientCnx.java | 5 +++ .../pulsar/common/protocol/Commands.java | 1 + 4 files changed, 56 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java index dfd52d494ae15..df6b1b8a8f92f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java @@ -20,13 +20,17 @@ import com.google.common.collect.Sets; import io.netty.channel.ChannelHandlerContext; +import java.lang.reflect.Field; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import org.apache.pulsar.PulsarVersion; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -124,4 +128,44 @@ public void testClientVersion() throws Exception { producer.close(); consumer.close(); } + + @Test + public void testSupportsGetPartitionedMetadataWithoutAutoCreation() throws Exception { + final String topic = BrokerTestUtil.newUniqueName( "persistent://" + NAMESPACE + "/tp"); + admin.topics().createNonPartitionedTopic(topic); + PulsarClientImpl clientWitBinaryLookup = (PulsarClientImpl) PulsarClient.builder() + .maxNumberOfRejectedRequestPerConnection(1) + .connectionMaxIdleSeconds(Integer.MAX_VALUE) + .serviceUrl(pulsar.getBrokerServiceUrl()) + .build(); + ProducerImpl producer = (ProducerImpl) clientWitBinaryLookup.newProducer().topic(topic).create(); + + // Verify: the variable "isSupportsGetPartitionedMetadataWithoutAutoCreation" responded from the broker is true. + Awaitility.await().untilAsserted(() -> { + ClientCnx clientCnx = producer.getClientCnx(); + Assert.assertNotNull(clientCnx); + Assert.assertTrue(clientCnx.isSupportsGetPartitionedMetadataWithoutAutoCreation()); + }); + Assert.assertEquals( + clientWitBinaryLookup.getPartitionsForTopic(topic, true).get().size(), 1); + + // Inject a "false" value for the variable "isSupportsGetPartitionedMetadataWithoutAutoCreation". + // Verify: client will get a not support error. + Field field = ClientCnx.class.getDeclaredField("supportsGetPartitionedMetadataWithoutAutoCreation"); + field.setAccessible(true); + for (CompletableFuture clientCnxFuture : clientWitBinaryLookup.getCnxPool().getConnections()) { + field.set(clientCnxFuture.get(), false); + } + try { + clientWitBinaryLookup.getPartitionsForTopic(topic, false).join(); + Assert.fail("Expected an error that the broker version is too old."); + } catch (Exception ex) { + Assert.assertTrue(ex.getMessage().contains("without auto-creation is not supported from the broker")); + } + + // cleanup. + producer.close(); + clientWitBinaryLookup.close(); + admin.topics().delete(topic, false); + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index b363d6e4366ad..bf015c564b9cc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -254,6 +254,12 @@ private CompletableFuture getPartitionedTopicMetadata( CompletableFuture partitionFuture = new CompletableFuture<>(); client.getCnxPool().getConnection(socketAddress).thenAccept(clientCnx -> { + if (!metadataAutoCreationEnabled && !clientCnx.isSupportsGetPartitionedMetadataWithoutAutoCreation()) { + partitionFuture.completeExceptionally(new PulsarClientException.NotSupportedException("The feature of" + + " getting partitions without auto-creation is not supported from the broker," + + " please upgrade the broker to the latest version.")); + return; + } long requestId = client.newRequestId(); ByteBuf request = Commands.newPartitionMetadataRequest(topicName.toString(), requestId, metadataAutoCreationEnabled); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index 03e0f406dd2f2..6f343a2ee5855 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -191,6 +191,8 @@ public class ClientCnx extends PulsarHandler { protected AuthenticationDataProvider authenticationDataProvider; private TransactionBufferHandler transactionBufferHandler; private boolean supportsTopicWatchers; + @Getter + private boolean supportsGetPartitionedMetadataWithoutAutoCreation; /** Idle stat. **/ @Getter @@ -400,6 +402,9 @@ protected void handleConnected(CommandConnected connected) { supportsTopicWatchers = connected.hasFeatureFlags() && connected.getFeatureFlags().isSupportsTopicWatchers(); + supportsGetPartitionedMetadataWithoutAutoCreation = + connected.hasFeatureFlags() + && connected.getFeatureFlags().isSupportsGetPartitionedMetadataWithoutAutoCreation(); // set remote protocol version to the correct version before we complete the connection future setRemoteEndpointProtocolVersion(connected.getProtocolVersion()); 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 cbee7f354c4df..224e093baf112 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 @@ -301,6 +301,7 @@ public static BaseCommand newConnectedCommand(int clientProtocolVersion, int max connected.setProtocolVersion(versionToAdvertise); connected.setFeatureFlags().setSupportsTopicWatchers(supportsTopicWatchers); + connected.setFeatureFlags().setSupportsGetPartitionedMetadataWithoutAutoCreation(true); return cmd; } From fb03d159129d10fdf1c277ba7b040025bed2ca61 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Thu, 6 Jun 2024 16:11:00 +0800 Subject: [PATCH 274/580] [improve][ci] Add arm64 image build (#22755) Signed-off-by: Zixuan Liu --- .github/workflows/pulsar-ci.yaml | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index c15d51f9cfcf6..1160a0d1ec363 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -465,6 +465,12 @@ jobs: timeout-minutes: 60 needs: ['preconditions', 'build-and-license-check'] if: ${{ needs.preconditions.outputs.docs_only != 'true'}} + strategy: + fail-fast: false + matrix: + platform: + - linux/amd64 + - linux/arm64 env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} @@ -509,14 +515,21 @@ jobs: cd $HOME $GITHUB_WORKSPACE/build/pulsar_ci_tool.sh restore_tar_from_github_actions_artifacts pulsar-maven-repository-binaries - - name: Build java-test-image docker image + - name: Set up QEMU + uses: docker/setup-qemu-action@v3 + with: + platforms: arm64 + + - name: Build java-test-image docker image - ${{ matrix.platform }} run: | # build docker image DOCKER_CLI_EXPERIMENTAL=enabled mvn -B -am -pl docker/pulsar,tests/docker-images/java-test-image install -Pcore-modules,-main,integrationTests,docker \ + -Ddocker.platforms=${{ matrix.platform }} \ -Dmaven.test.skip=true -DskipSourceReleaseAssembly=true \ -Dspotbugs.skip=true -Dlicense.skip=true -Dcheckstyle.skip=true -Drat.skip=true - name: save docker image apachepulsar/java-test-image:latest to Github artifact cache + if: ${{ matrix.platform == 'linux/amd64' }} run: | $GITHUB_WORKSPACE/build/pulsar_ci_tool.sh docker_save_image_to_github_actions_artifacts apachepulsar/java-test-image:latest pulsar-java-test-image From fb80007a47deaadb82d0b1b1e4fcd6ca04c05c9c Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Thu, 6 Jun 2024 16:16:14 +0800 Subject: [PATCH 275/580] [improve] [pip] PIP-357: Correct the conf name in load balance module. (#22823) --- pip/pip-357.md | 35 +++++++++++++++++++++++++++++++++++ 1 file changed, 35 insertions(+) create mode 100644 pip/pip-357.md diff --git a/pip/pip-357.md b/pip/pip-357.md new file mode 100644 index 0000000000000..716a7d5f5043c --- /dev/null +++ b/pip/pip-357.md @@ -0,0 +1,35 @@ +# PIP-357: Correct the conf name in load balance module. + +# Background knowledge + +We use `loadBalancerBandwithInResourceWeight` and `loadBalancerBandwithOutResourceWeight` to calculate the broker's load in the load balance module. However, the correct conf name should be `loadBalancerBandwidthInResourceWeight` and `loadBalancerBandwidthOutResourceWeight`. This PIP is to correct the conf name in the load balance module. + +# Motivation + +The current conf name is incorrect. + + +# Detailed Design + +- deprecated `loadBalancerBandwithInResourceWeight` and `loadBalancerBandwithOutResourceWeight` in the load balance module. +- add `loadBalancerBandwidthInResourceWeight` and `loadBalancerBandwidthOutResourceWeight` in the load balance module. + +In case of users upgrading to this version don't notice the change, we will still support the old conf name in following way: +- If a configuration is not the default configuration, use that configuration. +- If both the new and the old are configured different from the default value, use the new one. + +# Backward & Forward Compatibility + +Backward compatible, users can upgrade to this version without doing any changes and the old conf name will still work. +If user want to use the new conf name, they can change the conf name in the configuration file. +Just remember that if both the new and the old are configured different from the default value, the new one will be used. + +# General Notes + +# Links + + +* Mailing List discussion thread: https://lists.apache.org/thread/31wfq2hhprn4zknp4jv21lzf5809q6lf +* Mailing List voting thread: https://lists.apache.org/thread/0pggcploqw43mo134cwmk7b3p7t13848 From 9326a08eb173b8a7410bcb00c4ab7d3602064b4a Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 7 Jun 2024 11:09:09 +0800 Subject: [PATCH 276/580] [fix][broker] Fix NPE after publishing a tombstone to the service unit channel (#22859) --- .../loadbalance/extensions/manager/UnloadManager.java | 6 +++--- .../loadbalance/extensions/manager/UnloadManagerTest.java | 6 +----- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java index 6b745345c0a43..42fd2fc8473d7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.manager; +import static com.google.common.base.Preconditions.checkArgument; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigning; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.models.UnloadDecision.Label.Failure; @@ -201,9 +202,8 @@ public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable } } case Init -> { - if (data.force()) { - complete(serviceUnit, t); - } + checkArgument(data == null, "Init state must be associated with null data"); + complete(serviceUnit, t); } case Owned -> complete(serviceUnit, t); case Releasing -> LatencyMetric.RELEASE.endMeasurement(serviceUnit); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java index f7deb072688c5..be78cfcb595c5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java @@ -126,11 +126,7 @@ public void testSuccess() throws IllegalAccessException, ExecutionException, Int assertEquals(inFlightUnloadRequestMap.size(), 1); // Success with Init state. - manager.handleEvent(bundle, - new ServiceUnitStateData(ServiceUnitState.Init, null, srcBroker, false, VERSION_ID_INIT), null); - assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent(bundle, - new ServiceUnitStateData(ServiceUnitState.Init, null, srcBroker, true, VERSION_ID_INIT), null); + manager.handleEvent(bundle, null, null); assertEquals(inFlightUnloadRequestMap.size(), 0); future.get(); assertEquals(counter.getBreakdownCounters().get(Success).get(Admin).get(), 1); From 9692b67fd64f7acf90a93bf9adc21fab61842555 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Fri, 7 Jun 2024 14:29:15 +0800 Subject: [PATCH 277/580] [improve] [pip] PIP-357: Correct the conf name in load balance module. (#22824) Co-authored-by: Kai Wang --- conf/broker.conf | 8 ++-- conf/standalone.conf | 8 ++-- .../terraform-ansible/templates/broker.conf | 8 ++-- .../pulsar/broker/ServiceConfiguration.java | 47 ++++++++++++++++++- .../extensions/data/BrokerLoadData.java | 16 +++---- .../impl/LeastResourceUsageWithWeight.java | 12 ++--- .../loadbalance/impl/ThresholdShedder.java | 4 +- .../ModularLoadManagerStrategyTest.java | 8 ++-- .../extensions/data/BrokerLoadDataTest.java | 12 ++--- .../LeastResourceUsageWithWeightTest.java | 4 +- 10 files changed, 85 insertions(+), 42 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 02e294029893a..8fd266d609cf4 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1393,13 +1393,13 @@ loadBalancerMsgThroughputMultiplierDifferenceShedderThreshold=4 # It only takes effect in the ThresholdShedder strategy. loadBalancerHistoryResourcePercentage=0.9 -# The BandWithIn usage weight when calculating new resource usage. +# The BandWidthIn usage weight when calculating new resource usage. # It only takes effect in the ThresholdShedder strategy. -loadBalancerBandwithInResourceWeight=1.0 +loadBalancerBandwidthInResourceWeight=1.0 -# The BandWithOut usage weight when calculating new resource usage. +# The BandWidthOut usage weight when calculating new resource usage. # It only takes effect in the ThresholdShedder strategy. -loadBalancerBandwithOutResourceWeight=1.0 +loadBalancerBandwidthOutResourceWeight=1.0 # The CPU usage weight when calculating new resource usage. # It only takes effect in the ThresholdShedder strategy. diff --git a/conf/standalone.conf b/conf/standalone.conf index 07d19c7bee929..6b261ce11c6cd 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -910,13 +910,13 @@ loadBalancerBrokerThresholdShedderPercentage=10 # It only takes effect in the ThresholdShedder strategy. loadBalancerHistoryResourcePercentage=0.9 -# The BandWithIn usage weight when calculating new resource usage. +# The BandWidthIn usage weight when calculating new resource usage. # It only takes effect in the ThresholdShedder strategy. -loadBalancerBandwithInResourceWeight=1.0 +loadBalancerBandwidthInResourceWeight=1.0 -# The BandWithOut usage weight when calculating new resource usage. +# The BandWidthOut usage weight when calculating new resource usage. # It only takes effect in the ThresholdShedder strategy. -loadBalancerBandwithOutResourceWeight=1.0 +loadBalancerBandwidthOutResourceWeight=1.0 # The CPU usage weight when calculating new resource usage. # It only takes effect in the ThresholdShedder strategy. diff --git a/deployment/terraform-ansible/templates/broker.conf b/deployment/terraform-ansible/templates/broker.conf index 084d7f46ce1ce..ff3677174024c 100644 --- a/deployment/terraform-ansible/templates/broker.conf +++ b/deployment/terraform-ansible/templates/broker.conf @@ -953,13 +953,13 @@ loadBalancerBrokerThresholdShedderPercentage=10 # It only take effect in ThresholdShedder strategy. loadBalancerHistoryResourcePercentage=0.9 -# The BandWithIn usage weight when calculating new resource usage. +# The BandWidthIn usage weight when calculating new resourde usage. # It only take effect in ThresholdShedder strategy. -loadBalancerBandwithInResourceWeight=1.0 +loadBalancerBandwidthInResourceWeight=1.0 -# The BandWithOut usage weight when calculating new resource usage. +# The BandWidthOut usage weight when calculating new resourde usage. # It only take effect in ThresholdShedder strategy. -loadBalancerBandwithOutResourceWeight=1.0 +loadBalancerBandwidthOutResourceWeight=1.0 # The CPU usage weight when calculating new resource usage. # It only take effect in ThresholdShedder strategy. 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 204ea453bae59..63ef6f3efe6d0 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 @@ -2444,17 +2444,60 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece @FieldContext( dynamic = true, category = CATEGORY_LOAD_BALANCER, - doc = "BandwithIn Resource Usage Weight" + doc = "BandwidthIn Resource Usage Weight" + ) + private double loadBalancerBandwidthInResourceWeight = 1.0; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "BandwidthOut Resource Usage Weight" + ) + private double loadBalancerBandwidthOutResourceWeight = 1.0; + + @Deprecated + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "BandwidthIn Resource Usage Weight, Deprecated: Use loadBalancerBandwidthInResourceWeight" ) private double loadBalancerBandwithInResourceWeight = 1.0; + @Deprecated @FieldContext( dynamic = true, category = CATEGORY_LOAD_BALANCER, - doc = "BandwithOut Resource Usage Weight" + doc = "BandwidthOut Resource Usage Weight, Deprecated: Use loadBalancerBandwidthOutResourceWeight" ) private double loadBalancerBandwithOutResourceWeight = 1.0; + /** + * Get the load balancer bandwidth in resource weight. + * To be compatible with the old configuration, we still support the old configuration. + * If a configuration is not the default configuration, use that configuration. + * If both the new and the old are configured different from the default value, use the new one. + * @return + */ + public double getLoadBalancerBandwidthInResourceWeight() { + if (loadBalancerBandwidthInResourceWeight != 1.0) { + return loadBalancerBandwidthInResourceWeight; + } + if (loadBalancerBandwithInResourceWeight != 1.0) { + return loadBalancerBandwithInResourceWeight; + } + return 1.0; + } + + public double getLoadBalancerBandwidthOutResourceWeight() { + if (loadBalancerBandwidthOutResourceWeight != 1.0) { + return loadBalancerBandwidthOutResourceWeight; + } + if (loadBalancerBandwithOutResourceWeight != 1.0) { + return loadBalancerBandwithOutResourceWeight; + } + return 1.0; + } + @FieldContext( dynamic = true, category = CATEGORY_LOAD_BALANCER, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLoadData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLoadData.java index a8cb9e0829479..95d89932ed96d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLoadData.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLoadData.java @@ -69,8 +69,8 @@ public class BrokerLoadData { * loadBalancerCPUResourceWeight, * loadBalancerMemoryResourceWeight, * loadBalancerDirectMemoryResourceWeight, - * loadBalancerBandwithInResourceWeight, and - * loadBalancerBandwithOutResourceWeight. + * loadBalancerBandwidthInResourceWeight, and + * loadBalancerBandwidthOutResourceWeight. * * The historical resource percentage is configured by loadBalancerHistoryResourcePercentage. */ @@ -186,8 +186,8 @@ private void updateWeightedMaxEMA(ServiceConfiguration conf) { var weightedMax = getMaxResourceUsageWithWeight( conf.getLoadBalancerCPUResourceWeight(), conf.getLoadBalancerMemoryResourceWeight(), conf.getLoadBalancerDirectMemoryResourceWeight(), - conf.getLoadBalancerBandwithInResourceWeight(), - conf.getLoadBalancerBandwithOutResourceWeight()); + conf.getLoadBalancerBandwidthInResourceWeight(), + conf.getLoadBalancerBandwidthOutResourceWeight()); weightedMaxEMA = updatedAt == 0 ? weightedMax : weightedMaxEMA * historyPercentage + (1 - historyPercentage) * weightedMax; } @@ -220,9 +220,9 @@ public void clear() { public String toString(ServiceConfiguration conf) { return String.format("cpu= %.2f%%, memory= %.2f%%, directMemory= %.2f%%, " - + "bandwithIn= %.2f%%, bandwithOut= %.2f%%, " + + "bandwidthIn= %.2f%%, bandwidthOut= %.2f%%, " + "cpuWeight= %f, memoryWeight= %f, directMemoryWeight= %f, " - + "bandwithInResourceWeight= %f, bandwithOutResourceWeight= %f, " + + "bandwidthInResourceWeight= %f, bandwidthOutResourceWeight= %f, " + "msgThroughputIn= %.2f, msgThroughputOut= %.2f, msgRateIn= %.2f, msgRateOut= %.2f, " + "bundleCount= %d, " + "maxResourceUsage= %.2f%%, weightedMaxEMA= %.2f%%, msgThroughputEMA= %.2f, " @@ -233,8 +233,8 @@ public String toString(ServiceConfiguration conf) { conf.getLoadBalancerCPUResourceWeight(), conf.getLoadBalancerMemoryResourceWeight(), conf.getLoadBalancerDirectMemoryResourceWeight(), - conf.getLoadBalancerBandwithInResourceWeight(), - conf.getLoadBalancerBandwithOutResourceWeight(), + conf.getLoadBalancerBandwidthInResourceWeight(), + conf.getLoadBalancerBandwidthOutResourceWeight(), msgThroughputIn, msgThroughputOut, msgRateIn, msgRateOut, bundleCount, maxResourceUsage * 100, weightedMaxEMA * 100, msgThroughputEMA, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LeastResourceUsageWithWeight.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LeastResourceUsageWithWeight.java index ab3e63e9d133f..2baf58c9f05b5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LeastResourceUsageWithWeight.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LeastResourceUsageWithWeight.java @@ -68,8 +68,8 @@ private double getMaxResourceUsageWithWeight(final String broker, final BrokerDa localData.getDirectMemory().percentUsage(), localData.getBandwidthIn().percentUsage(), localData.getBandwidthOut().percentUsage(), conf.getLoadBalancerCPUResourceWeight(), conf.getLoadBalancerMemoryResourceWeight(), conf.getLoadBalancerDirectMemoryResourceWeight(), - conf.getLoadBalancerBandwithInResourceWeight(), - conf.getLoadBalancerBandwithOutResourceWeight()); + conf.getLoadBalancerBandwidthInResourceWeight(), + conf.getLoadBalancerBandwidthOutResourceWeight()); } if (log.isDebugEnabled()) { @@ -99,8 +99,8 @@ private double updateAndGetMaxResourceUsageWithWeight(String broker, BrokerData double resourceUsage = brokerData.getLocalData().getMaxResourceUsageWithWeight( conf.getLoadBalancerCPUResourceWeight(), conf.getLoadBalancerDirectMemoryResourceWeight(), - conf.getLoadBalancerBandwithInResourceWeight(), - conf.getLoadBalancerBandwithOutResourceWeight()); + conf.getLoadBalancerBandwidthInResourceWeight(), + conf.getLoadBalancerBandwidthOutResourceWeight()); historyUsage = historyUsage == null ? resourceUsage : historyUsage * historyPercentage + (1 - historyPercentage) * resourceUsage; if (log.isDebugEnabled()) { @@ -110,8 +110,8 @@ private double updateAndGetMaxResourceUsageWithWeight(String broker, BrokerData + "OUT weight: {} ", broker, historyUsage, historyPercentage, conf.getLoadBalancerCPUResourceWeight(), conf.getLoadBalancerMemoryResourceWeight(), conf.getLoadBalancerDirectMemoryResourceWeight(), - conf.getLoadBalancerBandwithInResourceWeight(), - conf.getLoadBalancerBandwithOutResourceWeight()); + conf.getLoadBalancerBandwidthInResourceWeight(), + conf.getLoadBalancerBandwidthOutResourceWeight()); } brokerAvgResourceUsageWithWeight.put(broker, historyUsage); return historyUsage; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ThresholdShedder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ThresholdShedder.java index ffa16c09e9b7a..aa556cd0ca5d3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ThresholdShedder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ThresholdShedder.java @@ -173,8 +173,8 @@ private double updateAvgResourceUsage(String broker, LocalBrokerData localBroker double resourceUsage = localBrokerData.getMaxResourceUsageWithWeight( conf.getLoadBalancerCPUResourceWeight(), conf.getLoadBalancerDirectMemoryResourceWeight(), - conf.getLoadBalancerBandwithInResourceWeight(), - conf.getLoadBalancerBandwithOutResourceWeight()); + conf.getLoadBalancerBandwidthInResourceWeight(), + conf.getLoadBalancerBandwidthOutResourceWeight()); historyUsage = historyUsage == null ? resourceUsage : historyUsage * historyPercentage + (1 - historyPercentage) * resourceUsage; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java index c64c9950a95a9..2b6bfb742eb04 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java @@ -88,8 +88,8 @@ public void testLeastResourceUsageWithWeight() { conf.setLoadBalancerCPUResourceWeight(1.0); conf.setLoadBalancerMemoryResourceWeight(0.1); conf.setLoadBalancerDirectMemoryResourceWeight(0.1); - conf.setLoadBalancerBandwithInResourceWeight(1.0); - conf.setLoadBalancerBandwithOutResourceWeight(1.0); + conf.setLoadBalancerBandwidthInResourceWeight(1.0); + conf.setLoadBalancerBandwidthOutResourceWeight(1.0); conf.setLoadBalancerHistoryResourcePercentage(0.5); conf.setLoadBalancerAverageResourceUsageDifferenceThresholdPercentage(5); @@ -167,8 +167,8 @@ public void testLeastResourceUsageWithWeightWithArithmeticException() conf.setLoadBalancerCPUResourceWeight(1.0); conf.setLoadBalancerMemoryResourceWeight(0.1); conf.setLoadBalancerDirectMemoryResourceWeight(0.1); - conf.setLoadBalancerBandwithInResourceWeight(1.0); - conf.setLoadBalancerBandwithOutResourceWeight(1.0); + conf.setLoadBalancerBandwidthInResourceWeight(1.0); + conf.setLoadBalancerBandwidthOutResourceWeight(1.0); conf.setLoadBalancerHistoryResourcePercentage(0.5); conf.setLoadBalancerAverageResourceUsageDifferenceThresholdPercentage(5); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLoadDataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLoadDataTest.java index 85792a7ba9387..295c157e3596a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLoadDataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLoadDataTest.java @@ -41,8 +41,8 @@ public void testUpdateBySystemResourceUsage() { conf.setLoadBalancerCPUResourceWeight(0.5); conf.setLoadBalancerMemoryResourceWeight(0.5); conf.setLoadBalancerDirectMemoryResourceWeight(0.5); - conf.setLoadBalancerBandwithInResourceWeight(0.5); - conf.setLoadBalancerBandwithOutResourceWeight(0.5); + conf.setLoadBalancerBandwidthInResourceWeight(0.5); + conf.setLoadBalancerBandwidthOutResourceWeight(0.5); conf.setLoadBalancerHistoryResourcePercentage(0.75); BrokerLoadData data = new BrokerLoadData(); @@ -108,9 +108,9 @@ public void testUpdateBySystemResourceUsage() { assertThat(data.getUpdatedAt(), greaterThanOrEqualTo(now)); assertEquals(data.getReportedAt(), 0l); assertEquals(data.toString(conf), "cpu= 300.00%, memory= 100.00%, directMemory= 2.00%, " - + "bandwithIn= 3.00%, bandwithOut= 4.00%, " + + "bandwidthIn= 3.00%, bandwidthOut= 4.00%, " + "cpuWeight= 0.500000, memoryWeight= 0.500000, directMemoryWeight= 0.500000, " - + "bandwithInResourceWeight= 0.500000, bandwithOutResourceWeight= 0.500000, " + + "bandwidthInResourceWeight= 0.500000, bandwidthOutResourceWeight= 0.500000, " + "msgThroughputIn= 5.00, msgThroughputOut= 6.00, " + "msgRateIn= 7.00, msgRateOut= 8.00, bundleCount= 9, " + "maxResourceUsage= 300.00%, weightedMaxEMA= 187.50%, msgThroughputEMA= 5.00, " @@ -126,8 +126,8 @@ public void testUpdateByBrokerLoadData() { conf.setLoadBalancerCPUResourceWeight(0.5); conf.setLoadBalancerMemoryResourceWeight(0.5); conf.setLoadBalancerDirectMemoryResourceWeight(0.5); - conf.setLoadBalancerBandwithInResourceWeight(0.5); - conf.setLoadBalancerBandwithOutResourceWeight(0.5); + conf.setLoadBalancerBandwidthInResourceWeight(0.5); + conf.setLoadBalancerBandwidthOutResourceWeight(0.5); conf.setLoadBalancerHistoryResourcePercentage(0.75); BrokerLoadData data = new BrokerLoadData(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java index b1e09bf2f3afb..5f3a08d493bc3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java @@ -196,8 +196,8 @@ public static LoadManagerContext getContext() { conf.setLoadBalancerCPUResourceWeight(1.0); conf.setLoadBalancerMemoryResourceWeight(0.1); conf.setLoadBalancerDirectMemoryResourceWeight(0.1); - conf.setLoadBalancerBandwithInResourceWeight(1.0); - conf.setLoadBalancerBandwithOutResourceWeight(1.0); + conf.setLoadBalancerBandwidthInResourceWeight(1.0); + conf.setLoadBalancerBandwidthOutResourceWeight(1.0); conf.setLoadBalancerHistoryResourcePercentage(0.5); conf.setLoadBalancerAverageResourceUsageDifferenceThresholdPercentage(5); var brokerLoadDataStore = new LoadDataStore() { From c81c0f684f8c55c2e39739c6e1de935dff2085d6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Jun 2024 15:25:35 +0300 Subject: [PATCH 278/580] [fix][cli] Fix the shell script parameter passthrough syntax (#22867) --- bin/bookkeeper | 12 ++++++------ bin/pulsar | 38 +++++++++++++++++++------------------- 2 files changed, 25 insertions(+), 25 deletions(-) diff --git a/bin/bookkeeper b/bin/bookkeeper index 0cc07dd49aba5..13d092f4c99a6 100755 --- a/bin/bookkeeper +++ b/bin/bookkeeper @@ -214,20 +214,20 @@ OPTS="$OPTS $BK_METADATA_OPTIONS" #Change to BK_HOME to support relative paths cd "$BK_HOME" if [ $COMMAND == "bookie" ]; then - exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.server.Main --conf $BOOKIE_CONF $@ + exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.server.Main --conf $BOOKIE_CONF "$@" elif [ $COMMAND == "autorecovery" ]; then - exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.replication.AutoRecoveryMain --conf $BOOKIE_CONF $@ + exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.replication.AutoRecoveryMain --conf $BOOKIE_CONF "$@" elif [ $COMMAND == "localbookie" ]; then NUMBER=$1 shift - exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.util.LocalBookKeeper $NUMBER $BOOKIE_CONF $@ + exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.util.LocalBookKeeper $NUMBER $BOOKIE_CONF "$@" elif [ $COMMAND == "upgrade" ]; then - exec $JAVA $OPTS org.apache.bookkeeper.bookie.FileSystemUpgrade --conf $BOOKIE_CONF $@ + exec $JAVA $OPTS org.apache.bookkeeper.bookie.FileSystemUpgrade --conf $BOOKIE_CONF "$@" elif [ $COMMAND == "shell" ]; then ENTRY_FORMATTER_ARG="-DentryFormatterClass=${ENTRY_FORMATTER_CLASS:-org.apache.bookkeeper.util.StringEntryFormatter}" - exec $JAVA $OPTS $ENTRY_FORMATTER_ARG org.apache.bookkeeper.bookie.BookieShell -conf $BOOKIE_CONF $@ + exec $JAVA $OPTS $ENTRY_FORMATTER_ARG org.apache.bookkeeper.bookie.BookieShell -conf $BOOKIE_CONF "$@" elif [ $COMMAND == "help" -o $COMMAND == "--help" -o $COMMAND == "-h" ]; then bookkeeper_help; else - exec $JAVA $OPTS $COMMAND $@ + exec $JAVA $OPTS $COMMAND "$@" fi diff --git a/bin/pulsar b/bin/pulsar index ab0029af5b0da..f6061601d88b1 100755 --- a/bin/pulsar +++ b/bin/pulsar @@ -329,56 +329,56 @@ fi cd "$PULSAR_HOME" if [ $COMMAND == "broker" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-broker.log"} - exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.PulsarBrokerStarter --broker-conf $PULSAR_BROKER_CONF $@ + exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.PulsarBrokerStarter --broker-conf $PULSAR_BROKER_CONF "$@" elif [ $COMMAND == "bookie" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"bookkeeper.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.server.Main --conf $PULSAR_BOOKKEEPER_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.server.Main --conf $PULSAR_BOOKKEEPER_CONF "$@" elif [ $COMMAND == "zookeeper" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"zookeeper.log"} - exec $JAVA ${ZK_OPTS} $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_ZK_CONF $@ + exec $JAVA ${ZK_OPTS} $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_ZK_CONF "$@" elif [ $COMMAND == "global-zookeeper" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"global-zookeeper.log"} # Allow global ZK to turn into read-only mode when it cannot reach the quorum OPTS="${OPTS} ${ZK_OPTS} -Dreadonlymode.enabled=true" - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_GLOBAL_ZK_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_GLOBAL_ZK_CONF "$@" elif [ $COMMAND == "configuration-store" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"configuration-store.log"} # Allow global ZK to turn into read-only mode when it cannot reach the quorum OPTS="${OPTS} ${ZK_OPTS} -Dreadonlymode.enabled=true" - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_CONFIGURATION_STORE_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_CONFIGURATION_STORE_CONF "$@" elif [ $COMMAND == "proxy" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-proxy.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.proxy.server.ProxyServiceStarter --config $PULSAR_PROXY_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.proxy.server.ProxyServiceStarter --config $PULSAR_PROXY_CONF "$@" elif [ $COMMAND == "websocket" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-websocket.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.websocket.service.WebSocketServiceStarter $PULSAR_WEBSOCKET_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.websocket.service.WebSocketServiceStarter $PULSAR_WEBSOCKET_CONF "$@" elif [ $COMMAND == "functions-worker" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-functions-worker.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.functions.worker.FunctionWorkerStarter -c $PULSAR_WORKER_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.functions.worker.FunctionWorkerStarter -c $PULSAR_WORKER_CONF "$@" elif [ $COMMAND == "standalone" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-standalone.log"} - exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS ${ZK_OPTS} -Dpulsar.log.file=$PULSAR_LOG_FILE -Dpulsar.config.file=$PULSAR_STANDALONE_CONF org.apache.pulsar.PulsarStandaloneStarter $@ + exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS ${ZK_OPTS} -Dpulsar.log.file=$PULSAR_LOG_FILE -Dpulsar.config.file=$PULSAR_STANDALONE_CONF org.apache.pulsar.PulsarStandaloneStarter "$@" elif [ ${COMMAND} == "autorecovery" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-autorecovery.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.replication.AutoRecoveryMain --conf $PULSAR_BOOKKEEPER_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.replication.AutoRecoveryMain --conf $PULSAR_BOOKKEEPER_CONF "$@" elif [ $COMMAND == "initialize-cluster-metadata" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataSetup $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataSetup "$@" elif [ $COMMAND == "delete-cluster-metadata" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataTeardown $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataTeardown "$@" elif [ $COMMAND == "initialize-transaction-coordinator-metadata" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarTransactionCoordinatorMetadataSetup $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarTransactionCoordinatorMetadataSetup "$@" elif [ $COMMAND == "initialize-namespace" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarInitialNamespaceSetup $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarInitialNamespaceSetup "$@" elif [ $COMMAND == "zookeeper-shell" ]; then - exec $JAVA $OPTS org.apache.zookeeper.ZooKeeperMain $@ + exec $JAVA $OPTS org.apache.zookeeper.ZooKeeperMain "$@" elif [ $COMMAND == "broker-tool" ]; then - exec $JAVA $OPTS org.apache.pulsar.broker.tools.BrokerTool $@ + exec $JAVA $OPTS org.apache.pulsar.broker.tools.BrokerTool "$@" elif [ $COMMAND == "compact-topic" ]; then - exec $JAVA $OPTS org.apache.pulsar.compaction.CompactorTool --broker-conf $PULSAR_BROKER_CONF $@ + exec $JAVA $OPTS org.apache.pulsar.compaction.CompactorTool --broker-conf $PULSAR_BROKER_CONF "$@" elif [ $COMMAND == "tokens" ]; then - exec $JAVA $OPTS org.apache.pulsar.utils.auth.tokens.TokensCliUtils $@ + exec $JAVA $OPTS org.apache.pulsar.utils.auth.tokens.TokensCliUtils "$@" elif [ $COMMAND == "version" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarVersionStarter $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarVersionStarter "$@" elif [ $COMMAND == "help" -o $COMMAND == "--help" -o $COMMAND == "-h" ]; then pulsar_help; else From 6f1f7bae76d04d877a7e06641275a3d4a73cadba Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Fri, 7 Jun 2024 20:27:27 +0800 Subject: [PATCH 279/580] [fix] [test] fix testGetMetrics in ExtensibleLoadManagerImplTest. (#22864) --- .../loadbalance/extensions/ExtensibleLoadManagerImplTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index 07855fda4d758..43c50a8ac54f4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -1292,6 +1292,8 @@ public void testRoleChange() throws Exception { @Test public void testGetMetrics() throws Exception { { + ServiceConfiguration conf = getDefaultConf(); + conf.setLoadBalancerMemoryResourceWeight(1); var brokerLoadDataReporter = mock(BrokerLoadDataReporter.class); FieldUtils.writeDeclaredField(primaryLoadManager, "brokerLoadDataReporter", brokerLoadDataReporter, true); BrokerLoadData loadData = new BrokerLoadData(); From c5cc25ebdc3a32d002b944e77fb59c9ccd1f14c1 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Jun 2024 18:36:52 +0300 Subject: [PATCH 280/580] [fix][cli] Fix Pulsar standalone shutdown - bkCluster wasn't closed (#22868) --- .../org/apache/pulsar/PulsarStandalone.java | 10 ++++ .../pulsar/PulsarStandaloneStarter.java | 58 ++++++++++++++----- .../apache/pulsar/PulsarStandaloneTest.java | 48 +++++++++++++-- .../configurations/pulsar_broker_test.conf | 26 ++++----- .../pulsar_broker_test_standalone.conf | 26 ++++----- ...r_broker_test_standalone_with_rocksdb.conf | 26 ++++----- .../standalone_no_client_auth.conf | 4 +- .../pulsar/metadata/bookkeeper/BKCluster.java | 43 +++++++++----- 8 files changed, 167 insertions(+), 74 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java index b785448cdacaf..7f80aa29f53d9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java @@ -420,18 +420,22 @@ public void close() { try { if (fnWorkerService != null) { fnWorkerService.stop(); + fnWorkerService = null; } if (broker != null) { broker.close(); + broker = null; } if (bkCluster != null) { bkCluster.close(); + bkCluster = null; } if (bkEnsemble != null) { bkEnsemble.stop(); + bkEnsemble = null; } } catch (Exception e) { log.error("Shutdown failed: {}", e.getMessage(), e); @@ -496,5 +500,11 @@ private static void processTerminator(int exitCode) { ShutdownUtil.triggerImmediateForcefulShutdown(exitCode); } + public String getBrokerServiceUrl() { + return broker.getBrokerServiceUrl(); + } + public String getWebServiceUrl() { + return broker.getWebServiceAddress(); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandaloneStarter.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandaloneStarter.java index 0ab731591da14..29feac8cb46eb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandaloneStarter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandaloneStarter.java @@ -19,9 +19,12 @@ package org.apache.pulsar; import static org.apache.commons.lang3.StringUtils.isBlank; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import java.io.FileInputStream; import java.util.Arrays; +import lombok.AccessLevel; +import lombok.Setter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; @@ -38,6 +41,9 @@ public class PulsarStandaloneStarter extends PulsarStandalone { @Option(names = {"-g", "--generate-docs"}, description = "Generate docs") private boolean generateDocs = false; + private Thread shutdownThread; + @Setter(AccessLevel.PACKAGE) + private boolean testMode; public PulsarStandaloneStarter(String[] args) throws Exception { @@ -108,30 +114,54 @@ public PulsarStandaloneStarter(String[] args) throws Exception { } } } + } + @Override + public synchronized void start() throws Exception { registerShutdownHook(); + super.start(); } protected void registerShutdownHook() { - Runtime.getRuntime().addShutdownHook(new Thread(() -> { + if (shutdownThread != null) { + throw new IllegalStateException("Shutdown hook already registered"); + } + shutdownThread = new Thread(() -> { try { - if (fnWorkerService != null) { - fnWorkerService.stop(); - } - - if (broker != null) { - broker.close(); - } - - if (bkEnsemble != null) { - bkEnsemble.stop(); - } + doClose(false); } catch (Exception e) { log.error("Shutdown failed: {}", e.getMessage(), e); } finally { - LogManager.shutdown(); + if (!testMode) { + LogManager.shutdown(); + } } - })); + }); + Runtime.getRuntime().addShutdownHook(shutdownThread); + } + + // simulate running the shutdown hook, for testing + @VisibleForTesting + void runShutdownHook() { + if (!testMode) { + throw new IllegalStateException("Not in test mode"); + } + Runtime.getRuntime().removeShutdownHook(shutdownThread); + shutdownThread.run(); + shutdownThread = null; + } + + @Override + public void close() { + doClose(true); + } + + private synchronized void doClose(boolean removeShutdownHook) { + super.close(); + if (shutdownThread != null && removeShutdownHook) { + Runtime.getRuntime().removeShutdownHook(shutdownThread); + shutdownThread = null; + } } protected void exit(int status) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/PulsarStandaloneTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/PulsarStandaloneTest.java index 6ed93a75a3fb5..3d22feb822e32 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/PulsarStandaloneTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/PulsarStandaloneTest.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.metadata.bookkeeper.BKCluster; import org.testng.Assert; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -46,12 +47,15 @@ public Object[][] enableBrokerClientAuth() { @Test public void testStandaloneWithRocksDB() throws Exception { String[] args = new String[]{"--config", - "./src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf"}; + "./src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf", + "-nss", + "-nfw"}; final int bookieNum = 3; final File tempDir = IOUtils.createTempDir("standalone", "test"); PulsarStandaloneStarter standalone = new PulsarStandaloneStarter(args); standalone.setBkDir(tempDir.getAbsolutePath()); + standalone.setBkPort(0); standalone.setNumOfBk(bookieNum); standalone.startBookieWithMetadataStore(); @@ -90,11 +94,12 @@ public void testMetadataInitialization(boolean enableBrokerClientAuth) throws Ex } final File bkDir = IOUtils.createTempDir("standalone", "bk"); standalone.setNumOfBk(1); + standalone.setBkPort(0); standalone.setBkDir(bkDir.getAbsolutePath()); standalone.start(); @Cleanup PulsarAdmin admin = PulsarAdmin.builder() - .serviceHttpUrl("http://localhost:8080") + .serviceHttpUrl(standalone.getWebServiceUrl()) .authentication(new MockTokenAuthenticationProvider.MockAuthentication()) .build(); if (enableBrokerClientAuth) { @@ -104,8 +109,8 @@ public void testMetadataInitialization(boolean enableBrokerClientAuth) throws Ex } else { assertTrue(admin.clusters().getClusters().isEmpty()); admin.clusters().createCluster("test_cluster", ClusterData.builder() - .serviceUrl("http://localhost:8080/") - .brokerServiceUrl("pulsar://localhost:6650/") + .serviceUrl(standalone.getWebServiceUrl()) + .brokerServiceUrl(standalone.getBrokerServiceUrl()) .build()); assertTrue(admin.tenants().getTenants().isEmpty()); admin.tenants().createTenant("public", TenantInfo.builder() @@ -125,4 +130,39 @@ public void testMetadataInitialization(boolean enableBrokerClientAuth) throws Ex cleanDirectory(bkDir); cleanDirectory(metadataDir); } + + + @Test + public void testShutdownHookClosesBkCluster() throws Exception { + File dataDir = IOUtils.createTempDir("data", ""); + File metadataDir = new File(dataDir, "metadata"); + File bkDir = new File(dataDir, "bookkeeper"); + @Cleanup + PulsarStandaloneStarter standalone = new PulsarStandaloneStarter(new String[] { + "--config", + "./src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf", + "-nss", + "-nfw", + "--metadata-dir", + metadataDir.getAbsolutePath(), + "--bookkeeper-dir", + bkDir.getAbsolutePath() + }); + standalone.setTestMode(true); + standalone.setBkPort(0); + standalone.start(); + BKCluster bkCluster = standalone.bkCluster; + standalone.runShutdownHook(); + assertTrue(bkCluster.isClosed()); + } + + @Test + public void testWipeData() throws Exception { + PulsarStandaloneStarter standalone = new PulsarStandaloneStarter(new String[] { + "--config", + "./src/test/resources/configurations/standalone_no_client_auth.conf", + "--wipe-data" + }); + assertTrue(standalone.isWipeData()); + } } diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf index f2316111f8017..ddda30d0a4bd9 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf @@ -17,17 +17,17 @@ # under the License. # -applicationName="pulsar_broker" -zookeeperServers="localhost" -configurationStoreServers="localhost" +applicationName=pulsar_broker +zookeeperServers=localhost +configurationStoreServers=localhost brokerServicePort=6650 -brokerServicePortTls=6651 +brokerServicePortTls= webServicePort=8080 -webServicePortTls=4443 +webServicePortTls= httpMaxRequestHeaderSize=1234 bindAddress=0.0.0.0 advertisedAddress= -clusterName="test_cluster" +clusterName=test_cluster brokerShutdownTimeoutMs=3000 backlogQuotaCheckEnabled=true backlogQuotaCheckIntervalInSeconds=60 @@ -42,17 +42,17 @@ clientLibraryVersionCheckEnabled=false clientLibraryVersionCheckAllowUnversioned=true statusFilePath=/tmp/status.html tlsEnabled=false -tlsCertificateFilePath=/usr/local/conf/pulsar/server.crt -tlsKeyFilePath=/home/local/conf/pulsar/server.key +tlsCertificateFilePath= +tlsKeyFilePath= tlsTrustCertsFilePath= tlsAllowInsecureConnection=false authenticationEnabled=false authorizationEnabled=false -superUserRoles="test_user" -brokerClientAuthenticationPlugin="org.apache.pulsar.client.impl.auth.AuthenticationDisabled" +superUserRoles=test_user +brokerClientAuthenticationPlugin=org.apache.pulsar.client.impl.auth.AuthenticationDisabled brokerClientAuthenticationParameters= -bookkeeperClientAuthenticationPlugin="test_auth_plugin" -bookkeeperClientAuthenticationAppId="test_auth_id" +bookkeeperClientAuthenticationPlugin= +bookkeeperClientAuthenticationAppId=test_auth_id bookkeeperClientTimeoutInSeconds=30 bookkeeperClientSpeculativeReadTimeoutInMillis=0 bookkeeperClientHealthCheckEnabled=true @@ -64,7 +64,7 @@ bookkeeperClientRegionawarePolicyEnabled=false bookkeeperClientMinNumRacksPerWriteQuorum=2 bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false bookkeeperClientReorderReadSequenceEnabled=false -bookkeeperClientIsolationGroups="test_group" +bookkeeperClientIsolationGroups=test_group managedLedgerDefaultEnsembleSize=3 managedLedgerDefaultWriteQuorum=2 managedLedgerDefaultAckQuorum=2 diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf index 4a40d9f0c6565..812c8dc9748f9 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf @@ -17,18 +17,18 @@ # under the License. # -applicationName="pulsar_broker" -metadataStoreUrl="zk:localhost:2181/ledger" -configurationMetadataStoreUrl="zk:localhost:2181" -brokerServicePort=6650 -brokerServicePortTls=6651 -webServicePort=8080 -webServicePortTls=4443 +applicationName=pulsar_broker +metadataStoreUrl=zk:localhost:2181/ledger +configurationMetadataStoreUrl=zk:localhost:2181 +brokerServicePort=0 +brokerServicePortTls= +webServicePort=0 +webServicePortTls= bindAddress=0.0.0.0 advertisedAddress= advertisedListeners=internal:pulsar://192.168.1.11:6660,internal:pulsar+ssl://192.168.1.11:6651 internalListenerName=internal -clusterName="test_cluster" +clusterName=test_cluster brokerShutdownTimeoutMs=3000 backlogQuotaCheckEnabled=true backlogQuotaCheckIntervalInSeconds=60 @@ -49,11 +49,11 @@ tlsTrustCertsFilePath= tlsAllowInsecureConnection=false authenticationEnabled=false authorizationEnabled=false -superUserRoles="test_user" -brokerClientAuthenticationPlugin="org.apache.pulsar.client.impl.auth.AuthenticationDisabled" +superUserRoles=test_user +brokerClientAuthenticationPlugin=org.apache.pulsar.client.impl.auth.AuthenticationDisabled brokerClientAuthenticationParameters= -bookkeeperClientAuthenticationPlugin="test_auth_plugin" -bookkeeperClientAuthenticationAppId="test_auth_id" +bookkeeperClientAuthenticationPlugin= +bookkeeperClientAuthenticationAppId= bookkeeperClientTimeoutInSeconds=30 bookkeeperClientSpeculativeReadTimeoutInMillis=0 bookkeeperClientHealthCheckEnabled=true @@ -65,7 +65,7 @@ bookkeeperClientRegionawarePolicyEnabled=false bookkeeperClientMinNumRacksPerWriteQuorum=2 bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false bookkeeperClientReorderReadSequenceEnabled=false -bookkeeperClientIsolationGroups="test_group" +bookkeeperClientIsolationGroups= managedLedgerDefaultEnsembleSize=3 managedLedgerDefaultWriteQuorum=2 managedLedgerDefaultAckQuorum=2 diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf index d8b26bbbfa99d..46c876686b05b 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf @@ -17,19 +17,19 @@ # under the License. # -applicationName="pulsar_broker" +applicationName=pulsar_broker metadataStoreUrl= configurationMetadataStoreUrl= -brokerServicePort=6650 -brokerServicePortTls=6651 -webServicePort=8080 +brokerServicePort=0 +brokerServicePortTls= +webServicePort=0 allowLoopback=true -webServicePortTls=4443 +webServicePortTls= bindAddress=0.0.0.0 advertisedAddress= advertisedListeners= internalListenerName=internal -clusterName="test_cluster" +clusterName=test_cluster brokerShutdownTimeoutMs=3000 backlogQuotaCheckEnabled=true backlogQuotaCheckIntervalInSeconds=60 @@ -44,17 +44,17 @@ clientLibraryVersionCheckEnabled=false clientLibraryVersionCheckAllowUnversioned=true statusFilePath=/tmp/status.html tlsEnabled=false -tlsCertificateFilePath=/usr/local/conf/pulsar/server.crt -tlsKeyFilePath=/home/local/conf/pulsar/server.key +tlsCertificateFilePath= +tlsKeyFilePath= tlsTrustCertsFilePath= tlsAllowInsecureConnection=false authenticationEnabled=false authorizationEnabled=false -superUserRoles="test_user" -brokerClientAuthenticationPlugin="org.apache.pulsar.client.impl.auth.AuthenticationDisabled" +superUserRoles=test_user +brokerClientAuthenticationPlugin=org.apache.pulsar.client.impl.auth.AuthenticationDisabled brokerClientAuthenticationParameters= -bookkeeperClientAuthenticationPlugin="test_auth_plugin" -bookkeeperClientAuthenticationAppId="test_auth_id" +bookkeeperClientAuthenticationPlugin= +bookkeeperClientAuthenticationAppId=test_auth_id bookkeeperClientTimeoutInSeconds=30 bookkeeperClientSpeculativeReadTimeoutInMillis=0 bookkeeperClientHealthCheckEnabled=true @@ -66,7 +66,7 @@ bookkeeperClientRegionawarePolicyEnabled=false bookkeeperClientMinNumRacksPerWriteQuorum=2 bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false bookkeeperClientReorderReadSequenceEnabled=false -bookkeeperClientIsolationGroups="test_group" +bookkeeperClientIsolationGroups=test_group managedLedgerDefaultEnsembleSize=3 managedLedgerDefaultWriteQuorum=2 managedLedgerDefaultAckQuorum=2 diff --git a/pulsar-broker/src/test/resources/configurations/standalone_no_client_auth.conf b/pulsar-broker/src/test/resources/configurations/standalone_no_client_auth.conf index 4e2fd40298354..6f0d82cef17bc 100644 --- a/pulsar-broker/src/test/resources/configurations/standalone_no_client_auth.conf +++ b/pulsar-broker/src/test/resources/configurations/standalone_no_client_auth.conf @@ -17,8 +17,8 @@ # under the License. # -brokerServicePort=6650 -webServicePort=8080 +brokerServicePort=0 +webServicePort=0 allowLoopback=true clusterName=test_cluster superUserRoles=admin diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java index c2f3f72ec21c0..8d3a90239efd3 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java @@ -30,6 +30,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Matcher; import java.util.regex.Pattern; import lombok.Getter; @@ -49,8 +50,8 @@ import org.apache.bookkeeper.replication.AutoRecoveryMain; import org.apache.bookkeeper.server.conf.BookieConfiguration; import org.apache.bookkeeper.util.IOUtils; -import org.apache.bookkeeper.util.PortManager; import org.apache.commons.io.FileUtils; +import org.apache.pulsar.common.util.PortManager; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @@ -74,6 +75,9 @@ public class BKCluster implements AutoCloseable { protected final ServerConfiguration baseConf; protected final ClientConfiguration baseClientConf; + private final List lockedPorts = new ArrayList<>(); + private final AtomicBoolean closed = new AtomicBoolean(false); + public static class BKClusterConf { private ServerConfiguration baseServerConfiguration; @@ -148,20 +152,24 @@ private BKCluster(BKClusterConf bkClusterConf) throws Exception { @Override public void close() throws Exception { - // stop bookkeeper service - try { - stopBKCluster(); - } catch (Exception e) { - log.error("Got Exception while trying to stop BKCluster", e); - } - // cleanup temp dirs - try { - cleanupTempDirs(); - } catch (Exception e) { - log.error("Got Exception while trying to cleanupTempDirs", e); - } + if (closed.compareAndSet(false, true)) { + // stop bookkeeper service + try { + stopBKCluster(); + } catch (Exception e) { + log.error("Got Exception while trying to stop BKCluster", e); + } + lockedPorts.forEach(PortManager::releaseLockedPort); + lockedPorts.clear(); + // cleanup temp dirs + try { + cleanupTempDirs(); + } catch (Exception e) { + log.error("Got Exception while trying to cleanupTempDirs", e); + } - this.store.close(); + this.store.close(); + } } private File createTempDir(String prefix, String suffix) throws IOException { @@ -229,7 +237,8 @@ private ServerConfiguration newServerConfiguration(int index) throws Exception { int port; if (baseConf.isEnableLocalTransport() || !baseConf.getAllowEphemeralPorts() || clusterConf.bkPort == 0) { - port = PortManager.nextFreePort(); + port = PortManager.nextLockedFreePort(); + lockedPorts.add(port); } else { // bk 4.15 cookie validation finds the same ip:port in case of port 0 // and 2nd bookie's cookie validation fails @@ -399,4 +408,8 @@ private static ServerConfiguration setLoopbackInterfaceAndAllowLoopback(ServerCo serverConf.setAllowLoopback(true); return serverConf; } + + public boolean isClosed() { + return closed.get(); + } } From d6dc4d3957e13b392c55324f3607a86d37a835a7 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Jun 2024 19:18:48 +0300 Subject: [PATCH 281/580] [fix][misc] Disable JFR based telemetry collection since it's not used (#22869) --- .../org/apache/pulsar/opentelemetry/OpenTelemetryService.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java index eb09e64fe731f..b5610fc485b3c 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java @@ -102,6 +102,9 @@ public OpenTelemetryService(String clusterName, // For a list of exposed metrics, see https://opentelemetry.io/docs/specs/semconv/runtime/jvm-metrics/ runtimeMetricsReference.set(RuntimeMetrics.builder(openTelemetrySdkReference.get()) + // disable JFR based telemetry and use only JMX telemetry + .disableAllFeatures() + // enable experimental JMX telemetry in addition .enableExperimentalJmxTelemetry() .build()); } From 80d1cf9e44f14b0753054e4c59419fbaf247481c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Jun 2024 20:00:00 +0300 Subject: [PATCH 282/580] [fix][cli] Fix healthcheck script pulsar-zookeeper-ruok.sh (#22873) --- docker/pulsar/scripts/pulsar-zookeeper-ruok.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/pulsar/scripts/pulsar-zookeeper-ruok.sh b/docker/pulsar/scripts/pulsar-zookeeper-ruok.sh index 7a0228c2386bd..045258696ff0b 100755 --- a/docker/pulsar/scripts/pulsar-zookeeper-ruok.sh +++ b/docker/pulsar/scripts/pulsar-zookeeper-ruok.sh @@ -20,7 +20,7 @@ # Check ZK server status -status=$(echo ruok | nc -q 1 localhost 2181) +status=$({ echo ruok; sleep 1; } | nc 127.0.0.1 2181) if [ "$status" == "imok" ]; then exit 0 else From 5af05951754f526c6066b16dc6e56797f041bca7 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Fri, 7 Jun 2024 10:12:37 -0700 Subject: [PATCH 283/580] [fix][broker] Support advertised listeners when gracefully transferring topics (ExtensibleLoadManagerImpl only) (#22862) --- .../pulsar/broker/service/ServerCnx.java | 41 +++++--- .../ExtensibleLoadManagerImplBaseTest.java | 15 ++- .../ExtensibleLoadManagerImplTest.java | 65 +++++++++++- ...anagerImplWithAdvertisedListenersTest.java | 98 +++++++++++++++++++ ...gerImplWithTransactionCoordinatorTest.java | 4 +- 5 files changed, 200 insertions(+), 23 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithAdvertisedListenersTest.java 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 26a00c00b5a6a..dc1cf913ab240 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 @@ -29,7 +29,6 @@ import static org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage.ignoreUnrecoverableBKException; import static org.apache.pulsar.common.api.proto.ProtocolVersion.v5; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; -import static org.apache.pulsar.common.protocol.Commands.newCloseConsumer; import static org.apache.pulsar.common.protocol.Commands.newLookupErrorResponse; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; @@ -71,6 +70,7 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.TransactionMetadataStoreService; @@ -82,6 +82,7 @@ import org.apache.pulsar.broker.limiter.ConnectionController; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.resources.TopicResources; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; @@ -149,6 +150,7 @@ import org.apache.pulsar.common.compression.CompressionCodec; import org.apache.pulsar.common.compression.CompressionCodecProvider; import org.apache.pulsar.common.intercept.InterceptException; +import org.apache.pulsar.common.lookup.data.LookupData; import org.apache.pulsar.common.naming.Metadata; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.SystemTopicNames; @@ -3186,15 +3188,28 @@ public void closeProducer(Producer producer, Optional assigned closeProducer(producer.getProducerId(), producer.getEpoch(), assignedBrokerLookupData); } + private LookupData getLookupData(BrokerLookupData lookupData) { + LookupOptions.LookupOptionsBuilder builder = LookupOptions.builder(); + if (StringUtils.isNotBlank((listenerName))) { + builder.advertisedListenerName(listenerName); + } + try { + return lookupData.toLookupResult(builder.build()).getLookupData(); + } catch (PulsarServerException e) { + log.error("Failed to get lookup data", e); + throw new RuntimeException(e); + } + } + private void closeProducer(long producerId, long epoch, Optional assignedBrokerLookupData) { if (getRemoteEndpointProtocolVersion() >= v5.getValue()) { - if (assignedBrokerLookupData.isPresent()) { - writeAndFlush(Commands.newCloseProducer(producerId, -1L, - assignedBrokerLookupData.get().pulsarServiceUrl(), - assignedBrokerLookupData.get().pulsarServiceUrlTls())); - } else { - writeAndFlush(Commands.newCloseProducer(producerId, -1L)); - } + assignedBrokerLookupData.ifPresentOrElse(lookup -> { + LookupData lookupData = getLookupData(lookup); + writeAndFlush(Commands.newCloseProducer(producerId, -1L, + lookupData.getBrokerUrl(), + lookupData.getBrokerUrlTls())); + }, + () -> writeAndFlush(Commands.newCloseProducer(producerId, -1L))); // The client does not necessarily know that the producer is closed, but the connection is still // active, and there could be messages in flight already. We want to ignore these messages for a time @@ -3220,9 +3235,13 @@ public void closeConsumer(Consumer consumer, Optional assigned private void closeConsumer(long consumerId, Optional assignedBrokerLookupData) { if (getRemoteEndpointProtocolVersion() >= v5.getValue()) { - writeAndFlush(newCloseConsumer(consumerId, -1L, - assignedBrokerLookupData.map(BrokerLookupData::pulsarServiceUrl).orElse(null), - assignedBrokerLookupData.map(BrokerLookupData::pulsarServiceUrlTls).orElse(null))); + assignedBrokerLookupData.ifPresentOrElse(lookup -> { + LookupData lookupData = getLookupData(lookup); + writeAndFlush(Commands.newCloseConsumer(consumerId, -1L, + lookupData.getBrokerUrl(), + lookupData.getBrokerUrlTls())); + }, + () -> writeAndFlush(Commands.newCloseConsumer(consumerId, -1L, null, null))); } else { close(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java index 32b7c5027281e..cce16061506a1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java @@ -21,7 +21,6 @@ import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import com.google.common.collect.Sets; - import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.commons.lang3.reflect.FieldUtils; @@ -65,7 +64,14 @@ protected ExtensibleLoadManagerImplBaseTest(String defaultTestNamespace) { this.defaultTestNamespace = defaultTestNamespace; } - protected ServiceConfiguration initConfig(ServiceConfiguration conf) { + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + updateConfig(conf); + } + + + protected ServiceConfiguration updateConfig(ServiceConfiguration conf) { conf.setForceDeleteNamespaceAllowed(true); conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); @@ -79,10 +85,9 @@ protected ServiceConfiguration initConfig(ServiceConfiguration conf) { @Override @BeforeClass(alwaysRun = true) protected void setup() throws Exception { - initConfig(conf); super.internalSetup(conf); pulsar1 = pulsar; - var conf2 = initConfig(getDefaultConf()); + var conf2 = updateConfig(getDefaultConf()); additionalPulsarTestContext = createAdditionalPulsarTestContext(conf2); pulsar2 = additionalPulsarTestContext.getPulsarService(); @@ -147,7 +152,7 @@ private void setSecondaryLoadManager() throws IllegalAccessException { FieldUtils.readField(secondaryLoadManager, "serviceUnitStateChannel", true); } - protected CompletableFuture getBundleAsync(PulsarService pulsar, TopicName topic) { + protected static CompletableFuture getBundleAsync(PulsarService pulsar, TopicName topic) { return pulsar.getNamespaceService().getBundleAsync(topic); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index 43c50a8ac54f4..4a9b80c798f86 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -111,6 +111,7 @@ import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -434,6 +435,19 @@ public Object[][] isPersistentTopicSubscriptionTypeTest() { @Test(timeOut = 30_000, dataProvider = "isPersistentTopicSubscriptionTypeTest") public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, SubscriptionType subscriptionType) throws Exception { + testTransferClientReconnectionWithoutLookup(topicDomain, subscriptionType, defaultTestNamespace, admin, + lookupUrl.toString(), pulsar1, pulsar2, primaryLoadManager, secondaryLoadManager); + } + + @Test(enabled = false) + public static void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, + SubscriptionType subscriptionType, + String defaultTestNamespace, + PulsarAdmin admin, String brokerServiceUrl, + PulsarService pulsar1, PulsarService pulsar2, + ExtensibleLoadManager primaryLoadManager, + ExtensibleLoadManager secondaryLoadManager) + throws Exception { var id = String.format("test-tx-client-reconnect-%s-%s", subscriptionType, UUID.randomUUID()); var topic = String.format("%s://%s/%s", topicDomain.toString(), defaultTestNamespace, id); var topicName = TopicName.get(topic); @@ -443,7 +457,8 @@ public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, var consumers = new ArrayList>(); try { var lookups = new ArrayList(); - + var pulsarClient = pulsarClient(brokerServiceUrl, 0); + clients.add(pulsarClient); @Cleanup var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); lookups.add(spyLookupService(pulsarClient)); @@ -451,7 +466,7 @@ public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, var consumerCount = subscriptionType == SubscriptionType.Exclusive ? 1 : 3; for (int i = 0; i < consumerCount; i++) { - var client = newPulsarClient(lookupUrl.toString(), 0); + var client = pulsarClient(brokerServiceUrl, 0); clients.add(client); var consumer = client.newConsumer(Schema.STRING). subscriptionName(id). @@ -478,7 +493,7 @@ public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, dstBrokerUrl = pulsar1.getBrokerId(); dstBrokerServiceUrl = pulsar1.getBrokerServiceUrl(); } - checkOwnershipState(broker, bundle); + checkOwnershipState(broker, bundle, primaryLoadManager, secondaryLoadManager, pulsar1); var messageCountBeforeUnloading = 100; var messageCountAfterUnloading = 100; @@ -572,6 +587,17 @@ public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicSubscriptionTypeTest") public void testUnloadClientReconnectionWithLookup(TopicDomain topicDomain, SubscriptionType subscriptionType) throws Exception { + testUnloadClientReconnectionWithLookup(topicDomain, subscriptionType, defaultTestNamespace, admin, + lookupUrl.toString(), pulsar1); + } + + @Test(enabled = false) + public static void testUnloadClientReconnectionWithLookup(TopicDomain topicDomain, + SubscriptionType subscriptionType, + String defaultTestNamespace, + PulsarAdmin admin, + String brokerServiceUrl, + PulsarService pulsar1) throws Exception { var id = String.format("test-unload-%s-client-reconnect-%s-%s", topicDomain, subscriptionType, UUID.randomUUID()); var topic = String.format("%s://%s/%s", topicDomain, defaultTestNamespace, id); @@ -580,6 +606,7 @@ public void testUnloadClientReconnectionWithLookup(TopicDomain topicDomain, var consumers = new ArrayList>(); try { @Cleanup + var pulsarClient = pulsarClient(brokerServiceUrl, 0); var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); var consumerCount = subscriptionType == SubscriptionType.Exclusive ? 1 : 3; @@ -651,6 +678,16 @@ public Object[][] isPersistentTopicTest() { @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicTest") public void testOptimizeUnloadDisable(TopicDomain topicDomain) throws Exception { + testOptimizeUnloadDisable(topicDomain, defaultTestNamespace, admin, lookupUrl.toString(), pulsar1, pulsar2); + } + + @Test(enabled = false) + public static void testOptimizeUnloadDisable(TopicDomain topicDomain, + String defaultTestNamespace, + PulsarAdmin admin, + String brokerServiceUrl, + PulsarService pulsar1, + PulsarService pulsar2) throws Exception { var id = String.format("test-optimize-unload-disable-%s-%s", topicDomain, UUID.randomUUID()); var topic = String.format("%s://%s/%s", topicDomain, defaultTestNamespace, id); var topicName = TopicName.get(topic); @@ -658,6 +695,9 @@ public void testOptimizeUnloadDisable(TopicDomain topicDomain) throws Exception pulsar1.getConfig().setLoadBalancerMultiPhaseBundleUnload(false); pulsar2.getConfig().setLoadBalancerMultiPhaseBundleUnload(false); + @Cleanup + var pulsarClient = pulsarClient(brokerServiceUrl, 0); + @Cleanup var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); @@ -719,13 +759,16 @@ public void testOptimizeUnloadDisable(TopicDomain topicDomain) throws Exception verify(lookup, times(2)).getBroker(topicName); } - private LookupService spyLookupService(PulsarClient client) throws IllegalAccessException { + protected static LookupService spyLookupService(PulsarClient client) throws IllegalAccessException { LookupService svc = (LookupService) FieldUtils.readDeclaredField(client, "lookup", true); var lookup = spy(svc); FieldUtils.writeDeclaredField(client, "lookup", lookup, true); return lookup; } - private void checkOwnershipState(String broker, NamespaceBundle bundle) + + protected static void checkOwnershipState(String broker, NamespaceBundle bundle, + ExtensibleLoadManager primaryLoadManager, + ExtensibleLoadManager secondaryLoadManager, PulsarService pulsar1) throws ExecutionException, InterruptedException { var targetLoadManager = secondaryLoadManager; var otherLoadManager = primaryLoadManager; @@ -737,6 +780,11 @@ private void checkOwnershipState(String broker, NamespaceBundle bundle) assertFalse(otherLoadManager.checkOwnershipAsync(Optional.empty(), bundle).get()); } + protected void checkOwnershipState(String broker, NamespaceBundle bundle) + throws ExecutionException, InterruptedException { + checkOwnershipState(broker, bundle, primaryLoadManager, secondaryLoadManager, pulsar1); + } + @Test(timeOut = 30 * 1000) public void testSplitBundleAdminAPI() throws Exception { final String namespace = "public/testSplitBundleAdminAPI"; @@ -1745,4 +1793,11 @@ public String name() { } + protected static PulsarClient pulsarClient(String url, int intervalInSecs) throws PulsarClientException { + return + PulsarClient.builder() + .serviceUrl(url) + .statsInterval(intervalInSecs, TimeUnit.SECONDS).build(); + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithAdvertisedListenersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithAdvertisedListenersTest.java new file mode 100644 index 0000000000000..bec7d4d78fe7e --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithAdvertisedListenersTest.java @@ -0,0 +1,98 @@ +/* + * 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.loadbalance.extensions; + +import static org.apache.pulsar.common.util.PortManager.nextLockedFreePort; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.common.naming.TopicDomain; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +/** + * Unit test for {@link ExtensibleLoadManagerImpl with AdvertisedListeners broker configs}. + */ +@Slf4j +@Test(groups = "flaky") +@SuppressWarnings("unchecked") +public class ExtensibleLoadManagerImplWithAdvertisedListenersTest extends ExtensibleLoadManagerImplBaseTest { + + public String brokerServiceUrl; + public ExtensibleLoadManagerImplWithAdvertisedListenersTest() { + super("public/test"); + } + + @Override + protected ServiceConfiguration updateConfig(ServiceConfiguration conf) { + super.updateConfig(conf); + int privatePulsarPort = nextLockedFreePort(); + int publicPulsarPort = nextLockedFreePort(); + conf.setInternalListenerName("internal"); + conf.setBindAddresses("external:pulsar://localhost:" + publicPulsarPort); + conf.setAdvertisedListeners( + "external:pulsar://localhost:" + publicPulsarPort + + ",internal:pulsar://localhost:" + privatePulsarPort); + conf.setWebServicePortTls(Optional.empty()); + conf.setBrokerServicePortTls(Optional.empty()); + conf.setBrokerServicePort(Optional.of(privatePulsarPort)); + conf.setWebServicePort(Optional.of(0)); + brokerServiceUrl = conf.getBindAddresses().replaceAll("external:", ""); + return conf; + } + + @DataProvider(name = "isPersistentTopicSubscriptionTypeTest") + public Object[][] isPersistentTopicSubscriptionTypeTest() { + return new Object[][]{ + {TopicDomain.non_persistent, SubscriptionType.Exclusive}, + {TopicDomain.persistent, SubscriptionType.Key_Shared} + }; + } + + @Test(timeOut = 30_000, dataProvider = "isPersistentTopicSubscriptionTypeTest") + public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, SubscriptionType subscriptionType) + throws Exception { + ExtensibleLoadManagerImplTest.testTransferClientReconnectionWithoutLookup(topicDomain, subscriptionType, + defaultTestNamespace, admin, + brokerServiceUrl, + pulsar1, pulsar2, primaryLoadManager, secondaryLoadManager); + } + + @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicSubscriptionTypeTest") + public void testUnloadClientReconnectionWithLookup(TopicDomain topicDomain, + SubscriptionType subscriptionType) throws Exception { + ExtensibleLoadManagerImplTest.testUnloadClientReconnectionWithLookup(topicDomain, subscriptionType, + defaultTestNamespace, admin, + brokerServiceUrl, + pulsar1); + } + + @DataProvider(name = "isPersistentTopicTest") + public Object[][] isPersistentTopicTest() { + return new Object[][]{{TopicDomain.persistent}, {TopicDomain.non_persistent}}; + } + + @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicTest") + public void testOptimizeUnloadDisable(TopicDomain topicDomain) throws Exception { + ExtensibleLoadManagerImplTest.testOptimizeUnloadDisable(topicDomain, defaultTestNamespace, admin, + brokerServiceUrl, pulsar1, pulsar2); + } + +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithTransactionCoordinatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithTransactionCoordinatorTest.java index 0c95dd85f28e0..ed99b502b7e29 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithTransactionCoordinatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithTransactionCoordinatorTest.java @@ -31,8 +31,8 @@ public ExtensibleLoadManagerImplWithTransactionCoordinatorTest() { } @Override - protected ServiceConfiguration initConfig(ServiceConfiguration conf) { - conf = super.initConfig(conf); + protected ServiceConfiguration updateConfig(ServiceConfiguration conf) { + conf = super.updateConfig(conf); conf.setTransactionCoordinatorEnabled(true); return conf; } From e31dbdd9782360aa8411a751e9b85f093645338d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Jun 2024 22:05:20 +0300 Subject: [PATCH 284/580] [improve][build] Require Java 17 or Java 21 for building Pulsar (#22875) --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index de385c9705932..6e32359f326b4 100644 --- a/pom.xml +++ b/pom.xml @@ -1979,8 +1979,8 @@ flexible messaging model and an intuitive client API. - 17 - Java 17+ is required to build Pulsar. + [17,18),[21,22) + Java 17 or Java 21 is required to build Pulsar. 3.6.1 From 5a8db3687852c21b7d533a64e98563fa323a0897 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 7 Jun 2024 12:13:04 -0700 Subject: [PATCH 285/580] [improve][broker] Reuse topic OpenTelemetry attributes (#22876) --- .../service/PersistentTopicAttributes.java | 73 +++++++++++++++++++ .../apache/pulsar/broker/service/Topic.java | 5 ++ .../broker/service/TopicAttributes.java | 44 +++++++++++ .../nonpersistent/NonPersistentTopic.java | 16 ++++ .../service/persistent/PersistentTopic.java | 14 ++++ .../broker/stats/OpenTelemetryTopicStats.java | 54 ++++---------- .../OpenTelemetryAttributes.java | 17 +++++ 7 files changed, 182 insertions(+), 41 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PersistentTopicAttributes.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicAttributes.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PersistentTopicAttributes.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PersistentTopicAttributes.java new file mode 100644 index 0000000000000..048edafe8848f --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PersistentTopicAttributes.java @@ -0,0 +1,73 @@ +/* + * 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.service; + +import io.opentelemetry.api.common.Attributes; +import lombok.Getter; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; + +@Getter +public class PersistentTopicAttributes extends TopicAttributes { + + private final Attributes timeBasedQuotaAttributes; + private final Attributes sizeBasedQuotaAttributes; + + private final Attributes compactionSuccessAttributes; + private final Attributes compactionFailureAttributes; + + private final Attributes transactionActiveAttributes; + private final Attributes transactionCommittedAttributes; + private final Attributes transactionAbortedAttributes; + + public PersistentTopicAttributes(TopicName topicName) { + super(topicName); + + timeBasedQuotaAttributes = Attributes.builder() + .putAll(commonAttributes) + .putAll(OpenTelemetryAttributes.BacklogQuotaType.TIME.attributes) + .build(); + sizeBasedQuotaAttributes = Attributes.builder() + .putAll(commonAttributes) + .putAll(OpenTelemetryAttributes.BacklogQuotaType.SIZE.attributes) + .build(); + + transactionActiveAttributes = Attributes.builder() + .putAll(commonAttributes) + .putAll(OpenTelemetryAttributes.TransactionStatus.ACTIVE.attributes) + .build(); + transactionCommittedAttributes = Attributes.builder() + .putAll(commonAttributes) + .putAll(OpenTelemetryAttributes.TransactionStatus.COMMITTED.attributes) + .build(); + transactionAbortedAttributes = Attributes.builder() + .putAll(commonAttributes) + .putAll(OpenTelemetryAttributes.TransactionStatus.ABORTED.attributes) + .build(); + + compactionSuccessAttributes = Attributes.builder() + .putAll(commonAttributes) + .putAll(OpenTelemetryAttributes.CompactionStatus.SUCCESS.attributes) + .build(); + compactionFailureAttributes = Attributes.builder() + .putAll(commonAttributes) + .putAll(OpenTelemetryAttributes.CompactionStatus.FAILURE.attributes) + .build(); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java index e902de8a45a10..50a28c7979277 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java @@ -384,4 +384,9 @@ default boolean isSystemTopic() { */ HierarchyTopicPolicies getHierarchyTopicPolicies(); + /** + * Get OpenTelemetry attribute set. + * @return + */ + TopicAttributes getTopicAttributes(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicAttributes.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicAttributes.java new file mode 100644 index 0000000000000..60dc9ae093964 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicAttributes.java @@ -0,0 +1,44 @@ +/* + * 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.service; + +import io.opentelemetry.api.common.Attributes; +import java.util.Objects; +import lombok.Getter; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; + +@Getter +public class TopicAttributes { + + protected final Attributes commonAttributes; + + public TopicAttributes(TopicName topicName) { + Objects.requireNonNull(topicName); + var builder = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, topicName.getDomain().toString()) + .put(OpenTelemetryAttributes.PULSAR_TENANT, topicName.getTenant()) + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, topicName.getNamespace()) + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName.getPartitionedTopicName()); + if (topicName.isPartitioned()) { + builder.put(OpenTelemetryAttributes.PULSAR_PARTITION_INDEX, topicName.getPartitionIndex()); + } + commonAttributes = builder.build(); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 86eab3d38b0aa..ad09e7b756d9d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -36,6 +36,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; import org.apache.commons.lang3.mutable.MutableInt; @@ -62,6 +63,7 @@ import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.SubscriptionOption; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.TopicAttributes; import org.apache.pulsar.broker.service.TopicPolicyListener; import org.apache.pulsar.broker.service.TransportCnx; import org.apache.pulsar.broker.stats.ClusterReplicationMetrics; @@ -117,6 +119,11 @@ protected TopicStats initialValue() { } }; + private volatile TopicAttributes topicAttributes = null; + private static final AtomicReferenceFieldUpdater + TOPIC_ATTRIBUTES_FIELD_UPDATER = AtomicReferenceFieldUpdater.newUpdater( + NonPersistentTopic.class, TopicAttributes.class, "topicAttributes"); + private static class TopicStats { public double averageMsgSize; public double aggMsgRateIn; @@ -1268,4 +1275,13 @@ public boolean isPersistent() { public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { return -1; } + + @Override + public TopicAttributes getTopicAttributes() { + if (topicAttributes != null) { + return topicAttributes; + } + return TOPIC_ATTRIBUTES_FIELD_UPDATER.updateAndGet(this, + old -> old != null ? old : new TopicAttributes(TopicName.get(topic))); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 2165247b1619e..eb15e31b49b56 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -123,6 +123,7 @@ import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.GetStatsOptions; +import org.apache.pulsar.broker.service.PersistentTopicAttributes; import org.apache.pulsar.broker.service.Producer; import org.apache.pulsar.broker.service.Replicator; import org.apache.pulsar.broker.service.StreamingStats; @@ -291,6 +292,11 @@ protected TopicStatsHelper initialValue() { @Getter private final PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); + private volatile PersistentTopicAttributes persistentTopicAttributes = null; + private static final AtomicReferenceFieldUpdater + PERSISTENT_TOPIC_ATTRIBUTES_FIELD_UPDATER = AtomicReferenceFieldUpdater.newUpdater( + PersistentTopic.class, PersistentTopicAttributes.class, "persistentTopicAttributes"); + private volatile TimeBasedBacklogQuotaCheckResult timeBasedBacklogQuotaCheckResult; private static final AtomicReferenceFieldUpdater TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER = AtomicReferenceFieldUpdater.newUpdater( @@ -4355,4 +4361,12 @@ protected boolean isExceedMaximumDeliveryDelay(ByteBuf headersAndPayload) { return false; } + @Override + public PersistentTopicAttributes getTopicAttributes() { + if (persistentTopicAttributes != null) { + return persistentTopicAttributes; + } + return PERSISTENT_TOPIC_ATTRIBUTES_FIELD_UPDATER.updateAndGet(this, + old -> old != null ? old : new PersistentTopicAttributes(TopicName.get(topic))); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStats.java index 1f0735c0ec1f7..b6d3f08907792 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStats.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.stats; -import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.metrics.BatchCallback; import io.opentelemetry.api.metrics.ObservableDoubleMeasurement; import io.opentelemetry.api.metrics.ObservableLongMeasurement; @@ -31,12 +30,10 @@ import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.stats.MetricsUtil; import org.apache.pulsar.compaction.CompactedTopicContext; import org.apache.pulsar.compaction.Compactor; -import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; public class OpenTelemetryTopicStats implements AutoCloseable { @@ -383,16 +380,8 @@ public void close() { } private void recordMetricsForTopic(Topic topic) { - var topicName = TopicName.get(topic.getName()); - var builder = Attributes.builder() - .put(OpenTelemetryAttributes.PULSAR_DOMAIN, topicName.getDomain().toString()) - .put(OpenTelemetryAttributes.PULSAR_TENANT, topicName.getTenant()) - .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, topicName.getNamespace()) - .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName.getPartitionedTopicName()); - if (topicName.isPartitioned()) { - builder.put(OpenTelemetryAttributes.PULSAR_PARTITION_INDEX, topicName.getPartitionIndex()); - } - var attributes = builder.build(); + var topicAttributes = topic.getTopicAttributes(); + var attributes = topicAttributes.getCommonAttributes(); if (topic instanceof AbstractTopic abstractTopic) { subscriptionCounter.record(abstractTopic.getSubscriptions().size(), attributes); @@ -410,6 +399,7 @@ private void recordMetricsForTopic(Topic topic) { } if (topic instanceof PersistentTopic persistentTopic) { + var persistentTopicAttributes = persistentTopic.getTopicAttributes(); var managedLedger = persistentTopic.getManagedLedger(); var managedLedgerStats = persistentTopic.getManagedLedger().getStats(); storageCounter.record(managedLedgerStats.getStoredMessagesSize(), attributes); @@ -428,29 +418,17 @@ private void recordMetricsForTopic(Topic topic) { backlogQuotaAge.record(topic.getBestEffortOldestUnacknowledgedMessageAgeSeconds(), attributes); var backlogQuotaMetrics = persistentTopic.getPersistentTopicMetrics().getBacklogQuotaMetrics(); backlogEvictionCounter.record(backlogQuotaMetrics.getSizeBasedBacklogQuotaExceededEvictionCount(), - Attributes.builder() - .putAll(attributes) - .put(OpenTelemetryAttributes.PULSAR_BACKLOG_QUOTA_TYPE, "size") - .build()); + persistentTopicAttributes.getSizeBasedQuotaAttributes()); backlogEvictionCounter.record(backlogQuotaMetrics.getTimeBasedBacklogQuotaExceededEvictionCount(), - Attributes.builder() - .putAll(attributes) - .put(OpenTelemetryAttributes.PULSAR_BACKLOG_QUOTA_TYPE, "time") - .build()); + persistentTopicAttributes.getTimeBasedQuotaAttributes()); var txnBuffer = persistentTopic.getTransactionBuffer(); - transactionCounter.record(txnBuffer.getOngoingTxnCount(), Attributes.builder() - .putAll(attributes) - .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "active") - .build()); - transactionCounter.record(txnBuffer.getCommittedTxnCount(), Attributes.builder() - .putAll(attributes) - .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "committed") - .build()); - transactionCounter.record(txnBuffer.getAbortedTxnCount(), Attributes.builder() - .putAll(attributes) - .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "aborted") - .build()); + transactionCounter.record(txnBuffer.getOngoingTxnCount(), + persistentTopicAttributes.getTransactionActiveAttributes()); + transactionCounter.record(txnBuffer.getCommittedTxnCount(), + persistentTopicAttributes.getTransactionCommittedAttributes()); + transactionCounter.record(txnBuffer.getAbortedTxnCount(), + persistentTopicAttributes.getTransactionAbortedAttributes()); Optional.ofNullable(pulsar.getNullableCompactor()) .map(Compactor::getStats) @@ -458,15 +436,9 @@ private void recordMetricsForTopic(Topic topic) { .ifPresent(compactionRecord -> { compactionRemovedCounter.record(compactionRecord.getCompactionRemovedEventCount(), attributes); compactionOperationCounter.record(compactionRecord.getCompactionSucceedCount(), - Attributes.builder() - .putAll(attributes) - .put(OpenTelemetryAttributes.PULSAR_COMPACTION_STATUS, "success") - .build()); + persistentTopicAttributes.getCompactionSuccessAttributes()); compactionOperationCounter.record(compactionRecord.getCompactionFailedCount(), - Attributes.builder() - .putAll(attributes) - .put(OpenTelemetryAttributes.PULSAR_COMPACTION_STATUS, "failure") - .build()); + persistentTopicAttributes.getCompactionFailureAttributes()); compactionDurationSeconds.record(MetricsUtil.convertToSeconds( compactionRecord.getCompactionDurationTimeInMills(), TimeUnit.MILLISECONDS), attributes); compactionBytesInCounter.record(compactionRecord.getCompactionReadBytes(), attributes); diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index a3e8a0c1e725c..9783f0e754f63 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -19,6 +19,7 @@ package org.apache.pulsar.opentelemetry; import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; import java.util.List; /** @@ -100,14 +101,30 @@ public interface OpenTelemetryAttributes { * The status of the Pulsar transaction. */ AttributeKey PULSAR_TRANSACTION_STATUS = AttributeKey.stringKey("pulsar.transaction.status"); + enum TransactionStatus { + ACTIVE, + COMMITTED, + ABORTED; + public final Attributes attributes = Attributes.of(PULSAR_TRANSACTION_STATUS, name().toLowerCase()); + } /** * The status of the Pulsar compaction operation. */ AttributeKey PULSAR_COMPACTION_STATUS = AttributeKey.stringKey("pulsar.compaction.status"); + enum CompactionStatus { + SUCCESS, + FAILURE; + public final Attributes attributes = Attributes.of(PULSAR_COMPACTION_STATUS, name().toLowerCase()); + } /** * The type of the backlog quota. */ AttributeKey PULSAR_BACKLOG_QUOTA_TYPE = AttributeKey.stringKey("pulsar.backlog.quota.type"); + enum BacklogQuotaType { + SIZE, + TIME; + public final Attributes attributes = Attributes.of(PULSAR_BACKLOG_QUOTA_TYPE, name().toLowerCase()); + } } From e236f49f4a8f3645eba3b755c6114b63585fa02e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Jun 2024 23:07:01 +0300 Subject: [PATCH 286/580] [fix][misc] Add proper nslookup (included in bind-tools) to docker image (#22878) --- docker/pulsar/Dockerfile | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index f3fea0e1e9d1e..b75519fa91a07 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -81,7 +81,7 @@ FROM apachepulsar/glibc-base:2.38 as glibc FROM alpine:3.19.1 ENV LANG C.UTF-8 -# Install some utilities +# Install some utilities, some are required by Pulsar scripts RUN apk add --no-cache \ bash \ python3 \ @@ -91,7 +91,8 @@ RUN apk add --no-cache \ gcompat \ ca-certificates \ procps \ - curl + curl \ + bind-tools # Upgrade all packages to get latest versions with security fixes RUN apk upgrade --no-cache From ef6fbf40f0ea99ba4b802f04dbeb1cf1c630c9bc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 8 Jun 2024 09:43:34 +0300 Subject: [PATCH 287/580] [improve][ci] Migrate from Gradle Enterprise to Develocity (#22880) --- .gitignore | 2 ++ ...a.groovy => develocity-custom-user-data.groovy} | 0 .mvn/{gradle-enterprise.xml => develocity.xml} | 14 +++++--------- .mvn/extensions.xml | 6 +++--- 4 files changed, 10 insertions(+), 12 deletions(-) rename .mvn/{gradle-enterprise-custom-user-data.groovy => develocity-custom-user-data.groovy} (100%) rename .mvn/{gradle-enterprise.xml => develocity.xml} (67%) diff --git a/.gitignore b/.gitignore index cd00c44200059..80d760cd29df7 100644 --- a/.gitignore +++ b/.gitignore @@ -97,3 +97,5 @@ test-reports/ # Gradle Enterprise .mvn/.gradle-enterprise/ +# Gradle Develocity +.mvn/.develocity/ diff --git a/.mvn/gradle-enterprise-custom-user-data.groovy b/.mvn/develocity-custom-user-data.groovy similarity index 100% rename from .mvn/gradle-enterprise-custom-user-data.groovy rename to .mvn/develocity-custom-user-data.groovy diff --git a/.mvn/gradle-enterprise.xml b/.mvn/develocity.xml similarity index 67% rename from .mvn/gradle-enterprise.xml rename to .mvn/develocity.xml index b9ae41d579944..5c0fbb47c7217 100644 --- a/.mvn/gradle-enterprise.xml +++ b/.mvn/develocity.xml @@ -19,25 +19,21 @@ under the License. --> - - - #{env['GRADLE_ENTERPRISE_ACCESS_KEY']?.trim() > '' and !(env['GITHUB_HEAD_REF']?.matches('(?i).*(experiment|wip|private).*') or env['GITHUB_REPOSITORY']?.matches('(?i).*(experiment|wip|private).*'))} + #{(env['GRADLE_ENTERPRISE_ACCESS_KEY']?.trim() > '' or env['DEVELOCITY_ACCESS_KEY']?.trim() > '') and !(env['GITHUB_HEAD_REF']?.matches('(?i).*(experiment|wip|private).*') or env['GITHUB_REPOSITORY']?.matches('(?i).*(experiment|wip|private).*'))} https://ge.apache.org false - true true true #{isFalse(env['GITHUB_ACTIONS'])} - ALWAYS - true #{{'0.0.0.0'}} @@ -50,4 +46,4 @@ false - + \ No newline at end of file diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index bc051debf614c..4a2117925f163 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -23,12 +23,12 @@ xsi:schemaLocation="http://maven.apache.org/EXTENSIONS/1.0.0 http://maven.apache.org/xsd/core-extensions-1.0.0.xsd"> com.gradle - gradle-enterprise-maven-extension - 1.20.1 + develocity-maven-extension + 1.21.4 com.gradle common-custom-user-data-maven-extension - 1.12.5 + 2.0 From 30f78353895818785b3fa09adef96a9b45057af2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 10 Jun 2024 01:39:11 +0300 Subject: [PATCH 288/580] [fix][build] Add re2/j dependency to pulsar-common and client shading (#22884) --- pulsar-client-admin-shaded/pom.xml | 1 + pulsar-client-all/pom.xml | 1 + pulsar-client-shaded/pom.xml | 1 + pulsar-common/pom.xml | 5 +++++ .../main/java-templates/org/apache/pulsar/PulsarVersion.java | 4 ++-- 5 files changed, 10 insertions(+), 2 deletions(-) diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index 7370ea42a4a5a..96ca2f8de9fd4 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -122,6 +122,7 @@ com.google.protobuf:protobuf-java com.google.guava:guava com.google.code.gson:gson + com.google.re2j:re2j com.fasterxml.jackson.*:* io.netty:* io.netty.incubator:* diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index b73c495ec1b69..27abc1a24c38c 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -166,6 +166,7 @@ com.google.errorprone:* com.google.j2objc:* com.google.code.gson:gson + com.google.re2j:re2j com.fasterxml.jackson.*:* io.netty:netty io.netty:netty-all diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index be2dc028498d8..ca018308731d6 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -144,6 +144,7 @@ com.google.errorprone:* com.google.j2objc:* com.google.code.gson:gson + com.google.re2j:re2j com.fasterxml.jackson.*:* io.netty:* io.netty.incubator:* diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index cdc30dac2897d..62e7bde25603c 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -206,6 +206,11 @@ protobuf-java + + com.google.re2j + re2j + + org.bouncycastle diff --git a/pulsar-common/src/main/java-templates/org/apache/pulsar/PulsarVersion.java b/pulsar-common/src/main/java-templates/org/apache/pulsar/PulsarVersion.java index c597dd327f672..119e46b9536f3 100644 --- a/pulsar-common/src/main/java-templates/org/apache/pulsar/PulsarVersion.java +++ b/pulsar-common/src/main/java-templates/org/apache/pulsar/PulsarVersion.java @@ -18,8 +18,8 @@ */ package org.apache.pulsar; -import com.google.re2j.Matcher; -import com.google.re2j.Pattern; +import java.util.regex.Matcher; +import java.util.regex.Pattern; public class PulsarVersion { From 540134c5b48ade391ee2a67235acb3416be1003c Mon Sep 17 00:00:00 2001 From: Shasank Sekhar Pandey <57795242+shasank112001@users.noreply.github.com> Date: Mon, 10 Jun 2024 13:51:51 +0200 Subject: [PATCH 289/580] [improve][pip] PIP-351: Additional options for Pulsar-Test client to support KeyStore based TLS (#22694) --- pip/pip-351.md | 166 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 166 insertions(+) create mode 100644 pip/pip-351.md diff --git a/pip/pip-351.md b/pip/pip-351.md new file mode 100644 index 0000000000000..17f88b4895533 --- /dev/null +++ b/pip/pip-351.md @@ -0,0 +1,166 @@ + + +# PIP-351: Additional options for Pulsar-Test client to support KeyStore based TLS + +# Background knowledge + + + +In both Pulsar Client and Pulsar Admin, we support the use of KeyStores. This feature is provided by means of the boolean +"useKeyStoreTls". The boolean is also the only way authentication mechanisms such as AuthenticationKeyStoreTls can be utilised +properly, as the logic to use keystores for SSL Connections, from either ClientConfigurationData stored in Pulsar Admin/Client +or AuthData hinges on the "useKeyStoreTls" boolean as can be seen below: + +AsyncHttpConnector.java +```java +if (conf.isUseKeyStoreTls()) { + KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : + new KeyStoreParams(conf.getTlsKeyStoreType(), conf.getTlsKeyStorePath(), + conf.getTlsKeyStorePassword()); + + final SSLContext sslCtx = KeyStoreSSLContext.createClientSslContext( + conf.getSslProvider(), + params.getKeyStoreType(), + params.getKeyStorePath(), + params.getKeyStorePassword(), + conf.isTlsAllowInsecureConnection(), + conf.getTlsTrustStoreType(), + conf.getTlsTrustStorePath(), + conf.getTlsTrustStorePassword(), + conf.getTlsCiphers(), + conf.getTlsProtocols()); + + JsseSslEngineFactory sslEngineFactory = new JsseSslEngineFactory(sslCtx); + confBuilder.setSslEngineFactory(sslEngineFactory); +} +``` + +None of these options can be currently configured when using Pulsar Test client. + +# Motivation + + + +As we already let users both extend authentication and use just the keystore and truststore properties to set up mTLS +connections, without using any authentication plugin class, a lot of them might want to use this method of authentication +during Performance Testing as well. + +I understand that currently mTLS (for testing purposes) can be achieved by using trust and client certificates. +However, the issue of users extending authentication plugin classes and utilizing keystores is still not covered +with the current options. Therefore, I propose we make these already existing options be configured in test clients, +increasing its usability. + +# Goals + +## In Scope + +Create new Arguments for the following properties, in PerformanceBaseArguments.java : +1. useKeyStoreTls +2. trustStoreType +3. trustStorePath +4. trustStorePass +5. keyStoreType +6. keyStorePath +7. keyStorePass + +Update the code to change between TrustCerts and TrustStore based on useKeyStoreTls. + + + +[//]: # (## Out of Scope) + + + + +[//]: # (# High Level Design) + + + +# Detailed Design + +## Design & Implementation Details + + + +Add the options for utilizing keystores as part of performance base arguments, along with forwarding their values +to the client/admin builders. + +## Public-facing Changes + + + +### CLI + +All places we utilize Pulsar Test client, for example Pulsar-Perf will have the following new options: + +1. --use-keystore-tls → Default value = false +2. --truststore-type → Default value = JKS, Possible values = JKS, PKCS12 +3. --truststore-path → Default value = "" +4. --truststore-pass → Default value = "" +5. --keystore-type → Default value = JKS, Possible values = JKS, PKCS12 +6. --keystore-path → Default value = "" +7. --keystore-pass → Default value = "" + + + +# Backward & Forward Compatibility + +The change will not affect any previous releases. The options can also be brought to previous versions, however, I have +noticed that Pulsar has moved away from JCommander in Version 3.2.x to Picocli (currently in master) +Therefore, to add these options to previous versions, the code has to be replicated to those versions. From f6eceedbded53cded4dd751206ebb51d2867e978 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 10 Jun 2024 19:30:24 +0300 Subject: [PATCH 290/580] [fix][cli] Fix Pulsar standalone "--wipe-data" (#22885) --- .../main/java/org/apache/pulsar/PulsarStandalone.java | 9 ++++++++- .../apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java | 2 ++ .../org/apache/pulsar/metadata/bookkeeper/BKCluster.java | 1 + 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java index 7f80aa29f53d9..d0118b06e7c05 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java @@ -18,12 +18,14 @@ */ package org.apache.pulsar; +import static org.apache.commons.io.FileUtils.cleanDirectory; import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; import static org.apache.pulsar.common.naming.SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; import io.netty.util.internal.PlatformDependent; import java.io.File; +import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; @@ -446,7 +448,12 @@ public void close() { void startBookieWithMetadataStore() throws Exception { if (StringUtils.isBlank(metadataStoreUrl)){ log.info("Starting BK with RocksDb metadata store"); - metadataStoreUrl = "rocksdb://" + Paths.get(metadataDir).toAbsolutePath(); + Path metadataDirPath = Paths.get(metadataDir); + metadataStoreUrl = "rocksdb://" + metadataDirPath.toAbsolutePath(); + if (wipeData && Files.exists(metadataDirPath)) { + log.info("Wiping RocksDb metadata store at {}", metadataStoreUrl); + cleanDirectory(metadataDirPath.toFile()); + } } else { log.info("Starting BK with metadata store: {}", metadataStoreUrl); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java b/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java index e8a503c46e006..cf1a30951ebdf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java @@ -194,6 +194,7 @@ private void runZookeeper(int maxCC) throws IOException { : createTempDirectory("zktest"); if (this.clearOldData) { + LOG.info("Wiping Zookeeper data directory at {}", zkDataDir.getAbsolutePath()); cleanDirectory(zkDataDir); } @@ -291,6 +292,7 @@ private void runBookies(ServerConfiguration baseConf) throws Exception { : createTempDirectory("bk" + i + "test"); if (this.clearOldData) { + LOG.info("Wiping Bookie data directory at {}", bkDataDir.getAbsolutePath()); cleanDirectory(bkDataDir); } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java index 8d3a90239efd3..fe2b981ffe995 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java @@ -232,6 +232,7 @@ private ServerConfiguration newServerConfiguration(int index) throws Exception { } if (clusterConf.clearOldData && dataDir.exists()) { + log.info("Wiping Bookie data directory at {}", dataDir.getAbsolutePath()); cleanDirectory(dataDir); } From f17d90e528687fc796cc7e9c5c5b7487a3e3723e Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 10 Jun 2024 12:39:49 -0700 Subject: [PATCH 291/580] [improve] Upgrade IPAddress to 5.5.0 (#22886) --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 25b6787d420df..6769df3903719 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -516,7 +516,7 @@ The Apache Software License, Version 2.0 - io.etcd-jetcd-core-0.7.7.jar - io.etcd-jetcd-grpc-0.7.7.jar * IPAddress - - com.github.seancfoley-ipaddress-5.3.3.jar + - com.github.seancfoley-ipaddress-5.5.0.jar * RxJava - io.reactivex.rxjava3-rxjava-3.0.1.jar * RoaringBitmap diff --git a/pom.xml b/pom.xml index 6e32359f326b4..1514b7da13a17 100644 --- a/pom.xml +++ b/pom.xml @@ -255,7 +255,7 @@ flexible messaging model and an intuitive client API. 0.3.0 2.0 1.10.12 - 5.3.3 + 5.5.0 3.4.3 1.5.2-3 2.0.6 From c326d8e2203b6e9be37f4f2066fd7e90a9b9fb54 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=90=A7=E6=98=93=E5=AE=A2?= Date: Tue, 11 Jun 2024 12:46:04 +0800 Subject: [PATCH 292/580] [fix][misc] Topic name from persistence name should decode local name (#22879) --- .../java/org/apache/pulsar/common/naming/TopicName.java | 5 ++--- .../java/org/apache/pulsar/common/naming/TopicNameTest.java | 6 ++++++ 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java index eebca0e0d7214..e051e01495dbe 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java @@ -358,17 +358,16 @@ public static String fromPersistenceNamingEncoding(String mlName) { String localName; if (parts.size() == 4) { tenant = parts.get(0); - cluster = null; namespacePortion = parts.get(1); domain = parts.get(2); - localName = parts.get(3); + localName = Codec.decode(parts.get(3)); return String.format("%s://%s/%s/%s", domain, tenant, namespacePortion, localName); } else if (parts.size() == 5) { tenant = parts.get(0); cluster = parts.get(1); namespacePortion = parts.get(2); domain = parts.get(3); - localName = parts.get(4); + localName = Codec.decode(parts.get(4)); return String.format("%s://%s/%s/%s/%s", domain, tenant, cluster, namespacePortion, localName); } else { throw new IllegalArgumentException("Invalid managedLedger name: " + mlName); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/naming/TopicNameTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/naming/TopicNameTest.java index 835045f9167dd..485bea3f1addb 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/naming/TopicNameTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/naming/TopicNameTest.java @@ -267,6 +267,12 @@ public void testFromPersistenceNamingEncoding() { } catch (IllegalArgumentException e) { // Exception is expected. } + + // case5: local name with special characters e.g. a:b:c + String topicName = "persistent://tenant/namespace/a:b:c"; + String persistentNamingEncoding = "tenant/namespace/persistent/a%3Ab%3Ac"; + assertEquals(TopicName.get(topicName).getPersistenceNamingEncoding(), persistentNamingEncoding); + assertEquals(TopicName.fromPersistenceNamingEncoding(persistentNamingEncoding), topicName); } From 6b76544433b43185e1990a919a6d5a5b8cf236d6 Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Tue, 11 Jun 2024 15:56:43 +0800 Subject: [PATCH 293/580] [improve] [pip] PIP-355: Enhancing Broker-Level Metrics for Pulsar (#22778) Co-authored-by: Dragos Misca --- pip/pip-355.md | 36 ++++++++++++++++++++++++++++++++++++ 1 file changed, 36 insertions(+) create mode 100644 pip/pip-355.md diff --git a/pip/pip-355.md b/pip/pip-355.md new file mode 100644 index 0000000000000..cb0e41faefd6d --- /dev/null +++ b/pip/pip-355.md @@ -0,0 +1,36 @@ +# PIP-355: Enhancing Broker-Level Metrics for Pulsar + +# Background Knowledge +Pulsar provides broker-level, namespace-level, and topic-level metrics to monitor and analyze the behavior of the Pulsar service. These metrics are accessible through the Prometheus metrics endpoint. Detailed explanations of all metrics can be found on the Pulsar website: [Pulsar Metrics Reference](https://pulsar.apache.org/docs/3.2.x/reference-metrics/) + +# Motivation +Within Pulsar's current metrics framework, the `pulsar_out_bytes_total` metric is utilized to expose the total bytes dispatched by the broker to consumers. However, there are notable limitations and challenges associated with this metric: +- Inclusion of system subscriptions in the total bytes out, alongside user subscriptions, complicates accurate calculation of user-specific data. +- The granularity of the metric (namespace-level vs. topic-subscription level) impacts the scalability and resource consumption when calculating cluster-level total out bytes. + +# Goals +This proposal aims to address the following objectives: +- Simplify the process of calculating cluster-level total out bytes. +- Enable the calculation of total out bytes dispatched to system subscriptions. + +# High-Level Design +To achieve the outlined goals, the proposal introduces two new broker-level metrics: +- `pulsar_broker_out_bytes_total{system_subscription="true|false"}`: Represents the total out bytes dispatched by the broker to consumers. The label `system_subscription="false"` represents total traffic dispatched to user subscriptions, while `system_subscription="true"` represents total traffic dispatched to system cursors and cursor names added by `additionalSystemCursorNames` introduced in [PIP-349](https://github.com/apache/pulsar/pull/22651). +- `pulsar_broker_in_bytes_total{system_topic="true|false"}`: Tracks the total in bytes sent by producers to the broker. The label `system_topic="false"` represents total traffic from user topics, while `system_topic="true"` represents total traffic from system topics. + +# Detailed Design +The implementation involves the introduction of the following broker-level metrics: +- `pulsar_broker_out_bytes_total{system_subscription="true|false"}`: Aggregates the total out bytes from all topics, presented as a broker-level metric. +- `pulsar_broker_in_bytes_total{system_topic="true|false"}`: Calculation of total in bytes across all topics. + +# Metrics +The proposal includes the addition of two new broker-level metrics: +- `pulsar_broker_out_bytes_total{system_subscription="true|false"}` +- `pulsar_broker_in_bytes_total{system_topic="true|false"}` + +# Backward & Forward Compatibility +The proposed changes ensure full compatibility with existing systems and pave the way for seamless integration with future enhancements. + +# Links +- Mailing List discussion thread: https://lists.apache.org/thread/n3vvh6pso9ml7sg3qpww870om5vcfnpv +- Mailing List voting thread: https://lists.apache.org/thread/h4rjcv77wppz96gc31cpr3hw17v9jc4o From 266243cae246a6fa52b4b6c626932885ad44cbf4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Tue, 11 Jun 2024 23:45:12 +0800 Subject: [PATCH 294/580] [improve][broker] Optimize PersistentTopic.getLastDispatchablePosition (#22707) ### Motivation [PersistentTopic#getLastDispatchablePosition](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java#L3776-L3788) is using by [Reader#hasMessageAvailable](https://github.com/apache/pulsar/blob/master/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Reader.java#L116) , [ConsumerImpl#hasMessageAvailable](https://github.com/apache/pulsar/blob/master/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java#L2440-L2448), [Consumer#getLastMessageIdAsync](https://github.com/apache/pulsar/blob/master/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java#L591-L615). The current implementation is read entries from Bookkeeper(or sth else), which leads to low throughput, high latency and heavy load, this PR is for the purpose of optimization. --- .../service/persistent/PersistentTopic.java | 66 +++++++++++++++---- .../buffer/impl/InMemTransactionBuffer.java | 14 +++- .../buffer/impl/TopicTransactionBuffer.java | 11 ++++ .../buffer/impl/TransactionBufferDisable.java | 14 +++- 4 files changed, 89 insertions(+), 16 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index eb15e31b49b56..d78dac899b732 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -317,6 +317,9 @@ private static class EstimateTimeBasedBacklogQuotaCheckResult { Long estimatedOldestUnacknowledgedMessageTimestamp; } + // The last position that can be dispatched to consumers + private volatile Position lastDispatchablePosition; + /*** * We use 3 futures to prevent a new closing if there is an in-progress deletion or closing. We make Pulsar return * the in-progress one when it is called the second time. @@ -3792,18 +3795,57 @@ public Position getLastPosition() { @Override public CompletableFuture getLastDispatchablePosition() { - return ManagedLedgerImplUtils.asyncGetLastValidPosition((ManagedLedgerImpl) ledger, entry -> { - MessageMetadata md = Commands.parseMessageMetadata(entry.getDataBuffer()); - // If a messages has marker will filter by AbstractBaseDispatcher.filterEntriesForConsumer - if (Markers.isServerOnlyMarker(md)) { - return false; - } else if (md.hasTxnidMostBits() && md.hasTxnidLeastBits()) { - // Filter-out transaction aborted messages. - TxnID txnID = new TxnID(md.getTxnidMostBits(), md.getTxnidLeastBits()); - return !isTxnAborted(txnID, (PositionImpl) entry.getPosition()); - } - return true; - }, getMaxReadPosition()); + if (lastDispatchablePosition != null) { + return CompletableFuture.completedFuture(lastDispatchablePosition); + } + return ManagedLedgerImplUtils + .asyncGetLastValidPosition((ManagedLedgerImpl) ledger, entry -> { + MessageMetadata md = Commands.parseMessageMetadata(entry.getDataBuffer()); + // If a messages has marker will filter by AbstractBaseDispatcher.filterEntriesForConsumer + if (Markers.isServerOnlyMarker(md)) { + return false; + } else if (md.hasTxnidMostBits() && md.hasTxnidLeastBits()) { + // Filter-out transaction aborted messages. + TxnID txnID = new TxnID(md.getTxnidMostBits(), md.getTxnidLeastBits()); + return !isTxnAborted(txnID, (PositionImpl) entry.getPosition()); + } + return true; + }, getMaxReadPosition()) + .thenApply(position -> { + // Update lastDispatchablePosition to the given position + updateLastDispatchablePosition(position); + return position; + }); + } + + /** + * Update lastDispatchablePosition if the given position is greater than the lastDispatchablePosition. + * + * @param position + */ + public synchronized void updateLastDispatchablePosition(Position position) { + // Update lastDispatchablePosition to null if the position is null, fallback to + // ManagedLedgerImplUtils#asyncGetLastValidPosition + if (position == null) { + lastDispatchablePosition = null; + return; + } + + PositionImpl position0 = (PositionImpl) position; + // If the position is greater than the maxReadPosition, ignore + if (position0.compareTo(getMaxReadPosition()) > 0) { + return; + } + // If the lastDispatchablePosition is null, set it to the position + if (lastDispatchablePosition == null) { + lastDispatchablePosition = position; + return; + } + // If the position is greater than the lastDispatchablePosition, update it + PositionImpl lastDispatchablePosition0 = (PositionImpl) lastDispatchablePosition; + if (position0.compareTo(lastDispatchablePosition0) > 0) { + lastDispatchablePosition = position; + } } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java index bab7b64c608c4..533d0716d413c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java @@ -377,8 +377,11 @@ public boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { @Override public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { - if (!isMarkerMessage && maxReadPositionCallBack != null) { - maxReadPositionCallBack.maxReadPositionMovedForward(null, position); + if (!isMarkerMessage) { + updateLastDispatchablePosition(position); + if (maxReadPositionCallBack != null) { + maxReadPositionCallBack.maxReadPositionMovedForward(null, position); + } } } @@ -436,4 +439,11 @@ public long getCommittedTxnCount() { .filter(txnBuffer -> txnBuffer.status.equals(TxnStatus.COMMITTED)) .count(); } + + // ThreadSafe + private void updateLastDispatchablePosition(Position position) { + if (topic instanceof PersistentTopic t) { + t.updateLastDispatchablePosition(position); + } + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index dfb73815e08d7..fbd4ddf7da053 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -297,6 +297,11 @@ private void handleTransactionMessage(TxnID txnId, Position position) { } } + // ThreadSafe + private void updateLastDispatchablePosition(Position position) { + topic.updateLastDispatchablePosition(position); + } + @Override public CompletableFuture openTransactionBufferReader(TxnID txnID, long startSequenceId) { return null; @@ -459,6 +464,8 @@ void removeTxnAndUpdateMaxReadPosition(TxnID txnID) { } else { updateMaxReadPosition((PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(), false); } + // Update the last dispatchable position to null if there is a TXN finished. + updateLastDispatchablePosition(null); } /** @@ -523,6 +530,10 @@ public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean i } } } + // If the message is a normal message, update the last dispatchable position. + if (!isMarkerMessage) { + updateLastDispatchablePosition(position); + } } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java index ebd61dbaa82ec..6f5dc0cd4d0dd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java @@ -99,8 +99,11 @@ public boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { @Override public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { - if (!isMarkerMessage && maxReadPositionCallBack != null) { - maxReadPositionCallBack.maxReadPositionMovedForward(null, position); + if (!isMarkerMessage) { + updateLastDispatchablePosition(position); + if (maxReadPositionCallBack != null) { + maxReadPositionCallBack.maxReadPositionMovedForward(null, position); + } } } @@ -148,4 +151,11 @@ public long getAbortedTxnCount() { public long getCommittedTxnCount() { return 0; } + + // ThreadSafe + private void updateLastDispatchablePosition(Position position) { + if (topic instanceof PersistentTopic t) { + t.updateLastDispatchablePosition(position); + } + } } From 1770cbc20c12da596fc7cf436ff23271b7771682 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Wed, 12 Jun 2024 10:47:28 +0800 Subject: [PATCH 295/580] [improve] [broker] Fail fast when it failed to create LoadSheddingStrategy instance (#22827) --- .../loadbalance/impl/ModularLoadManagerImpl.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index 5d08ea9c3c3be..764580e9b6d95 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -292,15 +292,8 @@ private void handleMetadataSessionEvent(SessionEvent e) { } private LoadSheddingStrategy createLoadSheddingStrategy() { - try { - return Reflections.createInstance(conf.getLoadBalancerLoadSheddingStrategy(), LoadSheddingStrategy.class, - Thread.currentThread().getContextClassLoader()); - } catch (Exception e) { - log.error("Error when trying to create load shedding strategy: {}", - conf.getLoadBalancerLoadPlacementStrategy(), e); - } - log.error("create load shedding strategy failed. using OverloadShedder instead."); - return new OverloadShedder(); + return Reflections.createInstance(conf.getLoadBalancerLoadSheddingStrategy(), LoadSheddingStrategy.class, + Thread.currentThread().getContextClassLoader()); } /** From c724f02e6ab4f342e805b21cc99c394b31aaf612 Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Wed, 12 Jun 2024 11:21:03 +0800 Subject: [PATCH 296/580] [improve] [broker] PIP-355: Enhancing Broker-Level Metrics for Pulsar (#22779) PIP: #22778 --- .../pulsar/broker/service/AbstractTopic.java | 21 +++++ .../nonpersistent/NonPersistentTopic.java | 6 ++ .../service/persistent/PersistentTopic.java | 13 ++- .../prometheus/AggregatedBrokerStats.java | 12 +++ .../prometheus/NamespaceStatsAggregator.java | 29 +++++- .../broker/stats/prometheus/TopicStats.java | 2 + .../broker/stats/PrometheusMetricsTest.java | 88 +++++++++++++++++++ .../policies/data/stats/TopicStatsImpl.java | 6 ++ 8 files changed, 170 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index b6ce43b060c6f..572b54e0d3e79 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; import static org.apache.bookkeeper.mledger.impl.ManagedLedgerMBeanImpl.ENTRY_LATENCY_BUCKETS_USEC; +import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; import com.google.common.base.MoreObjects; import java.util.ArrayList; import java.util.Arrays; @@ -32,6 +33,7 @@ import java.util.Optional; import java.util.Queue; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; @@ -128,6 +130,7 @@ public abstract class AbstractTopic implements Topic, TopicPolicyListener RATE_LIMITED_UPDATER = @@ -157,10 +160,13 @@ public abstract class AbstractTopic implements Topic, TopicPolicyListener> entryFilters; protected volatile boolean transferring = false; private volatile List activeRateLimiters; + protected Set additionalSystemCursorNames = new TreeSet<>(); + public AbstractTopic(String topic, BrokerService brokerService) { this.topic = topic; this.brokerService = brokerService; @@ -176,6 +182,8 @@ public AbstractTopic(String topic, BrokerService brokerService) { this.preciseTopicPublishRateLimitingEnable = config.isPreciseTopicPublishRateLimiterEnable(); topicPublishRateLimiter = new PublishRateLimiterImpl(brokerService.getPulsar().getMonotonicSnapshotClock()); updateActiveRateLimiters(); + + additionalSystemCursorNames = brokerService.pulsar().getConfiguration().getAdditionalSystemCursorNames(); } public SubscribeRate getSubscribeRate() { @@ -921,6 +929,10 @@ public void incrementPublishCount(Producer producer, int numOfMessages, long msg // increase counters bytesInCounter.add(msgSizeInBytes); msgInCounter.add(numOfMessages); + + if (isSystemTopic()) { + systemTopicBytesInCounter.add(msgSizeInBytes); + } } private void handlePublishThrottling(Producer producer, int numOfMessages, long msgSizeInBytes) { @@ -1184,6 +1196,10 @@ public long getMsgOutCounter() { + sumSubscriptions(AbstractSubscription::getMsgOutCounter); } + public long getSystemTopicBytesInCounter() { + return systemTopicBytesInCounter.longValue(); + } + public long getBytesOutCounter() { return bytesOutFromRemovedSubscriptions.longValue() + sumSubscriptions(AbstractSubscription::getBytesOutCounter); @@ -1369,4 +1385,9 @@ public static Optional getMigratedClusterUrl(PulsarService pulsar, S } return Optional.empty(); } + + public boolean isSystemCursor(String sub) { + return COMPACTION_SUBSCRIPTION.equals(sub) + || (additionalSystemCursorNames != null && additionalSystemCursorNames.contains(sub)); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index ad09e7b756d9d..a6f65f6da3284 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -940,9 +940,11 @@ public CompletableFuture asyncGetStats(GetStatsOptions stats.averageMsgSize = stats.msgRateIn == 0.0 ? 0.0 : (stats.msgThroughputIn / stats.msgRateIn); stats.msgInCounter = getMsgInCounter(); stats.bytesInCounter = getBytesInCounter(); + stats.systemTopicBytesInCounter = getSystemTopicBytesInCounter(); stats.waitingPublishers = getWaitingProducersCount(); stats.bytesOutCounter = bytesOutFromRemovedSubscriptions.longValue(); stats.msgOutCounter = msgOutFromRemovedSubscriptions.longValue(); + stats.bytesOutInternalCounter = bytesOutFromRemovedSystemSubscriptions.longValue(); subscriptions.forEach((name, subscription) -> { NonPersistentSubscriptionStatsImpl subStats = subscription.getStats(getStatsOptions); @@ -952,6 +954,10 @@ public CompletableFuture asyncGetStats(GetStatsOptions stats.bytesOutCounter += subStats.bytesOutCounter; stats.msgOutCounter += subStats.msgOutCounter; stats.getSubscriptions().put(name, subStats); + + if (isSystemCursor(name)) { + stats.bytesOutInternalCounter += subStats.bytesOutCounter; + } }); replicators.forEach((cluster, replicator) -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index d78dac899b732..d9f9c4689f6ed 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -40,7 +40,6 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -287,7 +286,6 @@ protected TopicStatsHelper initialValue() { private final ExecutorService orderedExecutor; private volatile CloseFutures closeFutures; - private Set additionalSystemCursorNames = new TreeSet<>(); @Getter private final PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); @@ -431,7 +429,6 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS } else { shadowSourceTopic = null; } - additionalSystemCursorNames = brokerService.pulsar().getConfiguration().getAdditionalSystemCursorNames(); } @Override @@ -1401,6 +1398,10 @@ void removeSubscription(String subscriptionName) { SubscriptionStatsImpl stats = sub.getStats(new GetStatsOptions(false, false, false, false, false)); bytesOutFromRemovedSubscriptions.add(stats.bytesOutCounter); msgOutFromRemovedSubscriptions.add(stats.msgOutCounter); + + if (isSystemCursor(subscriptionName)) { + bytesOutFromRemovedSystemSubscriptions.add(stats.bytesOutCounter); + } } } @@ -2566,10 +2567,12 @@ public CompletableFuture asyncGetStats(GetStatsOptions stats.averageMsgSize = stats.msgRateIn == 0.0 ? 0.0 : (stats.msgThroughputIn / stats.msgRateIn); stats.msgInCounter = getMsgInCounter(); stats.bytesInCounter = getBytesInCounter(); + stats.systemTopicBytesInCounter = getSystemTopicBytesInCounter(); stats.msgChunkPublished = this.msgChunkPublished; stats.waitingPublishers = getWaitingProducersCount(); stats.bytesOutCounter = bytesOutFromRemovedSubscriptions.longValue(); stats.msgOutCounter = msgOutFromRemovedSubscriptions.longValue(); + stats.bytesOutInternalCounter = bytesOutFromRemovedSystemSubscriptions.longValue(); stats.publishRateLimitedTimes = publishRateLimitedTimes; TransactionBuffer txnBuffer = getTransactionBuffer(); stats.ongoingTxnCount = txnBuffer.getOngoingTxnCount(); @@ -2596,6 +2599,10 @@ public CompletableFuture asyncGetStats(GetStatsOptions topicMetricBean.labelsAndValues = v.labelsAndValues; topicMetricBean.value += v.value; }); + + if (isSystemCursor(name)) { + stats.bytesOutInternalCounter += subStats.bytesOutCounter; + } }); replicators.forEach((cluster, replicator) -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java index 037fb29a999e3..85096be9b00f4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java @@ -35,6 +35,10 @@ public class AggregatedBrokerStats { public long msgBacklog; public long sizeBasedBacklogQuotaExceededEvictionCount; public long timeBasedBacklogQuotaExceededEvictionCount; + public long bytesInCounter; + public long bytesOutCounter; + public long systemTopicBytesInCounter; + public long bytesOutInternalCounter; @SuppressWarnings("DuplicatedCode") void updateStats(TopicStats stats) { @@ -54,6 +58,10 @@ void updateStats(TopicStats stats) { msgBacklog += stats.msgBacklog; timeBasedBacklogQuotaExceededEvictionCount += stats.timeBasedBacklogQuotaExceededEvictionCount; sizeBasedBacklogQuotaExceededEvictionCount += stats.sizeBasedBacklogQuotaExceededEvictionCount; + bytesInCounter += stats.bytesInCounter; + bytesOutCounter += stats.bytesOutCounter; + systemTopicBytesInCounter += stats.systemTopicBytesInCounter; + bytesOutInternalCounter += stats.bytesOutInternalCounter; } @SuppressWarnings("DuplicatedCode") @@ -74,5 +82,9 @@ public void reset() { msgBacklog = 0; sizeBasedBacklogQuotaExceededEvictionCount = 0; timeBasedBacklogQuotaExceededEvictionCount = 0; + bytesInCounter = 0; + bytesOutCounter = 0; + systemTopicBytesInCounter = 0; + bytesOutInternalCounter = 0; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java index 3728c3edd1e8b..3bbc9100b364f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java @@ -211,6 +211,8 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include stats.msgInCounter = tStatus.msgInCounter; stats.bytesInCounter = tStatus.bytesInCounter; stats.msgOutCounter = tStatus.msgOutCounter; + stats.systemTopicBytesInCounter = tStatus.systemTopicBytesInCounter; + stats.bytesOutInternalCounter = tStatus.getBytesOutInternalCounter(); stats.bytesOutCounter = tStatus.bytesOutCounter; stats.averageMsgSize = tStatus.averageMsgSize; stats.publishRateLimitedTimes = tStatus.publishRateLimitedTimes; @@ -358,6 +360,16 @@ private static void printBrokerStats(PrometheusMetricStreams stream, String clus brokerStats.timeBasedBacklogQuotaExceededEvictionCount, cluster, BacklogQuotaType.message_age); writeMetric(stream, "pulsar_broker_msg_backlog", brokerStats.msgBacklog, cluster); + long userOutBytes = brokerStats.bytesOutCounter - brokerStats.bytesOutInternalCounter; + writeMetric(stream, "pulsar_broker_out_bytes_total", + userOutBytes, cluster, "system_subscription", "false"); + writeMetric(stream, "pulsar_broker_out_bytes_total", + brokerStats.bytesOutInternalCounter, cluster, "system_subscription", "true"); + long userTopicInBytes = brokerStats.bytesInCounter - brokerStats.systemTopicBytesInCounter; + writeMetric(stream, "pulsar_broker_in_bytes_total", + userTopicInBytes, cluster, "system_topic", "false"); + writeMetric(stream, "pulsar_broker_in_bytes_total", + brokerStats.systemTopicBytesInCounter, cluster, "system_topic", "true"); } private static void printTopicsCountStats(PrometheusMetricStreams stream, Map namespaceTopicsCount, @@ -412,7 +424,8 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat namespace); stats.bucketDelayedIndexStats.forEach((k, metric) -> { - writeMetric(stream, metric.name, metric.value, cluster, namespace, metric.labelsAndValues); + String[] labels = ArrayUtils.addAll(new String[]{"namespace", namespace}, metric.labelsAndValues); + writeMetric(stream, metric.name, metric.value, cluster, labels); }); writePulsarMsgBacklog(stream, stats.msgBacklog, cluster, namespace); @@ -534,13 +547,21 @@ private static void writeMetric(PrometheusMetricStreams stream, String metricNam stream.writeSample(metricName, value, "cluster", cluster); } + private static void writeMetric(PrometheusMetricStreams stream, String metricName, Number value, + String cluster, String... extraLabelsAndValues) { + String[] labels = ArrayUtils.addAll(new String[]{"cluster", cluster}, extraLabelsAndValues); + stream.writeSample(metricName, value, labels); + } + + private static void writeMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, - String namespace, String... extraLabelsAndValues) { - String[] labelsAndValues = new String[]{"cluster", cluster, "namespace", namespace}; - String[] labels = ArrayUtils.addAll(labelsAndValues, extraLabelsAndValues); + String namespace) { + String[] labels = new String[]{"cluster", cluster, "namespace", namespace}; stream.writeSample(metricName, value, labels); } + + private static void writeReplicationStat(PrometheusMetricStreams stream, String metricName, AggregatedNamespaceStats namespaceStats, Function sampleValueFunction, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java index e8ab7b095dc3c..9eb4077225ca1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java @@ -56,6 +56,8 @@ class TopicStats { long msgOutCounter; @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.BYTES_OUT_COUNTER) long bytesOutCounter; + long systemTopicBytesInCounter; + long bytesOutInternalCounter; @PulsarDeprecatedMetric // Can be derived from MESSAGE_IN_COUNTER and BYTES_IN_COUNTER double averageMsgSize; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index 1fe0e99b49874..0d7f8eb0aa3e8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -205,6 +205,94 @@ public void testPublishRateLimitedTimes() throws Exception { producer3.close(); } + @Test + public void testBrokerMetrics() throws Exception { + cleanup(); + conf.setAdditionalSystemCursorNames(Set.of("test-cursor")); + setup(); + + Producer p1 = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic1").create(); + Producer p2 = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic2").create(); + // system topic + Producer p3 = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/__change_events").create(); + + Consumer c1 = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic1") + .subscriptionName("test") + .subscribe(); + + // additional system cursor + Consumer c2 = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/my-topic2") + .subscriptionName("test-cursor") + .subscribe(); + + Consumer c3 = pulsarClient.newConsumer() + .topic("persistent://my-property/use/my-ns/__change_events") + .subscriptionName("test-v1") + .subscribe(); + + final int messages = 10; + for (int i = 0; i < messages; i++) { + String message = "my-message-" + i; + p1.send(message.getBytes()); + p2.send(message.getBytes()); + p3.send(message.getBytes()); + } + + for (int i = 0; i < messages; i++) { + c1.acknowledge(c1.receive()); + c2.acknowledge(c2.receive()); + c3.acknowledge(c3.receive()); + } + + // unsubscribe to test remove cursor impact on metric + c1.unsubscribe(); + c2.unsubscribe(); + + //admin.topics().unload("persistent://my-property/use/my-ns/my-topic1"); + + ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); + String metricsStr = statsOut.toString(); + Multimap metrics = parseMetrics(metricsStr); + + metrics.entries().forEach(e -> { + System.out.println(e.getKey() + ": " + e.getValue()); + }); + + List bytesOutTotal = (List) metrics.get("pulsar_broker_out_bytes_total"); + List bytesInTotal = (List) metrics.get("pulsar_broker_in_bytes_total"); + assertEquals(bytesOutTotal.size(), 2); + assertEquals(bytesInTotal.size(), 2); + + double systemOutBytes = 0.0; + double userOutBytes = 0.0; + switch (bytesOutTotal.get(0).tags.get("system_subscription").toString()) { + case "true": + systemOutBytes = bytesOutTotal.get(0).value; + userOutBytes = bytesOutTotal.get(1).value; + case "false": + systemOutBytes = bytesOutTotal.get(1).value; + userOutBytes = bytesOutTotal.get(0).value; + } + + double systemInBytes = 0.0; + double userInBytes = 0.0; + switch (bytesInTotal.get(0).tags.get("system_topic").toString()) { + case "true": + systemInBytes = bytesInTotal.get(0).value; + userInBytes = bytesInTotal.get(1).value; + case "false": + systemInBytes = bytesInTotal.get(1).value; + userInBytes = bytesInTotal.get(0).value; + } + + assertEquals(userOutBytes / 2, systemOutBytes); + assertEquals(userInBytes / 2, systemInBytes); + assertEquals(userOutBytes + systemOutBytes, userInBytes + systemInBytes); + } + @Test public void testMetricsTopicCount() throws Exception { String ns1 = "prop/ns-abc1"; diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java index 70cf4cd341484..022fffd3a7e59 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java @@ -66,12 +66,18 @@ public class TopicStatsImpl implements TopicStats { /** Total messages published to the topic (msg). */ public long msgInCounter; + /** Total bytes published to the system topic (bytes). */ + public long systemTopicBytesInCounter; + /** Total bytes delivered to consumer (bytes). */ public long bytesOutCounter; /** Total messages delivered to consumer (msg). */ public long msgOutCounter; + /** Total bytes delivered to internal cursors. */ + public long bytesOutInternalCounter; + /** Average size of published messages (bytes). */ public double averageMsgSize; From 75d7e557d84bf2cca2ec791dfe8479b8a6df7875 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 13 Jun 2024 01:24:04 +0300 Subject: [PATCH 297/580] [improve][misc] Upgrade to Netty 4.1.111.Final and switch to use grpc-netty-shaded (#22892) --- distribution/server/pom.xml | 13 ++ .../server/src/assemble/LICENSE.bin.txt | 50 +++-- .../shell/src/assemble/LICENSE.bin.txt | 40 ++-- jetcd-core-shaded/pom.xml | 187 ++++++++++++++++++ pom.xml | 60 +++++- pulsar-broker/pom.xml | 12 ++ pulsar-functions/instance/pom.xml | 9 +- pulsar-metadata/pom.xml | 11 +- .../metadata/impl/EtcdMetadataStore.java | 6 +- 9 files changed, 329 insertions(+), 59 deletions(-) create mode 100644 jetcd-core-shaded/pom.xml diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index adabddfa31da4..c42b0a137850c 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -39,6 +39,19 @@ ${project.version} + + ${project.groupId} + pulsar-metadata + ${project.version} + + + + ${project.groupId} + jetcd-core-shaded + ${project.version} + shaded + + ${project.groupId} pulsar-docs-tools diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 6769df3903719..1a66ab6d70a2f 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -292,27 +292,27 @@ The Apache Software License, Version 2.0 - org.apache.commons-commons-lang3-3.11.jar - org.apache.commons-commons-text-1.10.0.jar * Netty - - io.netty-netty-buffer-4.1.108.Final.jar - - io.netty-netty-codec-4.1.108.Final.jar - - io.netty-netty-codec-dns-4.1.108.Final.jar - - io.netty-netty-codec-http-4.1.108.Final.jar - - io.netty-netty-codec-http2-4.1.108.Final.jar - - io.netty-netty-codec-socks-4.1.108.Final.jar - - io.netty-netty-codec-haproxy-4.1.108.Final.jar - - io.netty-netty-common-4.1.108.Final.jar - - io.netty-netty-handler-4.1.108.Final.jar - - io.netty-netty-handler-proxy-4.1.108.Final.jar - - io.netty-netty-resolver-4.1.108.Final.jar - - io.netty-netty-resolver-dns-4.1.108.Final.jar - - io.netty-netty-resolver-dns-classes-macos-4.1.108.Final.jar - - io.netty-netty-resolver-dns-native-macos-4.1.108.Final-osx-aarch_64.jar - - io.netty-netty-resolver-dns-native-macos-4.1.108.Final-osx-x86_64.jar - - io.netty-netty-transport-4.1.108.Final.jar - - io.netty-netty-transport-classes-epoll-4.1.108.Final.jar - - io.netty-netty-transport-native-epoll-4.1.108.Final-linux-aarch_64.jar - - io.netty-netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar - - io.netty-netty-transport-native-unix-common-4.1.108.Final.jar - - io.netty-netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar + - io.netty-netty-buffer-4.1.111.Final.jar + - io.netty-netty-codec-4.1.111.Final.jar + - io.netty-netty-codec-dns-4.1.111.Final.jar + - io.netty-netty-codec-http-4.1.111.Final.jar + - io.netty-netty-codec-http2-4.1.111.Final.jar + - io.netty-netty-codec-socks-4.1.111.Final.jar + - io.netty-netty-codec-haproxy-4.1.111.Final.jar + - io.netty-netty-common-4.1.111.Final.jar + - io.netty-netty-handler-4.1.111.Final.jar + - io.netty-netty-handler-proxy-4.1.111.Final.jar + - io.netty-netty-resolver-4.1.111.Final.jar + - io.netty-netty-resolver-dns-4.1.111.Final.jar + - io.netty-netty-resolver-dns-classes-macos-4.1.111.Final.jar + - io.netty-netty-resolver-dns-native-macos-4.1.111.Final-osx-aarch_64.jar + - io.netty-netty-resolver-dns-native-macos-4.1.111.Final-osx-x86_64.jar + - io.netty-netty-transport-4.1.111.Final.jar + - io.netty-netty-transport-classes-epoll-4.1.111.Final.jar + - io.netty-netty-transport-native-epoll-4.1.111.Final-linux-aarch_64.jar + - io.netty-netty-transport-native-epoll-4.1.111.Final-linux-x86_64.jar + - io.netty-netty-transport-native-unix-common-4.1.111.Final.jar + - io.netty-netty-transport-native-unix-common-4.1.111.Final-linux-x86_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.65.Final.jar - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar @@ -434,7 +434,6 @@ The Apache Software License, Version 2.0 - io.grpc-grpc-auth-1.56.0.jar - io.grpc-grpc-context-1.56.0.jar - io.grpc-grpc-core-1.56.0.jar - - io.grpc-grpc-netty-1.56.0.jar - io.grpc-grpc-protobuf-1.56.0.jar - io.grpc-grpc-protobuf-lite-1.56.0.jar - io.grpc-grpc-stub-1.56.0.jar @@ -498,7 +497,6 @@ The Apache Software License, Version 2.0 - io.vertx-vertx-core-4.5.8.jar - io.vertx-vertx-web-4.5.8.jar - io.vertx-vertx-web-common-4.5.8.jar - - io.vertx-vertx-grpc-4.5.8.jar * Apache ZooKeeper - org.apache.zookeeper-zookeeper-3.9.2.jar - org.apache.zookeeper-zookeeper-jute-3.9.2.jar @@ -510,11 +508,7 @@ The Apache Software License, Version 2.0 - com.google.http-client-google-http-client-1.41.0.jar - com.google.auto.value-auto-value-annotations-1.10.1.jar - com.google.re2j-re2j-1.7.jar - * Jetcd - - io.etcd-jetcd-api-0.7.7.jar - - io.etcd-jetcd-common-0.7.7.jar - - io.etcd-jetcd-core-0.7.7.jar - - io.etcd-jetcd-grpc-0.7.7.jar + * Jetcd - shaded * IPAddress - com.github.seancfoley-ipaddress-5.5.0.jar * RxJava diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 5c3b051cfdd70..ff590023ff3a5 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -347,23 +347,23 @@ The Apache Software License, Version 2.0 - commons-text-1.10.0.jar - commons-compress-1.26.0.jar * Netty - - netty-buffer-4.1.108.Final.jar - - netty-codec-4.1.108.Final.jar - - netty-codec-dns-4.1.108.Final.jar - - netty-codec-http-4.1.108.Final.jar - - netty-codec-socks-4.1.108.Final.jar - - netty-codec-haproxy-4.1.108.Final.jar - - netty-common-4.1.108.Final.jar - - netty-handler-4.1.108.Final.jar - - netty-handler-proxy-4.1.108.Final.jar - - netty-resolver-4.1.108.Final.jar - - netty-resolver-dns-4.1.108.Final.jar - - netty-transport-4.1.108.Final.jar - - netty-transport-classes-epoll-4.1.108.Final.jar - - netty-transport-native-epoll-4.1.108.Final-linux-aarch_64.jar - - netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.108.Final.jar - - netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar + - netty-buffer-4.1.111.Final.jar + - netty-codec-4.1.111.Final.jar + - netty-codec-dns-4.1.111.Final.jar + - netty-codec-http-4.1.111.Final.jar + - netty-codec-socks-4.1.111.Final.jar + - netty-codec-haproxy-4.1.111.Final.jar + - netty-common-4.1.111.Final.jar + - netty-handler-4.1.111.Final.jar + - netty-handler-proxy-4.1.111.Final.jar + - netty-resolver-4.1.111.Final.jar + - netty-resolver-dns-4.1.111.Final.jar + - netty-transport-4.1.111.Final.jar + - netty-transport-classes-epoll-4.1.111.Final.jar + - netty-transport-native-epoll-4.1.111.Final-linux-aarch_64.jar + - netty-transport-native-epoll-4.1.111.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.111.Final.jar + - netty-transport-native-unix-common-4.1.111.Final-linux-x86_64.jar - netty-tcnative-boringssl-static-2.0.65.Final.jar - netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar - netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar @@ -374,9 +374,9 @@ The Apache Software License, Version 2.0 - netty-incubator-transport-classes-io_uring-0.0.24.Final.jar - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-aarch_64.jar - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-x86_64.jar - - netty-resolver-dns-classes-macos-4.1.108.Final.jar - - netty-resolver-dns-native-macos-4.1.108.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.108.Final-osx-x86_64.jar + - netty-resolver-dns-classes-macos-4.1.111.Final.jar + - netty-resolver-dns-native-macos-4.1.111.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.111.Final-osx-x86_64.jar * Prometheus client - simpleclient-0.16.0.jar - simpleclient_log4j2-0.16.0.jar diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml new file mode 100644 index 0000000000000..d8819a1148a21 --- /dev/null +++ b/jetcd-core-shaded/pom.xml @@ -0,0 +1,187 @@ + + + + 4.0.0 + + org.apache.pulsar + pulsar + 3.4.0-SNAPSHOT + + + jetcd-core-shaded + Apache Pulsar :: jetcd-core shaded + + + + io.etcd + jetcd-core + + + io.grpc + grpc-netty + + + io.netty + * + + + + + io.grpc + grpc-netty-shaded + + + + dev.failsafe + failsafe + + + io.grpc + grpc-protobuf + + + io.grpc + grpc-stub + + + io.grpc + grpc-grpclb + + + io.grpc + grpc-util + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + true + true + false + + + io.etcd:* + io.vertx:* + + + + + + io.vertx + org.apache.pulsar.jetcd.shaded.io.vertx + + + + io.grpc.netty + io.grpc.netty.shaded.io.grpc.netty + + + + io.netty + io.grpc.netty.shaded.io.netty + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + META-INF/maven/${project.groupId}/${project.artifactId}/pom.xml + + + + + + + + META-INF/maven/${project.groupId}/${project.artifactId}/pom.xml + ${project.basedir}/dependency-reduced-pom.xml + + + + true + shaded + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + attach-shaded-jar + package + + attach-artifact + + + + + ${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar + jar + shaded + + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + ${maven-antrun-plugin.version} + + + unpack-shaded-jar + package + + run + + + + + + + + + + + + diff --git a/pom.xml b/pom.xml index 1514b7da13a17..71562619c18d5 100644 --- a/pom.xml +++ b/pom.xml @@ -148,7 +148,7 @@ flexible messaging model and an intuitive client API. 1.1.10.5 4.1.12.1 5.1.0 - 4.1.108.Final + 4.1.111.Final 0.0.24.Final 9.4.54.v20240208 2.5.2 @@ -302,6 +302,7 @@ flexible messaging model and an intuitive client API. 2.3.0 3.4.1 3.1.0 + 3.6.0 1.1.0 1.5.0 3.1.2 @@ -584,6 +585,10 @@ flexible messaging model and an intuitive client API. jose4j org.bitbucket.b_c + + io.grpc + grpc-netty + @@ -1053,12 +1058,51 @@ flexible messaging model and an intuitive client API. io.etcd jetcd-core ${jetcd.version} + + + io.grpc + grpc-netty + + - io.etcd jetcd-test ${jetcd.version} + + + io.grpc + grpc-netty + + + io.etcd + jetcd-core + + + io.etcd + jetcd-api + + + io.vertx + * + + + + + ${project.groupId} + jetcd-core-shaded + ${project.version} + shaded + + + io.etcd + * + + + io.vertx + * + + @@ -1152,6 +1196,10 @@ flexible messaging model and an intuitive client API. com.squareup.okio okio + + io.grpc + grpc-netty + @@ -2142,6 +2190,11 @@ flexible messaging model and an intuitive client API. docker-maven-plugin ${docker-maven.version} + + org.codehaus.mojo + build-helper-maven-plugin + ${build-helper-maven-plugin.version} + @@ -2387,6 +2440,7 @@ flexible messaging model and an intuitive client API. pulsar-client-messagecrypto-bc pulsar-metadata + jetcd-core-shaded jclouds-shaded @@ -2452,7 +2506,7 @@ flexible messaging model and an intuitive client API. distribution pulsar-metadata - + jetcd-core-shaded pulsar-package-management diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 73f55710c4f79..20117ed21db06 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -484,6 +484,18 @@ ${project.version} + + ${project.groupId} + jetcd-core-shaded + ${project.version} + shaded + test + + + io.grpc + grpc-netty-shaded + test + io.etcd jetcd-test diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index 99a87963f477f..160885a8ea4d7 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -101,7 +101,7 @@ io.grpc - grpc-all + * com.google.protobuf @@ -110,6 +110,11 @@ + + io.grpc + grpc-netty-shaded + + io.grpc grpc-stub @@ -215,7 +220,7 @@ - + diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index f2566fac653d7..e4a4dd5ec46ab 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -122,10 +122,15 @@ - io.etcd - jetcd-core + ${project.groupId} + jetcd-core-shaded + ${project.version} + shaded + + + io.grpc + grpc-netty-shaded - io.etcd diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java index 194b0d6a2f8a8..3937fd712dc9f 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java @@ -43,10 +43,10 @@ import io.etcd.jetcd.watch.WatchResponse; import io.grpc.Status; import io.grpc.StatusRuntimeException; -import io.grpc.netty.GrpcSslContexts; +import io.grpc.netty.shaded.io.grpc.netty.GrpcSslContexts; +import io.grpc.netty.shaded.io.netty.handler.ssl.SslContext; +import io.grpc.netty.shaded.io.netty.handler.ssl.SslProvider; import io.grpc.stub.StreamObserver; -import io.netty.handler.ssl.SslContext; -import io.netty.handler.ssl.SslProvider; import java.io.File; import java.io.IOException; import java.io.InputStream; From a91a172b4ee6d8b974a3fa905e435975557fcc57 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 13 Jun 2024 16:49:05 +0800 Subject: [PATCH 298/580] [fix][broker] The topic might reference a closed ledger (#22860) --- .../apache/pulsar/broker/PulsarService.java | 5 + .../pulsar/broker/service/BrokerService.java | 155 +++++++++--------- .../pulsar/broker/service/ReplicatorTest.java | 10 +- .../client/api/OrphanPersistentTopicTest.java | 68 ++++++++ 4 files changed, 151 insertions(+), 87 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 2e9f9dc6b0105..6cbc99e2cf4d4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -1964,6 +1964,11 @@ protected BrokerService newBrokerService(PulsarService pulsar) throws Exception return new BrokerService(pulsar, ioEventLoopGroup); } + @VisibleForTesting + public void setTransactionExecutorProvider(TransactionBufferProvider transactionBufferProvider) { + this.transactionBufferProvider = transactionBufferProvider; + } + private CompactionServiceFactory loadCompactionServiceFactory() { String compactionServiceFactoryClassName = config.getCompactionServiceFactoryClassName(); var compactionServiceFactory = 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 9a08578ee4088..82d7fad38740e 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 @@ -1001,38 +1001,38 @@ public CompletableFuture> getTopic(final String topic, boolean c return getTopic(TopicName.get(topic), createIfMissing, properties); } + /** + * Retrieves or creates a topic based on the specified parameters. + * 0. If disable PersistentTopics or NonPersistentTopics, it will return a failed future with NotAllowedException. + * 1. If topic future exists in the cache returned directly regardless of whether it fails or timeout. + * 2. If the topic metadata exists, the topic is created regardless of {@code createIfMissing}. + * 3. If the topic metadata not exists, and {@code createIfMissing} is false, + * returns an empty Optional in a CompletableFuture. And this empty future not be added to the map. + * 4. Otherwise, use computeIfAbsent. It returns the existing topic or creates and adds a new topicFuture. + * Any exceptions will remove the topicFuture from the map. + * + * @param topicName The name of the topic, potentially including partition information. + * @param createIfMissing If true, creates the topic if it does not exist. + * @param properties Topic configuration properties used during creation. + * @return CompletableFuture with an Optional of the topic if found or created, otherwise empty. + */ public CompletableFuture> getTopic(final TopicName topicName, boolean createIfMissing, Map properties) { try { - CompletableFuture> topicFuture = topics.get(topicName.toString()); - if (topicFuture != null) { - if (topicFuture.isCompletedExceptionally() - || (topicFuture.isDone() && !topicFuture.getNow(Optional.empty()).isPresent())) { - // Exceptional topics should be recreated. - topics.remove(topicName.toString(), topicFuture); - } else { - // a non-existing topic in the cache shouldn't prevent creating a topic - if (createIfMissing) { - if (topicFuture.isDone() && topicFuture.getNow(Optional.empty()).isPresent()) { - return topicFuture; - } else { - return topicFuture.thenCompose(value -> { - if (!value.isPresent()) { - // retry and create topic - return getTopic(topicName, createIfMissing, properties); - } else { - // in-progress future completed successfully - return CompletableFuture.completedFuture(value); - } - }); - } - } else { - return topicFuture; - } - } + // If topic future exists in the cache returned directly regardless of whether it fails or timeout. + CompletableFuture> tp = topics.get(topicName.toString()); + if (tp != null) { + return tp; } final boolean isPersistentTopic = topicName.getDomain().equals(TopicDomain.persistent); if (isPersistentTopic) { + if (!pulsar.getConfiguration().isEnablePersistentTopics()) { + if (log.isDebugEnabled()) { + log.debug("Broker is unable to load persistent topic {}", topicName); + } + return FutureUtil.failedFuture(new NotAllowedException( + "Broker is unable to load persistent topic")); + } return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topicName) .thenCompose(exists -> { if (!exists && !createIfMissing) { @@ -1047,44 +1047,48 @@ public CompletableFuture> getTopic(final TopicName topicName, bo throw FutureUtil.wrapToCompletionException(new ServiceUnitNotReadyException(errorInfo)); }).thenCompose(optionalTopicPolicies -> { final TopicPolicies topicPolicies = optionalTopicPolicies.orElse(null); - return topics.computeIfAbsent(topicName.toString(), (tpName) -> { - if (topicName.isPartitioned()) { - final TopicName topicNameEntity = TopicName.get(topicName.getPartitionedTopicName()); - return fetchPartitionedTopicMetadataAsync(topicNameEntity) - .thenCompose((metadata) -> { - // Allow crate non-partitioned persistent topic that name includes - // `partition` - if (metadata.partitions == 0 - || topicName.getPartitionIndex() < metadata.partitions) { - return loadOrCreatePersistentTopic(tpName, createIfMissing, - properties, topicPolicies); - } + if (topicName.isPartitioned()) { + final TopicName topicNameEntity = TopicName.get(topicName.getPartitionedTopicName()); + return fetchPartitionedTopicMetadataAsync(topicNameEntity) + .thenCompose((metadata) -> { + // Allow crate non-partitioned persistent topic that name includes + // `partition` + if (metadata.partitions == 0 + || topicName.getPartitionIndex() < metadata.partitions) { + return topics.computeIfAbsent(topicName.toString(), (tpName) -> + loadOrCreatePersistentTopic(tpName, + createIfMissing, properties, topicPolicies)); + } else { final String errorMsg = String.format("Illegal topic partition name %s with max allowed " + "%d partitions", topicName, metadata.partitions); log.warn(errorMsg); return FutureUtil.failedFuture( new BrokerServiceException.NotAllowedException(errorMsg)); - }); - } - return loadOrCreatePersistentTopic(tpName, createIfMissing, properties, topicPolicies); - }).thenCompose(optionalTopic -> { - if (!optionalTopic.isPresent() && createIfMissing) { - log.warn("[{}] Try to recreate the topic with createIfMissing=true " - + "but the returned topic is empty", topicName); - return getTopic(topicName, createIfMissing, properties); - } - return CompletableFuture.completedFuture(optionalTopic); - }); + } + }); + } else { + return topics.computeIfAbsent(topicName.toString(), (tpName) -> + loadOrCreatePersistentTopic(tpName, createIfMissing, properties, topicPolicies)); + } }); }); } else { - return topics.computeIfAbsent(topicName.toString(), (name) -> { + if (!pulsar.getConfiguration().isEnableNonPersistentTopics()) { + if (log.isDebugEnabled()) { + log.debug("Broker is unable to load non-persistent topic {}", topicName); + } + return FutureUtil.failedFuture(new NotAllowedException( + "Broker is unable to load persistent topic")); + } + if (!topics.containsKey(topicName.toString())) { topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.BEFORE); - if (topicName.isPartitioned()) { - final TopicName partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName()); - return this.fetchPartitionedTopicMetadataAsync(partitionedTopicName).thenCompose((metadata) -> { - if (topicName.getPartitionIndex() < metadata.partitions) { + } + if (topicName.isPartitioned()) { + final TopicName partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName()); + return this.fetchPartitionedTopicMetadataAsync(partitionedTopicName).thenCompose((metadata) -> { + if (topicName.getPartitionIndex() < metadata.partitions) { + return topics.computeIfAbsent(topicName.toString(), (name) -> { topicEventsDispatcher .notify(topicName.toString(), TopicEvent.CREATE, EventStage.BEFORE); @@ -1095,11 +1099,13 @@ public CompletableFuture> getTopic(final TopicName topicName, bo topicEventsDispatcher .notifyOnCompletion(eventFuture, topicName.toString(), TopicEvent.LOAD); return res; - } - topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.FAILURE); - return CompletableFuture.completedFuture(Optional.empty()); - }); - } else if (createIfMissing) { + }); + } + topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.FAILURE); + return CompletableFuture.completedFuture(Optional.empty()); + }); + } else if (createIfMissing) { + return topics.computeIfAbsent(topicName.toString(), (name) -> { topicEventsDispatcher.notify(topicName.toString(), TopicEvent.CREATE, EventStage.BEFORE); CompletableFuture> res = createNonPersistentTopic(name); @@ -1109,11 +1115,15 @@ public CompletableFuture> getTopic(final TopicName topicName, bo topicEventsDispatcher .notifyOnCompletion(eventFuture, topicName.toString(), TopicEvent.LOAD); return res; - } else { + }); + } else { + CompletableFuture> topicFuture = topics.get(topicName.toString()); + if (topicFuture == null) { topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.FAILURE); - return CompletableFuture.completedFuture(Optional.empty()); + topicFuture = CompletableFuture.completedFuture(Optional.empty()); } - }); + return topicFuture; + } } } catch (IllegalArgumentException e) { log.warn("[{}] Illegalargument exception when loading topic", topicName, e); @@ -1252,15 +1262,9 @@ private CompletableFuture> createNonPersistentTopic(String topic CompletableFuture> topicFuture = new CompletableFuture<>(); topicFuture.exceptionally(t -> { pulsarStats.recordTopicLoadFailed(); + pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); return null; }); - if (!pulsar.getConfiguration().isEnableNonPersistentTopics()) { - if (log.isDebugEnabled()) { - log.debug("Broker is unable to load non-persistent topic {}", topic); - } - return FutureUtil.failedFuture( - new NotAllowedException("Broker is not unable to load non-persistent topic")); - } final long topicCreateTimeMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()); NonPersistentTopic nonPersistentTopic; try { @@ -1283,7 +1287,6 @@ private CompletableFuture> createNonPersistentTopic(String topic }).exceptionally(ex -> { log.warn("Replication check failed. Removing topic from topics list {}, {}", topic, ex.getCause()); nonPersistentTopic.stopReplProducers().whenComplete((v, exception) -> { - pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); topicFuture.completeExceptionally(ex); }); return null; @@ -1534,14 +1537,6 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S final CompletableFuture> topicFuture = FutureUtil.createFutureWithTimeout( Duration.ofSeconds(pulsar.getConfiguration().getTopicLoadTimeoutSeconds()), executor(), () -> FAILED_TO_LOAD_TOPIC_TIMEOUT_EXCEPTION); - if (!pulsar.getConfiguration().isEnablePersistentTopics()) { - if (log.isDebugEnabled()) { - log.debug("Broker is unable to load persistent topic {}", topic); - } - topicFuture.completeExceptionally(new NotAllowedException( - "Broker is unable to load persistent topic")); - return topicFuture; - } checkTopicNsOwnership(topic) .thenRun(() -> { @@ -1556,6 +1551,7 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S // do not recreate topic if topic is already migrated and deleted by broker // so, avoid creating a new topic if migration is already started if (ex != null && (ex.getCause() instanceof TopicMigratedException)) { + pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); topicFuture.completeExceptionally(ex.getCause()); return null; } @@ -1570,6 +1566,7 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S } } }).exceptionally(ex -> { + pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); topicFuture.completeExceptionally(ex.getCause()); return null; }); @@ -1744,6 +1741,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { + " topic", topic, FutureUtil.getException(topicFuture)); executor().submit(() -> { persistentTopic.close().whenComplete((ignore, ex) -> { + topics.remove(topic, topicFuture); if (ex != null) { log.warn("[{}] Get an error when closing topic.", topic, ex); @@ -1760,6 +1758,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { + " Removing topic from topics list {}, {}", topic, ex); executor().submit(() -> { persistentTopic.close().whenComplete((ignore, closeEx) -> { + topics.remove(topic, topicFuture); if (closeEx != null) { log.warn("[{}] Get an error when closing topic.", topic, closeEx); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index 765727aeac319..b58f416ea1a57 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -19,12 +19,10 @@ package org.apache.pulsar.broker.service; import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; -import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.retryStrategically; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -1434,13 +1432,6 @@ public void testCleanupTopic() throws Exception { // Ok } - final CompletableFuture> timedOutTopicFuture = topicFuture; - // timeout topic future should be removed from cache - retryStrategically((test) -> pulsar1.getBrokerService().getTopic(topicName, false) != timedOutTopicFuture, 5, - 1000); - - assertNotEquals(timedOutTopicFuture, pulsar1.getBrokerService().getTopics().get(topicName)); - try { Consumer consumer = client1.newConsumer().topic(topicName).subscriptionType(SubscriptionType.Shared) .subscriptionName("my-subscriber-name").subscribeAsync().get(100, TimeUnit.MILLISECONDS); @@ -1452,6 +1443,7 @@ public void testCleanupTopic() throws Exception { ManagedLedgerImpl ml = (ManagedLedgerImpl) mlFactory.open(topicMlName + "-2"); mlFuture.complete(ml); + // Re-create topic will success. Consumer consumer = client1.newConsumer().topic(topicName).subscriptionName("my-subscriber-name") .subscriptionType(SubscriptionType.Shared).subscribeAsync() .get(2 * topicLoadTimeoutSeconds, TimeUnit.SECONDS); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java index 7cd9da7574dbb..d6473efd788d8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java @@ -18,7 +18,9 @@ */ package org.apache.pulsar.client.api; +import static org.apache.pulsar.broker.service.persistent.PersistentTopic.DEDUPLICATION_CURSOR_NAME; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import java.lang.reflect.Field; import java.util.List; @@ -27,6 +29,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; @@ -34,6 +37,9 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; import org.apache.pulsar.broker.service.TopicPolicyListener; +import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; +import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider; +import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferDisable; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.compaction.CompactionServiceFactory; @@ -108,6 +114,68 @@ public void testNoOrphanTopicAfterCreateTimeout() throws Exception { pulsar.getConfig().setTopicLoadTimeoutSeconds(originalTopicLoadTimeoutSeconds); } + @Test + public void testCloseLedgerThatTopicAfterCreateTimeout() throws Exception { + // Make the topic loading timeout faster. + long originalTopicLoadTimeoutSeconds = pulsar.getConfig().getTopicLoadTimeoutSeconds(); + int topicLoadTimeoutSeconds = 1; + pulsar.getConfig().setTopicLoadTimeoutSeconds(topicLoadTimeoutSeconds); + pulsar.getConfig().setBrokerDeduplicationEnabled(true); + pulsar.getConfig().setTransactionCoordinatorEnabled(true); + String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp2"); + + // Mock message deduplication recovery speed topicLoadTimeoutSeconds + String mlPath = BrokerService.MANAGED_LEDGER_PATH_ZNODE + "/" + + TopicName.get(tpName).getPersistenceNamingEncoding() + "/" + DEDUPLICATION_CURSOR_NAME; + mockZooKeeper.delay(topicLoadTimeoutSeconds * 1000, (op, path) -> { + if (mlPath.equals(path)) { + log.info("Topic load timeout: " + path); + return true; + } + return false; + }); + + // First load topic will trigger timeout + // The first topic load will trigger a timeout. When the topic closes, it will call transactionBuffer.close. + // Here, we simulate a sleep to ensure that the ledger is not immediately closed. + TransactionBufferProvider mockTransactionBufferProvider = new TransactionBufferProvider() { + @Override + public TransactionBuffer newTransactionBuffer(Topic originTopic) { + return new TransactionBufferDisable(originTopic) { + @SneakyThrows + @Override + public CompletableFuture closeAsync() { + Thread.sleep(500); + return super.closeAsync(); + } + }; + } + }; + TransactionBufferProvider originalTransactionBufferProvider = pulsar.getTransactionBufferProvider(); + pulsar.setTransactionExecutorProvider(mockTransactionBufferProvider); + CompletableFuture> firstLoad = pulsar.getBrokerService().getTopic(tpName, true); + Awaitility.await().ignoreExceptions().atMost(5, TimeUnit.SECONDS) + .pollInterval(100, TimeUnit.MILLISECONDS) + // assert first create topic timeout + .untilAsserted(() -> { + assertTrue(firstLoad.isCompletedExceptionally()); + }); + + // Once the first load topic times out, immediately to load the topic again. + Producer producer = pulsarClient.newProducer().topic(tpName).create(); + for (int i = 0; i < 10; i++) { + MessageId send = producer.send("msg".getBytes()); + Thread.sleep(100); + assertNotNull(send); + } + + // set to back + pulsar.setTransactionExecutorProvider(originalTransactionBufferProvider); + pulsar.getConfig().setTopicLoadTimeoutSeconds(originalTopicLoadTimeoutSeconds); + pulsar.getConfig().setBrokerDeduplicationEnabled(false); + pulsar.getConfig().setTransactionCoordinatorEnabled(false); + } + @Test public void testNoOrphanTopicIfInitFailed() throws Exception { String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); From 411f6973e85b0a6213e992386e1704f93d0aae42 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 13 Jun 2024 17:29:20 +0300 Subject: [PATCH 299/580] [improve][misc] Replace dependencies on PositionImpl with Position interface (#22891) Co-authored-by: Matteo Merli --- .../bookkeeper/mledger/ManagedCursor.java | 23 +- .../apache/bookkeeper/mledger/Position.java | 101 +++++- .../bookkeeper/mledger/PositionFactory.java | 59 ++++ .../bookkeeper/mledger/ReadOnlyCursor.java | 7 +- .../mledger/impl/AckSetPositionImpl.java | 89 ++++++ .../bookkeeper/mledger/impl/AckSetState.java} | 34 +- .../mledger/impl/AckSetStateUtil.java | 79 +++++ .../bookkeeper/mledger/impl/EntryImpl.java | 14 +- ...clable.java => ImmutablePositionImpl.java} | 50 +-- .../mledger/impl/ManagedCursorContainer.java | 18 +- .../mledger/impl/ManagedCursorImpl.java | 299 +++++++++--------- .../impl/ManagedLedgerFactoryImpl.java | 4 +- .../mledger/impl/ManagedLedgerImpl.java | 201 ++++++------ .../impl/ManagedLedgerOfflineBacklog.java | 32 +- .../mledger/impl/NonDurableCursorImpl.java | 12 +- .../bookkeeper/mledger/impl/OpAddEntry.java | 5 +- .../bookkeeper/mledger/impl/OpFindNewest.java | 12 +- .../bookkeeper/mledger/impl/OpReadEntry.java | 27 +- .../bookkeeper/mledger/impl/OpScan.java | 11 +- .../bookkeeper/mledger/impl/PositionImpl.java | 168 ---------- .../mledger/impl/PositionRecyclable.java | 77 +++++ .../mledger/impl/ReadOnlyCursorImpl.java | 7 +- .../impl/ReadOnlyManagedLedgerImpl.java | 17 +- .../mledger/impl/ShadowManagedLedgerImpl.java | 13 +- .../mledger/impl/cache/EntryCache.java | 6 +- .../impl/cache/EntryCacheDisabled.java | 6 +- .../impl/cache/RangeEntryCacheImpl.java | 23 +- .../mledger/util/ManagedLedgerImplUtils.java | 7 +- .../mledger/util/PositionAckSetUtil.java | 22 +- .../mledger/impl/EntryCacheManagerTest.java | 8 +- .../impl/ManagedCursorConcurrencyTest.java | 4 +- .../impl/ManagedCursorContainerTest.java | 233 +++++++------- ...edCursorIndividualDeletedMessagesTest.java | 14 +- .../mledger/impl/ManagedCursorTest.java | 231 +++++++------- .../mledger/impl/ManagedLedgerBkTest.java | 4 +- .../ManagedLedgerFactoryShutdownTest.java | 5 +- .../impl/ManagedLedgerFactoryTest.java | 7 +- .../impl/ManagedLedgerTerminationTest.java | 3 +- .../mledger/impl/ManagedLedgerTest.java | 157 ++++----- .../mledger/impl/NonDurableCursorTest.java | 87 ++--- .../mledger/impl/OffloadPrefixReadTest.java | 5 +- .../mledger/impl/OffloadPrefixTest.java | 25 +- .../bookkeeper/mledger/impl/PositionTest.java | 29 +- .../mledger/impl/ReadOnlyCursorTest.java | 17 +- .../impl/ShadowManagedLedgerImplTest.java | 3 +- .../util/ManagedLedgerImplUtilsTest.java | 7 +- .../mledger/util/PositionAckSetUtilTest.java | 52 +-- .../admin/impl/PersistentTopicsBase.java | 39 +-- .../broker/admin/impl/TransactionsBase.java | 4 +- .../broker/admin/v2/PersistentTopics.java | 4 +- .../pulsar/broker/admin/v3/Transactions.java | 5 +- .../delayed/DelayedDeliveryTracker.java | 4 +- .../InMemoryDelayedDeliveryTracker.java | 9 +- .../bucket/BucketDelayedDeliveryTracker.java | 9 +- .../rest/RestMessagePublishContext.java | 9 +- .../apache/pulsar/broker/rest/TopicsBase.java | 14 +- .../service/AbstractBaseDispatcher.java | 24 +- .../broker/service/BacklogQuotaManager.java | 10 +- .../pulsar/broker/service/Consumer.java | 68 ++-- .../pulsar/broker/service/Dispatcher.java | 4 +- .../service/InMemoryRedeliveryTracker.java | 5 +- .../pulsar/broker/service/Producer.java | 5 - .../pulsar/broker/service/ServerCnx.java | 19 +- .../pulsar/broker/service/Subscription.java | 3 +- .../NonPersistentDispatcher.java | 4 +- .../NonPersistentSubscription.java | 3 +- .../persistent/GeoPersistentReplicator.java | 3 +- .../persistent/MessageDeduplication.java | 10 +- .../MessageRedeliveryController.java | 7 +- ...PersistentDispatcherMultipleConsumers.java | 32 +- ...sistentDispatcherSingleActiveConsumer.java | 4 +- .../PersistentMessageExpiryMonitor.java | 18 +- .../persistent/PersistentReplicator.java | 9 +- ...tStickyKeyDispatcherMultipleConsumers.java | 43 ++- .../persistent/PersistentSubscription.java | 39 ++- .../service/persistent/PersistentTopic.java | 54 ++-- .../ReplicatedSubscriptionSnapshotCache.java | 11 +- .../ReplicatedSubscriptionsController.java | 8 +- ...eplicatedSubscriptionsSnapshotBuilder.java | 3 +- .../buffer/AbortedTxnProcessor.java | 8 +- .../transaction/buffer/TransactionBuffer.java | 9 +- .../buffer/impl/InMemTransactionBuffer.java | 9 +- ...SingleSnapshotAbortedTxnProcessorImpl.java | 17 +- ...napshotSegmentAbortedTxnProcessorImpl.java | 47 +-- .../buffer/impl/TopicTransactionBuffer.java | 52 +-- .../buffer/impl/TransactionBufferDisable.java | 9 +- .../pendingack/PendingAckHandle.java | 17 +- .../pendingack/PendingAckStore.java | 7 +- .../impl/InMemoryPendingAckStore.java | 6 +- .../impl/MLPendingAckReplyCallBack.java | 21 +- .../pendingack/impl/MLPendingAckStore.java | 65 ++-- .../impl/PendingAckHandleDisabled.java | 13 +- .../pendingack/impl/PendingAckHandleImpl.java | 151 ++++----- .../pulsar/compaction/CompactedTopic.java | 5 +- .../pulsar/compaction/CompactedTopicImpl.java | 22 +- .../compaction/CompactedTopicUtils.java | 10 +- .../PulsarTopicCompactionService.java | 3 +- .../pulsar/broker/admin/AdminApiTest.java | 4 +- .../admin/v3/AdminApiTransactionTest.java | 31 +- .../delayed/AbstractDeliveryTrackerTest.java | 4 +- .../broker/delayed/MockManagedCursor.java | 13 +- .../BucketDelayedDeliveryTrackerTest.java | 27 +- .../MangedLedgerInterceptorImplTest.java | 25 +- .../service/AbstractBaseDispatcherTest.java | 4 +- .../service/AbstractReplicatorTest.java | 4 +- .../BatchMessageWithBatchIndexLevelTest.java | 4 +- .../DeduplicationDisabledBrokerLevelTest.java | 6 +- .../pulsar/broker/service/MessageTTLTest.java | 6 +- ...sistentDispatcherFailoverConsumerTest.java | 4 +- .../service/PersistentMessageFinderTest.java | 32 +- .../broker/service/PersistentTopicTest.java | 14 +- .../pulsar/broker/service/ReplicatorTest.java | 6 +- .../pulsar/broker/service/ServerCnxTest.java | 15 +- .../service/TransactionMarkerDeleteTest.java | 10 +- .../persistent/MessageDuplicationTest.java | 14 +- .../MessageRedeliveryControllerTest.java | 25 +- ...ckyKeyDispatcherMultipleConsumersTest.java | 8 +- .../PersistentSubscriptionTest.java | 42 +-- .../persistent/PersistentTopicTest.java | 11 +- ...plicatedSubscriptionSnapshotCacheTest.java | 24 +- ...catedSubscriptionsSnapshotBuilderTest.java | 12 +- .../persistent/TopicDuplicationTest.java | 56 ++-- .../service/plugin/FilterEntryTest.java | 4 +- .../SegmentAbortedTxnProcessorTest.java | 21 +- .../TopicTransactionBufferRecoverTest.java | 9 +- .../transaction/TransactionConsumeTest.java | 7 +- .../transaction/TransactionProduceTest.java | 10 +- .../broker/transaction/TransactionTest.java | 31 +- .../buffer/TopicTransactionBufferTest.java | 21 +- .../buffer/TransactionLowWaterMarkTest.java | 14 +- .../buffer/TransactionStablePositionTest.java | 8 +- .../PendingAckInMemoryDeleteTest.java | 21 +- .../pendingack/PendingAckPersistentTest.java | 22 +- .../impl/MLPendingAckStoreTest.java | 24 +- .../client/api/KeySharedSubscriptionTest.java | 13 +- .../api/NonDurableSubscriptionTest.java | 45 +-- .../client/impl/MessageChunkingTest.java | 6 +- .../pulsar/client/impl/MessageParserTest.java | 4 +- .../client/impl/TransactionEndToEndTest.java | 5 +- .../compaction/CompactedTopicImplTest.java | 13 +- .../pulsar/compaction/CompactedTopicTest.java | 22 +- .../compaction/CompactedTopicUtilsTest.java | 11 +- .../pulsar/compaction/CompactorTest.java | 4 +- .../GetLastMessageIdCompactedTest.java | 7 +- .../TopicCompactionServiceTest.java | 8 +- .../impl/MLTransactionLogImpl.java | 6 +- .../impl/TxnBatchedPositionImpl.java | 29 +- .../impl/MLTransactionLogImplTest.java | 38 +-- .../impl/TxnBatchedPositionImplTest.java | 27 +- .../impl/TxnLogBufferedWriterTest.java | 19 +- .../impl/BlobStoreManagedLedgerOffloader.java | 6 +- 151 files changed, 2214 insertions(+), 1910 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/PositionFactory.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetPositionImpl.java rename managed-ledger/src/{test/java/org/apache/bookkeeper/mledger/impl/PositionImplRecyclableTest.java => main/java/org/apache/bookkeeper/mledger/impl/AckSetState.java} (56%) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetStateUtil.java rename managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/{PositionImplRecyclable.java => ImmutablePositionImpl.java} (50%) delete mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionRecyclable.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java index 227b5429abf77..4aa3226a4dc2b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java @@ -34,7 +34,6 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.SkipEntriesCallback; -import org.apache.bookkeeper.mledger.impl.PositionImpl; /** * A ManagedCursor is a persisted cursor inside a ManagedLedger. @@ -152,7 +151,7 @@ enum IndividualDeletedEntries { * max position can read */ void asyncReadEntries(int numberOfEntriesToRead, ReadEntriesCallback callback, Object ctx, - PositionImpl maxPosition); + Position maxPosition); /** @@ -165,7 +164,7 @@ void asyncReadEntries(int numberOfEntriesToRead, ReadEntriesCallback callback, O * @param maxPosition max position can read */ void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition); + Object ctx, Position maxPosition); /** * Asynchronously read entries from the ManagedLedger. @@ -178,7 +177,7 @@ void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, ReadEntriesC * @param skipCondition predicate of read filter out */ default void asyncReadEntriesWithSkip(int numberOfEntriesToRead, long maxSizeBytes, ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition, Predicate skipCondition) { + Object ctx, Position maxPosition, Predicate skipCondition) { asyncReadEntries(numberOfEntriesToRead, maxSizeBytes, callback, ctx, maxPosition); } @@ -256,7 +255,7 @@ List readEntriesOrWait(int maxEntries, long maxSizeBytes) * max position can read */ void asyncReadEntriesOrWait(int numberOfEntriesToRead, ReadEntriesCallback callback, Object ctx, - PositionImpl maxPosition); + Position maxPosition); /** * Asynchronously read entries from the ManagedLedger, up to the specified number and size. @@ -277,7 +276,7 @@ void asyncReadEntriesOrWait(int numberOfEntriesToRead, ReadEntriesCallback callb * max position can read */ void asyncReadEntriesOrWait(int maxEntries, long maxSizeBytes, ReadEntriesCallback callback, Object ctx, - PositionImpl maxPosition); + Position maxPosition); /** * Asynchronously read entries from the ManagedLedger, up to the specified number and size. @@ -298,7 +297,7 @@ void asyncReadEntriesOrWait(int maxEntries, long maxSizeBytes, ReadEntriesCallba * predicate of read filter out */ default void asyncReadEntriesWithSkipOrWait(int maxEntries, ReadEntriesCallback callback, Object ctx, - PositionImpl maxPosition, Predicate skipCondition) { + Position maxPosition, Predicate skipCondition) { asyncReadEntriesOrWait(maxEntries, callback, ctx, maxPosition); } @@ -323,15 +322,15 @@ default void asyncReadEntriesWithSkipOrWait(int maxEntries, ReadEntriesCallback * predicate of read filter out */ default void asyncReadEntriesWithSkipOrWait(int maxEntries, long maxSizeBytes, ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition, - Predicate skipCondition) { + Object ctx, Position maxPosition, + Predicate skipCondition) { asyncReadEntriesOrWait(maxEntries, maxSizeBytes, callback, ctx, maxPosition); } /** * Cancel a previously scheduled asyncReadEntriesOrWait operation. * - * @see #asyncReadEntriesOrWait(int, ReadEntriesCallback, Object, PositionImpl) + * @see #asyncReadEntriesOrWait(int, ReadEntriesCallback, Object, Position) * @return true if the read operation was canceled or false if there was no pending operation */ boolean cancelPendingReadRequest(); @@ -837,7 +836,7 @@ default void skipNonRecoverableLedger(long ledgerId){} * Get last individual deleted range. * @return range */ - Range getLastIndividualDeletedRange(); + Range getLastIndividualDeletedRange(); /** * Trim delete entries for the given entries. @@ -847,7 +846,7 @@ default void skipNonRecoverableLedger(long ledgerId){} /** * Get deleted batch indexes list for a batch message. */ - long[] getDeletedBatchIndexesAsLongArray(PositionImpl position); + long[] getDeletedBatchIndexesAsLongArray(Position position); /** * @return the managed cursor stats MBean diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/Position.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/Position.java index ac5810bbf01e7..d0d6d865c9558 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/Position.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/Position.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.mledger; +import java.util.Optional; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; @@ -26,16 +27,108 @@ */ @InterfaceAudience.LimitedPrivate @InterfaceStability.Stable -public interface Position { +public interface Position extends Comparable { + /** + * Get the ledger id of the entry pointed by this position. + * + * @return the ledger id + */ + long getLedgerId(); + + /** + * Get the entry id of the entry pointed by this position. + * + * @return the entry id + */ + long getEntryId(); + + /** + * Compare this position with another position. + * The comparison is first based on the ledger id, and then on the entry id. + * This is implements the Comparable interface. + * @param that the other position to be compared. + * @return -1 if this position is less than the other, 0 if they are equal, 1 if this position is greater than + * the other. + */ + default int compareTo(Position that) { + if (getLedgerId() != that.getLedgerId()) { + return Long.compare(getLedgerId(), that.getLedgerId()); + } + + return Long.compare(getEntryId(), that.getEntryId()); + } + + /** + * Compare this position with another position based on the ledger id and entry id. + * @param ledgerId the ledger id to compare + * @param entryId the entry id to compare + * @return -1 if this position is less than the other, 0 if they are equal, 1 if this position is greater than + * the other. + */ + default int compareTo(long ledgerId, long entryId) { + if (getLedgerId() != ledgerId) { + return Long.compare(getLedgerId(), ledgerId); + } + + return Long.compare(getEntryId(), entryId); + } + + /** + * Calculate the hash code for the position based on ledgerId and entryId. + * This is used in Position implementations to implement the hashCode method. + * @return hash code + */ + default int hashCodeForPosition() { + int result = Long.hashCode(getLedgerId()); + result = 31 * result + Long.hashCode(getEntryId()); + return result; + } + /** * Get the position of the entry next to this one. The returned position might point to a non-existing, or not-yet * existing entry * * @return the position of the next logical entry */ - Position getNext(); + default Position getNext() { + if (getEntryId() < 0) { + return PositionFactory.create(getLedgerId(), 0); + } else { + return PositionFactory.create(getLedgerId(), getEntryId() + 1); + } + } - long getLedgerId(); + /** + * Position after moving entryNum messages, + * if entryNum < 1, then return the current position. + * */ + default Position getPositionAfterEntries(int entryNum) { + if (entryNum < 1) { + return this; + } + if (getEntryId() < 0) { + return PositionFactory.create(getLedgerId(), entryNum - 1); + } else { + return PositionFactory.create(getLedgerId(), getEntryId() + entryNum); + } + } - long getEntryId(); + /** + * Check if the position implementation has an extension of the given class or interface. + * + * @param extensionClass the class of the extension + * @return true if the position has an extension of the given class, false otherwise + */ + default boolean hasExtension(Class extensionClass) { + return getExtension(extensionClass).isPresent(); + } + + /** + * Get the extension instance of the given class or interface that is attached to this position. + * If the position does not have an extension of the given class, an empty optional is returned. + * @param extensionClass the class of the extension + */ + default Optional getExtension(Class extensionClass) { + return Optional.empty(); + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/PositionFactory.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/PositionFactory.java new file mode 100644 index 0000000000000..0b119844a6268 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/PositionFactory.java @@ -0,0 +1,59 @@ +/* + * 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.bookkeeper.mledger; + +import org.apache.bookkeeper.mledger.impl.ImmutablePositionImpl; + +/** + * Factory for creating {@link Position} instances. + */ +public final class PositionFactory { + /** + * Earliest position. + */ + public static final Position EARLIEST = create(-1, -1); + /** + * Latest position. + */ + public static final Position LATEST = create(Long.MAX_VALUE, Long.MAX_VALUE); + + private PositionFactory() { + } + + /** + * Create a new position. + * + * @param ledgerId ledger id + * @param entryId entry id + * @return new position + */ + public static Position create(long ledgerId, long entryId) { + return new ImmutablePositionImpl(ledgerId, entryId); + } + + /** + * Create a new position. + * + * @param other other position + * @return new position + */ + public static Position create(Position other) { + return new ImmutablePositionImpl(other); + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyCursor.java index 18d412f893152..016298cb108bb 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyCursor.java @@ -24,7 +24,6 @@ import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; -import org.apache.bookkeeper.mledger.impl.PositionImpl; @InterfaceAudience.LimitedPrivate @InterfaceStability.Stable @@ -48,7 +47,7 @@ public interface ReadOnlyCursor { * @see #readEntries(int) */ void asyncReadEntries(int numberOfEntriesToRead, ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition); + Object ctx, Position maxPosition); /** * Asynchronously read entries from the ManagedLedger. @@ -60,7 +59,7 @@ void asyncReadEntries(int numberOfEntriesToRead, ReadEntriesCallback callback, * @param maxPosition max position can read */ void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition); + Object ctx, Position maxPosition); /** * Get the read position. This points to the next message to be read from the cursor. @@ -116,7 +115,7 @@ Position findNewestMatching(ManagedCursor.FindPositionConstraint constraint, Pre * @param range the range between two positions * @return the number of entries in range */ - long getNumberOfEntries(Range range); + long getNumberOfEntries(Range range); /** * Close the cursor and releases the associated resources. diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetPositionImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetPositionImpl.java new file mode 100644 index 0000000000000..22a99eb3607eb --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetPositionImpl.java @@ -0,0 +1,89 @@ +/* + * 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.bookkeeper.mledger.impl; + +import java.util.Optional; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; + +/** + * Position implementation that includes the ack set. + * Use {@link AckSetStateUtil#createPositionWithAckSet(long, long, long[])} to create instances. + */ +public class AckSetPositionImpl implements Position, AckSetState { + private final Optional ackSetStateExtension = Optional.of(this); + protected final long ledgerId; + protected final long entryId; + protected volatile long[] ackSet; + + public AckSetPositionImpl(long ledgerId, long entryId, long[] ackSet) { + this.ledgerId = ledgerId; + this.entryId = entryId; + this.ackSet = ackSet; + } + + public long[] getAckSet() { + return ackSet; + } + + public void setAckSet(long[] ackSet) { + this.ackSet = ackSet; + } + + public long getLedgerId() { + return ledgerId; + } + + public long getEntryId() { + return entryId; + } + + @Override + public Position getNext() { + if (entryId < 0) { + return PositionFactory.create(ledgerId, 0); + } else { + return PositionFactory.create(ledgerId, entryId + 1); + } + } + + @Override + public String toString() { + return ledgerId + ":" + entryId + " (ackSet " + (ackSet == null ? "is null" : + "with long[] size of " + ackSet.length) + ")"; + } + + @Override + public int hashCode() { + return hashCodeForPosition(); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof Position && compareTo((Position) obj) == 0; + } + + @Override + public Optional getExtension(Class extensionClass) { + if (extensionClass == AckSetState.class) { + return (Optional) ackSetStateExtension; + } + return Position.super.getExtension(extensionClass); + } +} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionImplRecyclableTest.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetState.java similarity index 56% rename from managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionImplRecyclableTest.java rename to managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetState.java index f46e3ec36b24c..363336e83113e 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionImplRecyclableTest.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetState.java @@ -18,17 +18,29 @@ */ package org.apache.bookkeeper.mledger.impl; -import static org.testng.Assert.assertNull; -import org.testng.annotations.Test; +/** + * Interface to manage the ackSet state attached to a position. + * Helpers in {@link AckSetStateUtil} to create positions with + * ackSet state and to extract the state. + */ +public interface AckSetState { + /** + * Get the ackSet bitset information encoded as a long array. + * @return the ackSet + */ + long[] getAckSet(); -public class PositionImplRecyclableTest { + /** + * Set the ackSet bitset information as a long array. + * @param ackSet the ackSet + */ + void setAckSet(long[] ackSet); - @Test - void shouldNotCarryStateInAckSetWhenRecycled() { - PositionImplRecyclable position = PositionImplRecyclable.create(); - position.ackSet = new long[]{1L, 2L, 3L}; - position.recycle(); - PositionImplRecyclable position2 = PositionImplRecyclable.create(); - assertNull(position2.ackSet); + /** + * Check if the ackSet is set. + * @return true if the ackSet is set, false otherwise + */ + default boolean hasAckSet() { + return getAckSet() != null; } -} \ No newline at end of file +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetStateUtil.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetStateUtil.java new file mode 100644 index 0000000000000..11ab520b68e92 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/AckSetStateUtil.java @@ -0,0 +1,79 @@ +/* + * 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.bookkeeper.mledger.impl; + +import java.util.Optional; +import lombok.experimental.UtilityClass; +import org.apache.bookkeeper.mledger.Position; + +/** + * Utility class to manage the ackSet state attached to a position. + */ +@UtilityClass +public class AckSetStateUtil { + /** + * Create a new position with the ackSet state. + * + * @param ledgerId ledger id + * @param entryId entry id + * @param ackSet ack set bitset information encoded as an array of longs + * @return new position + */ + public static Position createPositionWithAckSet(long ledgerId, long entryId, long[] ackSet) { + return new AckSetPositionImpl(ledgerId, entryId, ackSet); + } + + /** + * Get the AckSetState instance from the position if it exists. + * @param position position which possibly contains the AckSetState + */ + public static Optional maybeGetAckSetState(Position position) { + return position.getExtension(AckSetState.class); + } + + /** + * Get the ackSet bitset information encoded as a long array from the position if it exists. + * @param position position which possibly contains the AckSetState + * @return the ackSet or null if the position does not have the AckSetState, or it's not set + */ + public static long[] getAckSetArrayOrNull(Position position) { + return maybeGetAckSetState(position).map(AckSetState::getAckSet).orElse(null); + } + + /** + * Get the AckSetState instance from the position. + * @param position position which contains the AckSetState + * @return AckSetState instance + * @throws IllegalStateException if the position does not have AckSetState + */ + public static AckSetState getAckSetState(Position position) { + return maybeGetAckSetState(position) + .orElseThrow(() -> + new IllegalStateException("Position does not have AckSetState. position=" + position)); + } + + /** + * Check if position contains the ackSet information and it is set. + * @param position position which possibly contains the AckSetState + * @return true if the ackSet is set, false otherwise + */ + public static boolean hasAckSet(Position position) { + return maybeGetAckSetState(position).map(AckSetState::hasAckSet).orElse(false); + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java index 48a79a4ac529c..e0e2b859794b5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java @@ -26,11 +26,13 @@ import io.netty.util.ReferenceCounted; import org.apache.bookkeeper.client.api.LedgerEntry; import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.util.AbstractCASReferenceCounted; import org.apache.bookkeeper.mledger.util.RangeCache; public final class EntryImpl extends AbstractCASReferenceCounted implements Entry, Comparable, - RangeCache.ValueWithKeyValidation { + RangeCache.ValueWithKeyValidation { private static final Recycler RECYCLER = new Recycler() { @Override @@ -43,7 +45,7 @@ protected EntryImpl newObject(Handle handle) { private long timestamp; private long ledgerId; private long entryId; - private PositionImpl position; + private Position position; ByteBuf data; private Runnable onDeallocate; @@ -81,7 +83,7 @@ public static EntryImpl create(long ledgerId, long entryId, ByteBuf data) { return entry; } - public static EntryImpl create(PositionImpl position, ByteBuf data) { + public static EntryImpl create(Position position, ByteBuf data) { EntryImpl entry = RECYCLER.get(); entry.timestamp = System.nanoTime(); entry.ledgerId = position.getLedgerId(); @@ -152,9 +154,9 @@ public int getLength() { } @Override - public PositionImpl getPosition() { + public Position getPosition() { if (position == null) { - position = PositionImpl.get(ledgerId, entryId); + position = PositionFactory.create(ledgerId, entryId); } return position; } @@ -207,7 +209,7 @@ protected void deallocate() { } @Override - public boolean matchesKey(PositionImpl key) { + public boolean matchesKey(Position key) { return key.compareTo(ledgerId, entryId) == 0; } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImplRecyclable.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ImmutablePositionImpl.java similarity index 50% rename from managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImplRecyclable.java rename to managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ImmutablePositionImpl.java index eb2b33e858d63..06245a6b5f33a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImplRecyclable.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ImmutablePositionImpl.java @@ -18,33 +18,45 @@ */ package org.apache.bookkeeper.mledger.impl; -import io.netty.util.Recycler; -import io.netty.util.Recycler.Handle; import org.apache.bookkeeper.mledger.Position; -public class PositionImplRecyclable extends PositionImpl implements Position { +public final class ImmutablePositionImpl implements Position { + private final long ledgerId; + private final long entryId; - private final Handle recyclerHandle; + public ImmutablePositionImpl(long ledgerId, long entryId) { + this.ledgerId = ledgerId; + this.entryId = entryId; + } - private static final Recycler RECYCLER = new Recycler() { - @Override - protected PositionImplRecyclable newObject(Recycler.Handle recyclerHandle) { - return new PositionImplRecyclable(recyclerHandle); - } - }; + public ImmutablePositionImpl(Position other) { + this.ledgerId = other.getLedgerId(); + this.entryId = other.getEntryId(); + } - private PositionImplRecyclable(Handle recyclerHandle) { - super(PositionImpl.EARLIEST); - this.recyclerHandle = recyclerHandle; + public long getLedgerId() { + return ledgerId; } - public static PositionImplRecyclable create() { - return RECYCLER.get(); + public long getEntryId() { + return entryId; } - public void recycle() { - ackSet = null; - recyclerHandle.recycle(this); + /** + * String representation of virtual cursor - LedgerId:EntryId. + */ + @Override + public String toString() { + return ledgerId + ":" + entryId; } -} \ No newline at end of file + @Override + public int hashCode() { + return hashCodeForPosition(); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof Position && compareTo((Position) obj) == 0; + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java index 92f3d892b532d..ba901ece51c39 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainer.java @@ -54,7 +54,7 @@ public class ManagedCursorContainer implements Iterable { @Value public static class CursorInfo { ManagedCursor cursor; - PositionImpl position; + Position position; /** * Cursor info's version. @@ -67,10 +67,10 @@ public static class CursorInfo { private static class Item { final ManagedCursor cursor; - PositionImpl position; + Position position; int idx; - Item(ManagedCursor cursor, PositionImpl position, int idx) { + Item(ManagedCursor cursor, Position position, int idx) { this.cursor = cursor; this.position = position; this.idx = idx; @@ -160,7 +160,7 @@ public ManagedCursorContainer() {} public void add(ManagedCursor cursor, Position position) { long stamp = rwLock.writeLock(); try { - Item item = new Item(cursor, (PositionImpl) position, position != null ? heap.size() : -1); + Item item = new Item(cursor, position, position != null ? heap.size() : -1); cursors.put(cursor.getName(), item); if (position != null) { heap.add(item); @@ -229,7 +229,7 @@ public boolean removeCursor(String name) { * @return a pair of positions, representing the previous slowest reader and the new slowest reader (after the * update). */ - public Pair cursorUpdated(ManagedCursor cursor, Position newPosition) { + public Pair cursorUpdated(ManagedCursor cursor, Position newPosition) { requireNonNull(cursor); long stamp = rwLock.writeLock(); @@ -239,8 +239,8 @@ public Pair cursorUpdated(ManagedCursor cursor, Posi return null; } - PositionImpl previousSlowestConsumer = heap.get(0).position; - item.position = (PositionImpl) newPosition; + Position previousSlowestConsumer = heap.get(0).position; + item.position = newPosition; version = DataVersion.getNextVersion(version); if (heap.size() == 1) { @@ -254,7 +254,7 @@ public Pair cursorUpdated(ManagedCursor cursor, Posi } else { siftUp(item); } - PositionImpl newSlowestConsumer = heap.get(0).position; + Position newSlowestConsumer = heap.get(0).position; return Pair.of(previousSlowestConsumer, newSlowestConsumer); } finally { rwLock.unlockWrite(stamp); @@ -266,7 +266,7 @@ public Pair cursorUpdated(ManagedCursor cursor, Posi * * @return the slowest reader position */ - public PositionImpl getSlowestReaderPosition() { + public Position getSlowestReaderPosition() { long stamp = rwLock.readLock(); try { return heap.isEmpty() ? null : heap.get(0).position; diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 1d2065ef8e392..c0992e48dba8a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -85,6 +85,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.MetaStoreException; import org.apache.bookkeeper.mledger.ManagedLedgerException.NoMoreEntriesToReadException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ScanOutcome; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; @@ -127,21 +128,21 @@ public class ManagedCursorImpl implements ManagedCursor { private volatile Map cursorProperties; private final BookKeeper.DigestType digestType; - protected volatile PositionImpl markDeletePosition; + protected volatile Position markDeletePosition; // this position is have persistent mark delete position - protected volatile PositionImpl persistentMarkDeletePosition; - protected static final AtomicReferenceFieldUpdater + protected volatile Position persistentMarkDeletePosition; + protected static final AtomicReferenceFieldUpdater INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER = - AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, PositionImpl.class, + AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, Position.class, "inProgressMarkDeletePersistPosition"); - protected volatile PositionImpl inProgressMarkDeletePersistPosition; + protected volatile Position inProgressMarkDeletePersistPosition; - protected static final AtomicReferenceFieldUpdater READ_POSITION_UPDATER = - AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, PositionImpl.class, "readPosition"); - protected volatile PositionImpl readPosition; + protected static final AtomicReferenceFieldUpdater READ_POSITION_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, Position.class, "readPosition"); + protected volatile Position readPosition; // keeps sample of last read-position for validation and monitoring if read-position is not moving forward. - protected volatile PositionImpl statsLastReadPosition; + protected volatile Position statsLastReadPosition; protected static final AtomicReferenceFieldUpdater LAST_MARK_DELETE_ENTRY_UPDATER = AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, @@ -183,23 +184,17 @@ public class ManagedCursorImpl implements ManagedCursor { private volatile Stat cursorLedgerStat; private volatile ManagedCursorInfo managedCursorInfo; - private static final LongPairConsumer positionRangeConverter = PositionImpl::new; + private static final LongPairConsumer positionRangeConverter = PositionFactory::create; - private static final RangeBoundConsumer positionRangeReverseConverter = - (position) -> new LongPairRangeSet.LongPair(position.ledgerId, position.entryId); + private static final RangeBoundConsumer positionRangeReverseConverter = + (position) -> new LongPairRangeSet.LongPair(position.getLedgerId(), position.getEntryId()); - private static final LongPairConsumer recyclePositionRangeConverter = (key, value) -> { - PositionImplRecyclable position = PositionImplRecyclable.create(); - position.ledgerId = key; - position.entryId = value; - position.ackSet = null; - return position; - }; - protected final RangeSetWrapper individualDeletedMessages; + private static final LongPairConsumer recyclePositionRangeConverter = PositionRecyclable::get; + protected final RangeSetWrapper individualDeletedMessages; // Maintain the deletion status for batch messages // (ledgerId, entryId) -> deletion indexes - protected final ConcurrentSkipListMap batchDeletedIndexes; + protected final ConcurrentSkipListMap batchDeletedIndexes; private final ReadWriteLock lock = new ReentrantReadWriteLock(); private RateLimiter markDeleteLimiter; @@ -221,7 +216,7 @@ public class ManagedCursorImpl implements ManagedCursor { private volatile boolean isActive = false; class MarkDeleteEntry { - final PositionImpl newPosition; + final Position newPosition; final MarkDeleteCallback callback; final Object ctx; final Map properties; @@ -231,7 +226,7 @@ class MarkDeleteEntry { // group. List callbackGroup; - public MarkDeleteEntry(PositionImpl newPosition, Map properties, + public MarkDeleteEntry(Position newPosition, Map properties, MarkDeleteCallback callback, Object ctx) { this.newPosition = newPosition; this.properties = properties; @@ -495,7 +490,7 @@ public void operationComplete(ManagedCursorInfo info, Stat stat) { if (info.getCursorsLedgerId() == -1L) { // There is no cursor ledger to read the last position from. It means the cursor has been properly // closed and the last mark-delete position is stored in the ManagedCursorInfo itself. - PositionImpl recoveredPosition = new PositionImpl(info.getMarkDeleteLedgerId(), + Position recoveredPosition = PositionFactory.create(info.getMarkDeleteLedgerId(), info.getMarkDeleteEntryId()); if (info.getIndividualDeletedMessagesCount() > 0) { recoverIndividualDeletedMessages(info.getIndividualDeletedMessagesList()); @@ -599,7 +594,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac } } - PositionImpl position = new PositionImpl(positionInfo); + Position position = PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId()); if (positionInfo.getIndividualDeletedMessagesCount() > 0) { recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); } @@ -669,8 +664,10 @@ private void recoverBatchDeletedIndexes ( for (int i = 0; i < batchDeletedIndexInfo.getDeleteSetList().size(); i++) { array[i] = batchDeletedIndexInfo.getDeleteSetList().get(i); } - this.batchDeletedIndexes.put(PositionImpl.get(batchDeletedIndexInfo.getPosition().getLedgerId(), - batchDeletedIndexInfo.getPosition().getEntryId()), BitSetRecyclable.create().resetWords(array)); + this.batchDeletedIndexes.put( + PositionFactory.create(batchDeletedIndexInfo.getPosition().getLedgerId(), + batchDeletedIndexInfo.getPosition().getEntryId()), + BitSetRecyclable.create().resetWords(array)); } }); } finally { @@ -678,7 +675,7 @@ private void recoverBatchDeletedIndexes ( } } - private void recoveredCursor(PositionImpl position, Map properties, + private void recoveredCursor(Position position, Map properties, Map cursorProperties, LedgerHandle recoveredFromCursorLedger) { // if the position was at a ledger that didn't exist (since it will be deleted if it was previously empty), @@ -689,7 +686,7 @@ private void recoveredCursor(PositionImpl position, Map properties log.info("[{}] [{}] Couldn't find next next valid ledger for recovery {}", ledger.getName(), name, position); } - position = nextExistingLedger != null ? PositionImpl.get(nextExistingLedger, -1) : position; + position = nextExistingLedger != null ? PositionFactory.create(nextExistingLedger, -1) : position; } if (position.compareTo(ledger.getLastPosition()) > 0) { log.warn("[{}] [{}] Current position {} is ahead of last position {}", ledger.getName(), name, position, @@ -711,7 +708,7 @@ private void recoveredCursor(PositionImpl position, Map properties STATE_UPDATER.set(this, State.NoLedger); } - void initialize(PositionImpl position, Map properties, Map cursorProperties, + void initialize(Position position, Map properties, Map cursorProperties, final VoidCallback callback) { recoveredCursor(position, properties, cursorProperties, null); if (log.isDebugEnabled()) { @@ -757,7 +754,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { counter.countDown(); } - }, null, PositionImpl.LATEST); + }, null, PositionFactory.LATEST); counter.await(); @@ -770,19 +767,19 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { @Override public void asyncReadEntries(final int numberOfEntriesToRead, final ReadEntriesCallback callback, - final Object ctx, PositionImpl maxPosition) { + final Object ctx, Position maxPosition) { asyncReadEntries(numberOfEntriesToRead, NO_MAX_SIZE_LIMIT, callback, ctx, maxPosition); } @Override public void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition) { + Object ctx, Position maxPosition) { asyncReadEntriesWithSkip(numberOfEntriesToRead, maxSizeBytes, callback, ctx, maxPosition, null); } @Override public void asyncReadEntriesWithSkip(int numberOfEntriesToRead, long maxSizeBytes, ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition, Predicate skipCondition) { + Object ctx, Position maxPosition, Predicate skipCondition) { checkArgument(numberOfEntriesToRead > 0); if (isClosed()) { callback.readEntriesFailed(new ManagedLedgerException @@ -851,8 +848,8 @@ public void asyncGetNthEntry(int n, IndividualDeletedEntries deletedEntries, Rea return; } - PositionImpl startPosition = ledger.getNextValidPosition(markDeletePosition); - PositionImpl endPosition = ledger.getLastPosition(); + Position startPosition = ledger.getNextValidPosition(markDeletePosition); + Position endPosition = ledger.getLastPosition(); if (startPosition.compareTo(endPosition) <= 0) { long numOfEntries = getNumberOfEntries(Range.closed(startPosition, endPosition)); if (numOfEntries >= n) { @@ -860,7 +857,7 @@ public void asyncGetNthEntry(int n, IndividualDeletedEntries deletedEntries, Rea if (deletedEntries == IndividualDeletedEntries.Exclude) { deletedMessages = getNumIndividualDeletedEntriesToSkip(n); } - PositionImpl positionAfterN = ledger.getPositionAfterN(markDeletePosition, n + deletedMessages, + Position positionAfterN = ledger.getPositionAfterN(markDeletePosition, n + deletedMessages, PositionBound.startExcluded); ledger.asyncReadEntry(positionAfterN, callback, ctx); } else { @@ -903,7 +900,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { counter.countDown(); } - }, null, PositionImpl.LATEST); + }, null, PositionFactory.LATEST); counter.await(); @@ -916,27 +913,27 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { @Override public void asyncReadEntriesOrWait(int numberOfEntriesToRead, ReadEntriesCallback callback, Object ctx, - PositionImpl maxPosition) { + Position maxPosition) { asyncReadEntriesOrWait(numberOfEntriesToRead, NO_MAX_SIZE_LIMIT, callback, ctx, maxPosition); } @Override public void asyncReadEntriesOrWait(int maxEntries, long maxSizeBytes, ReadEntriesCallback callback, Object ctx, - PositionImpl maxPosition) { + Position maxPosition) { asyncReadEntriesWithSkipOrWait(maxEntries, maxSizeBytes, callback, ctx, maxPosition, null); } @Override public void asyncReadEntriesWithSkipOrWait(int maxEntries, ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition, - Predicate skipCondition) { + Object ctx, Position maxPosition, + Predicate skipCondition) { asyncReadEntriesWithSkipOrWait(maxEntries, NO_MAX_SIZE_LIMIT, callback, ctx, maxPosition, skipCondition); } @Override public void asyncReadEntriesWithSkipOrWait(int maxEntries, long maxSizeBytes, ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition, - Predicate skipCondition) { + Object ctx, Position maxPosition, + Predicate skipCondition) { checkArgument(maxEntries > 0); if (isClosed()) { callback.readEntriesFailed(new CursorAlreadyClosedException("Cursor was already closed"), ctx); @@ -1066,7 +1063,7 @@ public boolean hasMoreEntries() { // * Writer pointing to "invalid" entry -1 (meaning no entries in that ledger) --> Need to check if the reader // is // at the last entry in the previous ledger - PositionImpl writerPosition = ledger.getLastPosition(); + Position writerPosition = ledger.getLastPosition(); if (writerPosition.getEntryId() != -1) { return readPosition.compareTo(writerPosition) <= 0; } else { @@ -1093,8 +1090,8 @@ public long getNumberOfEntries() { public long getNumberOfEntriesSinceFirstNotAckedMessage() { // sometimes for already caught up consumer: due to race condition markDeletePosition > readPosition. so, // validate it before preparing range - PositionImpl markDeletePosition = this.markDeletePosition; - PositionImpl readPosition = this.readPosition; + Position markDeletePosition = this.markDeletePosition; + Position readPosition = this.readPosition; return (markDeletePosition != null && readPosition != null && markDeletePosition.compareTo(readPosition) < 0) ? ledger.getNumberOfEntries(Range.openClosed(markDeletePosition, readPosition)) : 0; @@ -1151,7 +1148,7 @@ public Position findNewestMatching(Predicate condition) throws Interrupte public CompletableFuture scan(Optional position, Predicate condition, int batchSize, long maxEntries, long timeOutMs) { - PositionImpl startPosition = (PositionImpl) position.orElseGet( + Position startPosition = position.orElseGet( () -> ledger.getNextValidPosition(markDeletePosition)); CompletableFuture future = new CompletableFuture<>(); OpScan op = new OpScan(this, batchSize, startPosition, condition, new ScanCallback() { @@ -1213,11 +1210,11 @@ public void asyncFindNewestMatching(FindPositionConstraint constraint, Predicate public void asyncFindNewestMatching(FindPositionConstraint constraint, Predicate condition, FindEntryCallback callback, Object ctx, boolean isFindFromLedger) { OpFindNewest op; - PositionImpl startPosition = null; + Position startPosition = null; long max = 0; switch (constraint) { case SearchAllAvailableEntries: - startPosition = (PositionImpl) getFirstPosition(); + startPosition = getFirstPosition(); max = ledger.getNumberOfEntries() - 1; break; case SearchActiveEntries: @@ -1271,15 +1268,15 @@ public void setAlwaysInactive() { @Override public Position getFirstPosition() { Long firstLedgerId = ledger.getLedgersInfo().firstKey(); - return firstLedgerId == null ? null : new PositionImpl(firstLedgerId, 0); + return firstLedgerId == null ? null : PositionFactory.create(firstLedgerId, 0); } - protected void internalResetCursor(PositionImpl proposedReadPosition, + protected void internalResetCursor(Position proposedReadPosition, AsyncCallbacks.ResetCursorCallback resetCursorCallback) { - final PositionImpl newReadPosition; - if (proposedReadPosition.equals(PositionImpl.EARLIEST)) { + final Position newReadPosition; + if (proposedReadPosition.equals(PositionFactory.EARLIEST)) { newReadPosition = ledger.getFirstPosition(); - } else if (proposedReadPosition.equals(PositionImpl.LATEST)) { + } else if (proposedReadPosition.equals(PositionFactory.LATEST)) { newReadPosition = ledger.getNextValidPosition(ledger.getLastPosition()); } else { newReadPosition = proposedReadPosition; @@ -1301,7 +1298,7 @@ protected void internalResetCursor(PositionImpl proposedReadPosition, final AsyncCallbacks.ResetCursorCallback callback = resetCursorCallback; - final PositionImpl newMarkDeletePosition = ledger.getPreviousPosition(newReadPosition); + final Position newMarkDeletePosition = ledger.getPreviousPosition(newReadPosition); VoidCallback finalCallback = new VoidCallback() { @Override @@ -1324,14 +1321,16 @@ public void operationComplete() { if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { batchDeletedIndexes.values().forEach(BitSetRecyclable::recycle); batchDeletedIndexes.clear(); - long[] resetWords = newReadPosition.ackSet; - if (resetWords != null) { - BitSetRecyclable ackSet = BitSetRecyclable.create().resetWords(resetWords); - batchDeletedIndexes.put(newReadPosition, ackSet); - } + AckSetStateUtil.maybeGetAckSetState(newReadPosition).ifPresent(ackSetState -> { + long[] resetWords = ackSetState.getAckSet(); + if (resetWords != null) { + BitSetRecyclable ackSet = BitSetRecyclable.create().resetWords(resetWords); + batchDeletedIndexes.put(newReadPosition, ackSet); + } + }); } - PositionImpl oldReadPosition = readPosition; + Position oldReadPosition = readPosition; if (oldReadPosition.compareTo(newReadPosition) >= 0) { log.info("[{}] reset readPosition to {} before current read readPosition {} on cursor {}", ledger.getName(), newReadPosition, oldReadPosition, name); @@ -1388,23 +1387,22 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { @Override public void asyncResetCursor(Position newPos, boolean forceReset, AsyncCallbacks.ResetCursorCallback callback) { - checkArgument(newPos instanceof PositionImpl); - final PositionImpl newPosition = (PositionImpl) newPos; + final Position newPosition = newPos; // order trim and reset operations on a ledger ledger.getExecutor().execute(() -> { - PositionImpl actualPosition = newPosition; + Position actualPosition = newPosition; if (!ledger.isValidPosition(actualPosition) - && !actualPosition.equals(PositionImpl.EARLIEST) - && !actualPosition.equals(PositionImpl.LATEST) + && !actualPosition.equals(PositionFactory.EARLIEST) + && !actualPosition.equals(PositionFactory.LATEST) && !forceReset) { actualPosition = ledger.getNextValidPosition(actualPosition); if (actualPosition == null) { // next valid position would only return null when newPos // is larger than all available positions, then it's latest in effect. - actualPosition = PositionImpl.LATEST; + actualPosition = PositionFactory.LATEST; } } @@ -1558,18 +1556,18 @@ public String toString() { positions.stream().filter(position -> !alreadyAcknowledgedPositions.contains(position)) .forEach(p ->{ - if (((PositionImpl) p).compareTo(this.readPosition) == 0) { + if (p.compareTo(this.readPosition) == 0) { this.setReadPosition(this.readPosition.getNext()); log.warn("[{}][{}] replayPosition{} equals readPosition{}," + " need set next readPosition", ledger.getName(), name, p, this.readPosition); } - ledger.asyncReadEntry((PositionImpl) p, cb, ctx); + ledger.asyncReadEntry(p, cb, ctx); }); return alreadyAcknowledgedPositions; } - protected long getNumberOfEntries(Range range) { + protected long getNumberOfEntries(Range range) { long allEntries = ledger.getNumberOfEntries(range); if (log.isDebugEnabled()) { @@ -1582,14 +1580,14 @@ protected long getNumberOfEntries(Range range) { try { if (getConfig().isUnackedRangesOpenCacheSetEnabled()) { int cardinality = individualDeletedMessages.cardinality( - range.lowerEndpoint().ledgerId, range.lowerEndpoint().entryId, - range.upperEndpoint().ledgerId, range.upperEndpoint().entryId); + range.lowerEndpoint().getLedgerId(), range.lowerEndpoint().getEntryId(), + range.upperEndpoint().getLedgerId(), range.upperEndpoint().getEntryId()); deletedEntries.addAndGet(cardinality); } else { individualDeletedMessages.forEach((r) -> { try { if (r.isConnected(range)) { - Range commonEntries = r.intersection(range); + Range commonEntries = r.intersection(range); long commonCount = ledger.getNumberOfEntries(commonEntries); if (log.isDebugEnabled()) { log.debug("[{}] [{}] Discounting {} entries for already deleted range {}", @@ -1599,9 +1597,9 @@ protected long getNumberOfEntries(Range range) { } return true; } finally { - if (r.lowerEndpoint() instanceof PositionImplRecyclable) { - ((PositionImplRecyclable) r.lowerEndpoint()).recycle(); - ((PositionImplRecyclable) r.upperEndpoint()).recycle(); + if (r.lowerEndpoint() instanceof PositionRecyclable) { + ((PositionRecyclable) r.lowerEndpoint()).recycle(); + ((PositionRecyclable) r.upperEndpoint()).recycle(); } } }, recyclePositionRangeConverter); @@ -1627,7 +1625,6 @@ public void markDelete(Position position) throws InterruptedException, ManagedLe public void markDelete(Position position, Map properties) throws InterruptedException, ManagedLedgerException { requireNonNull(position); - checkArgument(position instanceof PositionImpl); class Result { ManagedLedgerException exception = null; @@ -1752,7 +1749,7 @@ public void asyncSkipEntries(int numEntriesToSkip, IndividualDeletedEntries dele } asyncMarkDelete(ledger.getPositionAfterN(markDeletePosition, numEntriesToSkip + numDeletedMessages, - PositionBound.startExcluded), new MarkDeleteCallback() { + ManagedLedgerImpl.PositionBound.startExcluded), new MarkDeleteCallback() { @Override public void markDeleteComplete(Object ctx) { callback.skipEntriesComplete(ctx); @@ -1780,10 +1777,10 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { private static class InvidualDeletedMessagesHandlingState { long totalEntriesToSkip = 0L; long deletedMessages = 0L; - PositionImpl startPosition; - PositionImpl endPosition; + Position startPosition; + Position endPosition; - InvidualDeletedMessagesHandlingState(PositionImpl startPosition) { + InvidualDeletedMessagesHandlingState(Position startPosition) { this.startPosition = startPosition; } } @@ -1796,7 +1793,7 @@ long getNumIndividualDeletedEntriesToSkip(long numEntries) { try { state.endPosition = r.lowerEndpoint(); if (state.startPosition.compareTo(state.endPosition) <= 0) { - Range range = Range.openClosed(state.startPosition, state.endPosition); + Range range = Range.openClosed(state.startPosition, state.endPosition); long entries = ledger.getNumberOfEntries(range); if (state.totalEntriesToSkip + entries >= numEntries) { // do not process further @@ -1813,8 +1810,8 @@ long getNumIndividualDeletedEntriesToSkip(long numEntries) { } return true; } finally { - if (r.lowerEndpoint() instanceof PositionImplRecyclable) { - ((PositionImplRecyclable) r.lowerEndpoint()).recycle(); + if (r.lowerEndpoint() instanceof PositionRecyclable) { + ((PositionRecyclable) r.lowerEndpoint()).recycle(); } } }, recyclePositionRangeConverter); @@ -1824,15 +1821,15 @@ long getNumIndividualDeletedEntriesToSkip(long numEntries) { } } - boolean hasMoreEntries(PositionImpl position) { - PositionImpl lastPositionInLedger = ledger.getLastPosition(); + boolean hasMoreEntries(Position position) { + Position lastPositionInLedger = ledger.getLastPosition(); if (position.compareTo(lastPositionInLedger) <= 0) { return getNumberOfEntries(Range.closed(position, lastPositionInLedger)) > 0; } return false; } - void initializeCursorPosition(Pair lastPositionCounter) { + void initializeCursorPosition(Pair lastPositionCounter) { readPosition = ledger.getNextValidPosition(lastPositionCounter.getLeft()); ledger.onCursorReadPositionUpdated(this, readPosition); markDeletePosition = lastPositionCounter.getLeft(); @@ -1851,14 +1848,14 @@ void initializeCursorPosition(Pair lastPositionCounter) { * the new acknowledged position * @return the previous acknowledged position */ - PositionImpl setAcknowledgedPosition(PositionImpl newMarkDeletePosition) { + Position setAcknowledgedPosition(Position newMarkDeletePosition) { if (newMarkDeletePosition.compareTo(markDeletePosition) < 0) { throw new MarkDeletingMarkedPosition( "Mark deleting an already mark-deleted position. Current mark-delete: " + markDeletePosition + " -- attempted mark delete: " + newMarkDeletePosition); } - PositionImpl oldMarkDeletePosition = markDeletePosition; + Position oldMarkDeletePosition = markDeletePosition; if (!newMarkDeletePosition.equals(oldMarkDeletePosition)) { long skippedEntries = 0; @@ -1871,14 +1868,14 @@ PositionImpl setAcknowledgedPosition(PositionImpl newMarkDeletePosition) { skippedEntries = getNumberOfEntries(Range.openClosed(oldMarkDeletePosition, newMarkDeletePosition)); } - PositionImpl positionAfterNewMarkDelete = ledger.getNextValidPosition(newMarkDeletePosition); + Position positionAfterNewMarkDelete = ledger.getNextValidPosition(newMarkDeletePosition); // sometime ranges are connected but belongs to different ledgers so, they are placed sequentially // eg: (2:10..3:15] can be returned as (2:10..2:15],[3:0..3:15]. So, try to iterate over connected range and // found the last non-connected range which gives new markDeletePosition while (positionAfterNewMarkDelete.compareTo(ledger.lastConfirmedEntry) <= 0) { if (individualDeletedMessages.contains(positionAfterNewMarkDelete.getLedgerId(), positionAfterNewMarkDelete.getEntryId())) { - Range rangeToBeMarkDeleted = individualDeletedMessages.rangeContaining( + Range rangeToBeMarkDeleted = individualDeletedMessages.rangeContaining( positionAfterNewMarkDelete.getLedgerId(), positionAfterNewMarkDelete.getEntryId()); newMarkDeletePosition = rangeToBeMarkDeleted.upperEndpoint(); positionAfterNewMarkDelete = ledger.getNextValidPosition(newMarkDeletePosition); @@ -1904,7 +1901,7 @@ PositionImpl setAcknowledgedPosition(PositionImpl newMarkDeletePosition) { // If the position that is mark-deleted is past the read position, it // means that the client has skipped some entries. We need to move // read position forward - PositionImpl newReadPosition = ledger.getNextValidPosition(markDeletePosition); + Position newReadPosition = ledger.getNextValidPosition(markDeletePosition); if (log.isDebugEnabled()) { log.debug("[{}] Moved read position from: {} to: {}, and new mark-delete position {}", ledger.getName(), currentReadPosition, newReadPosition, markDeletePosition); @@ -1934,7 +1931,6 @@ public MarkDeletingMarkedPosition(String s) { public void asyncMarkDelete(final Position position, Map properties, final MarkDeleteCallback callback, final Object ctx) { requireNonNull(position); - checkArgument(position instanceof PositionImpl); if (isClosed()) { callback.markDeleteFailed(new ManagedLedgerException @@ -1958,12 +1954,14 @@ public void asyncMarkDelete(final Position position, Map propertie log.debug("[{}] Mark delete cursor {} up to position: {}", ledger.getName(), name, position); } - PositionImpl newPosition = (PositionImpl) position; + Position newPosition = position; + Optional ackSetStateOptional = AckSetStateUtil.maybeGetAckSetState(newPosition); if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { - if (newPosition.ackSet != null) { + if (ackSetStateOptional.isPresent()) { AtomicReference bitSetRecyclable = new AtomicReference<>(); - BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(newPosition.ackSet); + BitSetRecyclable givenBitSet = + BitSetRecyclable.create().resetWords(ackSetStateOptional.map(AckSetState::getAckSet).get()); // In order to prevent the batch index recorded in batchDeletedIndexes from rolling back, // only update batchDeletedIndexes when the submitted batch index is greater // than the recorded index. @@ -1985,15 +1983,19 @@ public void asyncMarkDelete(final Position position, Map propertie } newPosition = ledger.getPreviousPosition(newPosition); } - Map subMap = batchDeletedIndexes.subMap(PositionImpl.EARLIEST, newPosition); + Map subMap = batchDeletedIndexes.subMap(PositionFactory.EARLIEST, newPosition); subMap.values().forEach(BitSetRecyclable::recycle); subMap.clear(); - } else if (newPosition.ackSet != null) { - newPosition = ledger.getPreviousPosition(newPosition); - newPosition.ackSet = null; + } else { + if (ackSetStateOptional.isPresent()) { + AckSetState ackSetState = ackSetStateOptional.get(); + if (ackSetState.getAckSet() != null) { + newPosition = ledger.getPreviousPosition(newPosition); + } + } } - if (((PositionImpl) ledger.getLastConfirmedEntry()).compareTo(newPosition) < 0) { + if (ledger.getLastConfirmedEntry().compareTo(newPosition) < 0) { boolean shouldCursorMoveForward = false; try { long ledgerEntries = ledger.getLedgerInfo(markDeletePosition.getLedgerId()).get().getEntries(); @@ -2038,7 +2040,7 @@ public void asyncMarkDelete(final Position position, Map propertie internalAsyncMarkDelete(newPosition, properties, callback, ctx); } - protected void internalAsyncMarkDelete(final PositionImpl newPosition, Map properties, + protected void internalAsyncMarkDelete(final Position newPosition, Map properties, final MarkDeleteCallback callback, final Object ctx) { ledger.mbean.addMarkDeleteOp(); @@ -2093,7 +2095,7 @@ void internalMarkDelete(final MarkDeleteEntry mdEntry) { return; } - PositionImpl inProgressLatest = INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.updateAndGet(this, current -> { + Position inProgressLatest = INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.updateAndGet(this, current -> { if (current != null && current.compareTo(mdEntry.newPosition) > 0) { return current; } else { @@ -2144,8 +2146,8 @@ public void operationComplete() { individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()); if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { - Map subMap = batchDeletedIndexes.subMap(PositionImpl.EARLIEST, - false, PositionImpl.get(mdEntry.newPosition.getLedgerId(), + Map subMap = batchDeletedIndexes.subMap(PositionFactory.EARLIEST, + false, PositionFactory.create(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()), true); subMap.values().forEach(BitSetRecyclable::recycle); subMap.clear(); @@ -2258,7 +2260,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb return; } - PositionImpl newMarkDeletePosition = null; + Position newMarkDeletePosition = null; lock.writeLock().lock(); @@ -2269,8 +2271,8 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } for (Position pos : positions) { - PositionImpl position = (PositionImpl) requireNonNull(pos); - if (((PositionImpl) ledger.getLastConfirmedEntry()).compareTo(position) < 0) { + Position position = requireNonNull(pos); + if (ledger.getLastConfirmedEntry().compareTo(position) < 0) { if (log.isDebugEnabled()) { log.debug( "[{}] Failed mark delete due to invalid markDelete {} is ahead of last-confirmed-entry {} " @@ -2292,7 +2294,8 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } continue; } - if (position.ackSet == null) { + long[] ackSet = AckSetStateUtil.getAckSetArrayOrNull(position); + if (ackSet == null) { if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); if (bitSetRecyclable != null) { @@ -2301,7 +2304,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } // Add a range (prev, pos] to the set. Adding the previous entry as an open limit to the range will // make the RangeSet recognize the "continuity" between adjacent Positions. - PositionImpl previousPosition = ledger.getPreviousPosition(position); + Position previousPosition = ledger.getPreviousPosition(position); individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(), previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); MSG_CONSUMED_COUNTER_UPDATER.incrementAndGet(this); @@ -2311,14 +2314,14 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb individualDeletedMessages); } } else if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { - BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(position.ackSet); + BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(ackSet); BitSetRecyclable bitSet = batchDeletedIndexes.computeIfAbsent(position, (v) -> givenBitSet); if (givenBitSet != bitSet) { bitSet.and(givenBitSet); givenBitSet.recycle(); } if (bitSet.isEmpty()) { - PositionImpl previousPosition = ledger.getPreviousPosition(position); + Position previousPosition = ledger.getPreviousPosition(position); individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(), previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); @@ -2338,7 +2341,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb // If the lower bound of the range set is the current mark delete position, then we can trigger a new // mark-delete to the upper bound of the first range segment - Range range = individualDeletedMessages.firstRange(); + Range range = individualDeletedMessages.firstRange(); // If the upper bound is before the mark-delete position, we need to move ahead as these // individualDeletedMessages are now irrelevant @@ -2418,7 +2421,7 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { } // update lastMarkDeleteEntry field if newPosition is later than the current lastMarkDeleteEntry.newPosition - private void updateLastMarkDeleteEntryToLatest(final PositionImpl newPosition, + private void updateLastMarkDeleteEntryToLatest(final Position newPosition, final Map properties) { LAST_MARK_DELETE_ENTRY_UPDATER.updateAndGet(this, last -> { if (last != null && last.newPosition.compareTo(newPosition) > 0) { @@ -2443,13 +2446,13 @@ private void updateLastMarkDeleteEntryToLatest(final PositionImpl newPosition, List filterReadEntries(List entries) { lock.readLock().lock(); try { - Range entriesRange = Range.closed((PositionImpl) entries.get(0).getPosition(), - (PositionImpl) entries.get(entries.size() - 1).getPosition()); + Range entriesRange = Range.closed(entries.get(0).getPosition(), + entries.get(entries.size() - 1).getPosition()); if (log.isDebugEnabled()) { log.debug("[{}] [{}] Filtering entries {} - alreadyDeleted: {}", ledger.getName(), name, entriesRange, individualDeletedMessages); } - Range span = individualDeletedMessages.isEmpty() ? null : individualDeletedMessages.span(); + Range span = individualDeletedMessages.isEmpty() ? null : individualDeletedMessages.span(); if (span == null || !entriesRange.isConnected(span)) { // There are no individually deleted messages in this entry list, no need to perform filtering if (log.isDebugEnabled()) { @@ -2530,9 +2533,9 @@ public void rewind() { public void rewind(boolean readCompacted) { lock.writeLock().lock(); try { - PositionImpl newReadPosition = + Position newReadPosition = readCompacted ? markDeletePosition.getNext() : ledger.getNextValidPosition(markDeletePosition); - PositionImpl oldReadPosition = readPosition; + Position oldReadPosition = readPosition; log.info("[{}-{}] Rewind from {} to {}", ledger.getName(), name, oldReadPosition, newReadPosition); @@ -2545,8 +2548,7 @@ public void rewind(boolean readCompacted) { @Override public void seek(Position newReadPositionInt, boolean force) { - checkArgument(newReadPositionInt instanceof PositionImpl); - PositionImpl newReadPosition = (PositionImpl) newReadPositionInt; + Position newReadPosition = newReadPositionInt; lock.writeLock().lock(); try { @@ -2614,7 +2616,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { * @param callback * @param ctx */ - void persistPositionWhenClosing(PositionImpl position, Map properties, + void persistPositionWhenClosing(Position position, Map properties, final AsyncCallbacks.CloseCallback callback, final Object ctx) { if (shouldPersistUnackRangesToLedger()) { @@ -2661,7 +2663,7 @@ && getConfig().getMaxUnackedRangesToPersist() > 0 && individualDeletedMessages.size() > getConfig().getMaxUnackedRangesToPersistInMetadataStore(); } - private void persistPositionMetaStore(long cursorsLedgerId, PositionImpl position, Map properties, + private void persistPositionMetaStore(long cursorsLedgerId, Position position, Map properties, MetaStoreCallback callback, boolean persistIndividualDeletedMessageRanges) { if (state == State.Closed) { ledger.getExecutor().execute(() -> callback.operationFailed(new MetaStoreException( @@ -2779,10 +2781,9 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { * @param newReadPositionInt */ void setReadPosition(Position newReadPositionInt) { - checkArgument(newReadPositionInt instanceof PositionImpl); if (this.markDeletePosition == null - || ((PositionImpl) newReadPositionInt).compareTo(this.markDeletePosition) > 0) { - this.readPosition = (PositionImpl) newReadPositionInt; + || newReadPositionInt.compareTo(this.markDeletePosition) > 0) { + this.readPosition = newReadPositionInt; ledger.onCursorReadPositionUpdated(this, newReadPositionInt); } } @@ -2804,7 +2805,7 @@ public void skipNonRecoverableLedger(final long ledgerId){ log.warn("[{}] [{}] Since the ledger [{}] is lost and the autoSkipNonRecoverableData is true, this ledger will" + " be auto acknowledge in subscription", ledger.getName(), name, ledgerId); asyncDelete(() -> LongStream.range(0, ledgerInfo.getEntries()) - .mapToObj(i -> (Position) PositionImpl.get(ledgerId, i)).iterator(), + .mapToObj(i -> (Position) PositionFactory.create(ledgerId, i)).iterator(), new AsyncCallbacks.DeleteCallback() { @Override public void deleteComplete(Object ctx) { @@ -3075,9 +3076,9 @@ private List buildBatchEntryDeletio MLDataFormats.BatchedEntryDeletionIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats .BatchedEntryDeletionIndexInfo.newBuilder(); List result = new ArrayList<>(); - Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); + Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); while (iterator.hasNext() && result.size() < getConfig().getMaxBatchDeletedIndexToPersist()) { - Map.Entry entry = iterator.next(); + Map.Entry entry = iterator.next(); nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); @@ -3097,7 +3098,7 @@ private List buildBatchEntryDeletio } void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, final VoidCallback callback) { - PositionImpl position = mdEntry.newPosition; + Position position = mdEntry.newPosition; PositionInfo pi = PositionInfo.newBuilder().setLedgerId(position.getLedgerId()) .setEntryId(position.getEntryId()) .addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()) @@ -3167,7 +3168,7 @@ boolean rolloverLedgerIfNeeded(LedgerHandle lh1) { } void persistPositionToMetaStore(MarkDeleteEntry mdEntry, final VoidCallback callback) { - final PositionImpl newPosition = mdEntry.newPosition; + final Position newPosition = mdEntry.newPosition; STATE_UPDATER.compareAndSet(ManagedCursorImpl.this, State.Open, State.NoLedger); mbean.persistToLedger(false); // Before giving up, try to persist the position in the metadata store @@ -3257,7 +3258,7 @@ void notifyEntriesAvailable() { } PENDING_READ_OPS_UPDATER.incrementAndGet(this); - opReadEntry.readPosition = (PositionImpl) getReadPosition(); + opReadEntry.readPosition = getReadPosition(); ledger.asyncReadEntries(opReadEntry); } else { // No one is waiting to be notified. Ignore @@ -3397,9 +3398,10 @@ public static boolean isBkErrorNotRecoverable(int rc) { * * @param info */ - private PositionImpl getRollbackPosition(ManagedCursorInfo info) { - PositionImpl firstPosition = ledger.getFirstPosition(); - PositionImpl snapshottedPosition = new PositionImpl(info.getMarkDeleteLedgerId(), info.getMarkDeleteEntryId()); + private Position getRollbackPosition(ManagedCursorInfo info) { + Position firstPosition = ledger.getFirstPosition(); + Position snapshottedPosition = + PositionFactory.create(info.getMarkDeleteLedgerId(), info.getMarkDeleteEntryId()); if (firstPosition == null) { // There are no ledgers in the ML, any position is good return snapshottedPosition; @@ -3440,22 +3442,17 @@ public String getIndividuallyDeletedMessages() { } @VisibleForTesting - public LongPairRangeSet getIndividuallyDeletedMessagesSet() { + public LongPairRangeSet getIndividuallyDeletedMessagesSet() { return individualDeletedMessages; } public boolean isMessageDeleted(Position position) { - checkArgument(position instanceof PositionImpl); - return ((PositionImpl) position).compareTo(markDeletePosition) <= 0 + return position.compareTo(markDeletePosition) <= 0 || individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()); } //this method will return a copy of the position's ack set public long[] getBatchPositionAckSet(Position position) { - if (!(position instanceof PositionImpl)) { - return null; - } - if (batchDeletedIndexes != null) { BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.get(position); if (bitSetRecyclable == null) { @@ -3475,11 +3472,11 @@ public long[] getBatchPositionAckSet(Position position) { * @param position * @return next available position */ - public PositionImpl getNextAvailablePosition(PositionImpl position) { - Range range = individualDeletedMessages.rangeContaining(position.getLedgerId(), + public Position getNextAvailablePosition(Position position) { + Range range = individualDeletedMessages.rangeContaining(position.getLedgerId(), position.getEntryId()); if (range != null) { - PositionImpl nextPosition = range.upperEndpoint().getNext(); + Position nextPosition = range.upperEndpoint().getNext(); return (nextPosition != null && nextPosition.compareTo(position) > 0) ? nextPosition : position.getNext(); } return position.getNext(); @@ -3487,7 +3484,7 @@ public PositionImpl getNextAvailablePosition(PositionImpl position) { public Position getNextLedgerPosition(long currentLedgerId) { Long nextExistingLedger = ledger.getNextValidLedger(currentLedgerId); - return nextExistingLedger != null ? PositionImpl.get(nextExistingLedger, 0) : null; + return nextExistingLedger != null ? PositionFactory.create(nextExistingLedger, 0) : null; } public boolean isIndividuallyDeletedEntriesEmpty() { @@ -3532,7 +3529,7 @@ public ManagedLedger getManagedLedger() { } @Override - public Range getLastIndividualDeletedRange() { + public Range getLastIndividualDeletedRange() { return individualDeletedMessages.lastRange(); } @@ -3552,7 +3549,7 @@ private ManagedCursorImpl cursorImpl() { } @Override - public long[] getDeletedBatchIndexesAsLongArray(PositionImpl position) { + public long[] getDeletedBatchIndexesAsLongArray(Position position) { if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable bitSet = batchDeletedIndexes.get(position); return bitSet == null ? null : bitSet.toLongArray(); @@ -3626,7 +3623,7 @@ public int applyMaxSizeCap(int maxEntries, long maxSizeBytes) { @Override public boolean checkAndUpdateReadPositionChanged() { - PositionImpl lastEntry = ledger.lastConfirmedEntry; + Position lastEntry = ledger.lastConfirmedEntry; boolean isReadPositionOnTail = lastEntry == null || readPosition == null || (lastEntry.compareTo(readPosition) <= 0); boolean isReadPositionChanged = readPosition != null && !readPosition.equals(statsLastReadPosition); @@ -3674,7 +3671,7 @@ public ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) thro }); } if (batchDeletedIndexes != null) { - for (Map.Entry entry : this.batchDeletedIndexes.entrySet()) { + for (Map.Entry entry : this.batchDeletedIndexes.entrySet()) { BitSetRecyclable copiedBitSet = BitSetRecyclable.valueOf(entry.getValue()); newNonDurableCursor.batchDeletedIndexes.put(entry.getKey(), copiedBitSet); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index ed803a81462e1..0b0f66d14c98c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -18,7 +18,6 @@ */ package org.apache.bookkeeper.mledger.impl; -import static com.google.common.base.Preconditions.checkArgument; import static org.apache.bookkeeper.mledger.ManagedLedgerException.getManagedLedgerException; import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.NULL_OFFLOAD_PROMISE; import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; @@ -507,13 +506,12 @@ public void asyncOpenReadOnlyCursor(String managedLedgerName, Position startPosi callback.openReadOnlyCursorFailed(new ManagedLedgerException.ManagedLedgerFactoryClosedException(), ctx); return; } - checkArgument(startPosition instanceof PositionImpl); AsyncCallbacks.OpenReadOnlyManagedLedgerCallback openReadOnlyManagedLedgerCallback = new AsyncCallbacks.OpenReadOnlyManagedLedgerCallback() { @Override public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl readOnlyManagedLedger, Object ctx) { callback.openReadOnlyCursorComplete(readOnlyManagedLedger. - createReadOnlyCursor((PositionImpl) startPosition), ctx); + createReadOnlyCursor(startPosition), ctx); } @Override diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index a07aad973142c..8d1919dd0529c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -113,6 +113,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.ManagedLedgerMXBean; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.WaitingEntryCallBack; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.VoidCallback; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; @@ -213,8 +214,8 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { private final CallbackMutex trimmerMutex = new CallbackMutex(); private final CallbackMutex offloadMutex = new CallbackMutex(); - public static final CompletableFuture NULL_OFFLOAD_PROMISE = CompletableFuture - .completedFuture(PositionImpl.LATEST); + public static final CompletableFuture NULL_OFFLOAD_PROMISE = CompletableFuture + .completedFuture(PositionFactory.LATEST); protected volatile LedgerHandle currentLedger; protected volatile long currentLedgerEntries = 0; protected volatile long currentLedgerSize = 0; @@ -234,7 +235,7 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { private long maximumRolloverTimeMs; protected final Supplier> mlOwnershipChecker; - volatile PositionImpl lastConfirmedEntry; + volatile Position lastConfirmedEntry; protected ManagedLedgerInterceptor managedLedgerInterceptor; @@ -385,7 +386,9 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { ledgersStat = stat; if (mlInfo.hasTerminatedPosition()) { state = State.Terminated; - lastConfirmedEntry = new PositionImpl(mlInfo.getTerminatedPosition()); + NestedPositionInfo terminatedPosition = mlInfo.getTerminatedPosition(); + lastConfirmedEntry = + PositionFactory.create(terminatedPosition.getLedgerId(), terminatedPosition.getEntryId()); log.info("[{}] Recovering managed ledger terminated at {}", name, lastConfirmedEntry); } for (LedgerInfo ls : mlInfo.getLedgerInfoList()) { @@ -535,13 +538,14 @@ public void operationFailed(MetaStoreException e) { currentLedger = lh; currentLedgerTimeoutTriggered = new AtomicBoolean(); - lastConfirmedEntry = new PositionImpl(lh.getId(), -1); + lastConfirmedEntry = PositionFactory.create(lh.getId(), -1); // bypass empty ledgers, find last ledger with Message if possible. while (lastConfirmedEntry.getEntryId() == -1) { Map.Entry formerLedger = ledgers.lowerEntry(lastConfirmedEntry.getLedgerId()); if (formerLedger != null) { LedgerInfo ledgerInfo = formerLedger.getValue(); - lastConfirmedEntry = PositionImpl.get(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1); + lastConfirmedEntry = + PositionFactory.create(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1); } else { break; } @@ -654,7 +658,7 @@ private void addCursor(ManagedCursorImpl cursor) { if (cursor.isDurable()) { positionForOrdering = cursor.getMarkDeletedPosition(); if (positionForOrdering == null) { - positionForOrdering = PositionImpl.EARLIEST; + positionForOrdering = PositionFactory.EARLIEST; } } cursors.add(cursor, positionForOrdering); @@ -999,7 +1003,7 @@ public synchronized void asyncOpenCursor(final String cursorName, final InitialP final ManagedCursorImpl cursor = new ManagedCursorImpl(bookKeeper, this, cursorName); CompletableFuture cursorFuture = new CompletableFuture<>(); uninitializedCursors.put(cursorName, cursorFuture); - PositionImpl position = InitialPosition.Earliest == initialPosition ? getFirstPosition() : getLastPosition(); + Position position = InitialPosition.Earliest == initialPosition ? getFirstPosition() : getLastPosition(); cursor.initialize(position, properties, cursorProperties, new VoidCallback() { @Override public void operationComplete() { @@ -1130,7 +1134,7 @@ public ManagedCursor newNonDurableCursor(Position startCursorPosition, String cu } NonDurableCursorImpl cursor = new NonDurableCursorImpl(bookKeeper, this, cursorName, - (PositionImpl) startCursorPosition, initialPosition, isReadCompacted); + startCursorPosition, initialPosition, isReadCompacted); cursor.setActive(); log.info("[{}] Opened new cursor: {}", name, cursor); @@ -1169,7 +1173,7 @@ public long getNumberOfEntries() { @Override public long getNumberOfActiveEntries() { long totalEntries = getNumberOfEntries(); - PositionImpl pos = cursors.getSlowestReaderPosition(); + Position pos = cursors.getSlowestReaderPosition(); if (pos == null) { // If there are no consumers, there are no active entries return 0; @@ -1188,7 +1192,7 @@ public long getTotalSize() { @Override public long getEstimatedBacklogSize() { - PositionImpl pos = getMarkDeletePositionOfSlowestConsumer(); + Position pos = getMarkDeletePositionOfSlowestConsumer(); while (true) { if (pos == null) { @@ -1232,18 +1236,18 @@ public long getEstimatedBacklogSize() { @Override public CompletableFuture getEarliestMessagePublishTimeInBacklog() { - PositionImpl pos = getMarkDeletePositionOfSlowestConsumer(); + Position pos = getMarkDeletePositionOfSlowestConsumer(); return getEarliestMessagePublishTimeOfPos(pos); } - public CompletableFuture getEarliestMessagePublishTimeOfPos(PositionImpl pos) { + public CompletableFuture getEarliestMessagePublishTimeOfPos(Position pos) { CompletableFuture future = new CompletableFuture<>(); if (pos == null) { future.complete(0L); return future; } - PositionImpl nextPos = getNextValidPosition(pos); + Position nextPos = getNextValidPosition(pos); if (nextPos.compareTo(lastConfirmedEntry) > 0) { return CompletableFuture.completedFuture(-1L); @@ -1282,14 +1286,14 @@ public String toString() { /** * Get estimated backlog size from a specific position. */ - public long getEstimatedBacklogSize(PositionImpl pos) { + public long getEstimatedBacklogSize(Position pos) { if (pos == null) { return 0; } return estimateBacklogFromPosition(pos); } - long estimateBacklogFromPosition(PositionImpl pos) { + long estimateBacklogFromPosition(Position pos) { synchronized (this) { long sizeBeforePosLedger = ledgers.headMap(pos.getLedgerId()).values() .stream().mapToLong(LedgerInfo::getSize).sum(); @@ -1368,7 +1372,7 @@ public synchronized void asyncTerminate(TerminateCallback callback, Object ctx) if (rc != BKException.Code.OK) { callback.terminateFailed(createManagedLedgerException(rc), ctx); } else { - lastConfirmedEntry = new PositionImpl(lh.getId(), lh.getLastAddConfirmed()); + lastConfirmedEntry = PositionFactory.create(lh.getId(), lh.getLastAddConfirmed()); // Store the new state in metadata store.asyncUpdateLedgerIds(name, getManagedLedgerInfo(), ledgersStat, new MetaStoreCallback() { @Override @@ -1828,7 +1832,7 @@ public CompletableFuture asyncFindPosition(Predicate predicate) CompletableFuture future = new CompletableFuture<>(); Long firstLedgerId = ledgers.firstKey(); - final PositionImpl startPosition = firstLedgerId == null ? null : new PositionImpl(firstLedgerId, 0); + final Position startPosition = firstLedgerId == null ? null : PositionFactory.create(firstLedgerId, 0); if (startPosition == null) { future.complete(null); return future; @@ -1842,7 +1846,7 @@ public void findEntryComplete(Position position, Object ctx) { log.info("[{}] Unable to find position for predicate {}. Use the first position {} instead.", name, predicate, startPosition); } else { - finalPosition = getNextValidPosition((PositionImpl) position); + finalPosition = getNextValidPosition(position); } future.complete(finalPosition); } @@ -1893,7 +1897,7 @@ void asyncReadEntries(OpReadEntry opReadEntry) { if (ledgerInfo == null || ledgerInfo.getEntries() == 0) { // Cursor is pointing to an empty ledger, there's no need to try opening it. Skip this ledger and // move to the next one - opReadEntry.updateReadPosition(new PositionImpl(opReadEntry.readPosition.getLedgerId() + 1, 0)); + opReadEntry.updateReadPosition(PositionFactory.create(opReadEntry.readPosition.getLedgerId() + 1, 0)); opReadEntry.checkReadCompletion(); return; } @@ -2021,7 +2025,7 @@ public void invalidateLedgerHandle(ReadHandle ledgerHandle) { } } - public void asyncReadEntry(PositionImpl position, ReadEntryCallback callback, Object ctx) { + public void asyncReadEntry(Position position, ReadEntryCallback callback, Object ctx) { LedgerHandle currentLedger = this.currentLedger; if (log.isDebugEnabled()) { log.debug("[{}] Reading entry ledger {}: {}", name, position.getLedgerId(), position.getEntryId()); @@ -2054,7 +2058,7 @@ private void internalReadFromLedger(ReadHandle ledger, OpReadEntry opReadEntry) long firstEntry = opReadEntry.readPosition.getEntryId(); long lastEntryInLedger; - PositionImpl lastPosition = lastConfirmedEntry; + Position lastPosition = lastConfirmedEntry; if (ledger.getId() == lastPosition.getLedgerId()) { // For the current ledger, we only give read visibility to the last entry we have received a confirmation in @@ -2081,9 +2085,9 @@ private void internalReadFromLedger(ReadHandle ledger, OpReadEntry opReadEntry) // beginning of the next ledger Long nextLedgerId = ledgers.ceilingKey(ledger.getId() + 1); if (nextLedgerId != null) { - opReadEntry.updateReadPosition(new PositionImpl(nextLedgerId, 0)); + opReadEntry.updateReadPosition(PositionFactory.create(nextLedgerId, 0)); } else { - opReadEntry.updateReadPosition(new PositionImpl(ledger.getId() + 1, 0)); + opReadEntry.updateReadPosition(PositionFactory.create(ledger.getId() + 1, 0)); } } else { opReadEntry.updateReadPosition(opReadEntry.readPosition); @@ -2101,7 +2105,7 @@ private void internalReadFromLedger(ReadHandle ledger, OpReadEntry opReadEntry) long lastValidEntry = -1L; long entryId = firstEntry; for (; entryId <= lastEntry; entryId++) { - if (opReadEntry.skipCondition.test(PositionImpl.get(ledger.getId(), entryId))) { + if (opReadEntry.skipCondition.test(PositionFactory.create(ledger.getId(), entryId))) { if (firstValidEntry != -1L) { break; } @@ -2118,7 +2122,7 @@ private void internalReadFromLedger(ReadHandle ledger, OpReadEntry opReadEntry) // then manual call internalReadEntriesComplete to advance read position. if (firstValidEntry == -1L) { opReadEntry.internalReadEntriesComplete(Collections.emptyList(), opReadEntry.ctx, - PositionImpl.get(ledger.getId(), lastEntry)); + PositionFactory.create(ledger.getId(), lastEntry)); return; } @@ -2133,7 +2137,7 @@ private void internalReadFromLedger(ReadHandle ledger, OpReadEntry opReadEntry) asyncReadEntry(ledger, firstEntry, lastEntry, opReadEntry, opReadEntry.ctx); } - protected void asyncReadEntry(ReadHandle ledger, PositionImpl position, ReadEntryCallback callback, Object ctx) { + protected void asyncReadEntry(ReadHandle ledger, Position position, ReadEntryCallback callback, Object ctx) { mbean.addEntriesRead(1); if (config.getReadEntryTimeoutSeconds() > 0) { // set readOpCount to uniquely validate if ReadEntryCallbackWrapper is already recycled @@ -2318,8 +2322,8 @@ public ManagedLedgerMXBean getStats() { return mbean; } - public boolean hasMoreEntries(PositionImpl position) { - PositionImpl lastPos = lastConfirmedEntry; + public boolean hasMoreEntries(Position position) { + Position lastPos = lastConfirmedEntry; boolean result = position.compareTo(lastPos) <= 0; if (log.isDebugEnabled()) { log.debug("[{}] hasMoreEntries: pos={} lastPos={} res={}", name, position, lastPos, result); @@ -2340,7 +2344,7 @@ private void invalidateEntriesUpToSlowestReaderPosition() { return; } if (!activeCursors.isEmpty()) { - PositionImpl evictionPos = activeCursors.getSlowestReaderPosition(); + Position evictionPos = activeCursors.getSlowestReaderPosition(); if (evictionPos != null) { entryCache.invalidateEntries(evictionPos); } @@ -2349,7 +2353,7 @@ private void invalidateEntriesUpToSlowestReaderPosition() { } } - void onCursorMarkDeletePositionUpdated(ManagedCursorImpl cursor, PositionImpl newPosition) { + void onCursorMarkDeletePositionUpdated(ManagedCursorImpl cursor, Position newPosition) { if (config.isCacheEvictionByMarkDeletedPosition()) { updateActiveCursor(cursor, newPosition); } @@ -2357,15 +2361,15 @@ void onCursorMarkDeletePositionUpdated(ManagedCursorImpl cursor, PositionImpl ne // non-durable cursors aren't tracked for trimming return; } - Pair pair = cursors.cursorUpdated(cursor, newPosition); + Pair pair = cursors.cursorUpdated(cursor, newPosition); if (pair == null) { // Cursor has been removed in the meantime trimConsumedLedgersInBackground(); return; } - PositionImpl previousSlowestReader = pair.getLeft(); - PositionImpl currentSlowestReader = pair.getRight(); + Position previousSlowestReader = pair.getLeft(); + Position currentSlowestReader = pair.getRight(); if (previousSlowestReader.compareTo(currentSlowestReader) == 0) { // The slowest consumer has not changed position. Nothing to do right now @@ -2379,7 +2383,7 @@ void onCursorMarkDeletePositionUpdated(ManagedCursorImpl cursor, PositionImpl ne } private void updateActiveCursor(ManagedCursorImpl cursor, Position newPosition) { - Pair slowestPositions = activeCursors.cursorUpdated(cursor, newPosition); + Pair slowestPositions = activeCursors.cursorUpdated(cursor, newPosition); if (slowestPositions != null && !slowestPositions.getLeft().equals(slowestPositions.getRight())) { invalidateEntriesUpToSlowestReaderPosition(); @@ -2392,12 +2396,12 @@ public void onCursorReadPositionUpdated(ManagedCursorImpl cursor, Position newRe } } - PositionImpl startReadOperationOnLedger(PositionImpl position) { + Position startReadOperationOnLedger(Position position) { Long ledgerId = ledgers.ceilingKey(position.getLedgerId()); if (ledgerId != null && ledgerId != position.getLedgerId()) { // The ledger pointed by this position does not exist anymore. It was deleted because it was empty. We need // to skip on the next available ledger - position = new PositionImpl(ledgerId, 0); + position = PositionFactory.create(ledgerId, 0); } return position; @@ -2431,7 +2435,7 @@ public void addWaitingEntryCallBack(WaitingEntryCallBack cb) { public void maybeUpdateCursorBeforeTrimmingConsumedLedger() { for (ManagedCursor cursor : cursors) { - PositionImpl lastAckedPosition = (PositionImpl) cursor.getMarkDeletedPosition(); + Position lastAckedPosition = cursor.getMarkDeletedPosition(); LedgerInfo currPointedLedger = ledgers.get(lastAckedPosition.getLedgerId()); LedgerInfo nextPointedLedger = Optional.ofNullable(ledgers.higherEntry(lastAckedPosition.getLedgerId())) .map(Map.Entry::getValue).orElse(null); @@ -2440,7 +2444,7 @@ public void maybeUpdateCursorBeforeTrimmingConsumedLedger() { if (nextPointedLedger != null) { if (lastAckedPosition.getEntryId() != -1 && lastAckedPosition.getEntryId() + 1 >= currPointedLedger.getEntries()) { - lastAckedPosition = new PositionImpl(nextPointedLedger.getLedgerId(), -1); + lastAckedPosition = PositionFactory.create(nextPointedLedger.getLedgerId(), -1); } } else { log.debug("No need to reset cursor: {}, current ledger is the last ledger.", cursor); @@ -2480,7 +2484,7 @@ private void scheduleDeferredTrimming(boolean isTruncate, CompletableFuture p 100, TimeUnit.MILLISECONDS); } - public void maybeOffloadInBackground(CompletableFuture promise) { + public void maybeOffloadInBackground(CompletableFuture promise) { if (config.getLedgerOffloader() == null || config.getLedgerOffloader() == NullLedgerOffloader.INSTANCE || config.getLedgerOffloader().getOffloadPolicies() == null) { return; @@ -2497,7 +2501,7 @@ public void maybeOffloadInBackground(CompletableFuture promise) { } private void maybeOffload(long offloadThresholdInBytes, long offloadThresholdInSeconds, - CompletableFuture finalPromise) { + CompletableFuture finalPromise) { if (config.getLedgerOffloader() == null || config.getLedgerOffloader() == NullLedgerOffloader.INSTANCE || config.getLedgerOffloader().getOffloadPolicies() == null) { String msg = String.format("[%s] Nothing to offload due to offloader or offloadPolicies is NULL", name); @@ -2518,7 +2522,7 @@ private void maybeOffload(long offloadThresholdInBytes, long offloadThresholdInS return; } - CompletableFuture unlockingPromise = new CompletableFuture<>(); + CompletableFuture unlockingPromise = new CompletableFuture<>(); unlockingPromise.whenComplete((res, ex) -> { offloadMutex.unlock(); if (ex != null) { @@ -2564,7 +2568,7 @@ private void maybeOffload(long offloadThresholdInBytes, long offloadThresholdInS + ", total size = {}, already offloaded = {}, to offload = {}", name, toOffload.stream().map(LedgerInfo::getLedgerId).collect(Collectors.toList()), sizeSummed, alreadyOffloadedSize, toOffloadSize); - offloadLoop(unlockingPromise, toOffload, PositionImpl.LATEST, Optional.empty()); + offloadLoop(unlockingPromise, toOffload, PositionFactory.LATEST, Optional.empty()); } else { // offloadLoop will complete immediately with an empty list to offload log.debug("[{}] Nothing to offload, total size = {}, already offloaded = {}, " @@ -2572,7 +2576,7 @@ private void maybeOffload(long offloadThresholdInBytes, long offloadThresholdInS + "managedLedgerOffloadThresholdInSeconds:{}]", name, sizeSummed, alreadyOffloadedSize, offloadThresholdInBytes, TimeUnit.MILLISECONDS.toSeconds(offloadTimeThresholdMillis)); - unlockingPromise.complete(PositionImpl.LATEST); + unlockingPromise.complete(PositionFactory.LATEST); } } @@ -2653,7 +2657,7 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { // include lastLedger in the trimming. slowestReaderLedgerId = currentLedger.getId() + 1; } else { - PositionImpl slowestReaderPosition = cursors.getSlowestReaderPosition(); + Position slowestReaderPosition = cursors.getSlowestReaderPosition(); if (slowestReaderPosition != null) { // The slowest reader position is the mark delete position. // If the slowest reader position point the last entry in the ledger x, @@ -2843,7 +2847,7 @@ private boolean releaseReadHandleIfNoLongerRead(long ledgerId, long slowestNonDu } protected void doDeleteLedgers(List ledgersToDelete) { - PositionImpl currentLastConfirmedEntry = lastConfirmedEntry; + Position currentLastConfirmedEntry = lastConfirmedEntry; // Update metadata for (LedgerInfo ls : ledgersToDelete) { if (currentLastConfirmedEntry != null && ls.getLedgerId() == currentLastConfirmedEntry.getLedgerId()) { @@ -2875,7 +2879,7 @@ void advanceCursorsIfNecessary(List ledgersToDelete) throws LedgerNo // Just ack messages like a consumer. Normally, consumers will not confirm a position that does not exist, so // find the latest existing position to ack. - PositionImpl highestPositionToDelete = calculateLastEntryInLedgerList(ledgersToDelete); + Position highestPositionToDelete = calculateLastEntryInLedgerList(ledgersToDelete); if (highestPositionToDelete == null) { log.warn("[{}] The ledgers to be trim are all empty, skip to advance non-durable cursors: {}", name, ledgersToDelete); @@ -2885,8 +2889,8 @@ void advanceCursorsIfNecessary(List ledgersToDelete) throws LedgerNo // move the mark delete position to the highestPositionToDelete only if it is smaller than the add confirmed // to prevent the edge case where the cursor is caught up to the latest and highestPositionToDelete may be // larger than the last add confirmed - if (highestPositionToDelete.compareTo((PositionImpl) cursor.getMarkDeletedPosition()) > 0 - && highestPositionToDelete.compareTo((PositionImpl) cursor.getManagedLedger() + if (highestPositionToDelete.compareTo(cursor.getMarkDeletedPosition()) > 0 + && highestPositionToDelete.compareTo(cursor.getManagedLedger() .getLastConfirmedEntry()) <= 0 && !(!cursor.isDurable() && cursor instanceof NonDurableCursorImpl && ((NonDurableCursorImpl) cursor).isReadCompacted())) { cursor.asyncMarkDelete(highestPositionToDelete, cursor.getProperties(), new MarkDeleteCallback() { @@ -2907,11 +2911,11 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { /** * @return null if all ledgers is empty. */ - private PositionImpl calculateLastEntryInLedgerList(List ledgersToDelete) { + private Position calculateLastEntryInLedgerList(List ledgersToDelete) { for (int i = ledgersToDelete.size() - 1; i >= 0; i--) { LedgerInfo ledgerInfo = ledgersToDelete.get(i); if (ledgerInfo != null && ledgerInfo.hasEntries() && ledgerInfo.getEntries() > 0) { - return PositionImpl.get(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1); + return PositionFactory.create(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1); } } return null; @@ -3152,7 +3156,7 @@ public void asyncOffloadPrefix(Position pos, OffloadCallback callback, Object ct callback.offloadFailed(new ManagedLedgerException("NullLedgerOffloader"), ctx); return; } - PositionImpl requestOffloadTo = (PositionImpl) pos; + Position requestOffloadTo = pos; if (!isValidPosition(requestOffloadTo) // Also consider the case where the last ledger is currently // empty. In this the passed position is not technically @@ -3166,7 +3170,7 @@ public void asyncOffloadPrefix(Position pos, OffloadCallback callback, Object ct return; } - PositionImpl firstUnoffloaded; + Position firstUnoffloaded; Queue ledgersToOffload = new ConcurrentLinkedQueue<>(); synchronized (this) { @@ -3205,7 +3209,7 @@ public void asyncOffloadPrefix(Position pos, OffloadCallback callback, Object ct break; } } - firstUnoffloaded = PositionImpl.get(firstLedgerRetained, 0); + firstUnoffloaded = PositionFactory.create(firstLedgerRetained, 0); } if (ledgersToOffload.isEmpty()) { @@ -3218,7 +3222,7 @@ public void asyncOffloadPrefix(Position pos, OffloadCallback callback, Object ct log.info("[{}] Going to offload ledgers {}", name, ledgersToOffload.stream().map(LedgerInfo::getLedgerId).collect(Collectors.toList())); - CompletableFuture promise = new CompletableFuture<>(); + CompletableFuture promise = new CompletableFuture<>(); promise.whenComplete((result, exception) -> { offloadMutex.unlock(); if (exception != null) { @@ -3234,8 +3238,8 @@ public void asyncOffloadPrefix(Position pos, OffloadCallback callback, Object ct } } - void offloadLoop(CompletableFuture promise, Queue ledgersToOffload, - PositionImpl firstUnoffloaded, Optional firstError) { + void offloadLoop(CompletableFuture promise, Queue ledgersToOffload, + Position firstUnoffloaded, Optional firstError) { State currentState = getState(); if (currentState == State.Closed) { promise.completeExceptionally(new ManagedLedgerAlreadyClosedException( @@ -3300,7 +3304,7 @@ void offloadLoop(CompletableFuture promise, Queue ledg lastOffloadFailureTimestamp = System.currentTimeMillis(); log.warn("[{}] Exception occurred for ledgerId {} timestamp {} during offload", name, ledgerId, lastOffloadFailureTimestamp, exception); - PositionImpl newFirstUnoffloaded = PositionImpl.get(ledgerId, 0); + Position newFirstUnoffloaded = PositionFactory.create(ledgerId, 0); if (newFirstUnoffloaded.compareTo(firstUnoffloaded) > 0) { newFirstUnoffloaded = firstUnoffloaded; } @@ -3487,10 +3491,10 @@ private CompletableFuture completeLedgerInfoForOffloaded(long ledgerId, UU * the position range * @return the count of entries */ - long getNumberOfEntries(Range range) { - PositionImpl fromPosition = range.lowerEndpoint(); + long getNumberOfEntries(Range range) { + Position fromPosition = range.lowerEndpoint(); boolean fromIncluded = range.lowerBoundType() == BoundType.CLOSED; - PositionImpl toPosition = range.upperEndpoint(); + Position toPosition = range.upperEndpoint(); boolean toIncluded = range.upperBoundType() == BoundType.CLOSED; if (fromPosition.getLedgerId() == toPosition.getLedgerId()) { @@ -3534,7 +3538,7 @@ long getNumberOfEntries(Range range) { * specifies whether to include the start position in calculating the distance * @return the new position that is n entries ahead */ - public PositionImpl getPositionAfterN(final PositionImpl startPosition, long n, PositionBound startRange) { + public Position getPositionAfterN(final Position startPosition, long n, PositionBound startRange) { long entriesToSkip = n; long currentLedgerId; long currentEntryId; @@ -3542,7 +3546,7 @@ public PositionImpl getPositionAfterN(final PositionImpl startPosition, long n, currentLedgerId = startPosition.getLedgerId(); currentEntryId = startPosition.getEntryId(); } else { - PositionImpl nextValidPosition = getNextValidPosition(startPosition); + Position nextValidPosition = getNextValidPosition(startPosition); currentLedgerId = nextValidPosition.getLedgerId(); currentEntryId = nextValidPosition.getEntryId(); } @@ -3583,7 +3587,7 @@ public PositionImpl getPositionAfterN(final PositionImpl startPosition, long n, } } - PositionImpl positionToReturn = getPreviousPosition(PositionImpl.get(currentLedgerId, currentEntryId)); + Position positionToReturn = getPreviousPosition(PositionFactory.create(currentLedgerId, currentEntryId)); if (positionToReturn.compareTo(lastConfirmedEntry) > 0) { positionToReturn = lastConfirmedEntry; } @@ -3596,18 +3600,18 @@ public PositionImpl getPositionAfterN(final PositionImpl startPosition, long n, return positionToReturn; } - public boolean isNoMessagesAfterPos(PositionImpl pos) { - PositionImpl lac = (PositionImpl) getLastConfirmedEntry(); + public boolean isNoMessagesAfterPos(Position pos) { + Position lac = getLastConfirmedEntry(); return isNoMessagesAfterPosForSpecifiedLac(lac, pos); } - private boolean isNoMessagesAfterPosForSpecifiedLac(PositionImpl specifiedLac, PositionImpl pos) { + private boolean isNoMessagesAfterPosForSpecifiedLac(Position specifiedLac, Position pos) { if (pos.compareTo(specifiedLac) >= 0) { return true; } if (specifiedLac.getEntryId() < 0) { // Calculate the meaningful LAC. - PositionImpl actLac = getPreviousPosition(specifiedLac); + Position actLac = getPreviousPosition(specifiedLac); if (actLac.getEntryId() >= 0) { return pos.compareTo(actLac) >= 0; } else { @@ -3632,9 +3636,9 @@ private boolean isNoMessagesAfterPosForSpecifiedLac(PositionImpl specifiedLac, P * the current position * @return the previous position */ - public PositionImpl getPreviousPosition(PositionImpl position) { + public Position getPreviousPosition(Position position) { if (position.getEntryId() > 0) { - return PositionImpl.get(position.getLedgerId(), position.getEntryId() - 1); + return PositionFactory.create(position.getLedgerId(), position.getEntryId() - 1); } // The previous position will be the last position of an earlier ledgers @@ -3643,19 +3647,19 @@ public PositionImpl getPreviousPosition(PositionImpl position) { final Map.Entry firstEntry = headMap.firstEntry(); if (firstEntry == null) { // There is no previous ledger, return an invalid position in the current ledger - return PositionImpl.get(position.getLedgerId(), -1); + return PositionFactory.create(position.getLedgerId(), -1); } // We need to find the most recent non-empty ledger for (long ledgerId : headMap.descendingKeySet()) { LedgerInfo li = headMap.get(ledgerId); if (li != null && li.getEntries() > 0) { - return PositionImpl.get(li.getLedgerId(), li.getEntries() - 1); + return PositionFactory.create(li.getLedgerId(), li.getEntries() - 1); } } // in case there are only empty ledgers, we return a position in the first one - return PositionImpl.get(firstEntry.getKey(), -1); + return PositionFactory.create(firstEntry.getKey(), -1); } /** @@ -3665,8 +3669,8 @@ public PositionImpl getPreviousPosition(PositionImpl position) { * the position to validate * @return true if the position is valid, false otherwise */ - public boolean isValidPosition(PositionImpl position) { - PositionImpl lac = lastConfirmedEntry; + public boolean isValidPosition(Position position) { + Position lac = lastConfirmedEntry; if (log.isDebugEnabled()) { log.debug("IsValid position: {} -- last: {}", position, lac); } @@ -3710,37 +3714,26 @@ public Long getNextValidLedger(long ledgerId) { return ledgers.ceilingKey(ledgerId + 1); } - public PositionImpl getNextValidPosition(final PositionImpl position) { + public Position getNextValidPosition(final Position position) { return getValidPositionAfterSkippedEntries(position, 1); } - public PositionImpl getValidPositionAfterSkippedEntries(final PositionImpl position, int skippedEntryNum) { - PositionImpl skippedPosition = position.getPositionAfterEntries(skippedEntryNum); + public Position getValidPositionAfterSkippedEntries(final Position position, int skippedEntryNum) { + Position skippedPosition = position.getPositionAfterEntries(skippedEntryNum); while (!isValidPosition(skippedPosition)) { Long nextLedgerId = ledgers.ceilingKey(skippedPosition.getLedgerId() + 1); // This means it has jumped to the last position if (nextLedgerId == null) { if (currentLedgerEntries == 0 && currentLedger != null) { - return PositionImpl.get(currentLedger.getId(), 0); + return PositionFactory.create(currentLedger.getId(), 0); } return lastConfirmedEntry.getNext(); } - skippedPosition = PositionImpl.get(nextLedgerId, 0); + skippedPosition = PositionFactory.create(nextLedgerId, 0); } return skippedPosition; } - public PositionImpl getNextValidPositionInternal(final PositionImpl position) { - PositionImpl nextPosition = position.getNext(); - while (!isValidPosition(nextPosition)) { - Long nextLedgerId = ledgers.ceilingKey(nextPosition.getLedgerId() + 1); - if (nextLedgerId == null) { - throw new NullPointerException("nextLedgerId is null. No valid next position after " + position); - } - nextPosition = PositionImpl.get(nextLedgerId, 0); - } - return nextPosition; - } - public PositionImpl getFirstPosition() { + public Position getFirstPosition() { Long ledgerId = ledgers.firstKey(); if (ledgerId == null) { return null; @@ -3749,10 +3742,10 @@ public PositionImpl getFirstPosition() { checkState(ledgers.get(ledgerId).getEntries() == 0); ledgerId = lastConfirmedEntry.getLedgerId(); } - return new PositionImpl(ledgerId, -1); + return PositionFactory.create(ledgerId, -1); } - PositionImpl getLastPosition() { + Position getLastPosition() { return lastConfirmedEntry; } @@ -3761,16 +3754,16 @@ public ManagedCursor getSlowestConsumer() { return cursors.getSlowestReader(); } - PositionImpl getMarkDeletePositionOfSlowestConsumer() { + Position getMarkDeletePositionOfSlowestConsumer() { ManagedCursor slowestCursor = getSlowestConsumer(); - return slowestCursor == null ? null : (PositionImpl) slowestCursor.getMarkDeletedPosition(); + return slowestCursor == null ? null : slowestCursor.getMarkDeletedPosition(); } /** * Get the last position written in the managed ledger, alongside with the associated counter. */ - Pair getLastPositionAndCounter() { - PositionImpl pos; + Pair getLastPositionAndCounter() { + Position pos; long count; do { @@ -3786,10 +3779,10 @@ Pair getLastPositionAndCounter() { /** * Get the first position written in the managed ledger, alongside with the associated counter. */ - Pair getFirstPositionAndCounter() { - PositionImpl pos; + Pair getFirstPositionAndCounter() { + Position pos; long count; - Pair lastPositionAndCounter; + Pair lastPositionAndCounter; do { pos = getFirstPosition(); @@ -3808,7 +3801,7 @@ public void activateCursor(ManagedCursor cursor) { ? cursor.getMarkDeletedPosition() : cursor.getReadPosition(); if (positionForOrdering == null) { - positionForOrdering = PositionImpl.EARLIEST; + positionForOrdering = PositionFactory.EARLIEST; } activeCursors.add(cursor, positionForOrdering); } @@ -4561,10 +4554,10 @@ public void checkCursorsToCacheEntries() { } public Position getTheSlowestNonDurationReadPosition() { - PositionImpl theSlowestNonDurableReadPosition = PositionImpl.LATEST; + Position theSlowestNonDurableReadPosition = PositionFactory.LATEST; for (ManagedCursor cursor : cursors) { if (cursor instanceof NonDurableCursorImpl) { - PositionImpl readPosition = (PositionImpl) cursor.getReadPosition(); + Position readPosition = cursor.getReadPosition(); if (readPosition.compareTo(theSlowestNonDurableReadPosition) < 0) { theSlowestNonDurableReadPosition = readPosition; } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java index 81cd94e5bf96c..60e24e8df0498 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java @@ -35,6 +35,8 @@ import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.mledger.util.Errors; import org.apache.pulsar.common.naming.TopicName; @@ -63,11 +65,11 @@ public ManagedLedgerOfflineBacklog(DigestType digestType, byte[] password, Strin } // need a better way than to duplicate the functionality below from ML - private long getNumberOfEntries(Range range, + private long getNumberOfEntries(Range range, NavigableMap ledgers) { - PositionImpl fromPosition = range.lowerEndpoint(); + Position fromPosition = range.lowerEndpoint(); boolean fromIncluded = range.lowerBoundType() == BoundType.CLOSED; - PositionImpl toPosition = range.upperEndpoint(); + Position toPosition = range.upperEndpoint(); boolean toIncluded = range.upperBoundType() == BoundType.CLOSED; if (fromPosition.getLedgerId() == toPosition.getLedgerId()) { @@ -228,7 +230,8 @@ private void calculateCursorBacklogs(final ManagedLedgerFactoryImpl factory, fin ConcurrentOpenHashMap.newBuilder().build(); final MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo = ledgers.lastEntry().getValue(); - final PositionImpl lastLedgerPosition = new PositionImpl(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1); + final Position lastLedgerPosition = + PositionFactory.create(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1); if (log.isDebugEnabled()) { log.debug("[{}] Last ledger position {}", managedLedgerName, lastLedgerPosition); } @@ -312,14 +315,16 @@ public void readComplete(int rc, LedgerHandle lh, Enumeration seq, lh.getId()); return; } - final PositionImpl lastAckedMessagePosition = new PositionImpl(positionInfo); + final Position lastAckedMessagePosition = + PositionFactory.create(positionInfo.getLedgerId(), + positionInfo.getEntryId()); if (log.isDebugEnabled()) { log.debug("[{}] Cursor {} MD {} read last ledger position {}", managedLedgerName, cursorName, lastAckedMessagePosition, lastLedgerPosition); } // calculate cursor backlog - Range range = Range.openClosed(lastAckedMessagePosition, + Range range = Range.openClosed(lastAckedMessagePosition, lastLedgerPosition); if (log.isDebugEnabled()) { log.debug("[{}] Calculating backlog for cursor {} using range {}", @@ -351,9 +356,9 @@ public void operationComplete(MLDataFormats.ManagedCursorInfo info, bk.asyncOpenLedgerNoRecovery(cursorLedgerId, digestType, password, cursorLedgerOpenCb, null); } else { - PositionImpl lastAckedMessagePosition = new PositionImpl( + Position lastAckedMessagePosition = PositionFactory.create( info.getMarkDeleteLedgerId(), info.getMarkDeleteEntryId()); - Range range = Range.openClosed(lastAckedMessagePosition, + Range range = Range.openClosed(lastAckedMessagePosition, lastLedgerPosition); if (log.isDebugEnabled()) { log.debug("[{}] Calculating backlog for cursor {} using range {}", @@ -406,7 +411,7 @@ public void operationFailed(ManagedLedgerException.MetaStoreException e) { if (log.isDebugEnabled()) { log.debug("Cursor {} Ledger {} Trying to obtain MD from BkAdmin", cursorName, ledgerId); } - PositionImpl lastAckedMessagePosition = tryGetMDPosition(bk, ledgerId, cursorName); + Position lastAckedMessagePosition = tryGetMDPosition(bk, ledgerId, cursorName); if (lastAckedMessagePosition == null) { log.warn("[{}] Cursor {} read from ledger {}. Unable to determine cursor position", managedLedgerName, cursorName, ledgerId); @@ -416,7 +421,7 @@ public void operationFailed(ManagedLedgerException.MetaStoreException e) { cursorName, ledgerId, lastAckedMessagePosition); } // calculate cursor backlog - Range range = Range.openClosed(lastAckedMessagePosition, lastLedgerPosition); + Range range = Range.openClosed(lastAckedMessagePosition, lastLedgerPosition); if (log.isDebugEnabled()) { log.debug("[{}] Calculating backlog for cursor {} using range {}", managedLedgerName, cursorName, range); @@ -429,10 +434,10 @@ public void operationFailed(ManagedLedgerException.MetaStoreException e) { } } - private PositionImpl tryGetMDPosition(BookKeeper bookKeeper, long ledgerId, String cursorName) { + private Position tryGetMDPosition(BookKeeper bookKeeper, long ledgerId, String cursorName) { BookKeeperAdmin bookKeeperAdmin = null; long lastEntry = LedgerHandle.INVALID_ENTRY_ID; - PositionImpl lastAckedMessagePosition = null; + Position lastAckedMessagePosition = null; try { bookKeeperAdmin = new BookKeeperAdmin(bookKeeper); for (LedgerEntry ledgerEntry : bookKeeperAdmin.readEntries(ledgerId, 0, lastEntry)) { @@ -441,7 +446,8 @@ private PositionImpl tryGetMDPosition(BookKeeper bookKeeper, long ledgerId, Stri log.debug(" Read entry {} from ledger {} for cursor {}", lastEntry, ledgerId, cursorName); } MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(ledgerEntry.getEntry()); - lastAckedMessagePosition = new PositionImpl(positionInfo); + lastAckedMessagePosition = + PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId()); if (log.isDebugEnabled()) { log.debug("Cursor {} read position {}", cursorName, lastAckedMessagePosition); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java index 734eab20bc58e..326f8216f1e18 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java @@ -25,6 +25,8 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCursorCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.slf4j.Logger; @@ -35,7 +37,7 @@ public class NonDurableCursorImpl extends ManagedCursorImpl { private final boolean readCompacted; NonDurableCursorImpl(BookKeeper bookkeeper, ManagedLedgerImpl ledger, String cursorName, - PositionImpl startCursorPosition, CommandSubscribe.InitialPosition initialPosition, + Position startCursorPosition, CommandSubscribe.InitialPosition initialPosition, boolean isReadCompacted) { super(bookkeeper, ledger, cursorName); this.readCompacted = isReadCompacted; @@ -53,7 +55,7 @@ public class NonDurableCursorImpl extends ManagedCursorImpl { initializeCursorPosition(ledger.getFirstPositionAndCounter()); break; } - } else if (startCursorPosition.getLedgerId() == PositionImpl.EARLIEST.getLedgerId()) { + } else if (startCursorPosition.getLedgerId() == PositionFactory.EARLIEST.getLedgerId()) { // Start from invalid ledger to read from first available entry recoverCursor(ledger.getPreviousPosition(ledger.getFirstPosition())); } else { @@ -66,8 +68,8 @@ public class NonDurableCursorImpl extends ManagedCursorImpl { readPosition, markDeletePosition); } - private void recoverCursor(PositionImpl mdPosition) { - Pair lastEntryAndCounter = ledger.getLastPositionAndCounter(); + private void recoverCursor(Position mdPosition) { + Pair lastEntryAndCounter = ledger.getLastPositionAndCounter(); this.readPosition = isReadCompacted() ? mdPosition.getNext() : ledger.getNextValidPosition(mdPosition); markDeletePosition = ledger.getPreviousPosition(this.readPosition); @@ -96,7 +98,7 @@ void recover(final VoidCallback callback) { } @Override - protected void internalAsyncMarkDelete(final PositionImpl newPosition, Map properties, + protected void internalAsyncMarkDelete(final Position newPosition, Map properties, final MarkDeleteCallback callback, final Object ctx) { // Bypass persistence of mark-delete position and individually deleted messages info diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java index acbb0da5a4e74..539b62fe7fe4b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java @@ -36,6 +36,7 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; @@ -247,7 +248,7 @@ public void run() { entry.release(); } - PositionImpl lastEntry = PositionImpl.get(ledgerId, entryId); + Position lastEntry = PositionFactory.create(ledgerId, entryId); ManagedLedgerImpl.ENTRIES_ADDED_COUNTER_UPDATER.incrementAndGet(ml); ml.lastConfirmedEntry = lastEntry; @@ -288,7 +289,7 @@ public void closeComplete(int rc, LedgerHandle lh, Object ctx) { AddEntryCallback cb = callbackUpdater.getAndSet(this, null); if (cb != null) { - cb.addComplete(PositionImpl.get(lh.getId(), entryId), data.asReadOnly(), ctx); + cb.addComplete(PositionFactory.create(lh.getId(), entryId), data.asReadOnly(), ctx); ml.notifyCursors(); ml.notifyWaitingEntryCallBacks(); ReferenceCountUtil.release(data); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpFindNewest.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpFindNewest.java index 900af9322c791..707cb389eba1a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpFindNewest.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpFindNewest.java @@ -32,7 +32,7 @@ class OpFindNewest implements ReadEntryCallback { private final ManagedCursorImpl cursor; private final ManagedLedgerImpl ledger; - private final PositionImpl startPosition; + private final Position startPosition; private final FindEntryCallback callback; private final Predicate condition; private final Object ctx; @@ -41,13 +41,13 @@ enum State { checkFirst, checkLast, searching } - PositionImpl searchPosition; + Position searchPosition; long min; long max; Position lastMatchedPosition = null; State state; - public OpFindNewest(ManagedCursorImpl cursor, PositionImpl startPosition, Predicate condition, + public OpFindNewest(ManagedCursorImpl cursor, Position startPosition, Predicate condition, long numberOfEntries, FindEntryCallback callback, Object ctx) { this.cursor = cursor; this.ledger = cursor.ledger; @@ -63,7 +63,7 @@ public OpFindNewest(ManagedCursorImpl cursor, PositionImpl startPosition, Predic this.state = State.checkFirst; } - public OpFindNewest(ManagedLedgerImpl ledger, PositionImpl startPosition, Predicate condition, + public OpFindNewest(ManagedLedgerImpl ledger, Position startPosition, Predicate condition, long numberOfEntries, FindEntryCallback callback, Object ctx) { this.cursor = null; this.ledger = ledger; @@ -94,8 +94,10 @@ public void readEntryComplete(Entry entry, Object ctx) { lastMatchedPosition = position; // check last entry state = State.checkLast; - PositionImpl lastPosition = ledger.getLastPosition(); searchPosition = ledger.getPositionAfterN(searchPosition, max, PositionBound.startExcluded); + Position lastPosition = ledger.getLastPosition(); + searchPosition = + ledger.getPositionAfterN(searchPosition, max, ManagedLedgerImpl.PositionBound.startExcluded); if (lastPosition.compareTo(searchPosition) < 0) { if (log.isDebugEnabled()) { log.debug("first position {} matches, last should be {}, but moving to lastPos {}", position, diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index 534ef3d76cb0d..3fd7e36c433ae 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -30,26 +30,27 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; class OpReadEntry implements ReadEntriesCallback { ManagedCursorImpl cursor; - PositionImpl readPosition; + Position readPosition; private int count; private ReadEntriesCallback callback; Object ctx; // Results private List entries; - private PositionImpl nextReadPosition; - PositionImpl maxPosition; + private Position nextReadPosition; + Position maxPosition; - Predicate skipCondition; + Predicate skipCondition; - public static OpReadEntry create(ManagedCursorImpl cursor, PositionImpl readPositionRef, int count, - ReadEntriesCallback callback, Object ctx, PositionImpl maxPosition, Predicate skipCondition) { + public static OpReadEntry create(ManagedCursorImpl cursor, Position readPositionRef, int count, + ReadEntriesCallback callback, Object ctx, Position maxPosition, Predicate skipCondition) { OpReadEntry op = RECYCLER.get(); op.readPosition = cursor.ledger.startReadOperationOnLedger(readPositionRef); op.cursor = cursor; @@ -57,16 +58,16 @@ public static OpReadEntry create(ManagedCursorImpl cursor, PositionImpl readPosi op.callback = callback; op.entries = new ArrayList<>(); if (maxPosition == null) { - maxPosition = PositionImpl.LATEST; + maxPosition = PositionFactory.LATEST; } op.maxPosition = maxPosition; op.skipCondition = skipCondition; op.ctx = ctx; - op.nextReadPosition = PositionImpl.get(op.readPosition); + op.nextReadPosition = PositionFactory.create(op.readPosition); return op; } - void internalReadEntriesComplete(List returnedEntries, Object ctx, PositionImpl lastPosition) { + void internalReadEntriesComplete(List returnedEntries, Object ctx, Position lastPosition) { // Filter the returned entries for individual deleted messages int entriesCount = returnedEntries.size(); long entriesSize = 0; @@ -76,7 +77,7 @@ void internalReadEntriesComplete(List returnedEntries, Object ctx, Positi cursor.updateReadStats(entriesCount, entriesSize); if (entriesCount != 0) { - lastPosition = (PositionImpl) returnedEntries.get(entriesCount - 1).getPosition(); + lastPosition = returnedEntries.get(entriesCount - 1).getPosition(); } if (log.isDebugEnabled()) { log.debug("[{}][{}] Read entries succeeded batch_size={} cumulative_size={} requested_count={}", @@ -121,7 +122,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { if (exception instanceof ManagedLedgerException.LedgerNotExistException) { // try to find and move to next valid ledger nexReadPosition = cursor.getNextLedgerPosition(readPosition.getLedgerId()); - lostLedger = readPosition.ledgerId; + lostLedger = readPosition.getLedgerId(); } else { // Skip this read operation nexReadPosition = ledger.getValidPositionAfterSkippedEntries(readPosition, count); @@ -156,7 +157,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { } void updateReadPosition(Position newReadPosition) { - nextReadPosition = (PositionImpl) newReadPosition; + nextReadPosition = newReadPosition; cursor.setReadPosition(nextReadPosition); } @@ -194,7 +195,7 @@ private OpReadEntry(Handle recyclerHandle) { this.recyclerHandle = recyclerHandle; } - private static final Recycler RECYCLER = new Recycler() { + private static final Recycler RECYCLER = new Recycler<>() { @Override protected OpReadEntry newObject(Recycler.Handle recyclerHandle) { return new OpReadEntry(recyclerHandle); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpScan.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpScan.java index 6d68b042a7ad6..e4f6fd04ff4da 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpScan.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpScan.java @@ -30,7 +30,6 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.ScanOutcome; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound; @Slf4j class OpScan implements ReadEntriesCallback { @@ -44,11 +43,11 @@ class OpScan implements ReadEntriesCallback { private final long startTime = System.currentTimeMillis(); private final int batchSize; - PositionImpl searchPosition; + Position searchPosition; Position lastSeenPosition = null; public OpScan(ManagedCursorImpl cursor, int batchSize, - PositionImpl startPosition, Predicate condition, + Position startPosition, Predicate condition, ScanCallback callback, Object ctx, long maxEntries, long timeOutMs) { this.batchSize = batchSize; if (batchSize <= 0) { @@ -88,13 +87,13 @@ public void readEntriesComplete(List entries, Object ctx) { } } } - searchPosition = ledger.getPositionAfterN((PositionImpl) lastPositionForBatch, 1, - PositionBound.startExcluded); + searchPosition = ledger.getPositionAfterN(lastPositionForBatch, 1, + ManagedLedgerImpl.PositionBound.startExcluded); if (log.isDebugEnabled()) { log.debug("readEntryComplete {} at {} next is {}", lastPositionForBatch, searchPosition); } - if (searchPosition.compareTo((PositionImpl) lastPositionForBatch) == 0) { + if (searchPosition.compareTo(lastPositionForBatch) == 0) { // we have reached the end of the ledger, as we are not doing progress callback.scanComplete(lastSeenPosition, ScanOutcome.COMPLETED, OpScan.this.ctx); return; diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java deleted file mode 100644 index ee179b5d059c8..0000000000000 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java +++ /dev/null @@ -1,168 +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.bookkeeper.mledger.impl; - -import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.NestedPositionInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; - -public class PositionImpl implements Position, Comparable { - - protected long ledgerId; - protected long entryId; - protected long[] ackSet; - - public static final PositionImpl EARLIEST = new PositionImpl(-1, -1); - public static final PositionImpl LATEST = new PositionImpl(Long.MAX_VALUE, Long.MAX_VALUE); - - public PositionImpl(PositionInfo pi) { - this.ledgerId = pi.getLedgerId(); - this.entryId = pi.getEntryId(); - } - - public PositionImpl(NestedPositionInfo npi) { - this.ledgerId = npi.getLedgerId(); - this.entryId = npi.getEntryId(); - } - - public PositionImpl(long ledgerId, long entryId) { - this.ledgerId = ledgerId; - this.entryId = entryId; - } - - public PositionImpl(long ledgerId, long entryId, long[] ackSet) { - this.ledgerId = ledgerId; - this.entryId = entryId; - this.ackSet = ackSet; - } - - public PositionImpl(PositionImpl other) { - this.ledgerId = other.ledgerId; - this.entryId = other.entryId; - } - - public static PositionImpl get(long ledgerId, long entryId) { - return new PositionImpl(ledgerId, entryId); - } - - public static PositionImpl get(long ledgerId, long entryId, long[] ackSet) { - return new PositionImpl(ledgerId, entryId, ackSet); - } - - public static PositionImpl get(PositionImpl other) { - return new PositionImpl(other); - } - - public long[] getAckSet() { - return ackSet; - } - - public void setAckSet(long[] ackSet) { - this.ackSet = ackSet; - } - - public long getLedgerId() { - return ledgerId; - } - - public long getEntryId() { - return entryId; - } - - @Override - public PositionImpl getNext() { - if (entryId < 0) { - return PositionImpl.get(ledgerId, 0); - } else { - return PositionImpl.get(ledgerId, entryId + 1); - } - } - - /** - * Position after moving entryNum messages, - * if entryNum < 1, then return the current position. - * */ - public PositionImpl getPositionAfterEntries(int entryNum) { - if (entryNum < 1) { - return this; - } - if (entryId < 0) { - return PositionImpl.get(ledgerId, entryNum - 1); - } else { - return PositionImpl.get(ledgerId, entryId + entryNum); - } - } - - /** - * String representation of virtual cursor - LedgerId:EntryId. - */ - @Override - public String toString() { - return ledgerId + ":" + entryId; - } - - @Override - public int compareTo(PositionImpl that) { - if (this.ledgerId != that.ledgerId) { - return (this.ledgerId < that.ledgerId ? -1 : 1); - } - - if (this.entryId != that.entryId) { - return (this.entryId < that.entryId ? -1 : 1); - } - - return 0; - } - - public int compareTo(long ledgerId, long entryId) { - if (this.ledgerId != ledgerId) { - return (this.ledgerId < ledgerId ? -1 : 1); - } - - if (this.entryId != entryId) { - return (this.entryId < entryId ? -1 : 1); - } - - return 0; - } - - @Override - public int hashCode() { - int result = (int) (ledgerId ^ (ledgerId >>> 32)); - result = 31 * result + (int) (entryId ^ (entryId >>> 32)); - return result; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof PositionImpl) { - PositionImpl other = (PositionImpl) obj; - return ledgerId == other.ledgerId && entryId == other.entryId; - } - return false; - } - - public boolean hasAckSet() { - return ackSet != null; - } - - public PositionInfo getPositionInfo() { - return PositionInfo.newBuilder().setLedgerId(ledgerId).setEntryId(entryId).build(); - } -} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionRecyclable.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionRecyclable.java new file mode 100644 index 0000000000000..142abf903c2f3 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionRecyclable.java @@ -0,0 +1,77 @@ +/* + * 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.bookkeeper.mledger.impl; + +import io.netty.util.Recycler; +import io.netty.util.Recycler.Handle; +import org.apache.bookkeeper.mledger.Position; + +/** + * Recyclable implementation of Position that is used to reduce the overhead of creating new Position objects. + */ +public class PositionRecyclable implements Position { + private final Handle recyclerHandle; + + private static final Recycler RECYCLER = new Recycler() { + @Override + protected PositionRecyclable newObject(Recycler.Handle recyclerHandle) { + return new PositionRecyclable(recyclerHandle); + } + }; + + private long ledgerId; + private long entryId; + + private PositionRecyclable(Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + @Override + public long getLedgerId() { + return ledgerId; + } + + @Override + public long getEntryId() { + return entryId; + } + + public void recycle() { + ledgerId = -1; + entryId = -1; + recyclerHandle.recycle(this); + } + + @Override + public int hashCode() { + return hashCodeForPosition(); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof Position && compareTo((Position) obj) == 0; + } + + public static PositionRecyclable get(long ledgerId, long entryId) { + PositionRecyclable position = RECYCLER.get(); + position.ledgerId = ledgerId; + position.entryId = entryId; + return position; + } +} \ No newline at end of file diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java index 2461bcf780e99..bd3e461d94e5c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java @@ -23,6 +23,7 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound; import org.apache.bookkeeper.mledger.proto.MLDataFormats; @@ -31,10 +32,10 @@ public class ReadOnlyCursorImpl extends ManagedCursorImpl implements ReadOnlyCursor { public ReadOnlyCursorImpl(BookKeeper bookkeeper, ManagedLedgerImpl ledger, - PositionImpl startPosition, String cursorName) { + Position startPosition, String cursorName) { super(bookkeeper, ledger, cursorName); - if (startPosition.equals(PositionImpl.EARLIEST)) { + if (startPosition.equals(PositionFactory.EARLIEST)) { readPosition = ledger.getFirstPosition().getNext(); } else { readPosition = startPosition; @@ -67,7 +68,7 @@ public MLDataFormats.ManagedLedgerInfo.LedgerInfo getCurrentLedgerInfo() { } @Override - public long getNumberOfEntries(Range range) { + public long getNumberOfEntries(Range range) { return this.ledger.getNumberOfEntries(range); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java index d844963599995..e64941c3201cb 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java @@ -30,6 +30,8 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerNotFoundException; import org.apache.bookkeeper.mledger.ManagedLedgerException.MetaStoreException; import org.apache.bookkeeper.mledger.ManagedLedgerException.MetadataNotFoundException; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; import org.apache.bookkeeper.mledger.proto.MLDataFormats; @@ -126,20 +128,21 @@ public void operationFailed(MetaStoreException e) { return future; } - ReadOnlyCursor createReadOnlyCursor(PositionImpl startPosition) { + ReadOnlyCursor createReadOnlyCursor(Position startPosition) { if (ledgers.isEmpty()) { - lastConfirmedEntry = PositionImpl.EARLIEST; + lastConfirmedEntry = PositionFactory.EARLIEST; } else if (ledgers.lastEntry().getValue().getEntries() > 0) { // Last ledger has some of the entries - lastConfirmedEntry = new PositionImpl(ledgers.lastKey(), ledgers.lastEntry().getValue().getEntries() - 1); + lastConfirmedEntry = + PositionFactory.create(ledgers.lastKey(), ledgers.lastEntry().getValue().getEntries() - 1); } else { // Last ledger is empty. If there is a previous ledger, position on the last entry of that ledger if (ledgers.size() > 1) { long lastLedgerId = ledgers.lastKey(); LedgerInfo li = ledgers.headMap(lastLedgerId, false).lastEntry().getValue(); - lastConfirmedEntry = new PositionImpl(li.getLedgerId(), li.getEntries() - 1); + lastConfirmedEntry = PositionFactory.create(li.getLedgerId(), li.getEntries() - 1); } else { - lastConfirmedEntry = PositionImpl.EARLIEST; + lastConfirmedEntry = PositionFactory.EARLIEST; } } @@ -147,7 +150,7 @@ ReadOnlyCursor createReadOnlyCursor(PositionImpl startPosition) { } @Override - public void asyncReadEntry(PositionImpl position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { + public void asyncReadEntry(Position position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { this.getLedgerHandle(position.getLedgerId()) .thenAccept((ledger) -> asyncReadEntry(ledger, position, callback, ctx)) .exceptionally((ex) -> { @@ -160,7 +163,7 @@ public void asyncReadEntry(PositionImpl position, AsyncCallbacks.ReadEntryCallba @Override public long getNumberOfEntries() { - return getNumberOfEntries(Range.openClosed(PositionImpl.EARLIEST, getLastPosition())); + return getNumberOfEntries(Range.openClosed(PositionFactory.EARLIEST, getLastPosition())); } @Override diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java index ec5b006c4745b..546733f909e21 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java @@ -37,6 +37,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.pulsar.metadata.api.Stat; @@ -98,7 +99,9 @@ public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) } if (mlInfo.hasTerminatedPosition()) { - lastConfirmedEntry = new PositionImpl(mlInfo.getTerminatedPosition()); + MLDataFormats.NestedPositionInfo terminatedPosition = mlInfo.getTerminatedPosition(); + lastConfirmedEntry = + PositionFactory.create(terminatedPosition.getLedgerId(), terminatedPosition.getEntryId()); log.info("[{}][{}] Recovering managed ledger terminated at {}", name, sourceMLName, lastConfirmedEntry); } @@ -205,13 +208,13 @@ private void initLastConfirmedEntry() { if (currentLedger == null) { return; } - lastConfirmedEntry = new PositionImpl(currentLedger.getId(), currentLedger.getLastAddConfirmed()); + lastConfirmedEntry = PositionFactory.create(currentLedger.getId(), currentLedger.getLastAddConfirmed()); // bypass empty ledgers, find last ledger with Message if possible. while (lastConfirmedEntry.getEntryId() == -1) { Map.Entry formerLedger = ledgers.lowerEntry(lastConfirmedEntry.getLedgerId()); if (formerLedger != null) { LedgerInfo ledgerInfo = formerLedger.getValue(); - lastConfirmedEntry = PositionImpl.get(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1); + lastConfirmedEntry = PositionFactory.create(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1); } else { break; } @@ -281,7 +284,9 @@ private synchronized void processSourceManagedLedgerInfo(MLDataFormats.ManagedLe sourceLedgersStat = stat; if (mlInfo.hasTerminatedPosition()) { - lastConfirmedEntry = new PositionImpl(mlInfo.getTerminatedPosition()); + MLDataFormats.NestedPositionInfo terminatedPosition = mlInfo.getTerminatedPosition(); + lastConfirmedEntry = + PositionFactory.create(terminatedPosition.getLedgerId(), terminatedPosition.getEntryId()); log.info("[{}][{}] Process managed ledger terminated at {}", name, sourceMLName, lastConfirmedEntry); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCache.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCache.java index a67756ddeeae9..c2c5cd6bff43e 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCache.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCache.java @@ -21,8 +21,8 @@ import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback; +import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.EntryImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.tuple.Pair; /** @@ -54,7 +54,7 @@ public interface EntryCache extends Comparable { * @param lastPosition * the position of the last entry to be invalidated (non-inclusive) */ - void invalidateEntries(PositionImpl lastPosition); + void invalidateEntries(Position lastPosition); void invalidateEntriesBeforeTimestamp(long timestamp); @@ -115,7 +115,7 @@ void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boolean shou * @param ctx * the context object */ - void asyncReadEntry(ReadHandle lh, PositionImpl position, ReadEntryCallback callback, Object ctx); + void asyncReadEntry(ReadHandle lh, Position position, ReadEntryCallback callback, Object ctx); /** * Get the total size in bytes of all the entries stored in this cache. diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java index d1050e0062826..4f8f70bc81bab 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java @@ -27,9 +27,9 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; import org.apache.commons.lang3.tuple.Pair; @@ -56,7 +56,7 @@ public boolean insert(EntryImpl entry) { } @Override - public void invalidateEntries(PositionImpl lastPosition) { + public void invalidateEntries(Position lastPosition) { } @Override @@ -105,7 +105,7 @@ public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boole } @Override - public void asyncReadEntry(ReadHandle lh, PositionImpl position, AsyncCallbacks.ReadEntryCallback callback, + public void asyncReadEntry(ReadHandle lh, Position position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { lh.readAsync(position.getEntryId(), position.getEntryId()).whenCompleteAsync( (ledgerEntries, exception) -> { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java index 27aec6f178e39..254a517786a55 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java @@ -39,9 +39,10 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; import org.apache.bookkeeper.mledger.util.RangeCache; import org.apache.commons.lang3.tuple.Pair; @@ -61,7 +62,7 @@ public class RangeEntryCacheImpl implements EntryCache { private final RangeEntryCacheManagerImpl manager; final ManagedLedgerImpl ml; private ManagedLedgerInterceptor interceptor; - private final RangeCache entries; + private final RangeCache entries; private final boolean copyEntries; private final PendingReadsManager pendingReadsManager; @@ -130,7 +131,7 @@ public boolean insert(EntryImpl entry) { entry.getLength()); } - PositionImpl position = entry.getPosition(); + Position position = entry.getPosition(); if (entries.exists(position)) { return false; } @@ -182,8 +183,8 @@ private ByteBuf copyEntry(EntryImpl entry) { } @Override - public void invalidateEntries(final PositionImpl lastPosition) { - final PositionImpl firstPosition = PositionImpl.get(-1, 0); + public void invalidateEntries(final Position lastPosition) { + final Position firstPosition = PositionFactory.create(-1, 0); if (firstPosition.compareTo(lastPosition) > 0) { if (log.isDebugEnabled()) { @@ -206,8 +207,8 @@ public void invalidateEntries(final PositionImpl lastPosition) { @Override public void invalidateAllEntries(long ledgerId) { - final PositionImpl firstPosition = PositionImpl.get(ledgerId, 0); - final PositionImpl lastPosition = PositionImpl.get(ledgerId + 1, 0); + final Position firstPosition = PositionFactory.create(ledgerId, 0); + final Position lastPosition = PositionFactory.create(ledgerId + 1, 0); Pair removed = entries.removeRange(firstPosition, lastPosition, false); int entriesRemoved = removed.getLeft(); @@ -222,7 +223,7 @@ public void invalidateAllEntries(long ledgerId) { } @Override - public void asyncReadEntry(ReadHandle lh, PositionImpl position, final ReadEntryCallback callback, + public void asyncReadEntry(ReadHandle lh, Position position, final ReadEntryCallback callback, final Object ctx) { try { asyncReadEntry0(lh, position, callback, ctx); @@ -236,7 +237,7 @@ public void asyncReadEntry(ReadHandle lh, PositionImpl position, final ReadEntry } } - private void asyncReadEntry0(ReadHandle lh, PositionImpl position, final ReadEntryCallback callback, + private void asyncReadEntry0(ReadHandle lh, Position position, final ReadEntryCallback callback, final Object ctx) { if (log.isDebugEnabled()) { log.debug("[{}] Reading entry ledger {}: {}", ml.getName(), lh.getId(), position.getEntryId()); @@ -310,8 +311,8 @@ void asyncReadEntry0WithLimits(ReadHandle lh, long firstEntry, long lastEntry, b final long ledgerId = lh.getId(); final int entriesToRead = (int) (lastEntry - firstEntry) + 1; - final PositionImpl firstPosition = PositionImpl.get(lh.getId(), firstEntry); - final PositionImpl lastPosition = PositionImpl.get(lh.getId(), lastEntry); + final Position firstPosition = PositionFactory.create(lh.getId(), firstEntry); + final Position lastPosition = PositionFactory.create(lh.getId(), lastEntry); if (log.isDebugEnabled()) { log.debug("[{}] Reading entries range ledger {}: {} to {}", ml.getName(), ledgerId, firstEntry, lastEntry); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtils.java index 01de115290ab9..d13d71d0e5a13 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtils.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtils.java @@ -25,7 +25,6 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.common.classification.InterfaceStability; @InterfaceStability.Evolving @@ -36,7 +35,7 @@ public class ManagedLedgerImplUtils { */ public static CompletableFuture asyncGetLastValidPosition(final ManagedLedgerImpl ledger, final Predicate predicate, - final PositionImpl startPosition) { + final Position startPosition) { CompletableFuture future = new CompletableFuture<>(); internalAsyncReverseFindPositionOneByOne(ledger, predicate, startPosition, future); return future; @@ -44,7 +43,7 @@ public static CompletableFuture asyncGetLastValidPosition(final Manage private static void internalAsyncReverseFindPositionOneByOne(final ManagedLedgerImpl ledger, final Predicate predicate, - final PositionImpl position, + final Position position, final CompletableFuture future) { if (!ledger.isValidPosition(position)) { future.complete(position); @@ -59,7 +58,7 @@ public void readEntryComplete(Entry entry, Object ctx) { future.complete(position); return; } - PositionImpl previousPosition = ledger.getPreviousPosition((PositionImpl) position); + Position previousPosition = ledger.getPreviousPosition(position); internalAsyncReverseFindPositionOneByOne(ledger, predicate, previousPosition, future); } catch (Exception e) { future.completeExceptionally(e); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/PositionAckSetUtil.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/PositionAckSetUtil.java index 1c607582076a8..a7442215264e4 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/PositionAckSetUtil.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/PositionAckSetUtil.java @@ -18,7 +18,9 @@ */ package org.apache.bookkeeper.mledger.util; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.AckSetState; +import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.pulsar.common.util.collections.BitSetRecyclable; public class PositionAckSetUtil { @@ -41,11 +43,13 @@ public static boolean isAckSetOverlap(long[] currentAckSet, long[] otherAckSet) } //This method is do `and` operation for position's ack set - public static void andAckSet(PositionImpl currentPosition, PositionImpl otherPosition) { + public static void andAckSet(Position currentPosition, Position otherPosition) { if (currentPosition == null || otherPosition == null) { return; } - currentPosition.setAckSet(andAckSet(currentPosition.getAckSet(), otherPosition.getAckSet())); + AckSetState currentAckSetState = AckSetStateUtil.getAckSetState(currentPosition); + AckSetState otherAckSetState = AckSetStateUtil.getAckSetState(otherPosition); + currentAckSetState.setAckSet(andAckSet(currentAckSetState.getAckSet(), otherAckSetState.getAckSet())); } //This method is do `and` operation for ack set @@ -69,7 +73,7 @@ public static boolean isAckSetEmpty(long[] ackSet) { //This method is compare two position which position is bigger than another one. //When the ledgerId and entryId in this position is same to another one and two position all have ack set, it will //compare the ack set next bit index is bigger than another one. - public static int compareToWithAckSet(PositionImpl currentPosition, PositionImpl otherPosition) { + public static int compareToWithAckSet(Position currentPosition, Position otherPosition) { if (currentPosition == null || otherPosition == null) { throw new IllegalArgumentException("Two positions can't be null! " + "current position : [" + currentPosition + "] other position : [" + otherPosition + "]"); @@ -79,16 +83,18 @@ public static int compareToWithAckSet(PositionImpl currentPosition, PositionImpl BitSetRecyclable otherAckSet; BitSetRecyclable currentAckSet; - if (otherPosition.getAckSet() == null) { + long[] otherAckSetArr = AckSetStateUtil.getAckSetArrayOrNull(otherPosition); + if (otherAckSetArr == null) { otherAckSet = BitSetRecyclable.create(); } else { - otherAckSet = BitSetRecyclable.valueOf(otherPosition.getAckSet()); + otherAckSet = BitSetRecyclable.valueOf(otherAckSetArr); } - if (currentPosition.getAckSet() == null) { + long[] currentAckSetArr = AckSetStateUtil.getAckSetArrayOrNull(currentPosition); + if (currentAckSetArr == null) { currentAckSet = BitSetRecyclable.create(); } else { - currentAckSet = BitSetRecyclable.valueOf(currentPosition.getAckSet()); + currentAckSet = BitSetRecyclable.valueOf(currentAckSetArr); } if (currentAckSet.isEmpty() || otherAckSet.isEmpty()) { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java index 1ab3198498ac3..ece75a2de80d8 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java @@ -40,6 +40,8 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.cache.EntryCache; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheDisabled; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; @@ -120,7 +122,7 @@ public void simple() throws Exception { assertEquals(cache2.getSize(), 3); // Should remove 1 entry - cache2.invalidateEntries(new PositionImpl(2, 1)); + cache2.invalidateEntries(PositionFactory.create(2, 1)); assertEquals(cacheManager.getSize(), 2); assertEquals(cache2.getSize(), 2); @@ -330,7 +332,7 @@ public void verifyHitsMisses() throws Exception { assertEquals(factory2.getMbean().getCacheHitsThroughput(), 70.0); assertEquals(factory2.getMbean().getNumberOfCacheEvictions(), 0); - PositionImpl pos = (PositionImpl) entries.get(entries.size() - 1).getPosition(); + Position pos = entries.get(entries.size() - 1).getPosition(); c2.setReadPosition(pos); entries.forEach(Entry::release); @@ -390,7 +392,7 @@ void entryCacheDisabledAsyncReadEntry() throws Exception { EntryCache entryCache = cacheManager.getEntryCache(ml1); final CountDownLatch counter = new CountDownLatch(1); - entryCache.asyncReadEntry(lh, new PositionImpl(1L,1L), new AsyncCallbacks.ReadEntryCallback() { + entryCache.asyncReadEntry(lh, PositionFactory.create(1L,1L), new AsyncCallbacks.ReadEntryCallback() { public void readEntryComplete(Entry entry, Object ctx) { Assert.assertNotEquals(entry, null); entry.release(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorConcurrencyTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorConcurrencyTest.java index 7558f07db76ca..ebcbe31d5e784 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorConcurrencyTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorConcurrencyTest.java @@ -51,7 +51,7 @@ public class ManagedCursorConcurrencyTest extends MockedBookKeeperTestCase { private static final Logger log = LoggerFactory.getLogger(ManagedCursorConcurrencyTest.class); - + @DataProvider(name = "useOpenRangeSet") public static Object[][] useOpenRangeSet() { return new Object[][] { { Boolean.TRUE }, { Boolean.FALSE } }; @@ -325,7 +325,7 @@ public void testConcurrentReadOfSameEntry() throws Exception { for (int i = 0; i < N; i++) { ledger.addEntry(("entry" + i).getBytes()); } - long currentLedger = ((PositionImpl) cursors.get(0).getMarkDeletedPosition()).getLedgerId(); + long currentLedger = cursors.get(0).getMarkDeletedPosition().getLedgerId(); // empty the cache ((ManagedLedgerImpl) ledger).entryCache.invalidateAllEntries(currentLedger); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java index f0b3efe39d6b7..a387331f3c047 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java @@ -48,6 +48,7 @@ import org.apache.bookkeeper.mledger.ManagedCursorMXBean; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.testng.annotations.Test; public class ManagedCursorContainerTest { @@ -111,13 +112,13 @@ public List readEntries(int numberOfEntriesToRead) { @Override public void asyncReadEntries(int numberOfEntriesToRead, ReadEntriesCallback callback, Object ctx, - PositionImpl maxPosition) { + Position maxPosition) { callback.readEntriesComplete(null, ctx); } @Override public void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition) { + Object ctx, Position maxPosition) { callback.readEntriesComplete(null, ctx); } @@ -303,12 +304,12 @@ public List readEntriesOrWait(int numberOfEntriesToRead) { @Override public void asyncReadEntriesOrWait(int numberOfEntriesToRead, ReadEntriesCallback callback, Object ctx, - PositionImpl maxPosition) { + Position maxPosition) { } @Override public void asyncReadEntriesOrWait(int maxEntries, long maxSizeBytes, ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition) { + Object ctx, Position maxPosition) { } @@ -375,7 +376,7 @@ public ManagedLedger getManagedLedger() { } @Override - public Range getLastIndividualDeletedRange() { + public Range getLastIndividualDeletedRange() { return null; } @@ -385,7 +386,7 @@ public void trimDeletedEntries(List entries) { } @Override - public long[] getDeletedBatchIndexesAsLongArray(PositionImpl position) { + public long[] getDeletedBatchIndexesAsLongArray(Position position) { return new long[0]; } @@ -416,36 +417,36 @@ public void testSlowestReadPositionForActiveCursors() { assertNull(container.getSlowestReaderPosition()); // Add no durable cursor - PositionImpl position = PositionImpl.get(5,5); + Position position = PositionFactory.create(5,5); ManagedCursor cursor1 = spy(new MockManagedCursor(container, "test1", position)); doReturn(false).when(cursor1).isDurable(); doReturn(position).when(cursor1).getReadPosition(); container.add(cursor1, position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 5)); // Add no durable cursor - position = PositionImpl.get(1,1); + position = PositionFactory.create(1,1); ManagedCursor cursor2 = spy(new MockManagedCursor(container, "test2", position)); doReturn(false).when(cursor2).isDurable(); doReturn(position).when(cursor2).getReadPosition(); container.add(cursor2, position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(1, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(1, 1)); // Move forward cursor, cursor1 = 5:5, cursor2 = 5:6, slowest is 5:5 - position = PositionImpl.get(5,6); + position = PositionFactory.create(5,6); container.cursorUpdated(cursor2, position); doReturn(position).when(cursor2).getReadPosition(); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 5)); // Move forward cursor, cursor1 = 5:8, cursor2 = 5:6, slowest is 5:6 - position = PositionImpl.get(5,8); + position = PositionFactory.create(5,8); doReturn(position).when(cursor1).getReadPosition(); container.cursorUpdated(cursor1, position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 6)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); // Remove cursor, only cursor1 left, cursor1 = 5:8 container.removeCursor(cursor2.getName()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 8)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 8)); } @Test @@ -453,51 +454,51 @@ public void simple() throws Exception { ManagedCursorContainer container = new ManagedCursorContainer(); assertNull(container.getSlowestReaderPosition()); - ManagedCursor cursor1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); + ManagedCursor cursor1 = new MockManagedCursor(container, "test1", PositionFactory.create(5, 5)); container.add(cursor1, cursor1.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 5)); assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), - cursor1, new PositionImpl(5, 5)); + cursor1, PositionFactory.create(5, 5)); - ManagedCursor cursor2 = new MockManagedCursor(container, "test2", new PositionImpl(2, 2)); + ManagedCursor cursor2 = new MockManagedCursor(container, "test2", PositionFactory.create(2, 2)); container.add(cursor2, cursor2.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 2)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(2, 2)); assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), - cursor2, new PositionImpl(2, 2)); + cursor2, PositionFactory.create(2, 2)); - ManagedCursor cursor3 = new MockManagedCursor(container, "test3", new PositionImpl(2, 0)); + ManagedCursor cursor3 = new MockManagedCursor(container, "test3", PositionFactory.create(2, 0)); container.add(cursor3, cursor3.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 0)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(2, 0)); assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), - cursor3, new PositionImpl(2, 0)); + cursor3, PositionFactory.create(2, 0)); assertEquals(container.toString(), "[test1=5:5, test2=2:2, test3=2:0]"); - ManagedCursor cursor4 = new MockManagedCursor(container, "test4", new PositionImpl(4, 0)); + ManagedCursor cursor4 = new MockManagedCursor(container, "test4", PositionFactory.create(4, 0)); container.add(cursor4, cursor4.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 0)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(2, 0)); - ManagedCursor cursor5 = new MockManagedCursor(container, "test5", new PositionImpl(3, 5)); + ManagedCursor cursor5 = new MockManagedCursor(container, "test5", PositionFactory.create(3, 5)); container.add(cursor5, cursor5.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 0)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(2, 0)); - cursor3.markDelete(new PositionImpl(3, 0)); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 2)); + cursor3.markDelete(PositionFactory.create(3, 0)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(2, 2)); assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), - cursor2, new PositionImpl(2, 2)); + cursor2, PositionFactory.create(2, 2)); - cursor2.markDelete(new PositionImpl(10, 5)); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(3, 0)); + cursor2.markDelete(PositionFactory.create(10, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(3, 0)); container.removeCursor(cursor3.getName()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(3, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(3, 5)); container.removeCursor(cursor2.getName()); container.removeCursor(cursor5.getName()); container.removeCursor(cursor1.getName()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(4, 0)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(4, 0)); assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), - cursor4, new PositionImpl(4, 0)); + cursor4, PositionFactory.create(4, 0)); assertTrue(container.hasDurableCursors()); @@ -506,9 +507,9 @@ public void simple() throws Exception { assertFalse(container.hasDurableCursors()); - ManagedCursor cursor6 = new MockManagedCursor(container, "test6", new PositionImpl(6, 5)); + ManagedCursor cursor6 = new MockManagedCursor(container, "test6", PositionFactory.create(6, 5)); container.add(cursor6, cursor6.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(6, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(6, 5)); assertEquals(container.toString(), "[test6=6:5]"); } @@ -517,29 +518,29 @@ public void simple() throws Exception { public void updatingCursorOutsideContainer() { ManagedCursorContainer container = new ManagedCursorContainer(); - ManagedCursor cursor1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); + ManagedCursor cursor1 = new MockManagedCursor(container, "test1", PositionFactory.create(5, 5)); container.add(cursor1, cursor1.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 5)); - MockManagedCursor cursor2 = new MockManagedCursor(container, "test2", new PositionImpl(2, 2)); + MockManagedCursor cursor2 = new MockManagedCursor(container, "test2", PositionFactory.create(2, 2)); container.add(cursor2, cursor2.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 2)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(2, 2)); - cursor2.position = new PositionImpl(8, 8); + cursor2.position = PositionFactory.create(8, 8); // Until we don't update the container, the ordering will not change - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 2)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(2, 2)); container.cursorUpdated(cursor2, cursor2.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 5)); assertEqualsCursorAndPosition(container.getCursorWithOldestPosition(), - cursor1, new PositionImpl(5, 5)); + cursor1, PositionFactory.create(5, 5)); } private void assertEqualsCursorAndPosition(ManagedCursorContainer.CursorInfo cursorInfo, ManagedCursor expectedCursor, - PositionImpl expectedPosition) { + Position expectedPosition) { assertThat(cursorInfo.getCursor().getName()).isEqualTo(expectedCursor.getName()); assertThat(cursorInfo.getPosition()).isEqualTo(expectedPosition); } @@ -548,19 +549,19 @@ private void assertEqualsCursorAndPosition(ManagedCursorContainer.CursorInfo cur public void removingCursor() { ManagedCursorContainer container = new ManagedCursorContainer(); - ManagedCursor cursor1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); + ManagedCursor cursor1 = new MockManagedCursor(container, "test1", PositionFactory.create(5, 5)); container.add(cursor1, cursor1.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 5)); assertEquals(container.get("test1"), cursor1); - MockManagedCursor cursor2 = new MockManagedCursor(container, "test2", new PositionImpl(2, 2)); + MockManagedCursor cursor2 = new MockManagedCursor(container, "test2", PositionFactory.create(2, 2)); container.add(cursor2, cursor2.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(2, 2)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(2, 2)); assertEquals(container.get("test2"), cursor2); - MockManagedCursor cursor3 = new MockManagedCursor(container, "test3", new PositionImpl(1, 1)); + MockManagedCursor cursor3 = new MockManagedCursor(container, "test3", PositionFactory.create(1, 1)); container.add(cursor3, cursor3.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(1, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(1, 1)); assertEquals(container.get("test3"), cursor3); assertEquals(container, Lists.newArrayList(cursor1, cursor2, cursor3)); @@ -572,24 +573,24 @@ public void removingCursor() { assertNull(container.get("test2")); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(1, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(1, 1)); container.removeCursor("test3"); assertEquals(container, Lists.newArrayList(cursor1)); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 5)); } @Test public void ordering() throws Exception { ManagedCursorContainer container = new ManagedCursorContainer(); - ManagedCursor cursor1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); - ManagedCursor cursor2 = new MockManagedCursor(container, "test2", new PositionImpl(5, 1)); - ManagedCursor cursor3 = new MockManagedCursor(container, "test3", new PositionImpl(7, 1)); - ManagedCursor cursor4 = new MockManagedCursor(container, "test4", new PositionImpl(6, 4)); - ManagedCursor cursor5 = new MockManagedCursor(container, "test5", new PositionImpl(7, 0)); + ManagedCursor cursor1 = new MockManagedCursor(container, "test1", PositionFactory.create(5, 5)); + ManagedCursor cursor2 = new MockManagedCursor(container, "test2", PositionFactory.create(5, 1)); + ManagedCursor cursor3 = new MockManagedCursor(container, "test3", PositionFactory.create(7, 1)); + ManagedCursor cursor4 = new MockManagedCursor(container, "test4", PositionFactory.create(6, 4)); + ManagedCursor cursor5 = new MockManagedCursor(container, "test5", PositionFactory.create(7, 0)); container.add(cursor1, cursor1.getMarkDeletedPosition()); container.add(cursor2, cursor2.getMarkDeletedPosition()); @@ -597,19 +598,19 @@ public void ordering() throws Exception { container.add(cursor4, cursor4.getMarkDeletedPosition()); container.add(cursor5, cursor5.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 1)); container.removeCursor("test2"); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 5)); container.removeCursor("test1"); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(6, 4)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(6, 4)); container.removeCursor("test4"); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(7, 0)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(7, 0)); container.removeCursor("test5"); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(7, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(7, 1)); container.removeCursor("test3"); assertFalse(container.hasDurableCursors()); @@ -619,11 +620,11 @@ public void ordering() throws Exception { public void orderingWithUpdates() { ManagedCursorContainer container = new ManagedCursorContainer(); - MockManagedCursor c1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); - MockManagedCursor c2 = new MockManagedCursor(container, "test2", new PositionImpl(5, 1)); - MockManagedCursor c3 = new MockManagedCursor(container, "test3", new PositionImpl(7, 1)); - MockManagedCursor c4 = new MockManagedCursor(container, "test4", new PositionImpl(6, 4)); - MockManagedCursor c5 = new MockManagedCursor(container, "test5", new PositionImpl(7, 0)); + MockManagedCursor c1 = new MockManagedCursor(container, "test1", PositionFactory.create(5, 5)); + MockManagedCursor c2 = new MockManagedCursor(container, "test2", PositionFactory.create(5, 1)); + MockManagedCursor c3 = new MockManagedCursor(container, "test3", PositionFactory.create(7, 1)); + MockManagedCursor c4 = new MockManagedCursor(container, "test4", PositionFactory.create(6, 4)); + MockManagedCursor c5 = new MockManagedCursor(container, "test5", PositionFactory.create(7, 0)); container.add(c1, c1.getMarkDeletedPosition()); container.add(c2, c2.getMarkDeletedPosition()); @@ -631,50 +632,50 @@ public void orderingWithUpdates() { container.add(c4, c4.getMarkDeletedPosition()); container.add(c5, c5.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 1)); - c1.position = new PositionImpl(5, 8); + c1.position = PositionFactory.create(5, 8); container.cursorUpdated(c1, c1.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 1)); - c2.position = new PositionImpl(5, 6); + c2.position = PositionFactory.create(5, 6); container.cursorUpdated(c2, c2.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 6)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); - c1.position = new PositionImpl(6, 8); + c1.position = PositionFactory.create(6, 8); container.cursorUpdated(c1, c1.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 6)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); - c3.position = new PositionImpl(8, 5); + c3.position = PositionFactory.create(8, 5); container.cursorUpdated(c3, c3.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 6)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); - c1.position = new PositionImpl(8, 4); + c1.position = PositionFactory.create(8, 4); container.cursorUpdated(c1, c1.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 6)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); - c2.position = new PositionImpl(8, 4); + c2.position = PositionFactory.create(8, 4); container.cursorUpdated(c2, c2.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(6, 4)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(6, 4)); - c4.position = new PositionImpl(7, 1); + c4.position = PositionFactory.create(7, 1); container.cursorUpdated(c4, c4.position); // //// - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(7, 0)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(7, 0)); container.removeCursor("test5"); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(7, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(7, 1)); container.removeCursor("test4"); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(8, 4)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(8, 4)); container.removeCursor("test1"); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(8, 4)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(8, 4)); container.removeCursor("test2"); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(8, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(8, 5)); container.removeCursor("test3"); assertFalse(container.hasDurableCursors()); @@ -684,11 +685,11 @@ public void orderingWithUpdates() { public void orderingWithUpdatesAndReset() { ManagedCursorContainer container = new ManagedCursorContainer(); - MockManagedCursor c1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); - MockManagedCursor c2 = new MockManagedCursor(container, "test2", new PositionImpl(5, 1)); - MockManagedCursor c3 = new MockManagedCursor(container, "test3", new PositionImpl(7, 1)); - MockManagedCursor c4 = new MockManagedCursor(container, "test4", new PositionImpl(6, 4)); - MockManagedCursor c5 = new MockManagedCursor(container, "test5", new PositionImpl(7, 0)); + MockManagedCursor c1 = new MockManagedCursor(container, "test1", PositionFactory.create(5, 5)); + MockManagedCursor c2 = new MockManagedCursor(container, "test2", PositionFactory.create(5, 1)); + MockManagedCursor c3 = new MockManagedCursor(container, "test3", PositionFactory.create(7, 1)); + MockManagedCursor c4 = new MockManagedCursor(container, "test4", PositionFactory.create(6, 4)); + MockManagedCursor c5 = new MockManagedCursor(container, "test5", PositionFactory.create(7, 0)); container.add(c1, c1.getMarkDeletedPosition()); container.add(c2, c2.getMarkDeletedPosition()); @@ -696,50 +697,50 @@ public void orderingWithUpdatesAndReset() { container.add(c4, c4.getMarkDeletedPosition()); container.add(c5, c5.getMarkDeletedPosition()); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 1)); - c1.position = new PositionImpl(5, 8); + c1.position = PositionFactory.create(5, 8); container.cursorUpdated(c1, c1.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 1)); - c1.position = new PositionImpl(5, 6); + c1.position = PositionFactory.create(5, 6); container.cursorUpdated(c1, c1.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 1)); - c2.position = new PositionImpl(6, 8); + c2.position = PositionFactory.create(6, 8); container.cursorUpdated(c2, c2.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 6)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); - c3.position = new PositionImpl(8, 5); + c3.position = PositionFactory.create(8, 5); container.cursorUpdated(c3, c3.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(5, 6)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(5, 6)); - c1.position = new PositionImpl(8, 4); + c1.position = PositionFactory.create(8, 4); container.cursorUpdated(c1, c1.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(6, 4)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(6, 4)); - c2.position = new PositionImpl(4, 4); + c2.position = PositionFactory.create(4, 4); container.cursorUpdated(c2, c2.position); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(4, 4)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(4, 4)); - c4.position = new PositionImpl(7, 1); + c4.position = PositionFactory.create(7, 1); container.cursorUpdated(c4, c4.position); // //// - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(4, 4)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(4, 4)); container.removeCursor("test2"); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(7, 0)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(7, 0)); container.removeCursor("test5"); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(7, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(7, 1)); container.removeCursor("test1"); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(7, 1)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(7, 1)); container.removeCursor("test4"); - assertEquals(container.getSlowestReaderPosition(), new PositionImpl(8, 5)); + assertEquals(container.getSlowestReaderPosition(), PositionFactory.create(8, 5)); container.removeCursor("test3"); assertFalse(container.hasDurableCursors()); @@ -773,8 +774,8 @@ public void testDataVersion() { public void testVersions() { ManagedCursorContainer container = new ManagedCursorContainer(); - MockManagedCursor c1 = new MockManagedCursor(container, "test1", new PositionImpl(5, 5)); - MockManagedCursor c2 = new MockManagedCursor(container, "test2", new PositionImpl(5, 1)); + MockManagedCursor c1 = new MockManagedCursor(container, "test1", PositionFactory.create(5, 5)); + MockManagedCursor c2 = new MockManagedCursor(container, "test2", PositionFactory.create(5, 1)); container.add(c1, c1.getMarkDeletedPosition()); long version = container.getCursorWithOldestPosition().getVersion(); @@ -785,7 +786,7 @@ public void testVersions() { assertThat(ManagedCursorContainer.DataVersion.compareVersions(newVersion, version)).isPositive(); version = newVersion; - container.cursorUpdated(c2, new PositionImpl(5, 8)); + container.cursorUpdated(c2, PositionFactory.create(5, 8)); newVersion = container.getCursorWithOldestPosition().getVersion(); // newVersion > version assertThat(ManagedCursorContainer.DataVersion.compareVersions(newVersion, version)).isPositive(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java index 864c25c6c434b..3d4de5b1f4975 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java @@ -24,15 +24,15 @@ import static org.testng.Assert.assertEquals; import com.google.common.collect.Range; - import java.lang.reflect.Method; import java.util.ArrayList; import java.util.List; import java.util.NavigableMap; import java.util.concurrent.ConcurrentSkipListMap; - import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange; import org.apache.bookkeeper.mledger.proto.MLDataFormats.NestedPositionInfo; @@ -59,7 +59,7 @@ void testRecoverIndividualDeletedMessages() throws Exception { doReturn(config).when(ledger).getConfig(); ManagedCursorImpl cursor = spy(new ManagedCursorImpl(bookkeeper, ledger, "test-cursor")); - LongPairRangeSet deletedMessages = cursor.getIndividuallyDeletedMessagesSet(); + LongPairRangeSet deletedMessages = cursor.getIndividuallyDeletedMessagesSet(); Method recoverMethod = ManagedCursorImpl.class.getDeclaredMethod("recoverIndividualDeletedMessages", List.class); @@ -68,7 +68,7 @@ void testRecoverIndividualDeletedMessages() throws Exception { // (1) [(1:5..1:10]] List messageRangeList = new ArrayList(); messageRangeList.add(createMessageRange(1, 5, 1, 10)); - List> expectedRangeList = new ArrayList(); + List> expectedRangeList = new ArrayList(); expectedRangeList.add(createPositionRange(1, 5, 1, 10)); recoverMethod.invoke(cursor, messageRangeList); assertEquals(deletedMessages.size(), 1); @@ -120,9 +120,9 @@ private static MessageRange createMessageRange(long lowerLedgerId, long lowerEnt return messageRangeBuilder.build(); } - private static Range createPositionRange(long lowerLedgerId, long lowerEntryId, long upperLedgerId, + private static Range createPositionRange(long lowerLedgerId, long lowerEntryId, long upperLedgerId, long upperEntryId) { - return Range.openClosed(new PositionImpl(lowerLedgerId, lowerEntryId), - new PositionImpl(upperLedgerId, upperEntryId)); + return Range.openClosed(PositionFactory.create(lowerLedgerId, lowerEntryId), + PositionFactory.create(upperLedgerId, upperEntryId)); } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 4c95454e33a92..8913c4013b4ab 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -90,6 +90,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ScanOutcome; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.VoidCallback; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; @@ -141,11 +142,11 @@ public void testCloseCursor() throws Exception { ledger.addEntry(new byte[]{4}); ledger.addEntry(new byte[]{5}); // Persistent cursor info to ledger. - c1.delete(PositionImpl.get(c1.getReadPosition().getLedgerId(), c1.getReadPosition().getEntryId())); + c1.delete(PositionFactory.create(c1.getReadPosition().getLedgerId(), c1.getReadPosition().getEntryId())); Awaitility.await().until(() ->c1.getStats().getPersistLedgerSucceed() > 0); // Make cursor ledger can not work. closeCursorLedger(c1); - c1.delete(PositionImpl.get(c1.getReadPosition().getLedgerId(), c1.getReadPosition().getEntryId() + 2)); + c1.delete(PositionFactory.create(c1.getReadPosition().getLedgerId(), c1.getReadPosition().getEntryId() + 2)); ledger.close(); } @@ -253,7 +254,7 @@ void testPersistentMarkDeleteIfCreateCursorLedgerFailed() throws Exception { cursor.markDelete(lastEntry); // Assert persist mark deleted position to ZK was successful. - PositionImpl slowestReadPosition = ml.getCursors().getSlowestReaderPosition(); + Position slowestReadPosition = ml.getCursors().getSlowestReaderPosition(); assertTrue(slowestReadPosition.getLedgerId() >= lastEntry.getLedgerId()); assertTrue(slowestReadPosition.getEntryId() >= lastEntry.getEntryId()); assertEquals(cursor.getStats().getPersistLedgerSucceed(), 0); @@ -351,7 +352,7 @@ void testPersistentMarkDeleteIfSwitchCursorLedgerFailed() throws Exception { assertTrue(persistZookeeperSucceed2 > persistZookeeperSucceed1); // Assert persist mark deleted position to ZK was successful. - PositionImpl slowestReadPosition = ml.getCursors().getSlowestReaderPosition(); + Position slowestReadPosition = ml.getCursors().getSlowestReaderPosition(); assertTrue(slowestReadPosition.getLedgerId() >= lastEntry.getLedgerId()); assertTrue(slowestReadPosition.getEntryId() >= lastEntry.getEntryId()); assertEquals(cursor.getPersistentMarkDeletedPosition(), lastEntry); @@ -593,7 +594,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { fail(exception.getMessage()); } - }, null, PositionImpl.LATEST); + }, null, PositionFactory.LATEST); counter.await(); } @@ -621,7 +622,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { fail("async-call should not have failed"); } - }, null, PositionImpl.LATEST); + }, null, PositionFactory.LATEST); counter.await(); @@ -643,7 +644,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { counter2.countDown(); } - }, null, PositionImpl.LATEST); + }, null, PositionFactory.LATEST); counter2.await(); } @@ -670,7 +671,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { counter.countDown(); } - }, null, PositionImpl.LATEST); + }, null, PositionFactory.LATEST); counter.await(); } @@ -793,9 +794,9 @@ void testResetCursor() throws Exception { ledger.addEntry("dummy-entry-1".getBytes(Encoding)); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - PositionImpl lastPosition = (PositionImpl) ledger.addEntry("dummy-entry-4".getBytes(Encoding)); + Position lastPosition = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); final AtomicBoolean moveStatus = new AtomicBoolean(false); - PositionImpl resetPosition = new PositionImpl(lastPosition.getLedgerId(), lastPosition.getEntryId() - 2); + Position resetPosition = PositionFactory.create(lastPosition.getLedgerId(), lastPosition.getEntryId() - 2); try { cursor.resetCursor(resetPosition); moveStatus.set(true); @@ -814,21 +815,21 @@ void testResetCursor1() throws Exception { ManagedLedger ledger = factory.open("my_test_move_cursor_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(2)); ManagedCursor cursor = ledger.openCursor("trc1"); - PositionImpl actualEarliest = (PositionImpl) ledger.addEntry("dummy-entry-1".getBytes(Encoding)); + Position actualEarliest = ledger.addEntry("dummy-entry-1".getBytes(Encoding)); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - PositionImpl lastInPrev = (PositionImpl) ledger.addEntry("dummy-entry-4".getBytes(Encoding)); - PositionImpl firstInNext = (PositionImpl) ledger.addEntry("dummy-entry-5".getBytes(Encoding)); + Position lastInPrev = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); + Position firstInNext = ledger.addEntry("dummy-entry-5".getBytes(Encoding)); ledger.addEntry("dummy-entry-6".getBytes(Encoding)); ledger.addEntry("dummy-entry-7".getBytes(Encoding)); ledger.addEntry("dummy-entry-8".getBytes(Encoding)); ledger.addEntry("dummy-entry-9".getBytes(Encoding)); - PositionImpl last = (PositionImpl) ledger.addEntry("dummy-entry-10".getBytes(Encoding)); + Position last = ledger.addEntry("dummy-entry-10".getBytes(Encoding)); final AtomicBoolean moveStatus = new AtomicBoolean(false); // reset to earliest - PositionImpl earliest = PositionImpl.EARLIEST; + Position earliest = PositionFactory.EARLIEST; try { cursor.resetCursor(earliest); moveStatus.set(true); @@ -836,12 +837,12 @@ void testResetCursor1() throws Exception { log.warn("error in reset cursor", e.getCause()); } assertTrue(moveStatus.get()); - PositionImpl earliestPos = new PositionImpl(actualEarliest.getLedgerId(), -1); + Position earliestPos = PositionFactory.create(actualEarliest.getLedgerId(), -1); assertEquals(cursor.getReadPosition(), earliestPos); moveStatus.set(false); // reset to one after last entry in a ledger should point to the first entry in the next ledger - PositionImpl resetPosition = new PositionImpl(lastInPrev.getLedgerId(), lastInPrev.getEntryId() + 1); + Position resetPosition = PositionFactory.create(lastInPrev.getLedgerId(), lastInPrev.getEntryId() + 1); try { cursor.resetCursor(resetPosition); moveStatus.set(true); @@ -853,7 +854,7 @@ void testResetCursor1() throws Exception { moveStatus.set(false); // reset to a non exist larger ledger should point to the first non-exist entry in the next ledger - PositionImpl latest = new PositionImpl(last.getLedgerId() + 2, 0); + Position latest = PositionFactory.create(last.getLedgerId() + 2, 0); try { cursor.resetCursor(latest); moveStatus.set(true); @@ -861,14 +862,14 @@ void testResetCursor1() throws Exception { log.warn("error in reset cursor", e.getCause()); } assertTrue(moveStatus.get()); - PositionImpl lastPos = new PositionImpl(last.getLedgerId() + 1, 0); + Position lastPos = PositionFactory.create(last.getLedgerId() + 1, 0); Awaitility.await().untilAsserted(() -> { assertEquals(lastPos, cursor.getReadPosition()); }); moveStatus.set(false); // reset to latest should point to the first non-exist entry in the next ledger - PositionImpl anotherLast = PositionImpl.LATEST; + Position anotherLast = PositionFactory.LATEST; try { cursor.resetCursor(anotherLast); moveStatus.set(true); @@ -890,10 +891,10 @@ void testasyncResetCursor() throws Exception { ledger.addEntry("dummy-entry-1".getBytes(Encoding)); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - PositionImpl lastPosition = (PositionImpl) ledger.addEntry("dummy-entry-4".getBytes(Encoding)); + Position lastPosition = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); final AtomicBoolean moveStatus = new AtomicBoolean(false); CountDownLatch countDownLatch = new CountDownLatch(1); - PositionImpl resetPosition = new PositionImpl(lastPosition.getLedgerId(), lastPosition.getEntryId() - 2); + Position resetPosition = PositionFactory.create(lastPosition.getLedgerId(), lastPosition.getEntryId() - 2); cursor.asyncResetCursor(resetPosition, false, new AsyncCallbacks.ResetCursorCallback() { @Override @@ -930,7 +931,7 @@ void testConcurrentResetCursor() throws Exception { for (int i = 0; i < Messages; i++) { ledger.addEntry("test".getBytes()); } - final PositionImpl lastPosition = (PositionImpl) ledger.addEntry("dummy-entry-4".getBytes(Encoding)); + final Position lastPosition = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); for (int i = 0; i < Consumers; i++) { final ManagedCursor cursor = ledger.openCursor("tcrc" + i); @@ -943,14 +944,14 @@ public AtomicBoolean call() throws Exception { final AtomicBoolean moveStatus = new AtomicBoolean(false); CountDownLatch countDownLatch = new CountDownLatch(1); - final PositionImpl resetPosition = new PositionImpl(lastPosition.getLedgerId(), + final Position resetPosition = PositionFactory.create(lastPosition.getLedgerId(), lastPosition.getEntryId() - (5 * idx)); cursor.asyncResetCursor(resetPosition, false, new AsyncCallbacks.ResetCursorCallback() { @Override public void resetComplete(Object ctx) { moveStatus.set(true); - PositionImpl pos = (PositionImpl) ctx; + Position pos = (Position) ctx; log.info("move to [{}] completed for consumer [{}]", pos.toString(), idx); countDownLatch.countDown(); } @@ -958,7 +959,7 @@ public void resetComplete(Object ctx) { @Override public void resetFailed(ManagedLedgerException exception, Object ctx) { moveStatus.set(false); - PositionImpl pos = (PositionImpl) ctx; + Position pos = (Position) ctx; log.warn("move to [{}] failed for consumer [{}]", pos.toString(), idx); countDownLatch.countDown(); } @@ -985,9 +986,9 @@ void testLastActiveAfterResetCursor() throws Exception { ManagedLedger ledger = factory.open("test_cursor_ledger"); ManagedCursor cursor = ledger.openCursor("tla"); - PositionImpl lastPosition = null; + Position lastPosition = null; for (int i = 0; i < 3; i++) { - lastPosition = (PositionImpl) ledger.addEntry("dummy-entry".getBytes(Encoding)); + lastPosition = ledger.addEntry("dummy-entry".getBytes(Encoding)); } final AtomicBoolean moveStatus = new AtomicBoolean(false); @@ -1028,9 +1029,9 @@ void seekPosition() throws Exception { ledger.addEntry("dummy-entry-1".getBytes(Encoding)); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - PositionImpl lastPosition = (PositionImpl) ledger.addEntry("dummy-entry-4".getBytes(Encoding)); + Position lastPosition = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); - cursor.seek(new PositionImpl(lastPosition.getLedgerId(), lastPosition.getEntryId() - 1)); + cursor.seek(PositionFactory.create(lastPosition.getLedgerId(), lastPosition.getEntryId() - 1)); } @Test(timeOut = 20000) @@ -1039,12 +1040,12 @@ void seekPosition2() throws Exception { ManagedCursor cursor = ledger.openCursor("c1"); ledger.addEntry("dummy-entry-1".getBytes(Encoding)); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); - PositionImpl seekPosition = (PositionImpl) ledger.addEntry("dummy-entry-3".getBytes(Encoding)); + Position seekPosition = ledger.addEntry("dummy-entry-3".getBytes(Encoding)); ledger.addEntry("dummy-entry-4".getBytes(Encoding)); ledger.addEntry("dummy-entry-5".getBytes(Encoding)); ledger.addEntry("dummy-entry-6".getBytes(Encoding)); - cursor.seek(new PositionImpl(seekPosition.getLedgerId(), seekPosition.getEntryId())); + cursor.seek(PositionFactory.create(seekPosition.getLedgerId(), seekPosition.getEntryId())); } @Test(timeOut = 20000) @@ -1054,11 +1055,11 @@ void seekPosition3() throws Exception { ledger.addEntry("dummy-entry-1".getBytes(Encoding)); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - PositionImpl seekPosition = (PositionImpl) ledger.addEntry("dummy-entry-4".getBytes(Encoding)); + Position seekPosition = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); Position entry5 = ledger.addEntry("dummy-entry-5".getBytes(Encoding)); Position entry6 = ledger.addEntry("dummy-entry-6".getBytes(Encoding)); - cursor.seek(new PositionImpl(seekPosition.getLedgerId(), seekPosition.getEntryId())); + cursor.seek(PositionFactory.create(seekPosition.getLedgerId(), seekPosition.getEntryId())); assertEquals(cursor.getReadPosition(), seekPosition); List entries = cursor.readEntries(1); @@ -1155,7 +1156,7 @@ void markDeleteSkippingMessage() throws Exception { Position p1 = ledger.addEntry("dummy-entry-1".getBytes(Encoding)); Position p2 = ledger.addEntry("dummy-entry-2".getBytes(Encoding)); ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - PositionImpl p4 = (PositionImpl) ledger.addEntry("dummy-entry-4".getBytes(Encoding)); + Position p4 = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); assertEquals(cursor.getNumberOfEntries(), 4); @@ -1174,7 +1175,7 @@ void markDeleteSkippingMessage() throws Exception { assertFalse(cursor.hasMoreEntries()); assertEquals(cursor.getNumberOfEntries(), 0); - assertEquals(cursor.getReadPosition(), new PositionImpl(p4.getLedgerId(), p4.getEntryId() + 1)); + assertEquals(cursor.getReadPosition(), PositionFactory.create(p4.getLedgerId(), p4.getEntryId() + 1)); } @Test(timeOut = 20000) @@ -1772,7 +1773,7 @@ void testSkipEntries(boolean useOpenRangeSet) throws Exception { // skip entries until end of ledger c1.skipEntries(1, IndividualDeletedEntries.Exclude); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getReadPosition(), new PositionImpl(ledger.currentLedger.getId(), 0)); + assertEquals(c1.getReadPosition(), PositionFactory.create(ledger.currentLedger.getId(), 0)); assertEquals(c1.getMarkDeletedPosition(), pos); // skip entries across ledgers @@ -1812,7 +1813,7 @@ void testSkipEntriesWithIndividualDeletedMessages(boolean useOpenRangeSet) throw c1.skipEntries(3, IndividualDeletedEntries.Exclude); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getReadPosition(), new PositionImpl(pos5.getLedgerId() + 1, 0)); + assertEquals(c1.getReadPosition(), PositionFactory.create(pos5.getLedgerId() + 1, 0)); assertEquals(c1.getMarkDeletedPosition(), pos5); pos1 = ledger.addEntry("dummy-entry-1".getBytes(Encoding)); @@ -2002,7 +2003,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { log.error("Error reading", exception); } - }, null, PositionImpl.LATEST); + }, null, PositionFactory.LATEST); } ledger.addEntry("test".getBytes()); @@ -2695,7 +2696,7 @@ void internalTestFindNewestMatchingAllEntries(final String name, final int entri Thread.sleep(100); Position newPosition = ledger.addEntry(getEntryPublishTime("expectedresetposition")); long timestamp = System.currentTimeMillis(); - long ledgerId = ((PositionImpl) newPosition).getLedgerId(); + long ledgerId = newPosition.getLedgerId(); Thread.sleep(2); ledger.addEntry(getEntryPublishTime("not-read")); @@ -2710,11 +2711,11 @@ void internalTestFindNewestMatchingAllEntries(final String name, final int entri ledger = factory.open(ledgerAndCursorName, config); c1 = (ManagedCursorImpl) ledger.openCursor(ledgerAndCursorName); - PositionImpl found = (PositionImpl) findPositionFromAllEntries(c1, timestamp); + Position found = findPositionFromAllEntries(c1, timestamp); assertEquals(found.getLedgerId(), ledgerId); assertEquals(found.getEntryId(), expectedEntryId); - found = (PositionImpl) findPositionFromAllEntries(c1, 0); + found = findPositionFromAllEntries(c1, 0); assertNull(found); } @@ -2753,13 +2754,13 @@ void testReplayEntries() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger"); ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); - PositionImpl p1 = (PositionImpl) ledger.addEntry("entry1".getBytes(Encoding)); - PositionImpl p2 = (PositionImpl) ledger.addEntry("entry2".getBytes(Encoding)); - PositionImpl p3 = (PositionImpl) ledger.addEntry("entry3".getBytes(Encoding)); + Position p1 = ledger.addEntry("entry1".getBytes(Encoding)); + Position p2 = ledger.addEntry("entry2".getBytes(Encoding)); + Position p3 = ledger.addEntry("entry3".getBytes(Encoding)); ledger.addEntry("entry4".getBytes(Encoding)); // 1. Replay empty position set should return empty entry set - Set positions = new HashSet(); + Set positions = new HashSet(); assertTrue(c1.replayEntries(positions).isEmpty()); positions.add(p1); @@ -2775,7 +2776,7 @@ void testReplayEntries() throws Exception { entries.forEach(Entry::release); // 3. Fail on reading non-existing position - PositionImpl invalidPosition = new PositionImpl(100, 100); + Position invalidPosition = PositionFactory.create(100, 100); positions.add(invalidPosition); try { @@ -2802,24 +2803,24 @@ void testGetLastIndividualDeletedRange() throws Exception { ManagedLedger ledger = factory.open("test_last_individual_deleted"); ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); - PositionImpl markDeletedPosition = (PositionImpl) c1.getMarkDeletedPosition(); + Position markDeletedPosition = c1.getMarkDeletedPosition(); for(int i = 0; i < 10; i++) { ledger.addEntry(("entry" + i).getBytes(Encoding)); } - PositionImpl p1 = PositionImpl.get(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 1); - PositionImpl p2 = PositionImpl.get(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 2); - PositionImpl p3 = PositionImpl.get(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 5); - PositionImpl p4 = PositionImpl.get(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 6); + Position p1 = PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 1); + Position p2 = PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 2); + Position p3 = PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 5); + Position p4 = PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 6); c1.delete(Lists.newArrayList(p1, p2, p3, p4)); - assertEquals(c1.getLastIndividualDeletedRange(), Range.openClosed(PositionImpl.get(p3.getLedgerId(), + assertEquals(c1.getLastIndividualDeletedRange(), Range.openClosed(PositionFactory.create(p3.getLedgerId(), p3.getEntryId() - 1), p4)); - PositionImpl p5 = PositionImpl.get(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 8); + Position p5 = PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 8); c1.delete(p5); - assertEquals(c1.getLastIndividualDeletedRange(), Range.openClosed(PositionImpl.get(p5.getLedgerId(), + assertEquals(c1.getLastIndividualDeletedRange(), Range.openClosed(PositionFactory.create(p5.getLedgerId(), p5.getEntryId() - 1), p5)); } @@ -2829,14 +2830,14 @@ void testTrimDeletedEntries() throws ManagedLedgerException, InterruptedExceptio ManagedLedger ledger = factory.open("my_test_ledger"); ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); - PositionImpl markDeletedPosition = (PositionImpl) c1.getMarkDeletedPosition(); + Position markDeletedPosition = c1.getMarkDeletedPosition(); for(int i = 0; i < 10; i++) { ledger.addEntry(("entry" + i).getBytes(Encoding)); } - PositionImpl p1 = PositionImpl.get(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 1); - PositionImpl p2 = PositionImpl.get(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 2); - PositionImpl p3 = PositionImpl.get(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 5); - PositionImpl p4 = PositionImpl.get(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 6); + Position p1 = PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 1); + Position p2 = PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 2); + Position p3 = PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 5); + Position p4 = PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 6); c1.delete(Lists.newArrayList(p1, p2, p3, p4)); @@ -2849,7 +2850,7 @@ void testTrimDeletedEntries() throws ManagedLedgerException, InterruptedExceptio List entries = Lists.newArrayList(entry1, entry2, entry3, entry4, entry5); c1.trimDeletedEntries(entries); assertEquals(entries.size(), 1); - assertEquals(entries.get(0).getPosition(), PositionImpl.get(markDeletedPosition.getLedgerId(), + assertEquals(entries.get(0).getPosition(), PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId() + 7)); assertEquals(entry1.refCnt(), 0); @@ -2925,7 +2926,7 @@ void testGetEntryAfterN() throws Exception { List entries = c1.readEntries(4); entries.forEach(Entry::release); - long currentLedger = ((PositionImpl) c1.getMarkDeletedPosition()).getLedgerId(); + long currentLedger = (c1.getMarkDeletedPosition()).getLedgerId(); // check if the first message is returned for '0' Entry e = c1.getNthEntry(1, IndividualDeletedEntries.Exclude); @@ -2948,8 +2949,8 @@ void testGetEntryAfterN() throws Exception { assertNull(e); // check that the mark delete and read positions have not been updated after all the previous operations - assertEquals(c1.getMarkDeletedPosition(), new PositionImpl(currentLedger, -1)); - assertEquals(c1.getReadPosition(), new PositionImpl(currentLedger, 4)); + assertEquals(c1.getMarkDeletedPosition(), PositionFactory.create(currentLedger, -1)); + assertEquals(c1.getReadPosition(), PositionFactory.create(currentLedger, 4)); c1.markDelete(pos4); assertEquals(c1.getMarkDeletedPosition(), pos4); @@ -3006,7 +3007,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { counter.countDown(); } - }, null, PositionImpl.LATEST); + }, null, PositionFactory.LATEST); assertTrue(c1.cancelPendingReadRequest()); @@ -3022,7 +3023,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { counter2.countDown(); } - }, null, PositionImpl.LATEST); + }, null, PositionFactory.LATEST); ledger.addEntry("entry-1".getBytes(Encoding)); @@ -3371,7 +3372,7 @@ public void testInvalidMarkDelete() throws Exception { // validate: cursor.asyncMarkDelete(..) CountDownLatch markDeleteCallbackLatch = new CountDownLatch(1); - Position position = PositionImpl.get(100, 100); + Position position = PositionFactory.create(100, 100); AtomicBoolean markDeleteCallFailed = new AtomicBoolean(false); cursor.asyncMarkDelete(position, new MarkDeleteCallback() { @Override @@ -3436,8 +3437,8 @@ public void testEstimatedUnackedSize() throws Exception { @Test(timeOut = 20000) public void testRecoverCursorAheadOfLastPosition() throws Exception { final String mlName = "my_test_ledger"; - final PositionImpl lastPosition = new PositionImpl(1L, 10L); - final PositionImpl nextPosition = new PositionImpl(3L, -1L); + final Position lastPosition = PositionFactory.create(1L, 10L); + final Position nextPosition = PositionFactory.create(3L, -1L); final String cursorName = "my_test_cursor"; final long cursorsLedgerId = -1L; @@ -3495,7 +3496,7 @@ public void testRecoverCursorAfterResetToLatestForNewEntry() throws Exception { assertEquals(c.getReadPosition().getEntryId(), 0); assertEquals(ml.getLastConfirmedEntry().getEntryId(), -1); - c.resetCursor(PositionImpl.LATEST); + c.resetCursor(PositionFactory.LATEST); // A reset cursor starts out with these values. The rest of the test assumes this, so we assert it here. assertEquals(c.getMarkDeletedPosition().getEntryId(), -1); @@ -3549,7 +3550,7 @@ public void testRecoverCursorAfterResetToLatestForMultipleEntries() throws Excep assertEquals(c.getReadPosition().getEntryId(), 0); assertEquals(ml.getLastConfirmedEntry().getEntryId(), -1); - c.resetCursor(PositionImpl.LATEST); + c.resetCursor(PositionFactory.LATEST); // A reset cursor starts out with these values. The rest of the test assumes this, so we assert it here. assertEquals(c.getMarkDeletedPosition().getEntryId(), -1); @@ -3562,7 +3563,7 @@ public void testRecoverCursorAfterResetToLatestForMultipleEntries() throws Excep ml.addEntry(new byte[1]); ml.addEntry(new byte[1]); - c.resetCursor(PositionImpl.LATEST); + c.resetCursor(PositionFactory.LATEST); assertEquals(c.getMarkDeletedPosition().getEntryId(), 3); assertEquals(c.getReadPosition().getEntryId(), 4); @@ -3623,7 +3624,7 @@ void testAlwaysInactive() throws Exception { @Test void testNonDurableCursorActive() throws Exception { ManagedLedger ml = factory.open("testInactive"); - ManagedCursor cursor = ml.newNonDurableCursor(PositionImpl.LATEST, "c1"); + ManagedCursor cursor = ml.newNonDurableCursor(PositionFactory.LATEST, "c1"); assertTrue(cursor.isActive()); @@ -3683,19 +3684,19 @@ public void testBatchIndexMarkdelete() throws ManagedLedgerException, Interrupte } assertEquals(cursor.getNumberOfEntries(), totalEntries); markDeleteBatchIndex(cursor, positions[0], 10, 3); - List deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[0]), 10); + List deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[0]), 10); Assert.assertEquals(1, deletedIndexes.size()); Assert.assertEquals(0, deletedIndexes.get(0).getStart()); Assert.assertEquals(3, deletedIndexes.get(0).getEnd()); markDeleteBatchIndex(cursor, positions[0], 10, 4); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[0]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[0]), 10); Assert.assertEquals(1, deletedIndexes.size()); Assert.assertEquals(0, deletedIndexes.get(0).getStart()); Assert.assertEquals(4, deletedIndexes.get(0).getEnd()); markDeleteBatchIndex(cursor, positions[0], 10, 2); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[0]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[0]), 10); Assert.assertEquals(1, deletedIndexes.size()); Assert.assertEquals(0, deletedIndexes.get(0).getStart()); Assert.assertEquals(4, deletedIndexes.get(0).getEnd()); @@ -3714,19 +3715,19 @@ public void testBatchIndexDelete() throws ManagedLedgerException, InterruptedExc } assertEquals(cursor.getNumberOfEntries(), totalEntries); deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(new IntRange().setStart(2).setEnd(4))); - List deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[0]), 10); + List deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[0]), 10); Assert.assertEquals(1, deletedIndexes.size()); Assert.assertEquals(2, deletedIndexes.get(0).getStart()); Assert.assertEquals(4, deletedIndexes.get(0).getEnd()); deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(new IntRange().setStart(3).setEnd(8))); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[0]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[0]), 10); Assert.assertEquals(1, deletedIndexes.size()); Assert.assertEquals(2, deletedIndexes.get(0).getStart()); Assert.assertEquals(8, deletedIndexes.get(0).getEnd()); deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(new IntRange().setStart(0).setEnd(0))); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[0]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[0]), 10); Assert.assertEquals(2, deletedIndexes.size()); Assert.assertEquals(0, deletedIndexes.get(0).getStart()); Assert.assertEquals(0, deletedIndexes.get(0).getEnd()); @@ -3735,24 +3736,24 @@ public void testBatchIndexDelete() throws ManagedLedgerException, InterruptedExc deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(new IntRange().setStart(1).setEnd(1))); deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(new IntRange().setStart(9).setEnd(9))); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[0]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[0]), 10); Assert.assertNull(deletedIndexes); Assert.assertEquals(positions[0], cursor.getMarkDeletedPosition()); deleteBatchIndex(cursor, positions[1], 10, Lists.newArrayList(new IntRange().setStart(0).setEnd(5))); cursor.delete(positions[1]); deleteBatchIndex(cursor, positions[1], 10, Lists.newArrayList(new IntRange().setStart(6).setEnd(8))); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[1]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[1]), 10); Assert.assertNull(deletedIndexes); deleteBatchIndex(cursor, positions[2], 10, Lists.newArrayList(new IntRange().setStart(0).setEnd(5))); cursor.markDelete(positions[3]); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[2]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[2]), 10); Assert.assertNull(deletedIndexes); deleteBatchIndex(cursor, positions[3], 10, Lists.newArrayList(new IntRange().setStart(0).setEnd(5))); cursor.resetCursor(positions[0]); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[3]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[3]), 10); Assert.assertNull(deletedIndexes); } @@ -3789,17 +3790,17 @@ public void testBatchIndexesDeletionPersistAndRecover() throws ManagedLedgerExce ledger = factory.open("test_batch_indexes_deletion_persistent", managedLedgerConfig); cursor = ledger.openCursor("c1"); - List deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[5]), 10); + List deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[5]), 10); Assert.assertEquals(deletedIndexes.size(), 1); Assert.assertEquals(deletedIndexes.get(0).getStart(), 3); Assert.assertEquals(deletedIndexes.get(0).getEnd(), 6); Assert.assertEquals(cursor.getMarkDeletedPosition(), positions[4]); deleteBatchIndex(cursor, positions[5], 10, Lists.newArrayList(new IntRange().setStart(0).setEnd(9))); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[5]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[5]), 10); Assert.assertNull(deletedIndexes); Assert.assertEquals(cursor.getMarkDeletedPosition(), positions[5]); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[6]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray(positions[6]), 10); Assert.assertEquals(deletedIndexes.size(), 1); Assert.assertEquals(deletedIndexes.get(0).getStart(), 1); Assert.assertEquals(deletedIndexes.get(0).getEnd(), 3); @@ -3808,13 +3809,12 @@ public void testBatchIndexesDeletionPersistAndRecover() throws ManagedLedgerExce private void deleteBatchIndex(ManagedCursor cursor, Position position, int batchSize, List deleteIndexes) throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); - PositionImpl pos = (PositionImpl) position; BitSet bitSet = new BitSet(batchSize); bitSet.set(0, batchSize); deleteIndexes.forEach(intRange -> { bitSet.clear(intRange.getStart(), intRange.getEnd() + 1); }); - pos.ackSet = bitSet.toLongArray(); + Position pos = AckSetStateUtil.createPositionWithAckSet(position.getLedgerId(), position.getEntryId(), bitSet.toLongArray()); cursor.asyncDelete(pos, new DeleteCallback() { @@ -3829,18 +3829,16 @@ public void deleteFailed(ManagedLedgerException exception, Object ctx) { } }, null); latch.await(); - pos.ackSet = null; } private void markDeleteBatchIndex(ManagedCursor cursor, Position position, int batchSize, int batchIndex ) throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); - PositionImpl pos = (PositionImpl) position; BitSetRecyclable bitSet = new BitSetRecyclable(); bitSet.set(0, batchSize); bitSet.clear(0, batchIndex + 1); - pos.ackSet = bitSet.toLongArray(); + Position pos = AckSetStateUtil.createPositionWithAckSet(position.getLedgerId(), position.getEntryId(), bitSet.toLongArray()); cursor.asyncMarkDelete(pos, new MarkDeleteCallback() { @Override @@ -3854,7 +3852,6 @@ public void markDeleteComplete(Object ctx) { } }, null); latch.await(); - pos.ackSet = null; } private List getAckedIndexRange(long[] bitSetLongArray, int batchSize) { @@ -3906,8 +3903,8 @@ public void testReadEntriesOrWaitWithMaxPosition() throws Exception { int sendNumber = 20; ManagedLedger ledger = factory.open("testReadEntriesOrWaitWithMaxPosition"); ManagedCursor c = ledger.openCursor("c"); - Position position = PositionImpl.EARLIEST; - Position maxCanReadPosition = PositionImpl.EARLIEST; + Position position = PositionFactory.EARLIEST; + Position maxCanReadPosition = PositionFactory.EARLIEST; for (int i = 0; i < sendNumber; i++) { if (i == readMaxNumber - 1) { position = ledger.addEntry(new byte[1024]); @@ -3929,7 +3926,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { completableFuture.completeExceptionally(exception); } - }, null, (PositionImpl) position); + }, null, position); int number = completableFuture.get(); assertEquals(number, readMaxNumber); @@ -3944,7 +3941,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { completableFuture.completeExceptionally(exception); } - }, null, (PositionImpl) maxCanReadPosition); + }, null, maxCanReadPosition); assertEquals(number, sendNumber - readMaxNumber); @@ -4117,11 +4114,11 @@ public void testConsistencyOfIndividualMessages() throws Exception { ManagedLedger ledger1 = factory.open("testConsistencyOfIndividualMessages"); ManagedCursorImpl c1 = (ManagedCursorImpl) ledger1.openCursor("c"); - PositionImpl p1 = (PositionImpl) ledger1.addEntry(new byte[1024]); + Position p1 = ledger1.addEntry(new byte[1024]); c1.markDelete(p1); // Artificially add a position that is before the current mark-delete position - LongPairRangeSet idm = c1.getIndividuallyDeletedMessagesSet(); + LongPairRangeSet idm = c1.getIndividuallyDeletedMessagesSet(); idm.addOpenClosed(p1.getLedgerId() - 1, 0, p1.getLedgerId() - 1, 10); List positions = new ArrayList<>(); @@ -4218,7 +4215,7 @@ public void testCursorGetBacklog() throws Exception { ((ConcurrentSkipListMap) field.get(ledger)).remove(position.getLedgerId()); field = ManagedCursorImpl.class.getDeclaredField("markDeletePosition"); field.setAccessible(true); - field.set(managedCursor, PositionImpl.get(position1.getLedgerId(), -1)); + field.set(managedCursor, PositionFactory.create(position1.getLedgerId(), -1)); Assert.assertEquals(managedCursor.getNumberOfEntriesInBacklog(true), 2); @@ -4281,7 +4278,7 @@ public void testReadEmptyEntryList() throws Exception { .open("testReadEmptyEntryList", managedLedgerConfig); ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor("test"); - PositionImpl lastPosition = (PositionImpl) ledger.addEntry("test".getBytes(Encoding)); + Position lastPosition = ledger.addEntry("test".getBytes(Encoding)); ledger.rollCurrentLedgerIfFull(); AtomicBoolean flag = new AtomicBoolean(); @@ -4302,10 +4299,10 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { // op readPosition is bigger than maxReadPosition OpReadEntry opReadEntry = OpReadEntry.create(cursor, ledger.lastConfirmedEntry, 10, callback, - null, PositionImpl.get(lastPosition.getLedgerId(), -1), null); + null, PositionFactory.create(lastPosition.getLedgerId(), -1), null); Field field = ManagedCursorImpl.class.getDeclaredField("readPosition"); field.setAccessible(true); - field.set(cursor, PositionImpl.EARLIEST); + field.set(cursor, PositionFactory.EARLIEST); ledger.asyncReadEntries(opReadEntry); // when readPosition is bigger than maxReadPosition, should complete the opReadEntry @@ -4347,7 +4344,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { final int numReadRequests = 3; for (int i = 0; i < numReadRequests; i++) { - cursor.asyncReadEntriesOrWait(1, callback, null, new PositionImpl(0, 0)); + cursor.asyncReadEntriesOrWait(1, callback, null, PositionFactory.create(0, 0)); } Awaitility.await().atMost(Duration.ofSeconds(1)) .untilAsserted(() -> assertEquals(ledger.waitingCursors.size(), 1)); @@ -4434,8 +4431,8 @@ public void testReadEntriesWithSkip() throws ManagedLedgerException, Interrupted int sendNumber = 20; ManagedLedger ledger = factory.open("testReadEntriesWithSkip"); ManagedCursor cursor = ledger.openCursor("c"); - Position position = PositionImpl.EARLIEST; - Position maxCanReadPosition = PositionImpl.EARLIEST; + Position position = PositionFactory.EARLIEST; + Position maxCanReadPosition = PositionFactory.EARLIEST; for (int i = 0; i < sendNumber; i++) { if (i == readMaxNumber - 1) { position = ledger.addEntry(new byte[1024]); @@ -4464,7 +4461,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { completableFuture.completeExceptionally(exception); } - }, null, (PositionImpl) position, pos -> { + }, null, position, pos -> { return pos.getEntryId() % 2 != 0; }); @@ -4491,7 +4488,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { completableFuture2.completeExceptionally(exception); } - }, null, (PositionImpl) maxCanReadPosition, pos -> { + }, null, maxCanReadPosition, pos -> { return pos.getEntryId() % 2 != 0; }); @@ -4500,7 +4497,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { assertEquals(cursor.getReadPosition().getEntryId(), 20L); - cursor.seek(PositionImpl.EARLIEST); + cursor.seek(PositionFactory.EARLIEST); CompletableFuture completableFuture3 = new CompletableFuture<>(); cursor.asyncReadEntriesWithSkipOrWait(sendNumber, new ReadEntriesCallback() { @Override @@ -4512,13 +4509,13 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { completableFuture3.completeExceptionally(exception); } - }, null, (PositionImpl) maxCanReadPosition, pos -> false); + }, null, maxCanReadPosition, pos -> false); int number3 = completableFuture3.get(); assertEquals(number3, sendNumber); assertEquals(cursor.getReadPosition().getEntryId(), 20L); - cursor.seek(PositionImpl.EARLIEST); + cursor.seek(PositionFactory.EARLIEST); CompletableFuture completableFuture4 = new CompletableFuture<>(); cursor.asyncReadEntriesWithSkipOrWait(sendNumber, new ReadEntriesCallback() { @Override @@ -4530,7 +4527,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { completableFuture4.completeExceptionally(exception); } - }, null, (PositionImpl) maxCanReadPosition, pos -> true); + }, null, maxCanReadPosition, pos -> true); int number4 = completableFuture4.get(); assertEquals(number4, 0); @@ -4594,7 +4591,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { f0.completeExceptionally(exception); } - }, null, PositionImpl.get(maxReadPosition.getLedgerId(), maxReadPosition.getEntryId()).getNext()); + }, null, PositionFactory.create(maxReadPosition.getLedgerId(), maxReadPosition.getEntryId()).getNext()); f0.get(); @@ -4610,7 +4607,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { f1.completeExceptionally(exception); } - }, null, PositionImpl.get(maxReadPosition.getLedgerId(), maxReadPosition.getEntryId()).getNext()); + }, null, PositionFactory.create(maxReadPosition.getLedgerId(), maxReadPosition.getEntryId()).getNext()); f1.get(); @@ -4626,7 +4623,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { f2.completeExceptionally(exception); } - }, null, PositionImpl.get(maxReadPosition.getLedgerId(), maxReadPosition.getEntryId()).getNext()); + }, null, PositionFactory.create(maxReadPosition.getLedgerId(), maxReadPosition.getEntryId()).getNext()); f2.get(); @@ -4673,8 +4670,8 @@ public void testReadEntriesWithSkipDeletedEntriesAndWithSkipConditions() throws map.put(i, maxReadPosition0); } - PositionImpl maxReadPosition = - PositionImpl.get(maxReadPosition0.getLedgerId(), maxReadPosition0.getEntryId()).getNext(); + Position maxReadPosition = + PositionFactory.create(maxReadPosition0.getLedgerId(), maxReadPosition0.getEntryId()).getNext(); Set deletedPositions = new HashSet<>(); deletedPositions.add(map.get(1)); @@ -4690,7 +4687,7 @@ public void testReadEntriesWithSkipDeletedEntriesAndWithSkipConditions() throws skippedPositions.add(map.get(15).getEntryId()); skippedPositions.add(map.get(16).getEntryId()); - Predicate skipCondition = position -> skippedPositions.contains(position.getEntryId()); + Predicate skipCondition = position -> skippedPositions.contains(position.getEntryId()); List readEntries = new ArrayList<>(); CompletableFuture f0 = new CompletableFuture<>(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java index bb505200ba75e..cd1dcf05c3708 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java @@ -358,7 +358,7 @@ public void ledgerFencedByAutoReplication() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger" + testName, config); ManagedCursor c1 = ledger.openCursor("c1"); - PositionImpl p1 = (PositionImpl) ledger.addEntry("entry-1".getBytes()); + Position p1 = ledger.addEntry("entry-1".getBytes()); // Trigger the closure of the data ledger bkc.openLedger(p1.getLedgerId(), BookKeeper.DigestType.CRC32C, new byte[] {}); @@ -368,7 +368,7 @@ public void ledgerFencedByAutoReplication() throws Exception { assertEquals(2, c1.getNumberOfEntries()); assertEquals(2, c1.getNumberOfEntriesInBacklog(false)); - PositionImpl p3 = (PositionImpl) ledger.addEntry("entry-3".getBytes()); + Position p3 = ledger.addEntry("entry-3".getBytes()); // Now entry-2 should have been written before entry-3 assertEquals(3, c1.getNumberOfEntries()); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryShutdownTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryShutdownTest.java index 1c9fb29066b3d..00fc151c6d792 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryShutdownTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryShutdownTest.java @@ -44,6 +44,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.pulsar.metadata.api.GetResult; @@ -167,7 +168,7 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { } }, null); - factory.asyncOpenReadOnlyCursor(ledgerName, PositionImpl.EARLIEST, new ManagedLedgerConfig(), + factory.asyncOpenReadOnlyCursor(ledgerName, PositionFactory.EARLIEST, new ManagedLedgerConfig(), new AsyncCallbacks.OpenReadOnlyCursorCallback() { @Override public void openReadOnlyCursorComplete(ReadOnlyCursor cursor, Object ctx) { @@ -194,6 +195,6 @@ public void openReadOnlyCursorFailed(ManagedLedgerException exception, Object ct Assert.assertThrows(ManagedLedgerException.ManagedLedgerFactoryClosedException.class, () -> factory.open(ledgerName)); Assert.assertThrows(ManagedLedgerException.ManagedLedgerFactoryClosedException.class, - () -> factory.openReadOnlyCursor(ledgerName, PositionImpl.EARLIEST, new ManagedLedgerConfig())); + () -> factory.openReadOnlyCursor(ledgerName, PositionFactory.EARLIEST, new ManagedLedgerConfig())); } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java index a953b140aba63..dfff9ecb49a3a 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java @@ -28,6 +28,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerInfo; import org.apache.bookkeeper.mledger.ManagedLedgerInfo.CursorInfo; import org.apache.bookkeeper.mledger.ManagedLedgerInfo.MessageRangeInfo; +import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.awaitility.Awaitility; import org.testng.Assert; @@ -42,9 +43,9 @@ public void testGetManagedLedgerInfoWithClose() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testGetManagedLedgerInfo", conf); ManagedCursor c1 = ledger.openCursor("c1"); - PositionImpl p1 = (PositionImpl) ledger.addEntry("entry1".getBytes()); - PositionImpl p2 = (PositionImpl) ledger.addEntry("entry2".getBytes()); - PositionImpl p3 = (PositionImpl) ledger.addEntry("entry3".getBytes()); + Position p1 = ledger.addEntry("entry1".getBytes()); + Position p2 = ledger.addEntry("entry2".getBytes()); + Position p3 = ledger.addEntry("entry3".getBytes()); ledger.addEntry("entry4".getBytes()); c1.delete(p2); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTerminationTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTerminationTest.java index 2150e80b29593..11feb5a41cf08 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTerminationTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTerminationTest.java @@ -31,6 +31,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerTerminatedException; import org.apache.bookkeeper.mledger.ManagedLedgerException.NoMoreEntriesToReadException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.testng.annotations.Test; @@ -141,7 +142,7 @@ public void terminateWithNonDurableCursor() throws Exception { assertTrue(ledger.isTerminated()); assertEquals(lastPosition, p1); - ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.EARLIEST); + ManagedCursor c1 = ledger.newNonDurableCursor(PositionFactory.EARLIEST); List entries = c1.readEntries(10); assertEquals(entries.size(), 2); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 4f521f1e99e91..e3b272babb7bb 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -123,6 +123,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryMXBean; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.VoidCallback; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; import org.apache.bookkeeper.mledger.impl.cache.EntryCache; @@ -599,7 +600,7 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { fail(exception.getMessage()); } - }, cursor, PositionImpl.LATEST); + }, cursor, PositionFactory.LATEST); } @Override @@ -645,8 +646,8 @@ public void spanningMultipleLedgers() throws Exception { assertEquals(entries.size(), 11); assertFalse(cursor.hasMoreEntries()); - PositionImpl first = (PositionImpl) entries.get(0).getPosition(); - PositionImpl last = (PositionImpl) entries.get(entries.size() - 1).getPosition(); + Position first = entries.get(0).getPosition(); + Position last = entries.get(entries.size() - 1).getPosition(); entries.forEach(Entry::release); log.info("First={} Last={}", first, last); @@ -670,8 +671,8 @@ public void testStartReadOperationOnLedgerWithEmptyLedgers() throws ManagedLedge LedgerInfo ledgerInfo = ledgers.firstEntry().getValue(); ledgers.clear(); ManagedCursor c1 = ledger.openCursor("c1"); - PositionImpl position = new PositionImpl(ledgerInfo.getLedgerId(), 0); - PositionImpl maxPosition = new PositionImpl(ledgerInfo.getLedgerId(), 99); + Position position = PositionFactory.create(ledgerInfo.getLedgerId(), 0); + Position maxPosition = PositionFactory.create(ledgerInfo.getLedgerId(), 99); OpReadEntry opReadEntry = OpReadEntry.create((ManagedCursorImpl) c1, position, 20, new ReadEntriesCallback() { @@ -712,8 +713,8 @@ public void spanningMultipleLedgersWithSize() throws Exception { assertEquals(entries.size(), 3); assertFalse(cursor.hasMoreEntries()); - PositionImpl first = (PositionImpl) entries.get(0).getPosition(); - PositionImpl last = (PositionImpl) entries.get(entries.size() - 1).getPosition(); + Position first = entries.get(0).getPosition(); + Position last = entries.get(entries.size() - 1).getPosition(); entries.forEach(Entry::release); // Read again, from next ledger id @@ -1332,7 +1333,7 @@ public void closeLedgerWithError() throws Exception { public void deleteWithErrors1() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger"); - PositionImpl position = (PositionImpl) ledger.addEntry("dummy-entry-1".getBytes(Encoding)); + Position position = ledger.addEntry("dummy-entry-1".getBytes(Encoding)); assertEquals(ledger.getNumberOfEntries(), 1); // Force delete a ledger and test that deleting the ML still happens @@ -1702,7 +1703,7 @@ public void previousPosition() throws Exception { Position p0 = cursor.getMarkDeletedPosition(); // This is expected because p0 is already an "invalid" position (since no entry has been mark-deleted yet) - assertEquals(ledger.getPreviousPosition((PositionImpl) p0), p0); + assertEquals(ledger.getPreviousPosition(p0), p0); // Force to close an empty ledger ledger.close(); @@ -1714,8 +1715,8 @@ public void previousPosition() throws Exception { ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(2)); - PositionImpl pBeforeWriting = ledger.getLastPosition(); - PositionImpl p1 = (PositionImpl) ledger.addEntry("entry".getBytes()); + Position pBeforeWriting = ledger.getLastPosition(); + Position p1 = ledger.addEntry("entry".getBytes()); ledger.close(); ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", @@ -1725,9 +1726,9 @@ public void previousPosition() throws Exception { Position p4 = ledger.addEntry("entry".getBytes()); assertEquals(ledger.getPreviousPosition(p1), pBeforeWriting); - assertEquals(ledger.getPreviousPosition((PositionImpl) p2), p1); - assertEquals(ledger.getPreviousPosition((PositionImpl) p3), p2); - assertEquals(ledger.getPreviousPosition((PositionImpl) p4), p3); + assertEquals(ledger.getPreviousPosition(p2), p1); + assertEquals(ledger.getPreviousPosition(p3), p2); + assertEquals(ledger.getPreviousPosition(p4), p3); } /** @@ -1785,10 +1786,10 @@ public void invalidateConsumedEntriesFromCache() throws Exception { ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); ManagedCursorImpl c2 = (ManagedCursorImpl) ledger.openCursor("c2"); - PositionImpl p1 = (PositionImpl) ledger.addEntry("entry-1".getBytes()); - PositionImpl p2 = (PositionImpl) ledger.addEntry("entry-2".getBytes()); - PositionImpl p3 = (PositionImpl) ledger.addEntry("entry-3".getBytes()); - PositionImpl p4 = (PositionImpl) ledger.addEntry("entry-4".getBytes()); + Position p1 = ledger.addEntry("entry-1".getBytes()); + Position p2 = ledger.addEntry("entry-2".getBytes()); + Position p3 = ledger.addEntry("entry-3".getBytes()); + Position p4 = ledger.addEntry("entry-4".getBytes()); assertEquals(entryCache.getSize(), 7 * 4); assertEquals(cacheManager.getSize(), entryCache.getSize()); @@ -1835,10 +1836,10 @@ public void invalidateEntriesFromCacheByMarkDeletePosition() throws Exception { ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); ManagedCursorImpl c2 = (ManagedCursorImpl) ledger.openCursor("c2"); - PositionImpl p1 = (PositionImpl) ledger.addEntry("entry-1".getBytes()); - PositionImpl p2 = (PositionImpl) ledger.addEntry("entry-2".getBytes()); - PositionImpl p3 = (PositionImpl) ledger.addEntry("entry-3".getBytes()); - PositionImpl p4 = (PositionImpl) ledger.addEntry("entry-4".getBytes()); + Position p1 = ledger.addEntry("entry-1".getBytes()); + Position p2 = ledger.addEntry("entry-2".getBytes()); + Position p3 = ledger.addEntry("entry-3".getBytes()); + Position p4 = ledger.addEntry("entry-4".getBytes()); assertEquals(entryCache.getSize(), 7 * 4); assertEquals(cacheManager.getSize(), entryCache.getSize()); @@ -2101,10 +2102,10 @@ public void totalSizeTest() throws Exception { assertEquals(ledger.getTotalSize(), 8); - PositionImpl p2 = (PositionImpl) ledger.addEntry(new byte[12], 2, 5); + Position p2 = ledger.addEntry(new byte[12], 2, 5); assertEquals(ledger.getTotalSize(), 13); - c1.markDelete(new PositionImpl(p2.getLedgerId(), -1)); + c1.markDelete(PositionFactory.create(p2.getLedgerId(), -1)); // Wait for background trimming Thread.sleep(400); @@ -2347,7 +2348,7 @@ public void testRetention0WithEmptyLedger() throws Exception { ml.deleteCursor(c1.getName()); ml.internalTrimConsumedLedgers(CompletableFuture.completedFuture(null)); - assertTrue(ml.getFirstPosition().ledgerId <= ml.lastConfirmedEntry.ledgerId); + assertTrue(ml.getFirstPosition().getLedgerId() <= ml.lastConfirmedEntry.getLedgerId()); ml.close(); } @@ -2373,8 +2374,8 @@ public void testRetention0WithEmptyLedgerWithoutCursors() throws Exception { ml = (ManagedLedgerImpl) factory.open("deletion_after_retention_test_ledger", config); ml.internalTrimConsumedLedgers(CompletableFuture.completedFuture(null)); - assertTrue(ml.getFirstPosition().ledgerId <= ml.lastConfirmedEntry.ledgerId); - assertFalse(ml.getLedgersInfo().containsKey(ml.lastConfirmedEntry.ledgerId), + assertTrue(ml.getFirstPosition().getLedgerId() <= ml.lastConfirmedEntry.getLedgerId()); + assertFalse(ml.getLedgersInfo().containsKey(ml.lastConfirmedEntry.getLedgerId()), "the ledger at lastConfirmedEntry has not been trimmed!"); ml.close(); } @@ -2592,9 +2593,9 @@ public void testGetPositionAfterN() throws Exception { long firstLedger = managedLedger.getLedgersInfo().firstKey(); long secondLedger = managedLedger.getLedgersInfoAsList().get(1).getLedgerId(); - PositionImpl startPosition = new PositionImpl(firstLedger, 0); + Position startPosition = PositionFactory.create(firstLedger, 0); - PositionImpl targetPosition = managedLedger.getPositionAfterN(startPosition, 1, ManagedLedgerImpl.PositionBound.startExcluded); + Position targetPosition = managedLedger.getPositionAfterN(startPosition, 1, ManagedLedgerImpl.PositionBound.startExcluded); assertEquals(targetPosition.getLedgerId(), firstLedger); assertEquals(targetPosition.getEntryId(), 1); @@ -2603,7 +2604,7 @@ public void testGetPositionAfterN() throws Exception { assertEquals(targetPosition.getEntryId(), 4); // test for expiry situation - PositionImpl searchPosition = managedLedger.getNextValidPosition((PositionImpl) managedCursor.getMarkDeletedPosition()); + Position searchPosition = managedLedger.getNextValidPosition(managedCursor.getMarkDeletedPosition()); long length = managedCursor.getNumberOfEntriesInStorage(); // return the last confirm entry position if searchPosition is exceed the last confirm entry targetPosition = managedLedger.getPositionAfterN(searchPosition, length, ManagedLedgerImpl.PositionBound.startExcluded); @@ -2612,18 +2613,18 @@ public void testGetPositionAfterN() throws Exception { assertEquals(targetPosition.getEntryId(), 4); // test for n > NumberOfEntriesInStorage - searchPosition = new PositionImpl(secondLedger, 0); + searchPosition = PositionFactory.create(secondLedger, 0); targetPosition = managedLedger.getPositionAfterN(searchPosition, 100, ManagedLedgerImpl.PositionBound.startIncluded); assertEquals(targetPosition.getLedgerId(), secondLedger); assertEquals(targetPosition.getEntryId(), 4); // test for startPosition > current ledger - searchPosition = new PositionImpl(999, 0); + searchPosition = PositionFactory.create(999, 0); targetPosition = managedLedger.getPositionAfterN(searchPosition, 0, ManagedLedgerImpl.PositionBound.startIncluded); assertEquals(targetPosition.getLedgerId(), secondLedger); assertEquals(targetPosition.getEntryId(), 4); - searchPosition = new PositionImpl(999, 0); + searchPosition = PositionFactory.create(999, 0); targetPosition = managedLedger.getPositionAfterN(searchPosition, 10, ManagedLedgerImpl.PositionBound.startExcluded); assertEquals(targetPosition.getLedgerId(), secondLedger); assertEquals(targetPosition.getEntryId(), 4); @@ -2694,22 +2695,22 @@ public void testGetNextValidPosition() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testGetNextValidPosition", conf); ManagedCursor c1 = ledger.openCursor("c1"); - PositionImpl p1 = (PositionImpl) ledger.addEntry("entry1".getBytes()); - PositionImpl p2 = (PositionImpl) ledger.addEntry("entry2".getBytes()); - PositionImpl p3 = (PositionImpl) ledger.addEntry("entry3".getBytes()); + Position p1 = ledger.addEntry("entry1".getBytes()); + Position p2 = ledger.addEntry("entry2".getBytes()); + Position p3 = ledger.addEntry("entry3".getBytes()); - assertEquals(ledger.getNextValidPosition((PositionImpl) c1.getMarkDeletedPosition()), p1); + assertEquals(ledger.getNextValidPosition(c1.getMarkDeletedPosition()), p1); assertEquals(ledger.getNextValidPosition(p1), p2); Awaitility.await().untilAsserted(() -> { - assertEquals(ledger.getNextValidPosition(p3), PositionImpl.get(p3.getLedgerId() + 1, 0)); + assertEquals(ledger.getNextValidPosition(p3), PositionFactory.create(p3.getLedgerId() + 1, 0)); }); Awaitility.await().untilAsserted(() -> { - assertEquals(ledger.getNextValidPosition(PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1)), - PositionImpl.get(p3.getLedgerId() + 1, 0)); + assertEquals(ledger.getNextValidPosition(PositionFactory.create(p3.getLedgerId(), p3.getEntryId() + 1)), + PositionFactory.create(p3.getLedgerId() + 1, 0)); }); Awaitility.await().untilAsserted(() -> { - assertEquals(ledger.getNextValidPosition(PositionImpl.get(p3.getLedgerId() + 1, p3.getEntryId() + 1)), - PositionImpl.get(p3.getLedgerId() + 1, 0)); + assertEquals(ledger.getNextValidPosition(PositionFactory.create(p3.getLedgerId() + 1, p3.getEntryId() + 1)), + PositionFactory.create(p3.getLedgerId() + 1, 0)); }); } @@ -3058,10 +3059,10 @@ public void testConsumerSubscriptionInitializePosition() throws Exception{ ManagedCursor earliestCursor = ledger.openCursor("c2", InitialPosition.Earliest); // Since getReadPosition returns the next position, we decrease the entryId by 1 - PositionImpl p2 = (PositionImpl) earliestCursor.getReadPosition(); + Position p2 = earliestCursor.getReadPosition(); - Pair latestPositionAndCounter = ledger.getLastPositionAndCounter(); - Pair earliestPositionAndCounter = ledger.getFirstPositionAndCounter(); + Pair latestPositionAndCounter = ledger.getLastPositionAndCounter(); + Pair earliestPositionAndCounter = ledger.getFirstPositionAndCounter(); // The read position is the valid next position of the last position instead of the next position. assertEquals(ledger.getNextValidPosition(latestPositionAndCounter.getLeft()), latestCursor.getReadPosition()); assertEquals(ledger.getNextValidPosition(earliestPositionAndCounter.getLeft()), p2); @@ -3135,11 +3136,11 @@ public void testManagedLedgerWithReadEntryTimeOut() throws Exception { String ctxStr = "timeoutCtx"; CompletableFuture entriesFuture = new CompletableFuture<>(); ReadHandle ledgerHandle = mock(ReadHandle.class); - doReturn(entriesFuture).when(ledgerHandle).readAsync(PositionImpl.EARLIEST.getLedgerId(), - PositionImpl.EARLIEST.getEntryId()); + doReturn(entriesFuture).when(ledgerHandle).readAsync(PositionFactory.EARLIEST.getLedgerId(), + PositionFactory.EARLIEST.getEntryId()); // (1) test read-timeout for: ManagedLedger.asyncReadEntry(..) - ledger.asyncReadEntry(ledgerHandle, PositionImpl.EARLIEST, new ReadEntryCallback() { + ledger.asyncReadEntry(ledgerHandle, PositionFactory.EARLIEST, new ReadEntryCallback() { @Override public void readEntryComplete(Entry entry, Object ctx) { responseException1.set(null); @@ -3159,7 +3160,7 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { // (2) test read-timeout for: ManagedLedger.asyncReadEntry(..) AtomicReference responseException2 = new AtomicReference<>(); - PositionImpl readPositionRef = PositionImpl.EARLIEST; + Position readPositionRef = PositionFactory.EARLIEST; ManagedCursorImpl cursor = new ManagedCursorImpl(bk, ledger, "cursor1"); OpReadEntry opReadEntry = OpReadEntry.create(cursor, readPositionRef, 1, new ReadEntriesCallback() { @@ -3173,8 +3174,8 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { responseException2.set(exception); } - }, null, PositionImpl.LATEST, null); - ledger.asyncReadEntry(ledgerHandle, PositionImpl.EARLIEST.getEntryId(), PositionImpl.EARLIEST.getEntryId(), + }, null, PositionFactory.LATEST, null); + ledger.asyncReadEntry(ledgerHandle, PositionFactory.EARLIEST.getEntryId(), PositionFactory.EARLIEST.getEntryId(), opReadEntry, ctxStr); retryStrategically((test) -> { return responseException2.get() != null; @@ -3680,7 +3681,7 @@ public void testAsyncTruncateLedgerSlowestCursor() throws Exception { ManagedLedgerImpl ledger2 = (ManagedLedgerImpl)factory.open("truncate_ledger", config); ledger2.addEntry("test-entry-2".getBytes(Encoding)); ManagedCursor cursor3 = ledger2.openCursor("test-cursor"); - cursor3.resetCursor(new PositionImpl(ledger2.getLastPosition())); + cursor3.resetCursor(PositionFactory.create(ledger2.getLastPosition())); CompletableFuture future = ledger2.asyncTruncate(); future.get(); @@ -3856,8 +3857,8 @@ public void testReadOtherManagedLedgersEntry() throws Exception { ManagedLedgerImpl managedLedgerA = (ManagedLedgerImpl) factory.open("my_test_ledger_a"); ManagedLedgerImpl managedLedgerB = (ManagedLedgerImpl) factory.open("my_test_ledger_b"); - PositionImpl pa = (PositionImpl) managedLedgerA.addEntry("dummy-entry-a".getBytes(Encoding)); - PositionImpl pb = (PositionImpl) managedLedgerB.addEntry("dummy-entry-b".getBytes(Encoding)); + Position pa = managedLedgerA.addEntry("dummy-entry-a".getBytes(Encoding)); + Position pb = managedLedgerB.addEntry("dummy-entry-b".getBytes(Encoding)); // read managedLegerA's entry using managedLedgerA CompletableFuture completableFutureA = new CompletableFuture<>(); @@ -4071,15 +4072,15 @@ public void testGetTheSlowestNonDurationReadPosition() throws Exception { positions.add(ledger.addEntry(("entry-" + i).getBytes(UTF_8))); } - Assert.assertEquals(ledger.getTheSlowestNonDurationReadPosition(), PositionImpl.LATEST); + Assert.assertEquals(ledger.getTheSlowestNonDurationReadPosition(), PositionFactory.LATEST); - ManagedCursor nonDurableCursor = ledger.newNonDurableCursor(PositionImpl.EARLIEST); + ManagedCursor nonDurableCursor = ledger.newNonDurableCursor(PositionFactory.EARLIEST); Assert.assertEquals(ledger.getTheSlowestNonDurationReadPosition(), positions.get(0)); ledger.deleteCursor(nonDurableCursor.getName()); - Assert.assertEquals(ledger.getTheSlowestNonDurationReadPosition(), PositionImpl.LATEST); + Assert.assertEquals(ledger.getTheSlowestNonDurationReadPosition(), PositionFactory.LATEST); ledger.close(); } @@ -4110,28 +4111,28 @@ public void testIsNoMessagesAfterPos() throws Exception { final ManagedCursor managedCursor = ml.openCursor(cursorName); // One ledger. - PositionImpl p1 = (PositionImpl) ml.addEntry(data); - PositionImpl p2 = (PositionImpl) ml.addEntry(data); - PositionImpl p3 = (PositionImpl) ml.addEntry(data); + Position p1 = ml.addEntry(data); + Position p2 = ml.addEntry(data); + Position p3 = ml.addEntry(data); assertFalse(ml.isNoMessagesAfterPos(p1)); assertFalse(ml.isNoMessagesAfterPos(p2)); assertTrue(ml.isNoMessagesAfterPos(p3)); - assertTrue(ml.isNoMessagesAfterPos(PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1))); - assertTrue(ml.isNoMessagesAfterPos(PositionImpl.get(p3.getLedgerId() + 1, -1))); + assertTrue(ml.isNoMessagesAfterPos(PositionFactory.create(p3.getLedgerId(), p3.getEntryId() + 1))); + assertTrue(ml.isNoMessagesAfterPos(PositionFactory.create(p3.getLedgerId() + 1, -1))); // More than one ledger. ml.ledgerClosed(ml.currentLedger); - PositionImpl p4 = (PositionImpl) ml.addEntry(data); - PositionImpl p5 = (PositionImpl) ml.addEntry(data); - PositionImpl p6 = (PositionImpl) ml.addEntry(data); + Position p4 = ml.addEntry(data); + Position p5 = ml.addEntry(data); + Position p6 = ml.addEntry(data); assertFalse(ml.isNoMessagesAfterPos(p1)); assertFalse(ml.isNoMessagesAfterPos(p2)); assertFalse(ml.isNoMessagesAfterPos(p3)); assertFalse(ml.isNoMessagesAfterPos(p4)); assertFalse(ml.isNoMessagesAfterPos(p5)); assertTrue(ml.isNoMessagesAfterPos(p6)); - assertTrue(ml.isNoMessagesAfterPos(PositionImpl.get(p6.getLedgerId(), p6.getEntryId() + 1))); - assertTrue(ml.isNoMessagesAfterPos(PositionImpl.get(p6.getLedgerId() + 1, -1))); + assertTrue(ml.isNoMessagesAfterPos(PositionFactory.create(p6.getLedgerId(), p6.getEntryId() + 1))); + assertTrue(ml.isNoMessagesAfterPos(PositionFactory.create(p6.getLedgerId() + 1, -1))); // Switch ledger and make the entry id of Last confirmed entry is -1; ml.ledgerClosed(ml.currentLedger); @@ -4139,11 +4140,11 @@ public void testIsNoMessagesAfterPos() throws Exception { Awaitility.await().untilAsserted(() -> { assertEquals(ml.currentLedgerEntries, 0); }); - ml.lastConfirmedEntry = PositionImpl.get(ml.currentLedger.getId(), -1); + ml.lastConfirmedEntry = PositionFactory.create(ml.currentLedger.getId(), -1); assertFalse(ml.isNoMessagesAfterPos(p5)); assertTrue(ml.isNoMessagesAfterPos(p6)); - assertTrue(ml.isNoMessagesAfterPos(PositionImpl.get(p6.getLedgerId(), p6.getEntryId() + 1))); - assertTrue(ml.isNoMessagesAfterPos(PositionImpl.get(p6.getLedgerId() + 1, -1))); + assertTrue(ml.isNoMessagesAfterPos(PositionFactory.create(p6.getLedgerId(), p6.getEntryId() + 1))); + assertTrue(ml.isNoMessagesAfterPos(PositionFactory.create(p6.getLedgerId() + 1, -1))); // Trim ledgers to make there is no entries in ML. ml.deleteCursor(cursorName); @@ -4158,8 +4159,8 @@ public void testIsNoMessagesAfterPos() throws Exception { assertTrue(ml.isNoMessagesAfterPos(p4)); assertTrue(ml.isNoMessagesAfterPos(p5)); assertTrue(ml.isNoMessagesAfterPos(p6)); - assertTrue(ml.isNoMessagesAfterPos(PositionImpl.get(p6.getLedgerId(), p6.getEntryId() + 1))); - assertTrue(ml.isNoMessagesAfterPos(PositionImpl.get(p6.getLedgerId() + 1, -1))); + assertTrue(ml.isNoMessagesAfterPos(PositionFactory.create(p6.getLedgerId(), p6.getEntryId() + 1))); + assertTrue(ml.isNoMessagesAfterPos(PositionFactory.create(p6.getLedgerId() + 1, -1))); // cleanup. ml.close(); @@ -4177,9 +4178,9 @@ public void testGetEstimatedBacklogSize() throws Exception { positions.add(ledger.addEntry(new byte[1])); } - Assert.assertEquals(ledger.getEstimatedBacklogSize(new PositionImpl(-1, -1)), 10); - Assert.assertEquals(ledger.getEstimatedBacklogSize(((PositionImpl) positions.get(1))), 8); - Assert.assertEquals(ledger.getEstimatedBacklogSize(((PositionImpl) positions.get(9)).getNext()), 0); + Assert.assertEquals(ledger.getEstimatedBacklogSize(PositionFactory.create(-1, -1)), 10); + Assert.assertEquals(ledger.getEstimatedBacklogSize((positions.get(1))), 8); + Assert.assertEquals(ledger.getEstimatedBacklogSize((positions.get(9)).getNext()), 0); ledger.close(); } @@ -4228,7 +4229,7 @@ public void testNonDurableCursorCreateForInactiveLedger() throws Exception { }, 5, 1000); assertTrue(isRolledOver.booleanValue()); - Position Position = new PositionImpl(-1L, -1L); + Position Position = PositionFactory.create(-1L, -1L); assertNotNull(ml.newNonDurableCursor(Position)); } @@ -4299,11 +4300,11 @@ public void testNoCleanupOffloadLedgerWhenMetadataExceptionHappens() throws Exce metadataPutCallCount.incrementAndGet() == 2); // prepare the arguments for the offloadLoop method - CompletableFuture future = new CompletableFuture<>(); + CompletableFuture future = new CompletableFuture<>(); Queue ledgersToOffload = new LinkedList<>(); LedgerInfo ledgerInfo = LedgerInfo.getDefaultInstance().toBuilder().setLedgerId(1).setEntries(10).build(); ledgersToOffload.add(ledgerInfo); - PositionImpl firstUnoffloaded = new PositionImpl(1, 0); + Position firstUnoffloaded = PositionFactory.create(1, 0); Optional firstError = Optional.empty(); // mock the read handle to make the offload successful diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java index 82141bfd0eeeb..3e1bae7ea7b44 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java @@ -51,6 +51,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,7 +66,7 @@ public class NonDurableCursorTest extends MockedBookKeeperTestCase { void readFromEmptyLedger() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger"); - ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.EARLIEST); + ManagedCursor c1 = ledger.newNonDurableCursor(PositionFactory.EARLIEST); List entries = c1.readEntries(10); assertEquals(entries.size(), 0); entries.forEach(Entry::release); @@ -89,14 +90,14 @@ void testOpenNonDurableCursorAtNonExistentMessageId() throws Exception { ManagedLedger ledger = factory.open("non_durable_cursor_at_non_existent_msgid"); ManagedLedgerImpl mlImpl = (ManagedLedgerImpl) ledger; - PositionImpl position = mlImpl.getLastPosition(); + Position position = mlImpl.getLastPosition(); - ManagedCursor c1 = ledger.newNonDurableCursor(new PositionImpl( + ManagedCursor c1 = ledger.newNonDurableCursor(PositionFactory.create( position.getLedgerId(), position.getEntryId() - 1 )); - assertEquals(c1.getReadPosition(), new PositionImpl( + assertEquals(c1.getReadPosition(), PositionFactory.create( position.getLedgerId(), 0 )); @@ -109,7 +110,7 @@ void testOpenNonDurableCursorAtNonExistentMessageId() throws Exception { void testZNodeBypassed() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger"); - ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.EARLIEST); + ManagedCursor c1 = ledger.newNonDurableCursor(PositionFactory.EARLIEST); assertTrue(ledger.getCursors().iterator().hasNext()); c1.close(); @@ -125,8 +126,8 @@ void readTwice() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setRetentionTime(1, TimeUnit.HOURS).setRetentionSizeInMB(1)); - ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.LATEST); - ManagedCursor c2 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c1 = ledger.newNonDurableCursor(PositionFactory.LATEST); + ManagedCursor c2 = ledger.newNonDurableCursor(PositionFactory.LATEST); ledger.addEntry("entry-1".getBytes(Encoding)); ledger.addEntry("entry-2".getBytes(Encoding)); @@ -158,8 +159,8 @@ void readWithCacheDisabled() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(1) .setRetentionTime(1, TimeUnit.HOURS).setRetentionSizeInMB(1)); - ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.LATEST); - ManagedCursor c2 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c1 = ledger.newNonDurableCursor(PositionFactory.LATEST); + ManagedCursor c2 = ledger.newNonDurableCursor(PositionFactory.LATEST); ledger.addEntry("entry-1".getBytes(Encoding)); ledger.addEntry("entry-2".getBytes(Encoding)); @@ -188,7 +189,7 @@ void readFromClosedLedger() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(1) .setRetentionTime(1, TimeUnit.HOURS).setRetentionSizeInMB(1)); - ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c1 = ledger.newNonDurableCursor(PositionFactory.LATEST); ledger.close(); @@ -205,15 +206,15 @@ void testNumberOfEntries() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(2) .setRetentionTime(1, TimeUnit.HOURS).setRetentionSizeInMB(1)); - ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c1 = ledger.newNonDurableCursor(PositionFactory.LATEST); ledger.addEntry("dummy-entry-1".getBytes(Encoding)); - ManagedCursor c2 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c2 = ledger.newNonDurableCursor(PositionFactory.LATEST); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); - ManagedCursor c3 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c3 = ledger.newNonDurableCursor(PositionFactory.LATEST); ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - ManagedCursor c4 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c4 = ledger.newNonDurableCursor(PositionFactory.LATEST); ledger.addEntry("dummy-entry-4".getBytes(Encoding)); - ManagedCursor c5 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c5 = ledger.newNonDurableCursor(PositionFactory.LATEST); assertEquals(c1.getNumberOfEntries(), 4); assertTrue(c1.hasMoreEntries()); @@ -242,15 +243,15 @@ void testNumberOfEntriesInBacklog() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(2) .setRetentionTime(1, TimeUnit.HOURS).setRetentionSizeInMB(1)); - ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c1 = ledger.newNonDurableCursor(PositionFactory.LATEST); Position p1 = ledger.addEntry("dummy-entry-1".getBytes(Encoding)); - ManagedCursor c2 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c2 = ledger.newNonDurableCursor(PositionFactory.LATEST); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); - ManagedCursor c3 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c3 = ledger.newNonDurableCursor(PositionFactory.LATEST); Position p3 = ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - ManagedCursor c4 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c4 = ledger.newNonDurableCursor(PositionFactory.LATEST); Position p4 = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); - ManagedCursor c5 = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor c5 = ledger.newNonDurableCursor(PositionFactory.LATEST); assertEquals(c1.getNumberOfEntriesInBacklog(false), 4); assertEquals(c2.getNumberOfEntriesInBacklog(false), 3); @@ -334,7 +335,7 @@ void markDeleteAcrossLedgers() throws Exception { @Test(timeOut = 20000) void markDeleteGreaterThanLastConfirmedEntry() throws Exception { ManagedLedger ml1 = factory.open("my_test_ledger"); - ManagedCursor mc1 = ml1.newNonDurableCursor(PositionImpl.get(Long.MAX_VALUE - 1, Long.MAX_VALUE - 1)); + ManagedCursor mc1 = ml1.newNonDurableCursor(PositionFactory.create(Long.MAX_VALUE - 1, Long.MAX_VALUE - 1)); assertEquals(mc1.getMarkDeletedPosition(), ml1.getLastConfirmedEntry()); } @@ -342,13 +343,13 @@ void markDeleteGreaterThanLastConfirmedEntry() throws Exception { void testResetCursor() throws Exception { ManagedLedger ledger = factory.open("my_test_move_cursor_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(10)); - ManagedCursor cursor = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor cursor = ledger.newNonDurableCursor(PositionFactory.LATEST); ledger.addEntry("dummy-entry-1".getBytes(Encoding)); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - PositionImpl lastPosition = (PositionImpl) ledger.addEntry("dummy-entry-4".getBytes(Encoding)); + Position lastPosition = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); final AtomicBoolean moveStatus = new AtomicBoolean(false); - PositionImpl resetPosition = new PositionImpl(lastPosition.getLedgerId(), lastPosition.getEntryId() - 2); + Position resetPosition = PositionFactory.create(lastPosition.getLedgerId(), lastPosition.getEntryId() - 2); try { cursor.resetCursor(resetPosition); moveStatus.set(true); @@ -366,14 +367,14 @@ void testResetCursor() throws Exception { void testasyncResetCursor() throws Exception { ManagedLedger ledger = factory.open("my_test_move_cursor_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(10)); - ManagedCursor cursor = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor cursor = ledger.newNonDurableCursor(PositionFactory.LATEST); ledger.addEntry("dummy-entry-1".getBytes(Encoding)); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - PositionImpl lastPosition = (PositionImpl) ledger.addEntry("dummy-entry-4".getBytes(Encoding)); + Position lastPosition = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); final AtomicBoolean moveStatus = new AtomicBoolean(false); CountDownLatch countDownLatch = new CountDownLatch(1); - PositionImpl resetPosition = new PositionImpl(lastPosition.getLedgerId(), lastPosition.getEntryId() - 2); + Position resetPosition = PositionFactory.create(lastPosition.getLedgerId(), lastPosition.getEntryId() - 2); cursor.asyncResetCursor(resetPosition, false, new AsyncCallbacks.ResetCursorCallback() { @Override @@ -399,7 +400,7 @@ public void resetFailed(ManagedLedgerException exception, Object ctx) { void rewind() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(2) .setRetentionTime(1, TimeUnit.HOURS).setRetentionSizeInMB(1)); - ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.EARLIEST); + ManagedCursor c1 = ledger.newNonDurableCursor(PositionFactory.EARLIEST); Position p1 = ledger.addEntry("dummy-entry-1".getBytes(Encoding)); Position p2 = ledger.addEntry("dummy-entry-2".getBytes(Encoding)); Position p3 = ledger.addEntry("dummy-entry-3".getBytes(Encoding)); @@ -450,11 +451,11 @@ void rewind() throws Exception { @Test(timeOut = 20000) void markDeleteSkippingMessage() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(10)); - ManagedCursor cursor = ledger.newNonDurableCursor(PositionImpl.EARLIEST); + ManagedCursor cursor = ledger.newNonDurableCursor(PositionFactory.EARLIEST); Position p1 = ledger.addEntry("dummy-entry-1".getBytes(Encoding)); Position p2 = ledger.addEntry("dummy-entry-2".getBytes(Encoding)); ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - PositionImpl p4 = (PositionImpl) ledger.addEntry("dummy-entry-4".getBytes(Encoding)); + Position p4 = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); assertEquals(cursor.getNumberOfEntries(), 4); @@ -473,7 +474,7 @@ void markDeleteSkippingMessage() throws Exception { assertFalse(cursor.hasMoreEntries()); assertEquals(cursor.getNumberOfEntries(), 0); - assertEquals(cursor.getReadPosition(), new PositionImpl(p4.getLedgerId(), p4.getEntryId() + 1)); + assertEquals(cursor.getReadPosition(), PositionFactory.create(p4.getLedgerId(), p4.getEntryId() + 1)); } @Test(timeOut = 20000) @@ -546,7 +547,7 @@ void unorderedMarkDelete() throws Exception { void testSingleDelete() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig().setMaxEntriesPerLedger(3) .setRetentionTime(1, TimeUnit.HOURS).setRetentionSizeInMB(1)); - ManagedCursor cursor = ledger.newNonDurableCursor(PositionImpl.LATEST); + ManagedCursor cursor = ledger.newNonDurableCursor(PositionFactory.LATEST); Position p1 = ledger.addEntry("entry1".getBytes()); Position p2 = ledger.addEntry("entry2".getBytes()); @@ -592,9 +593,9 @@ void subscribeToEarliestPositionWithImmediateDeletion() throws Exception { /* Position p3 = */ ledger.addEntry("entry-3".getBytes()); Thread.sleep(300); - ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.EARLIEST); - assertEquals(c1.getReadPosition(), new PositionImpl(6, 0)); - assertEquals(c1.getMarkDeletedPosition(), new PositionImpl(6, -1)); + ManagedCursor c1 = ledger.newNonDurableCursor(PositionFactory.EARLIEST); + assertEquals(c1.getReadPosition(), PositionFactory.create(6, 0)); + assertEquals(c1.getMarkDeletedPosition(), PositionFactory.create(6, -1)); } @Test // (timeOut = 20000) @@ -609,9 +610,9 @@ void subscribeToEarliestPositionWithDeferredDeletion() throws Exception { /* Position p5 = */ ledger.addEntry("entry-5".getBytes()); /* Position p6 = */ ledger.addEntry("entry-6".getBytes()); - ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.EARLIEST); + ManagedCursor c1 = ledger.newNonDurableCursor(PositionFactory.EARLIEST); assertEquals(c1.getReadPosition(), p1); - assertEquals(c1.getMarkDeletedPosition(), new PositionImpl(3, -1)); + assertEquals(c1.getMarkDeletedPosition(), PositionFactory.create(3, -1)); assertEquals(c1.getNumberOfEntries(), 6); assertEquals(c1.getNumberOfEntriesInBacklog(false), 6); @@ -674,7 +675,7 @@ public void testGetSlowestConsumer() throws Exception { // The slowest reader should still be the durable cursor since non-durable readers are not taken into account assertEquals(p3, ledger.getCursors().getSlowestReaderPosition()); - PositionImpl earliestPos = new PositionImpl(-1, -2); + Position earliestPos = PositionFactory.create(-1, -2); ManagedCursor nonCursorEarliest = ledger.newNonDurableCursor(earliestPos, ncEarliest); @@ -702,7 +703,7 @@ public void testBacklogStatsWhenDroppingData() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testBacklogStatsWhenDroppingData", new ManagedLedgerConfig().setMaxEntriesPerLedger(1)); ManagedCursor c1 = ledger.openCursor("c1"); - ManagedCursor nonDurableCursor = ledger.newNonDurableCursor(PositionImpl.EARLIEST); + ManagedCursor nonDurableCursor = ledger.newNonDurableCursor(PositionFactory.EARLIEST); assertEquals(nonDurableCursor.getNumberOfEntries(), 0); assertEquals(nonDurableCursor.getNumberOfEntriesInBacklog(true), 0); @@ -746,7 +747,7 @@ public void testInvalidateReadHandleWithSlowNonDurableCursor() throws Exception new ManagedLedgerConfig().setMaxEntriesPerLedger(1).setRetentionTime(-1, TimeUnit.SECONDS) .setRetentionSizeInMB(-1)); ManagedCursor c1 = ledger.openCursor("c1"); - ManagedCursor nonDurableCursor = ledger.newNonDurableCursor(PositionImpl.EARLIEST); + ManagedCursor nonDurableCursor = ledger.newNonDurableCursor(PositionFactory.EARLIEST); List positions = new ArrayList<>(); for (int i = 0; i < 10; i++) { @@ -755,7 +756,7 @@ public void testInvalidateReadHandleWithSlowNonDurableCursor() throws Exception CountDownLatch latch = new CountDownLatch(10); for (int i = 0; i < 10; i++) { - ledger.asyncReadEntry((PositionImpl) positions.get(i), new AsyncCallbacks.ReadEntryCallback() { + ledger.asyncReadEntry(positions.get(i), new AsyncCallbacks.ReadEntryCallback() { @Override public void readEntryComplete(Entry entry, Object ctx) { latch.countDown(); @@ -819,7 +820,7 @@ void testCursorWithNameIsNotNull() throws Exception { void deleteNonDurableCursorWithName() throws Exception { ManagedLedger ledger = factory.open("deleteManagedLedgerWithNonDurableCursor"); - ManagedCursor c = ledger.newNonDurableCursor(PositionImpl.EARLIEST, "custom-name"); + ManagedCursor c = ledger.newNonDurableCursor(PositionFactory.EARLIEST, "custom-name"); assertEquals(Iterables.size(ledger.getCursors()), 1); ledger.deleteCursor(c.getName()); @@ -831,7 +832,7 @@ public void testMessagesConsumedCounterInitializedCorrect() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testMessagesConsumedCounterInitializedCorrect", new ManagedLedgerConfig().setRetentionTime(1, TimeUnit.HOURS).setRetentionSizeInMB(1)); Position position = ledger.addEntry("1".getBytes(Encoding)); - NonDurableCursorImpl cursor = (NonDurableCursorImpl) ledger.newNonDurableCursor(PositionImpl.EARLIEST); + NonDurableCursorImpl cursor = (NonDurableCursorImpl) ledger.newNonDurableCursor(PositionFactory.EARLIEST); cursor.delete(position); assertEquals(cursor.getMessagesConsumedCounter(), 1); assertTrue(cursor.getMessagesConsumedCounter() <= ledger.getEntriesAddedCounter()); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java index cd224e33e2734..29138145d1505 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java @@ -54,6 +54,7 @@ import org.apache.bookkeeper.mledger.LedgerOffloader; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.mledger.util.MockClock; import org.apache.bookkeeper.net.BookieId; @@ -93,7 +94,7 @@ public void testOffloadRead() throws Exception { UUID secondLedgerUUID = new UUID(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getUidMsb(), ledger.getLedgersInfoAsList().get(1).getOffloadContext().getUidLsb()); - ManagedCursor cursor = ledger.newNonDurableCursor(PositionImpl.EARLIEST); + ManagedCursor cursor = ledger.newNonDurableCursor(PositionFactory.EARLIEST); int i = 0; for (Entry e : cursor.readEntries(10)) { assertEquals(new String(e.getData()), "entry-" + i++); @@ -163,7 +164,7 @@ public void testBookkeeperFirstOffloadRead() throws Exception { UUID secondLedgerUUID = new UUID(secondLedger.getOffloadContext().getUidMsb(), secondLedger.getOffloadContext().getUidLsb()); - ManagedCursor cursor = ledger.newNonDurableCursor(PositionImpl.EARLIEST); + ManagedCursor cursor = ledger.newNonDurableCursor(PositionFactory.EARLIEST); int i = 0; for (Entry e : cursor.readEntries(10)) { Assert.assertEquals(new String(e.getData()), "entry-" + i++); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java index 2cdb14fb71e41..331e7b0317394 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java @@ -49,6 +49,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; @@ -201,13 +202,13 @@ public void testPositionOutOfRange() throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), 3); try { - ledger.offloadPrefix(PositionImpl.EARLIEST); + ledger.offloadPrefix(PositionFactory.EARLIEST); fail("Should have thrown an exception"); } catch (ManagedLedgerException.InvalidCursorPositionException e) { // expected } try { - ledger.offloadPrefix(PositionImpl.LATEST); + ledger.offloadPrefix(PositionFactory.LATEST); fail("Should have thrown an exception"); } catch (ManagedLedgerException.InvalidCursorPositionException e) { // expected @@ -241,7 +242,7 @@ public void testPositionOnEdgeOfLedger() throws Exception { ledger.addEntry("entry-blah".getBytes()); assertEquals(ledger.getLedgersInfoAsList().size(), 3); - PositionImpl firstUnoffloaded = (PositionImpl)ledger.offloadPrefix(p); + Position firstUnoffloaded = ledger.offloadPrefix(p); // only the first ledger should have been offloaded assertEquals(ledger.getLedgersInfoAsList().size(), 3); @@ -254,7 +255,7 @@ public void testPositionOnEdgeOfLedger() throws Exception { assertEquals(firstUnoffloaded.getEntryId(), 0); // offload again, with the position in the third ledger - PositionImpl firstUnoffloaded2 = (PositionImpl)ledger.offloadPrefix(ledger.getLastConfirmedEntry()); + Position firstUnoffloaded2 = ledger.offloadPrefix(ledger.getLastConfirmedEntry()); assertEquals(ledger.getLedgersInfoAsList().size(), 3); assertEquals(offloader.offloadedLedgers().size(), 2); assertTrue(offloader.offloadedLedgers().contains(ledger.getLedgersInfoAsList().get(0).getLedgerId())); @@ -291,9 +292,9 @@ public void testPositionOnLastEmptyLedger() throws Exception { assertEquals(ledger.getLedgersInfoAsList().get(1).getSize(), 0); // position past the end of first ledger - Position p = new PositionImpl(ledger.getLedgersInfoAsList().get(1).getLedgerId(), 0); + Position p = PositionFactory.create(ledger.getLedgersInfoAsList().get(1).getLedgerId(), 0); - PositionImpl firstUnoffloaded = (PositionImpl)ledger.offloadPrefix(p); + Position firstUnoffloaded = ledger.offloadPrefix(p); // only the first ledger should have been offloaded assertEquals(ledger.getLedgersInfoAsList().size(), 2); @@ -335,8 +336,8 @@ public CompletableFuture offload(ReadHandle ledger, } assertEquals(ledger.getLedgersInfoAsList().size(), 3); - PositionImpl startOfSecondLedger = PositionImpl.get(ledger.getLedgersInfoAsList().get(1).getLedgerId(), 0); - PositionImpl startOfThirdLedger = PositionImpl.get(ledger.getLedgersInfoAsList().get(2).getLedgerId(), 0); + Position startOfSecondLedger = PositionFactory.create(ledger.getLedgersInfoAsList().get(1).getLedgerId(), 0); + Position startOfThirdLedger = PositionFactory.create(ledger.getLedgersInfoAsList().get(2).getLedgerId(), 0); // trigger an offload which should offload the first two ledgers OffloadCallbackPromise cbPromise = new OffloadCallbackPromise(); @@ -398,8 +399,8 @@ public CompletableFuture offload(ReadHandle ledger, } assertEquals(ledger.getLedgersInfoAsList().size(), 3); - PositionImpl startOfSecondLedger = PositionImpl.get(ledger.getLedgersInfoAsList().get(1).getLedgerId(), 0); - PositionImpl startOfThirdLedger = PositionImpl.get(ledger.getLedgersInfoAsList().get(2).getLedgerId(), 0); + Position startOfSecondLedger = PositionFactory.create(ledger.getLedgersInfoAsList().get(1).getLedgerId(), 0); + Position startOfThirdLedger = PositionFactory.create(ledger.getLedgersInfoAsList().get(2).getLedgerId(), 0); // trigger an offload which should offload the first two ledgers OffloadCallbackPromise cbPromise = new OffloadCallbackPromise(); @@ -829,7 +830,7 @@ public void testDontOffloadEmpty() throws Exception { ledgers.put(secondLedgerId, ledgers.get(secondLedgerId).toBuilder().setEntries(0).setSize(0).build()); - PositionImpl firstUnoffloaded = (PositionImpl)ledger.offloadPrefix(ledger.getLastConfirmedEntry()); + Position firstUnoffloaded = ledger.offloadPrefix(ledger.getLastConfirmedEntry()); assertEquals(firstUnoffloaded.getLedgerId(), fourthLedgerId); assertEquals(firstUnoffloaded.getEntryId(), 0); @@ -1073,7 +1074,7 @@ public CompletableFuture offload(ReadHandle ledger, } else if (sizeThreshold != null && sizeThreshold.equals(100L) && timeThreshold == null) { // the last 2 ledgers won't be offloaded. assertEquals(cbPromise.join(), - PositionImpl.get(ledger.getLedgersInfoAsList().get(1).getLedgerId(), 0)); + PositionFactory.create(ledger.getLedgersInfoAsList().get(1).getLedgerId(), 0)); assertEventuallyTrue(() -> offloader.offloadedLedgers().size() == 2); assertEquals(offloader.offloadedLedgers(), Set.of(ledger.getLedgersInfoAsList().get(0).getLedgerId(), diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionTest.java index f2b1a7062b5e3..763146b6c3fbb 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionTest.java @@ -21,33 +21,35 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; import org.testng.annotations.Test; public class PositionTest { @Test(expectedExceptions = NullPointerException.class) public void nullParam() { - new PositionImpl((PositionInfo) null); + PositionFactory.create(null); } @Test public void simpleTest() { - PositionImpl pos = new PositionImpl(1, 2); + Position pos = PositionFactory.create(1, 2); assertEquals(pos.getLedgerId(), 1); assertEquals(pos.getEntryId(), 2); - assertEquals(pos, new PositionImpl(1, 2)); + assertEquals(pos, PositionFactory.create(1, 2)); - assertNotEquals(new PositionImpl(1, 3), pos); - assertNotEquals(new PositionImpl(3, 2), pos); + assertNotEquals(PositionFactory.create(1, 3), pos); + assertNotEquals(PositionFactory.create(3, 2), pos); assertNotEquals(pos, "1:2"); } @Test public void comparisons() { - PositionImpl pos1_1 = new PositionImpl(1, 1); - PositionImpl pos2_5 = new PositionImpl(2, 5); - PositionImpl pos10_0 = new PositionImpl(10, 0); - PositionImpl pos10_1 = new PositionImpl(10, 1); + Position pos1_1 = PositionFactory.create(1, 1); + Position pos2_5 = PositionFactory.create(2, 5); + Position pos10_0 = PositionFactory.create(10, 0); + Position pos10_1 = PositionFactory.create(10, 1); assertEquals(0, pos1_1.compareTo(pos1_1)); assertEquals(-1, pos1_1.compareTo(pos2_5)); @@ -72,10 +74,13 @@ public void comparisons() { @Test public void hashes() throws Exception { - PositionImpl p1 = new PositionImpl(5, 15); - PositionImpl p2 = new PositionImpl(PositionInfo.parseFrom(p1.getPositionInfo().toByteArray())); + Position p1 = PositionFactory.create(5, 15); + PositionInfo positionInfo = + PositionInfo.newBuilder().setLedgerId(p1.getLedgerId()).setEntryId(p1.getEntryId()).build(); + PositionInfo parsed = PositionInfo.parseFrom(positionInfo.toByteArray()); + Position p2 = PositionFactory.create(parsed.getLedgerId(), parsed.getEntryId()); assertEquals(p2.getLedgerId(), 5); assertEquals(p2.getEntryId(), 15); - assertEquals(new PositionImpl(5, 15).hashCode(), p2.hashCode()); + assertEquals(PositionFactory.create(5, 15).hashCode(), p2.hashCode()); } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorTest.java index 0386966ad2284..66a33560b67b4 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorTest.java @@ -31,6 +31,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerNotFoundException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.testng.annotations.Test; @@ -40,7 +41,7 @@ public class ReadOnlyCursorTest extends MockedBookKeeperTestCase { @Test void notFound() throws Exception { try { - factory.openReadOnlyCursor("notFound", PositionImpl.EARLIEST, new ManagedLedgerConfig()); + factory.openReadOnlyCursor("notFound", PositionFactory.EARLIEST, new ManagedLedgerConfig()); fail("Should have failed"); } catch (ManagedLedgerNotFoundException e) { // Expected @@ -59,7 +60,7 @@ void simple() throws Exception { ledger.addEntry(("entry-" + i).getBytes()); } - ReadOnlyCursor cursor = factory.openReadOnlyCursor("simple", PositionImpl.EARLIEST, new ManagedLedgerConfig()); + ReadOnlyCursor cursor = factory.openReadOnlyCursor("simple", PositionFactory.EARLIEST, new ManagedLedgerConfig()); assertEquals(cursor.getNumberOfEntries(), N); assertTrue(cursor.hasMoreEntries()); @@ -78,7 +79,7 @@ void simple() throws Exception { } // Open a new cursor - cursor = factory.openReadOnlyCursor("simple", PositionImpl.EARLIEST, new ManagedLedgerConfig()); + cursor = factory.openReadOnlyCursor("simple", PositionFactory.EARLIEST, new ManagedLedgerConfig()); assertEquals(cursor.getNumberOfEntries(), 2 * N); assertTrue(cursor.hasMoreEntries()); @@ -114,7 +115,7 @@ void skip() throws Exception { ledger.addEntry(("entry-" + i).getBytes()); } - ReadOnlyCursor cursor = factory.openReadOnlyCursor("skip", PositionImpl.EARLIEST, new ManagedLedgerConfig()); + ReadOnlyCursor cursor = factory.openReadOnlyCursor("skip", PositionFactory.EARLIEST, new ManagedLedgerConfig()); assertEquals(cursor.getNumberOfEntries(), N); assertTrue(cursor.hasMoreEntries()); @@ -138,7 +139,7 @@ void skipAll() throws Exception { ledger.addEntry(("entry-" + i).getBytes()); } - ReadOnlyCursor cursor = factory.openReadOnlyCursor("skip-all", PositionImpl.EARLIEST, + ReadOnlyCursor cursor = factory.openReadOnlyCursor("skip-all", PositionFactory.EARLIEST, new ManagedLedgerConfig()); assertEquals(cursor.getNumberOfEntries(), N); @@ -166,7 +167,7 @@ void skipMultiple() throws Exception { ledger.addEntry(("entry-" + i).getBytes()); } - ReadOnlyCursor cursor = factory.openReadOnlyCursor("skip", PositionImpl.EARLIEST, new ManagedLedgerConfig()); + ReadOnlyCursor cursor = factory.openReadOnlyCursor("skip", PositionFactory.EARLIEST, new ManagedLedgerConfig()); assertEquals(cursor.getNumberOfEntries(), N); assertTrue(cursor.hasMoreEntries()); @@ -188,7 +189,7 @@ void skipMultiple() throws Exception { void empty() throws Exception { factory.open("empty", new ManagedLedgerConfig().setRetentionTime(1, TimeUnit.HOURS)); - ReadOnlyCursor cursor = factory.openReadOnlyCursor("empty", PositionImpl.EARLIEST, new ManagedLedgerConfig()); + ReadOnlyCursor cursor = factory.openReadOnlyCursor("empty", PositionFactory.EARLIEST, new ManagedLedgerConfig()); assertEquals(cursor.getNumberOfEntries(), 0); assertFalse(cursor.hasMoreEntries()); @@ -206,7 +207,7 @@ void specifyStartPosition() throws Exception { ledger.addEntry(("entry-" + i).getBytes()); } - ReadOnlyCursor cursor = factory.openReadOnlyCursor("simple", PositionImpl.EARLIEST, new ManagedLedgerConfig()); + ReadOnlyCursor cursor = factory.openReadOnlyCursor("simple", PositionFactory.EARLIEST, new ManagedLedgerConfig()); assertEquals(cursor.getNumberOfEntries(), N); assertTrue(cursor.hasMoreEntries()); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java index 13dee4812b464..fc5450f2c4cfc 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java @@ -32,6 +32,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.awaitility.Awaitility; import org.testng.annotations.Test; @@ -127,7 +128,7 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { } {// test write entry with ledgerId > currentLedger - PositionImpl fakePos = PositionImpl.get(newPos.getLedgerId() + 1, newPos.getEntryId()); + Position fakePos = PositionFactory.create(newPos.getLedgerId() + 1, newPos.getEntryId()); CompletableFuture future = new CompletableFuture<>(); shadowML.asyncAddEntry(data, new AsyncCallbacks.AddEntryCallback() { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java index cce593fbb38ec..84842c74cd22a 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/ManagedLedgerImplUtilsTest.java @@ -29,7 +29,6 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.testng.annotations.Test; @@ -54,8 +53,8 @@ public void testGetLastValidPosition() throws Exception { }; // New ledger will return the last position, regardless of whether the conditions are met or not. - Position position = ManagedLedgerImplUtils.asyncGetLastValidPosition((ManagedLedgerImpl) ledger, - predicate, (PositionImpl) ledger.getLastConfirmedEntry()).get(); + Position position = ManagedLedgerImplUtils.asyncGetLastValidPosition((ManagedLedgerImpl) ledger, + predicate, ledger.getLastConfirmedEntry()).get(); assertEquals(ledger.getLastConfirmedEntry(), position); for (int i = 0; i < maxEntriesPerLedger - 1; i++) { @@ -68,7 +67,7 @@ public void testGetLastValidPosition() throws Exception { // Returns last position of entry is "match-entry" position = ManagedLedgerImplUtils.asyncGetLastValidPosition((ManagedLedgerImpl) ledger, - predicate, (PositionImpl) ledger.getLastConfirmedEntry()).get(); + predicate, ledger.getLastConfirmedEntry()).get(); assertEquals(position, lastMatchPosition); ledger.close(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/PositionAckSetUtilTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/PositionAckSetUtilTest.java index 088220cd35e5f..d9c0c5a11eaee 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/PositionAckSetUtilTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/util/PositionAckSetUtilTest.java @@ -18,18 +18,19 @@ */ package org.apache.bookkeeper.mledger.util; -import org.apache.bookkeeper.mledger.impl.PositionImpl; -import org.apache.pulsar.common.util.collections.BitSetRecyclable; -import org.testng.annotations.Test; - -import java.util.BitSet; - import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.andAckSet; import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.compareToWithAckSet; import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.isAckSetOverlap; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; +import java.util.BitSet; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.AckSetState; +import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; +import org.apache.pulsar.common.util.collections.BitSetRecyclable; +import org.testng.annotations.Test; public class PositionAckSetUtilTest { @@ -50,16 +51,16 @@ public void isAckSetRepeatedTest() { @Test public void compareToWithAckSetForCumulativeAckTest() { - PositionImpl positionOne = PositionImpl.get(1, 1); - PositionImpl positionTwo = PositionImpl.get(1, 2); + Position positionOne = PositionFactory.create(1, 1); + Position positionTwo = PositionFactory.create(1, 2); assertEquals(compareToWithAckSet(positionOne, positionTwo), -1); - positionTwo = PositionImpl.get(2, 1); + positionTwo = PositionFactory.create(2, 1); assertEquals(compareToWithAckSet(positionOne, positionTwo), -1); - positionTwo = PositionImpl.get(0, 1); + positionTwo = PositionFactory.create(0, 1); assertEquals(compareToWithAckSet(positionOne, positionTwo), 1); - positionTwo = PositionImpl.get(1, 0); + positionTwo = PositionFactory.create(1, 0); assertEquals(compareToWithAckSet(positionOne, positionTwo), 1); - positionTwo = PositionImpl.get(1, 1); + positionTwo = PositionFactory.create(1, 1); assertEquals(compareToWithAckSet(positionOne, positionTwo), 0); BitSet bitSetOne = new BitSet(); @@ -68,23 +69,24 @@ public void compareToWithAckSetForCumulativeAckTest() { bitSetTwo.set(0, 63); bitSetOne.clear(0, 10); bitSetTwo.clear(0, 10); - positionOne.setAckSet(bitSetOne.toLongArray()); - positionTwo.setAckSet(bitSetTwo.toLongArray()); + positionOne = AckSetStateUtil.createPositionWithAckSet(1, 1, bitSetOne.toLongArray()); + positionTwo = AckSetStateUtil.createPositionWithAckSet(1, 1, bitSetTwo.toLongArray()); assertEquals(compareToWithAckSet(positionOne, positionTwo), 0); bitSetOne.clear(10, 12); - positionOne.setAckSet(bitSetOne.toLongArray()); + AckSetState positionOneAckSetState = AckSetStateUtil.getAckSetState(positionOne); + positionOneAckSetState.setAckSet(bitSetOne.toLongArray()); assertEquals(compareToWithAckSet(positionOne, positionTwo), 2); bitSetOne.set(8, 12); - positionOne.setAckSet(bitSetOne.toLongArray()); + positionOneAckSetState.setAckSet(bitSetOne.toLongArray()); assertEquals(compareToWithAckSet(positionOne, positionTwo), -2); } @Test public void andAckSetTest() { - PositionImpl positionOne = PositionImpl.get(1, 1); - PositionImpl positionTwo = PositionImpl.get(1, 2); + Position positionOne = AckSetStateUtil.createPositionWithAckSet(1, 1, new long[0]); + Position positionTwo = AckSetStateUtil.createPositionWithAckSet(1, 2, new long[0]); BitSet bitSetOne = new BitSet(); BitSet bitSetTwo = new BitSet(); bitSetOne.set(0); @@ -92,20 +94,22 @@ public void andAckSetTest() { bitSetOne.set(4); bitSetOne.set(6); bitSetOne.set(8); - positionOne.setAckSet(bitSetOne.toLongArray()); - positionTwo.setAckSet(bitSetTwo.toLongArray()); + AckSetState positionOneAckSetState = AckSetStateUtil.getAckSetState(positionOne); + positionOneAckSetState.setAckSet(bitSetOne.toLongArray()); + AckSetState positionTwoAckSetState = AckSetStateUtil.getAckSetState(positionTwo); + positionTwoAckSetState.setAckSet(bitSetTwo.toLongArray()); andAckSet(positionOne, positionTwo); - BitSetRecyclable bitSetRecyclable = BitSetRecyclable.valueOf(positionOne.getAckSet()); + BitSetRecyclable bitSetRecyclable = BitSetRecyclable.valueOf(positionOneAckSetState.getAckSet()); assertTrue(bitSetRecyclable.isEmpty()); bitSetTwo.set(2); bitSetTwo.set(4); - positionOne.setAckSet(bitSetOne.toLongArray()); - positionTwo.setAckSet(bitSetTwo.toLongArray()); + positionOneAckSetState.setAckSet(bitSetOne.toLongArray()); + positionTwoAckSetState.setAckSet(bitSetTwo.toLongArray()); andAckSet(positionOne, positionTwo); - bitSetRecyclable = BitSetRecyclable.valueOf(positionOne.getAckSet()); + bitSetRecyclable = BitSetRecyclable.valueOf(positionOneAckSetState.getAckSet()); BitSetRecyclable bitSetRecyclableTwo = BitSetRecyclable.valueOf(bitSetTwo.toLongArray()); assertEquals(bitSetRecyclable, bitSetRecyclableTwo); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index bc933cc5c1adb..ff764b368eb83 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -63,11 +63,12 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerInfo; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ScanOutcome; +import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerOfflineBacklog; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.StringUtils; @@ -2286,7 +2287,7 @@ private void internalCreateSubscriptionForNonPartitionedTopic( // Mark the cursor as "inactive" as it was created without a real consumer connected ((PersistentSubscription) subscription).deactivateCursor(); return subscription.resetCursor( - PositionImpl.get(targetMessageId.getLedgerId(), targetMessageId.getEntryId())); + PositionFactory.create(targetMessageId.getLedgerId(), targetMessageId.getEntryId())); }).thenRun(() -> { log.info("[{}][{}] Successfully created subscription {} at message id {}", clientAppId(), topicName, subscriptionName, targetMessageId); @@ -2557,7 +2558,7 @@ protected void internalResetCursorOnPosition(AsyncResponse asyncResponse, String CompletableFuture batchSizeFuture = new CompletableFuture<>(); getEntryBatchSize(batchSizeFuture, (PersistentTopic) topic, messageId, batchIndex); batchSizeFuture.thenAccept(bi -> { - PositionImpl seekPosition = calculatePositionAckSet(isExcluded, bi, batchIndex, + Position seekPosition = calculatePositionAckSet(isExcluded, bi, batchIndex, messageId); sub.resetCursor(seekPosition).thenRun(() -> { log.info("[{}][{}] successfully reset cursor on subscription {}" @@ -2601,7 +2602,7 @@ private void getEntryBatchSize(CompletableFuture batchSizeFuture, Persi if (batchIndex >= 0) { try { ManagedLedgerImpl ledger = (ManagedLedgerImpl) topic.getManagedLedger(); - ledger.asyncReadEntry(new PositionImpl(messageId.getLedgerId(), + ledger.asyncReadEntry(PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()), new AsyncCallbacks.ReadEntryCallback() { @Override public void readEntryFailed(ManagedLedgerException exception, Object ctx) { @@ -2650,9 +2651,9 @@ public String toString() { } } - private PositionImpl calculatePositionAckSet(boolean isExcluded, int batchSize, + private Position calculatePositionAckSet(boolean isExcluded, int batchSize, int batchIndex, MessageIdImpl messageId) { - PositionImpl seekPosition; + Position seekPosition; if (batchSize > 0) { long[] ackSet; BitSetRecyclable bitSet = BitSetRecyclable.create(); @@ -2661,25 +2662,25 @@ private PositionImpl calculatePositionAckSet(boolean isExcluded, int batchSize, bitSet.clear(0, Math.max(batchIndex + 1, 0)); if (bitSet.length() > 0) { ackSet = bitSet.toLongArray(); - seekPosition = PositionImpl.get(messageId.getLedgerId(), + seekPosition = AckSetStateUtil.createPositionWithAckSet(messageId.getLedgerId(), messageId.getEntryId(), ackSet); } else { - seekPosition = PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId()); + seekPosition = PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()); seekPosition = seekPosition.getNext(); } } else { if (batchIndex - 1 >= 0) { bitSet.clear(0, batchIndex); ackSet = bitSet.toLongArray(); - seekPosition = PositionImpl.get(messageId.getLedgerId(), + seekPosition = AckSetStateUtil.createPositionWithAckSet(messageId.getLedgerId(), messageId.getEntryId(), ackSet); } else { - seekPosition = PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId()); + seekPosition = PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()); } } bitSet.recycle(); } else { - seekPosition = PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId()); + seekPosition = PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()); seekPosition = isExcluded ? seekPosition.getNext() : seekPosition; } return seekPosition; @@ -2713,7 +2714,7 @@ protected CompletableFuture internalGetMessageById(long ledgerId, long CompletableFuture results = new CompletableFuture<>(); ManagedLedgerImpl ledger = (ManagedLedgerImpl) ((PersistentTopic) topic).getManagedLedger(); - ledger.asyncReadEntry(new PositionImpl(ledgerId, entryId), + ledger.asyncReadEntry(PositionFactory.create(ledgerId, entryId), new AsyncCallbacks.ReadEntryCallback() { @Override public void readEntryFailed(ManagedLedgerException exception, @@ -2929,12 +2930,12 @@ protected CompletableFuture internalExamineMessageAsync(String initial throw new RestException(Status.PRECONDITION_FAILED, "Could not examine messages due to the total message is zero"); } - PositionImpl startPosition = persistentTopic.getFirstPosition(); + Position startPosition = persistentTopic.getFirstPosition(); long messageToSkip = initialPositionLocal.equals("earliest") ? messagePositionLocal : totalMessage - messagePositionLocal + 1; CompletableFuture future = new CompletableFuture<>(); - PositionImpl readPosition = persistentTopic.getPositionAfterN(startPosition, messageToSkip); + Position readPosition = persistentTopic.getPositionAfterN(startPosition, messageToSkip); persistentTopic.asyncReadEntry(readPosition, new AsyncCallbacks.ReadEntryCallback() { @Override public void readEntryComplete(Entry entry, Object ctx) { @@ -2976,7 +2977,7 @@ public String toString() { private Response generateResponseWithEntry(Entry entry, PersistentTopic persistentTopic) throws IOException { checkNotNull(entry); - PositionImpl pos = (PositionImpl) entry.getPosition(); + Position pos = entry.getPosition(); ByteBuf metadataAndPayload = entry.getDataBuffer(); long totalSize = metadataAndPayload.readableBytes(); @@ -3094,10 +3095,10 @@ private Response generateResponseWithEntry(Entry entry, PersistentTopic persiste } if (metadata.hasTxnidMostBits() && metadata.hasTxnidLeastBits()) { TxnID txnID = new TxnID(metadata.getTxnidMostBits(), metadata.getTxnidLeastBits()); - boolean isTxnAborted = persistentTopic.isTxnAborted(txnID, (PositionImpl) entry.getPosition()); + boolean isTxnAborted = persistentTopic.isTxnAborted(txnID, entry.getPosition()); responseBuilder.header("X-Pulsar-txn-aborted", isTxnAborted); } - boolean isTxnUncommitted = ((PositionImpl) entry.getPosition()) + boolean isTxnUncommitted = (entry.getPosition()) .compareTo(persistentTopic.getMaxReadPosition()) > 0; responseBuilder.header("X-Pulsar-txn-uncommitted", isTxnUncommitted); @@ -3216,7 +3217,7 @@ protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, .thenCompose(unused -> getTopicReferenceAsync(topicName)) .thenAccept(t -> { PersistentTopic topic = (PersistentTopic) t; - PositionImpl pos = new PositionImpl(messageId.getLedgerId(), + Position pos = PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()); if (topic == null) { asyncResponse.resume(new RestException(Status.NOT_FOUND, @@ -4052,7 +4053,7 @@ private CompletableFuture internalExpireMessagesNonPartitionedTopicByPosit getEntryBatchSize(batchSizeFuture, topic, messageId, batchIndex); batchSizeFuture.thenAccept(bi -> { - PositionImpl position = calculatePositionAckSet(isExcluded, bi, batchIndex, messageId); + Position position = calculatePositionAckSet(isExcluded, bi, batchIndex, messageId); try { if (messageExpirer.expireMessages(position)) { log.info("[{}] Message expire started up to {} on {} {}", clientAppId(), position, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TransactionsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TransactionsBase.java index 4fef0802ed413..55767136f8151 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TransactionsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TransactionsBase.java @@ -34,7 +34,7 @@ import javax.ws.rs.core.Response; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedLedger; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.service.Topic; @@ -548,7 +548,7 @@ protected CompletableFuture internalScaleTransactionCoordinators(int repli } protected CompletableFuture internalGetPositionStatsPendingAckStats( - boolean authoritative, String subName, PositionImpl position, Integer batchIndex) { + boolean authoritative, String subName, Position position, Integer batchIndex) { CompletableFuture completableFuture = new CompletableFuture<>(); getExistingPersistentTopicAsync(authoritative) .thenAccept(topic -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 8a1f4e0dc5600..0a8bf22c42d91 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -45,7 +45,7 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -1872,7 +1872,7 @@ public void analyzeSubscriptionBacklog( try { Optional positionImpl; if (position != null) { - positionImpl = Optional.of(new PositionImpl(position.getLedgerId(), + positionImpl = Optional.of(PositionFactory.create(position.getLedgerId(), position.getEntryId())); } else { positionImpl = Optional.empty(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java index 7e3806aa9b47b..089ec53069287 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java @@ -39,7 +39,8 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.admin.impl.TransactionsBase; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.web.RestException; @@ -438,7 +439,7 @@ public void getPositionStatsInPendingAck(@Suspended final AsyncResponse asyncRes try { checkTransactionCoordinatorEnabled(); validateTopicName(tenant, namespace, encodedTopic); - PositionImpl position = new PositionImpl(ledgerId, entryId); + Position position = PositionFactory.create(ledgerId, entryId); internalGetPositionStatsPendingAckStats(authoritative, subName, position, batchIndex) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java index 78229fef25a5a..81ed4894dc6ad 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java @@ -21,7 +21,7 @@ import com.google.common.annotations.Beta; import java.util.NavigableSet; import java.util.concurrent.CompletableFuture; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; /** * Represent the tracker for the delayed delivery of messages for a particular subscription. @@ -59,7 +59,7 @@ public interface DelayedDeliveryTracker extends AutoCloseable { /** * Get a set of position of messages that have already reached the delivery time. */ - NavigableSet getScheduledMessages(int maxMessages); + NavigableSet getScheduledMessages(int maxMessages); /** * Tells whether the dispatcher should pause any message deliveries, until the DelayedDeliveryTracker has diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java index 58358b06a46bb..8bd9fafa13715 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java @@ -26,7 +26,8 @@ import java.util.concurrent.CompletableFuture; import lombok.Getter; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue; @@ -114,9 +115,9 @@ public boolean hasMessageAvailable() { * Get a set of position of messages that have already reached. */ @Override - public NavigableSet getScheduledMessages(int maxMessages) { + public NavigableSet getScheduledMessages(int maxMessages) { int n = maxMessages; - NavigableSet positions = new TreeSet<>(); + NavigableSet positions = new TreeSet<>(); long cutoffTime = getCutoffTime(); while (n > 0 && !priorityQueue.isEmpty()) { @@ -127,7 +128,7 @@ public NavigableSet getScheduledMessages(int maxMessages) { long ledgerId = priorityQueue.peekN2(); long entryId = priorityQueue.peekN3(); - positions.add(new PositionImpl(ledgerId, entryId)); + positions.add(PositionFactory.create(ledgerId, entryId)); priorityQueue.pop(); --n; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index f98c9e000f150..063793f2dd1fa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -48,7 +48,8 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.mutable.MutableLong; @@ -550,7 +551,7 @@ public long getBufferMemoryUsage() { } @Override - public synchronized NavigableSet getScheduledMessages(int maxMessages) { + public synchronized NavigableSet getScheduledMessages(int maxMessages) { if (!checkPendingLoadDone()) { if (log.isDebugEnabled()) { log.debug("[{}] Skip getScheduledMessages to wait for bucket snapshot load finish.", @@ -563,7 +564,7 @@ public synchronized NavigableSet getScheduledMessages(int maxMessa lastMutableBucket.moveScheduledMessageToSharedQueue(cutoffTime, sharedBucketPriorityQueue); - NavigableSet positions = new TreeSet<>(); + NavigableSet positions = new TreeSet<>(); int n = maxMessages; while (n > 0 && !sharedBucketPriorityQueue.isEmpty()) { @@ -647,7 +648,7 @@ public synchronized NavigableSet getScheduledMessages(int maxMessa } } - positions.add(new PositionImpl(ledgerId, entryId)); + positions.add(PositionFactory.create(ledgerId, entryId)); sharedBucketPriorityQueue.pop(); removeIndexBit(ledgerId, entryId); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/RestMessagePublishContext.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/RestMessagePublishContext.java index 3c9adbd3e4fe4..f3b84090056be 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/RestMessagePublishContext.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/RestMessagePublishContext.java @@ -22,7 +22,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.service.Topic; /** @@ -33,7 +34,7 @@ public class RestMessagePublishContext implements Topic.PublishContext { private Topic topic; private long startTimeNs; - private CompletableFuture positionFuture; + private CompletableFuture positionFuture; /** * Executed from managed ledger thread when the message is persisted. @@ -54,13 +55,13 @@ public void completed(Exception exception, long ledgerId, long entryId) { topic.getName(), ledgerId, entryId); } topic.recordAddLatency(System.nanoTime() - startTimeNs, TimeUnit.NANOSECONDS); - positionFuture.complete(PositionImpl.get(ledgerId, entryId)); + positionFuture.complete(PositionFactory.create(ledgerId, entryId)); } recycle(); } // recycler - public static RestMessagePublishContext get(CompletableFuture positionFuture, Topic topic, + public static RestMessagePublishContext get(CompletableFuture positionFuture, Topic topic, long startTimeNs) { RestMessagePublishContext callback = RECYCLER.get(); callback.positionFuture = positionFuture; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java index 20c35b4f7769c..8f55df1107d0f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java @@ -53,7 +53,7 @@ import org.apache.avro.io.Decoder; import org.apache.avro.io.DecoderFactory; import org.apache.bookkeeper.mledger.ManagedLedgerException; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; @@ -194,7 +194,7 @@ private void internalPublishMessagesToPartition(TopicName topicName, ProducerMes String producerName = (null == request.getProducerName() || request.getProducerName().isEmpty()) ? defaultProducerName : request.getProducerName(); List messages = buildMessage(request, schema, producerName, topicName); - List> publishResults = new ArrayList<>(); + List> publishResults = new ArrayList<>(); List produceMessageResults = new ArrayList<>(); for (int index = 0; index < messages.size(); index++) { ProducerAck produceMessageResult = new ProducerAck(); @@ -235,7 +235,7 @@ private void internalPublishMessages(TopicName topicName, ProducerMessages reque String producerName = (null == request.getProducerName() || request.getProducerName().isEmpty()) ? defaultProducerName : request.getProducerName(); List messages = buildMessage(request, schema, producerName, topicName); - List> publishResults = new ArrayList<>(); + List> publishResults = new ArrayList<>(); List produceMessageResults = new ArrayList<>(); // Try to publish messages to all partitions this broker owns in round robin mode. for (int index = 0; index < messages.size(); index++) { @@ -266,8 +266,8 @@ private void internalPublishMessages(TopicName topicName, ProducerMessages reque } } - private CompletableFuture publishSingleMessageToPartition(String topic, Message message) { - CompletableFuture publishResult = new CompletableFuture<>(); + private CompletableFuture publishSingleMessageToPartition(String topic, Message message) { + CompletableFuture publishResult = new CompletableFuture<>(); pulsar().getBrokerService().getTopic(topic, false) .thenAccept(t -> { // TODO: Check message backlog and fail if backlog too large. @@ -297,11 +297,11 @@ private CompletableFuture publishSingleMessageToPartition(String t // Process results for all message publishing attempts private void processPublishMessageResults(List produceMessageResults, - List> publishResults) { + List> publishResults) { // process publish message result for (int index = 0; index < publishResults.size(); index++) { try { - PositionImpl position = publishResults.get(index).get(); + Position position = publishResults.get(index).get(); MessageId messageId = new MessageIdImpl(position.getLedgerId(), position.getEntryId(), Integer.parseInt(produceMessageResults.get(index).getMessageId())); produceMessageResults.get(index).setMessageId(messageId.toString()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java index 2f38ad67d4f30..fb5c457fcc874 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java @@ -32,7 +32,8 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.ServiceConfiguration; @@ -125,7 +126,7 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i int filteredEntryCount = 0; long filteredBytesCount = 0; List entriesToFiltered = hasFilter ? new ArrayList<>() : null; - List entriesToRedeliver = hasFilter ? new ArrayList<>() : null; + List entriesToRedeliver = hasFilter ? new ArrayList<>() : null; for (int i = 0, entriesSize = entries.size(); i < entriesSize; i++) { final Entry entry = entries.get(i); if (entry == null) { @@ -161,7 +162,7 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i entry.release(); continue; } else if (filterResult == EntryFilter.FilterResult.RESCHEDULE) { - entriesToRedeliver.add((PositionImpl) entry.getPosition()); + entriesToRedeliver.add(entry.getPosition()); entries.set(i, null); // FilterResult will be always `ACCEPTED` when there is No Filter // dont need to judge whether `hasFilter` is true or not. @@ -186,7 +187,7 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i } } else if (((PersistentTopic) subscription.getTopic()) .isTxnAborted(new TxnID(msgMetadata.getTxnidMostBits(), msgMetadata.getTxnidLeastBits()), - (PositionImpl) entry.getPosition())) { + entry.getPosition())) { individualAcknowledgeMessageIfNeeded(Collections.singletonList(entry.getPosition()), Collections.emptyMap()); entries.set(i, null); @@ -196,7 +197,7 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i } if (msgMetadata == null || (Markers.isServerOnlyMarker(msgMetadata))) { - PositionImpl pos = (PositionImpl) entry.getPosition(); + Position pos = entry.getPosition(); // Message metadata was corrupted or the messages was a server-only marker if (Markers.isReplicatedSubscriptionSnapshotMarker(msgMetadata)) { @@ -229,24 +230,25 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i int batchSize = msgMetadata.getNumMessagesInBatch(); long[] ackSet = null; if (indexesAcks != null && cursor != null) { - PositionImpl position = PositionImpl.get(entry.getLedgerId(), entry.getEntryId()); + Position position = PositionFactory.create(entry.getLedgerId(), entry.getEntryId()); ackSet = cursor .getDeletedBatchIndexesAsLongArray(position); // some batch messages ack bit sit will be in pendingAck state, so don't send all bit sit to consumer if (subscription instanceof PersistentSubscription && ((PersistentSubscription) subscription) .getPendingAckHandle() instanceof PendingAckHandleImpl) { - PositionImpl positionInPendingAck = + Position positionInPendingAck = ((PersistentSubscription) subscription).getPositionInPendingAck(position); // if this position not in pendingAck state, don't need to do any op if (positionInPendingAck != null) { - if (positionInPendingAck.hasAckSet()) { + long[] pendingAckSet = AckSetStateUtil.getAckSetArrayOrNull(positionInPendingAck); + if (pendingAckSet != null) { // need to or ackSet in pendingAck state and cursor ackSet which bit sit has been acked if (ackSet != null) { - ackSet = andAckSet(ackSet, positionInPendingAck.getAckSet()); + ackSet = andAckSet(ackSet, pendingAckSet); } else { // if actSet is null, use pendingAck ackSet - ackSet = positionInPendingAck.getAckSet(); + ackSet = pendingAckSet; } // if the result of pendingAckSet(in pendingAckHandle) AND the ackSet(in cursor) is empty // filter this entry @@ -347,7 +349,7 @@ protected boolean isConsumersExceededOnSubscription(AbstractTopic topic, int con && maxConsumersPerSubscription <= consumerSize; } - private void processReplicatedSubscriptionSnapshot(PositionImpl pos, ByteBuf headersAndPayload) { + private void processReplicatedSubscriptionSnapshot(Position pos, ByteBuf headersAndPayload) { // Remove the protobuf headers Commands.skipMessageMetadata(headersAndPayload); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java index c889062088e00..012cbcad1e26d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java @@ -28,8 +28,8 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursor.IndividualDeletedEntries; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.resources.NamespaceResources; @@ -226,8 +226,8 @@ private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuo } ManagedLedgerInfo.LedgerInfo ledgerInfo = mLedger.getLedgerInfo(oldestPosition.getLedgerId()).get(); if (ledgerInfo == null) { - PositionImpl nextPosition = - PositionImpl.get(mLedger.getNextValidLedger(oldestPosition.getLedgerId()), -1); + Position nextPosition = + PositionFactory.create(mLedger.getNextValidLedger(oldestPosition.getLedgerId()), -1); slowestConsumer.markDelete(nextPosition); continue; } @@ -235,8 +235,8 @@ private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuo if (ledgerInfo.getTimestamp() > 0 && currentMillis - ledgerInfo.getTimestamp() > SECONDS.toMillis(quota.getLimitTime())) { // skip whole ledger for the slowest cursor - PositionImpl nextPosition = - PositionImpl.get(mLedger.getNextValidLedger(ledgerInfo.getLedgerId()), -1); + Position nextPosition = + PositionFactory.create(mLedger.getNextValidLedger(ledgerInfo.getLedgerId()), -1); if (!nextPosition.equals(oldestPosition)) { slowestConsumer.markDelete(nextPosition); continue; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index 19711bfa718f4..02e21c44c9179 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -43,7 +43,8 @@ import lombok.Setter; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.authentication.AuthenticationDataSubscription; @@ -144,7 +145,7 @@ public class Consumer { private static final double avgPercent = 0.9; private boolean preciseDispatcherFlowControl; - private PositionImpl readPositionWhenJoining; + private Position readPositionWhenJoining; private final String clientAddress; // IP address only, no port number included private final MessageId startMessageId; private final boolean isAcknowledgmentAtBatchIndexLevelEnabled; @@ -490,20 +491,20 @@ public CompletableFuture messageAcked(CommandAck ack) { return CompletableFuture.completedFuture(null); } - PositionImpl position; + Position position; MessageIdData msgId = ack.getMessageIdAt(0); if (msgId.getAckSetsCount() > 0) { long[] ackSets = new long[msgId.getAckSetsCount()]; for (int j = 0; j < msgId.getAckSetsCount(); j++) { ackSets[j] = msgId.getAckSetAt(j); } - position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId(), ackSets); + position = AckSetStateUtil.createPositionWithAckSet(msgId.getLedgerId(), msgId.getEntryId(), ackSets); } else { - position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); + position = PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId()); } if (ack.hasTxnidMostBits() && ack.hasTxnidLeastBits()) { - List positionsAcked = Collections.singletonList(position); + List positionsAcked = Collections.singletonList(position); future = transactionCumulativeAcknowledge(ack.getTxnidMostBits(), ack.getTxnidLeastBits(), positionsAcked) .thenApply(unused -> 1L); @@ -534,7 +535,7 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map individualAckNormal(CommandAck ack, Map individualAckNormal(CommandAck ack, Map individualAckNormal(CommandAck ack, Map positionsAcked.forEach(position -> { //check if the position can remove from the consumer pending acks. // the bit set is empty in pending ack handle. - if (((PositionImpl) position).getAckSet() != null) { + if (AckSetStateUtil.hasAckSet(position)) { if (((PersistentSubscription) subscription) - .checkIsCanDeleteConsumerPendingAck((PositionImpl) position)) { - removePendingAcks((PositionImpl) position); + .checkIsCanDeleteConsumerPendingAck(position)) { + removePendingAcks(position); } } })); @@ -589,7 +590,7 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map individualAckWithTransaction(CommandAck ack) { // Individual ack - List> positionsAcked = new ArrayList<>(); + List> positionsAcked = new ArrayList<>(); if (!isTransactionEnabled()) { return FutureUtil.failedFuture( new BrokerServiceException.NotAllowedException("Server don't support transaction ack!")); @@ -598,7 +599,7 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { LongAdder totalAckCount = new LongAdder(); for (int i = 0; i < ack.getMessageIdsCount(); i++) { MessageIdData msgId = ack.getMessageIdAt(i); - PositionImpl position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); + Position position = AckSetStateUtil.createPositionWithAckSet(msgId.getLedgerId(), msgId.getEntryId(), null); // acked count at least one long ackedCount = 0; long batchSize = 0; @@ -618,7 +619,7 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { for (int j = 0; j < msgId.getAckSetsCount(); j++) { ackSets[j] = msgId.getAckSetAt(j); } - position.setAckSet(ackSets); + AckSetStateUtil.getAckSetState(position).setAckSet(ackSets); ackedCount = getAckedCountForTransactionAck(batchSize, ackSets); } @@ -636,7 +637,7 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { if (Subscription.isIndividualAckMode(subType)) { completableFuture.whenComplete((v, e) -> positionsAcked.forEach(positionLongMutablePair -> { - if (positionLongMutablePair.getLeft().getAckSet() != null) { + if (AckSetStateUtil.hasAckSet(positionLongMutablePair.getLeft())) { if (((PersistentSubscription) subscription) .checkIsCanDeleteConsumerPendingAck(positionLongMutablePair.left)) { removePendingAcks(positionLongMutablePair.left); @@ -665,7 +666,7 @@ private long getBatchSize(MessageIdData msgId) { return batchSize; } - private long getAckedCountForMsgIdNoAckSets(long batchSize, PositionImpl position, Consumer consumer) { + private long getAckedCountForMsgIdNoAckSets(long batchSize, Position position, Consumer consumer) { if (isAcknowledgmentAtBatchIndexLevelEnabled && Subscription.isIndividualAckMode(subType)) { long[] cursorAckSet = getCursorAckSet(position); if (cursorAckSet != null) { @@ -675,7 +676,7 @@ private long getAckedCountForMsgIdNoAckSets(long batchSize, PositionImpl positio return batchSize; } - private long getAckedCountForBatchIndexLevelEnabled(PositionImpl position, long batchSize, long[] ackSets, + private long getAckedCountForBatchIndexLevelEnabled(Position position, long batchSize, long[] ackSets, Consumer consumer) { long ackedCount = 0; if (isAcknowledgmentAtBatchIndexLevelEnabled && Subscription.isIndividualAckMode(subType) @@ -704,7 +705,7 @@ private long getAckedCountForTransactionAck(long batchSize, long[] ackSets) { return ackedCount; } - private long getUnAckedCountForBatchIndexLevelEnabled(PositionImpl position, long batchSize) { + private long getUnAckedCountForBatchIndexLevelEnabled(Position position, long batchSize) { long unAckedCount = batchSize; if (isAcknowledgmentAtBatchIndexLevelEnabled) { long[] cursorAckSet = getCursorAckSet(position); @@ -717,14 +718,14 @@ private long getUnAckedCountForBatchIndexLevelEnabled(PositionImpl position, lon return unAckedCount; } - private void checkAckValidationError(CommandAck ack, PositionImpl position) { + private void checkAckValidationError(CommandAck ack, Position position) { if (ack.hasValidationError()) { log.error("[{}] [{}] Received ack for corrupted message at {} - Reason: {}", subscription, consumerId, position, ack.getValidationError()); } } - private boolean checkCanRemovePendingAcksAndHandle(PositionImpl position, MessageIdData msgId) { + private boolean checkCanRemovePendingAcksAndHandle(Position position, MessageIdData msgId) { if (Subscription.isIndividualAckMode(subType) && msgId.getAckSetsCount() == 0) { return removePendingAcks(position); } @@ -746,7 +747,7 @@ private Consumer getAckOwnerConsumer(long ledgerId, long entryId) { return ackOwnerConsumer; } - private long[] getCursorAckSet(PositionImpl position) { + private long[] getCursorAckSet(Position position) { if (!(subscription instanceof PersistentSubscription)) { return null; } @@ -762,7 +763,7 @@ private boolean isTransactionEnabled() { private CompletableFuture transactionIndividualAcknowledge( long txnidMostBits, long txnidLeastBits, - List> positionList) { + List> positionList) { if (subscription instanceof PersistentSubscription) { TxnID txnID = new TxnID(txnidMostBits, txnidLeastBits); return ((PersistentSubscription) subscription).transactionIndividualAcknowledge(txnID, positionList); @@ -774,7 +775,7 @@ private CompletableFuture transactionIndividualAcknowledge( } private CompletableFuture transactionCumulativeAcknowledge(long txnidMostBits, long txnidLeastBits, - List positionList) { + List positionList) { if (!isTransactionEnabled()) { return FutureUtil.failedFuture( new BrokerServiceException.NotAllowedException("Server don't support transaction ack!")); @@ -1018,7 +1019,7 @@ public int hashCode() { * * @param position */ - private boolean removePendingAcks(PositionImpl position) { + private boolean removePendingAcks(Position position) { Consumer ackOwnedConsumer = null; if (pendingAcks.get(position.getLedgerId(), position.getEntryId()) == null) { for (Consumer consumer : subscription.getConsumers()) { @@ -1075,16 +1076,17 @@ public void redeliverUnacknowledgedMessages(long consumerEpoch) { } if (pendingAcks != null) { - List pendingPositions = new ArrayList<>((int) pendingAcks.size()); + List pendingPositions = new ArrayList<>((int) pendingAcks.size()); MutableInt totalRedeliveryMessages = new MutableInt(0); pendingAcks.forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { - int unAckedCount = (int) getUnAckedCountForBatchIndexLevelEnabled(PositionImpl.get(ledgerId, entryId), - batchSize); + int unAckedCount = + (int) getUnAckedCountForBatchIndexLevelEnabled(PositionFactory.create(ledgerId, entryId), + batchSize); totalRedeliveryMessages.add(unAckedCount); - pendingPositions.add(new PositionImpl(ledgerId, entryId)); + pendingPositions.add(PositionFactory.create(ledgerId, entryId)); }); - for (PositionImpl p : pendingPositions) { + for (Position p : pendingPositions) { pendingAcks.remove(p.getLedgerId(), p.getEntryId()); } @@ -1101,9 +1103,9 @@ public void redeliverUnacknowledgedMessages(long consumerEpoch) { public void redeliverUnacknowledgedMessages(List messageIds) { int totalRedeliveryMessages = 0; - List pendingPositions = new ArrayList<>(); + List pendingPositions = new ArrayList<>(); for (MessageIdData msg : messageIds) { - PositionImpl position = PositionImpl.get(msg.getLedgerId(), msg.getEntryId()); + Position position = PositionFactory.create(msg.getLedgerId(), msg.getEntryId()); LongPair longPair = pendingAcks.get(position.getLedgerId(), position.getEntryId()); if (longPair != null) { int unAckedCount = (int) getUnAckedCountForBatchIndexLevelEnabled(position, longPair.first); @@ -1164,7 +1166,7 @@ public boolean isPreciseDispatcherFlowControl() { return preciseDispatcherFlowControl; } - public void setReadPositionWhenJoining(PositionImpl readPositionWhenJoining) { + public void setReadPositionWhenJoining(Position readPositionWhenJoining) { this.readPositionWhenJoining = readPositionWhenJoining; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java index fcd4c52ee3795..d1d44709a9c52 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java @@ -23,7 +23,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.ManagedCursor; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; @@ -92,7 +92,7 @@ CompletableFuture disconnectAllConsumers(boolean isResetCursor, void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpoch); - void redeliverUnacknowledgedMessages(Consumer consumer, List positions); + void redeliverUnacknowledgedMessages(Consumer consumer, List positions); void addUnAckedMessages(int unAckMessages); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/InMemoryRedeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/InMemoryRedeliveryTracker.java index 8c992d2f7a90b..12e28793557b3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/InMemoryRedeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/InMemoryRedeliveryTracker.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap; import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap.LongPair; @@ -34,7 +33,7 @@ public class InMemoryRedeliveryTracker implements RedeliveryTracker { @Override public int incrementAndGetRedeliveryCount(Position position) { - PositionImpl positionImpl = (PositionImpl) position; + Position positionImpl = position; LongPair count = trackerCache.get(positionImpl.getLedgerId(), positionImpl.getEntryId()); int newCount = (int) (count != null ? count.first + 1 : 1); trackerCache.put(positionImpl.getLedgerId(), positionImpl.getEntryId(), newCount, 0L); @@ -49,7 +48,7 @@ public int getRedeliveryCount(long ledgerId, long entryId) { @Override public void remove(Position position) { - PositionImpl positionImpl = (PositionImpl) position; + Position positionImpl = position; trackerCache.remove(positionImpl.getLedgerId(), positionImpl.getEntryId()); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java index c10e33818ed3a..cf54ffea7db66 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java @@ -390,11 +390,6 @@ private static final class MessagePublishContext implements PublishContext, Runn private long entryTimestamp; - @Override - public Position getNext() { - return null; - } - @Override public long getLedgerId() { return ledgerId; 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 dc1cf913ab240..6901097bbbb27 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 @@ -66,8 +66,9 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.pulsar.broker.PulsarServerException; @@ -1906,7 +1907,7 @@ protected void handleSend(CommandSend send, ByteBuf headersAndPayload) { // This position is only used for shadow replicator Position position = send.hasMessageId() - ? PositionImpl.get(send.getMessageId().getLedgerId(), send.getMessageId().getEntryId()) : null; + ? PositionFactory.create(send.getMessageId().getLedgerId(), send.getMessageId().getEntryId()) : null; // Persist the message if (send.hasHighestSequenceId() && send.getSequenceId() <= send.getHighestSequenceId()) { @@ -2075,7 +2076,7 @@ protected void handleSeek(CommandSeek seek) { } } - Position position = new PositionImpl(msgIdData.getLedgerId(), + Position position = AckSetStateUtil.createPositionWithAckSet(msgIdData.getLedgerId(), msgIdData.getEntryId(), ackSet); @@ -2249,8 +2250,8 @@ protected void handleGetLastMessageId(CommandGetLastMessageId getLastMessageId) getLargestBatchIndexWhenPossible( topic, - (PositionImpl) lastPosition, - (PositionImpl) markDeletePosition, + lastPosition, + markDeletePosition, partitionIndex, requestId, consumer.getSubscription().getName(), @@ -2269,8 +2270,8 @@ protected void handleGetLastMessageId(CommandGetLastMessageId getLastMessageId) private void getLargestBatchIndexWhenPossible( Topic topic, - PositionImpl lastPosition, - PositionImpl markDeletePosition, + Position lastPosition, + Position markDeletePosition, int partitionIndex, long requestId, String subscriptionName, @@ -2307,7 +2308,7 @@ private void getLargestBatchIndexWhenPossible( return; } - if (compactionHorizon != null && lastPosition.compareTo((PositionImpl) compactionHorizon) <= 0) { + if (compactionHorizon != null && lastPosition.compareTo(compactionHorizon) <= 0) { handleLastMessageIdFromCompactionService(persistentTopic, requestId, partitionIndex, markDeletePosition); return; @@ -2368,7 +2369,7 @@ public String toString() { }); } private void handleLastMessageIdFromCompactionService(PersistentTopic persistentTopic, long requestId, - int partitionIndex, PositionImpl markDeletePosition) { + int partitionIndex, Position markDeletePosition) { persistentTopic.getTopicCompactionService().readLastCompactedEntry().thenAccept(entry -> { if (entry != null) { try { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java index 61107b7b0dbb3..452c30b45febb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java @@ -24,7 +24,6 @@ import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.common.api.proto.CommandAck.AckType; @@ -89,7 +88,7 @@ default long getNumberOfEntriesDelayed() { void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpoch); - void redeliverUnacknowledgedMessages(Consumer consumer, List positions); + void redeliverUnacknowledgedMessages(Consumer consumer, List positions); void markTopicWithBatchMessagePublished(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcher.java index 0a8f254f12189..af14fad0ee24a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcher.java @@ -20,7 +20,7 @@ import java.util.List; import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.common.stats.Rate; @@ -40,7 +40,7 @@ default void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpo } @Override - default void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { + default void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { // No-op } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java index cfe05cc32b77d..e92eef5cb7bff 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java @@ -27,7 +27,6 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.CollectionUtils; import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; @@ -521,7 +520,7 @@ public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, long } @Override - public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { + public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { // No-op } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java index 4ef2710fea61a..1314b2d2ed06b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java @@ -24,7 +24,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.client.api.transaction.TxnID; @@ -92,7 +91,7 @@ protected boolean replicateEntries(List entries) { if (msg.getMessageBuilder().hasTxnidLeastBits() && msg.getMessageBuilder().hasTxnidMostBits()) { TxnID tx = new TxnID(msg.getMessageBuilder().getTxnidMostBits(), msg.getMessageBuilder().getTxnidLeastBits()); - if (topic.isTxnAborted(tx, (PositionImpl) entry.getPosition())) { + if (topic.isTxnAborted(tx, entry.getPosition())) { cursor.asyncDelete(entry.getPosition(), this, entry.getPosition()); entry.release(); msg.recycle(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java index ab3b799093be6..9d970479400ba 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java @@ -37,7 +37,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.Topic.PublishContext; import org.apache.pulsar.common.api.proto.MessageMetadata; @@ -205,7 +205,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { future.completeExceptionally(exception); } - }, null, PositionImpl.LATEST); + }, null, PositionFactory.LATEST); } public Status getStatus() { @@ -403,7 +403,7 @@ public MessageDupStatus isDuplicate(PublishContext publishContext, ByteBuf heade /** * Call this method whenever a message is persisted to get the chance to trigger a snapshot. */ - public void recordMessagePersisted(PublishContext publishContext, PositionImpl position) { + public void recordMessagePersisted(PublishContext publishContext, Position position) { if (!isEnabled() || publishContext.isMarkerMessage()) { return; } @@ -547,11 +547,11 @@ public void takeSnapshot() { || currentTimeStamp - lastSnapshotTimestamp < TimeUnit.SECONDS.toMillis(interval)) { return; } - PositionImpl position = (PositionImpl) managedLedger.getLastConfirmedEntry(); + Position position = managedLedger.getLastConfirmedEntry(); if (position == null) { return; } - PositionImpl markDeletedPosition = (PositionImpl) managedCursor.getMarkDeletedPosition(); + Position markDeletedPosition = managedCursor.getMarkDeletedPosition(); if (markDeletedPosition != null && position.compareTo(markDeletedPosition) <= 0) { return; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java index 6380317724207..526874a7ae34b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java @@ -24,7 +24,8 @@ import java.util.NavigableSet; import java.util.Set; import javax.annotation.concurrent.NotThreadSafe; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.util.collections.ConcurrentLongLongHashMap; import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap; import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap.LongPair; @@ -145,7 +146,7 @@ public boolean containsStickyKeyHashes(Set stickyKeyHashes) { return false; } - public NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { - return messagesToRedeliver.items(maxMessagesToRead, PositionImpl::new); + public NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { + return messagesToRedeliver.items(maxMessagesToRead, PositionFactory::create); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index f20750fa0c20d..e64fe18a5f118 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -44,7 +44,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.NoMoreEntriesToReadException; import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.delayed.DelayedDeliveryTracker; @@ -87,7 +87,7 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul protected final PersistentTopic topic; protected final ManagedCursor cursor; - protected volatile Range lastIndividualDeletedRangeFromCursorRecovery; + protected volatile Range lastIndividualDeletedRangeFromCursorRecovery; private CompletableFuture closeFuture = null; protected final MessageRedeliveryController redeliveryMessages; @@ -97,7 +97,7 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul protected volatile boolean havePendingRead = false; protected volatile boolean havePendingReplayRead = false; - protected volatile PositionImpl minReplayedPosition = null; + protected volatile Position minReplayedPosition = null; protected boolean shouldRewindBeforeReadingOrReplaying = false; protected final String name; private boolean sendInProgress = false; @@ -333,8 +333,8 @@ public synchronized void readMoreEntries() { return; } - NavigableSet messagesToReplayNow = getMessagesToReplayNow(messagesToRead); - NavigableSet messagesToReplayFiltered = filterOutEntriesWillBeDiscarded(messagesToReplayNow); + NavigableSet messagesToReplayNow = getMessagesToReplayNow(messagesToRead); + NavigableSet messagesToReplayFiltered = filterOutEntriesWillBeDiscarded(messagesToReplayNow); if (!messagesToReplayFiltered.isEmpty()) { if (log.isDebugEnabled()) { log.debug("[{}] Schedule replay of {} messages for {} consumers", name, @@ -348,8 +348,8 @@ public synchronized void readMoreEntries() { : asyncReplayEntries(messagesToReplayFiltered); // clear already acked positions from replay bucket - deletedMessages.forEach(position -> redeliveryMessages.remove(((PositionImpl) position).getLedgerId(), - ((PositionImpl) position).getEntryId())); + deletedMessages.forEach(position -> redeliveryMessages.remove(position.getLedgerId(), + position.getEntryId())); // if all the entries are acked-entries and cleared up from redeliveryMessages, try to read // next entries as readCompletedEntries-callback was never called if ((messagesToReplayFiltered.size() - deletedMessages.size()) == 0) { @@ -374,7 +374,7 @@ public synchronized void readMoreEntries() { consumerList.size()); } havePendingRead = true; - NavigableSet toReplay = getMessagesToReplayNow(1); + NavigableSet toReplay = getMessagesToReplayNow(1); if (!toReplay.isEmpty()) { minReplayedPosition = toReplay.first(); redeliveryMessages.add(minReplayedPosition.getLedgerId(), minReplayedPosition.getEntryId()); @@ -384,7 +384,7 @@ public synchronized void readMoreEntries() { // Filter out and skip read delayed messages exist in DelayedDeliveryTracker if (delayedDeliveryTracker.isPresent()) { - Predicate skipCondition = null; + Predicate skipCondition = null; final DelayedDeliveryTracker deliveryTracker = delayedDeliveryTracker.get(); if (deliveryTracker instanceof BucketDelayedDeliveryTracker) { skipCondition = position -> ((BucketDelayedDeliveryTracker) deliveryTracker) @@ -928,7 +928,7 @@ public synchronized void readEntriesFailed(ManagedLedgerException exception, Obj } else { havePendingReplayRead = false; if (exception instanceof ManagedLedgerException.InvalidReplayPositionException) { - PositionImpl markDeletePosition = (PositionImpl) cursor.getMarkDeletedPosition(); + Position markDeletePosition = cursor.getMarkDeletedPosition(); redeliveryMessages.removeAllUpTo(markDeletePosition.getLedgerId(), markDeletePosition.getEntryId()); } } @@ -956,7 +956,7 @@ private boolean needTrimAckedMessages() { return false; } else { return lastIndividualDeletedRangeFromCursorRecovery.upperEndpoint() - .compareTo((PositionImpl) cursor.getReadPosition()) > 0; + .compareTo(cursor.getReadPosition()) > 0; } } @@ -1007,7 +1007,7 @@ public boolean isConsumerAvailable(Consumer consumer) { public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpoch) { consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { if (addMessageToReplay(ledgerId, entryId, stickyKeyHash)) { - redeliveryTracker.incrementAndGetRedeliveryCount((PositionImpl.get(ledgerId, entryId))); + redeliveryTracker.incrementAndGetRedeliveryCount((PositionFactory.create(ledgerId, entryId))); } }); if (log.isDebugEnabled()) { @@ -1018,7 +1018,7 @@ public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, long } @Override - public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { + public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { positions.forEach(position -> { // TODO: We want to pass a sticky key hash as a third argument to guarantee the order of the messages // on Key_Shared subscription, but it's difficult to get the sticky key here @@ -1183,10 +1183,10 @@ public boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata } } - protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { + protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { if (delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().hasMessageAvailable()) { delayedDeliveryTracker.get().resetTickTime(topic.getDelayedDeliveryTickTimeMillis()); - NavigableSet messagesAvailableNow = + NavigableSet messagesAvailableNow = delayedDeliveryTracker.get().getScheduledMessages(maxMessagesToRead); messagesAvailableNow.forEach(p -> redeliveryMessages.add(p.getLedgerId(), p.getEntryId())); } @@ -1207,7 +1207,7 @@ protected synchronized NavigableSet getMessagesToReplayNow(int max * - The order guarantee mechanism of Key_Shared mode filtered out all the entries. * - Delivery non entry to the client, but we did a BK read. */ - protected NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + protected NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { return src; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index adaa5a66a0cbe..600fbb26eb511 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -36,7 +36,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.ConcurrentWaitCallbackException; import org.apache.bookkeeper.mledger.ManagedLedgerException.NoMoreEntriesToReadException; import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.service.AbstractDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.Consumer; @@ -308,7 +308,7 @@ private synchronized void internalRedeliverUnacknowledgedMessages(Consumer consu } @Override - public void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { + public void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { // We cannot redeliver single messages to single consumers to preserve ordering. redeliverUnacknowledgedMessages(consumer, DEFAULT_CONSUMER_EPOCH); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java index 2478a7a2538d0..9a8a39c8e9a12 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java @@ -32,9 +32,9 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.LedgerNotExistException; import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.pulsar.broker.service.MessageExpirer; import org.apache.pulsar.client.impl.MessageImpl; @@ -128,8 +128,8 @@ private void checkExpiryByLedgerClosureTime(ManagedCursor cursor, int messageTTL info = ledgerInfo; } if (info != null && info.getLedgerId() > -1) { - PositionImpl position = PositionImpl.get(info.getLedgerId(), info.getEntries() - 1); - if (((PositionImpl) managedLedger.getLastConfirmedEntry()).compareTo(position) < 0) { + Position position = PositionFactory.create(info.getLedgerId(), info.getEntries() - 1); + if (managedLedger.getLastConfirmedEntry().compareTo(position) < 0) { findEntryComplete(managedLedger.getLastConfirmedEntry(), null); } else { findEntryComplete(position, null); @@ -141,8 +141,8 @@ private void checkExpiryByLedgerClosureTime(ManagedCursor cursor, int messageTTL @Override public boolean expireMessages(Position messagePosition) { // If it's beyond last position of this topic, do nothing. - PositionImpl topicLastPosition = (PositionImpl) this.topic.getLastPosition(); - if (topicLastPosition.compareTo((PositionImpl) messagePosition) < 0) { + Position topicLastPosition = this.topic.getLastPosition(); + if (topicLastPosition.compareTo(messagePosition) < 0) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Ignore expire-message scheduled task, given position {} is beyond " + "current topic's last position {}", topicName, subName, messagePosition, @@ -157,7 +157,7 @@ public boolean expireMessages(Position messagePosition) { cursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchActiveEntries, entry -> { try { // If given position larger than entry position. - return ((PositionImpl) entry.getPosition()).compareTo((PositionImpl) messagePosition) <= 0; + return entry.getPosition().compareTo(messagePosition) <= 0; } finally { entry.release(); } @@ -242,16 +242,16 @@ public void findEntryFailed(ManagedLedgerException exception, Optional long failedLedgerId = failedReadPosition.get().getLedgerId(); ManagedLedgerImpl ledger = ((ManagedLedgerImpl) cursor.getManagedLedger()); Position lastPositionInLedger = ledger.getOptionalLedgerInfo(failedLedgerId) - .map(ledgerInfo -> PositionImpl.get(failedLedgerId, ledgerInfo.getEntries() - 1)) + .map(ledgerInfo -> PositionFactory.create(failedLedgerId, ledgerInfo.getEntries() - 1)) .orElseGet(() -> { Long nextExistingLedger = ledger.getNextValidLedger(failedReadPosition.get().getLedgerId()); if (nextExistingLedger == null) { log.info("[{}] [{}] Couldn't find next next valid ledger for expiry monitor when find " + "entry failed {}", ledger.getName(), ledger.getName(), failedReadPosition); - return (PositionImpl) failedReadPosition.get(); + return failedReadPosition.get(); } else { - return PositionImpl.get(nextExistingLedger, -1); + return PositionFactory.create(nextExistingLedger, -1); } }); log.info("[{}][{}] ledger not existed, will complete the last position of the non-existed" diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index c3a27a15e9d90..6263c512997fa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -45,7 +45,6 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.CursorAlreadyClosedException; import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.AbstractReplicator; @@ -576,11 +575,11 @@ public void deleteFailed(ManagedLedgerException exception, Object ctx) { terminate(); return; } - if (ctx instanceof PositionImpl) { - PositionImpl deletedEntry = (PositionImpl) ctx; - if (deletedEntry.compareTo((PositionImpl) cursor.getMarkDeletedPosition()) > 0) { + if (ctx instanceof Position) { + Position deletedEntry = (Position) ctx; + if (deletedEntry.compareTo(cursor.getMarkDeletedPosition()) > 0) { brokerService.getPulsar().getExecutor().schedule( - () -> cursor.asyncDelete(deletedEntry, (PersistentReplicator) this, deletedEntry), 10, + () -> cursor.asyncDelete(deletedEntry, this, deletedEntry), 10, TimeUnit.SECONDS); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 2df9f38531f5d..766f45ad9908c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -38,7 +38,6 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.MapUtils; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -72,7 +71,7 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi * This means that, in order to preserve ordering, new consumers can only receive old * messages, until the mark-delete position will move past this point. */ - private final LinkedHashMap recentlyJoinedConsumers; + private final LinkedHashMap recentlyJoinedConsumers; PersistentStickyKeyDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor, Subscription subscription, ServiceConfiguration conf, KeySharedMeta ksm) { @@ -125,7 +124,7 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { }) ).thenRun(() -> { synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) { - PositionImpl readPositionWhenJoining = (PositionImpl) cursor.getReadPosition(); + Position readPositionWhenJoining = cursor.getReadPosition(); consumer.setReadPositionWhenJoining(readPositionWhenJoining); // If this was the 1st consumer, or if all the messages are already acked, then we // don't need to do anything special @@ -168,10 +167,10 @@ protected Map> initialValue() throws Exception { } }; - private static final FastThreadLocal>> localGroupedPositions = - new FastThreadLocal>>() { + private static final FastThreadLocal>> localGroupedPositions = + new FastThreadLocal>>() { @Override - protected Map> initialValue() throws Exception { + protected Map> initialValue() throws Exception { return new HashMap<>(); } }; @@ -197,9 +196,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis // A corner case that we have to retry a readMoreEntries in order to preserver order delivery. // This may happen when consumer closed. See issue #12885 for details. if (!allowOutOfOrderDelivery) { - NavigableSet messagesToReplayNow = this.getMessagesToReplayNow(1); + NavigableSet messagesToReplayNow = this.getMessagesToReplayNow(1); if (messagesToReplayNow != null && !messagesToReplayNow.isEmpty()) { - PositionImpl replayPosition = messagesToReplayNow.first(); + Position replayPosition = messagesToReplayNow.first(); // We have received a message potentially from the delayed tracker and, since we're not using it // right now, it needs to be added to the redelivery tracker or we won't attempt anymore to @@ -352,7 +351,7 @@ private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List= 0) { + if ((entries.get(i)).compareTo(maxReadPosition) >= 0) { // We have already crossed the divider line. All messages in the list are now // newer than what we can currently dispatch to this consumer return i; @@ -413,14 +412,14 @@ private boolean removeConsumersFromRecentJoinedConsumers() { if (MapUtils.isEmpty(recentlyJoinedConsumers)) { return false; } - Iterator> itr = recentlyJoinedConsumers.entrySet().iterator(); + Iterator> itr = recentlyJoinedConsumers.entrySet().iterator(); boolean hasConsumerRemovedFromTheRecentJoinedConsumers = false; - PositionImpl mdp = (PositionImpl) cursor.getMarkDeletedPosition(); + Position mdp = cursor.getMarkDeletedPosition(); if (mdp != null) { - PositionImpl nextPositionOfTheMarkDeletePosition = + Position nextPositionOfTheMarkDeletePosition = ((ManagedLedgerImpl) cursor.getManagedLedger()).getNextValidPosition(mdp); while (itr.hasNext()) { - Map.Entry entry = itr.next(); + Map.Entry entry = itr.next(); if (entry.getValue().compareTo(nextPositionOfTheMarkDeletePosition) <= 0) { itr.remove(); hasConsumerRemovedFromTheRecentJoinedConsumers = true; @@ -433,7 +432,7 @@ private boolean removeConsumersFromRecentJoinedConsumers() { } @Override - protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { + protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { if (isDispatcherStuckOnReplays) { // If we're stuck on replay, we want to move forward reading on the topic (until the overall max-unacked // messages kicks in), instead of keep replaying the same old messages, since the consumer that these @@ -456,7 +455,7 @@ private int getAvailablePermits(Consumer c) { } @Override - protected synchronized NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + protected synchronized NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", // So skip this filter out. if (isAllowOutOfOrderDelivery()) { @@ -465,11 +464,11 @@ protected synchronized NavigableSet filterOutEntriesWillBeDiscarde if (src.isEmpty()) { return src; } - NavigableSet res = new TreeSet<>(); + NavigableSet res = new TreeSet<>(); // Group positions. - final Map> groupedPositions = localGroupedPositions.get(); + final Map> groupedPositions = localGroupedPositions.get(); groupedPositions.clear(); - for (PositionImpl pos : src) { + for (Position pos : src) { Long stickyKeyHash = redeliveryMessages.getHash(pos.getLedgerId(), pos.getEntryId()); if (stickyKeyHash == null) { res.add(pos); @@ -483,7 +482,7 @@ protected synchronized NavigableSet filterOutEntriesWillBeDiscarde groupedPositions.computeIfAbsent(c, k -> new ArrayList<>()).add(pos); } // Filter positions by the Recently Joined Position rule. - for (Map.Entry> item : groupedPositions.entrySet()) { + for (Map.Entry> item : groupedPositions.entrySet()) { int availablePermits = getAvailablePermits(item.getKey()); if (availablePermits == 0) { continue; @@ -548,7 +547,7 @@ public boolean hasSameKeySharedPolicy(KeySharedMeta ksm) { && ksm.isAllowOutOfOrderDelivery() == this.allowOutOfOrderDelivery); } - public LinkedHashMap getRecentlyJoinedConsumers() { + public LinkedHashMap getRecentlyJoinedConsumers() { return recentlyJoinedConsumers; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index dbbf92aa76dce..7da339a420c89 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -53,7 +53,6 @@ import org.apache.bookkeeper.mledger.ScanOutcome; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.ServiceConfiguration; @@ -411,7 +410,7 @@ public void acknowledgeMessage(List positions, AckType ackType, Map c.localSubscriptionUpdated(subName, snapshot)); @@ -429,19 +428,19 @@ public void acknowledgeMessage(List positions, AckType ackType, Map transactionIndividualAcknowledge( TxnID txnId, - List> positions) { + List> positions) { return pendingAckHandle.individualAcknowledgeMessage(txnId, positions); } - public CompletableFuture transactionCumulativeAcknowledge(TxnID txnId, List positions) { + public CompletableFuture transactionCumulativeAcknowledge(TxnID txnId, List positions) { return pendingAckHandle.cumulativeAcknowledgeMessage(txnId, positions); } private final MarkDeleteCallback markDeleteCallback = new MarkDeleteCallback() { @Override public void markDeleteComplete(Object ctx) { - PositionImpl oldMD = (PositionImpl) ctx; - PositionImpl newMD = (PositionImpl) cursor.getMarkDeletedPosition(); + Position oldMD = (Position) ctx; + Position newMD = cursor.getMarkDeletedPosition(); if (log.isDebugEnabled()) { log.debug("[{}][{}] Mark deleted messages to position {} from position {}", topicName, subName, newMD, oldMD); @@ -478,7 +477,7 @@ public void deleteComplete(Object context) { if (dispatcher != null) { dispatcher.afterAckMessages(null, context); } - notifyTheMarkDeletePositionMoveForwardIfNeeded((PositionImpl) context); + notifyTheMarkDeletePositionMoveForwardIfNeeded((Position) context); } @Override @@ -492,8 +491,8 @@ public void deleteFailed(ManagedLedgerException exception, Object ctx) { }; private void notifyTheMarkDeletePositionMoveForwardIfNeeded(Position oldPosition) { - PositionImpl oldMD = (PositionImpl) oldPosition; - PositionImpl newMD = (PositionImpl) cursor.getMarkDeletedPosition(); + Position oldMD = oldPosition; + Position newMD = cursor.getMarkDeletedPosition(); if (dispatcher != null && newMD.compareTo(oldMD) > 0) { dispatcher.markDeletePositionMoveForward(); } @@ -837,7 +836,7 @@ private void resetCursor(Position finalPosition, CompletableFuture future) forceReset.complete(false); } else { topic.getTopicCompactionService().getLastCompactedPosition().thenAccept(lastCompactedPosition -> { - PositionImpl resetTo = (PositionImpl) finalPosition; + Position resetTo = finalPosition; if (lastCompactedPosition != null && resetTo.compareTo(lastCompactedPosition.getLedgerId(), lastCompactedPosition.getEntryId()) <= 0) { forceReset.complete(true); @@ -1268,14 +1267,14 @@ public SubscriptionStatsImpl getStats(GetStatsOptions getStatsOptions) { subStats.msgBacklog = getNumberOfEntriesInBacklog(getStatsOptions.isGetPreciseBacklog()); if (getStatsOptions.isSubscriptionBacklogSize()) { subStats.backlogSize = ((ManagedLedgerImpl) topic.getManagedLedger()) - .getEstimatedBacklogSize((PositionImpl) cursor.getMarkDeletedPosition()); + .getEstimatedBacklogSize(cursor.getMarkDeletedPosition()); } else { subStats.backlogSize = -1; } if (getStatsOptions.isGetEarliestTimeInBacklog()) { if (subStats.msgBacklog > 0) { ManagedLedgerImpl managedLedger = ((ManagedLedgerImpl) cursor.getManagedLedger()); - PositionImpl markDeletedPosition = (PositionImpl) cursor.getMarkDeletedPosition(); + Position markDeletedPosition = cursor.getMarkDeletedPosition(); long result = 0; try { result = managedLedger.getEarliestMessagePublishTimeOfPos(markDeletedPosition).get(); @@ -1300,7 +1299,7 @@ public SubscriptionStatsImpl getStats(GetStatsOptions getStatsOptions) { subStats.allowOutOfOrderDelivery = keySharedDispatcher.isAllowOutOfOrderDelivery(); subStats.keySharedMode = keySharedDispatcher.getKeySharedMode().toString(); - LinkedHashMap recentlyJoinedConsumers = keySharedDispatcher + LinkedHashMap recentlyJoinedConsumers = keySharedDispatcher .getRecentlyJoinedConsumers(); if (recentlyJoinedConsumers != null && recentlyJoinedConsumers.size() > 0) { recentlyJoinedConsumers.forEach((k, v) -> { @@ -1323,16 +1322,16 @@ public void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpoc } @Override - public void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { + public void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { Dispatcher dispatcher = getDispatcher(); if (dispatcher != null) { dispatcher.redeliverUnacknowledgedMessages(consumer, positions); } } - private void trimByMarkDeletePosition(List positions) { + private void trimByMarkDeletePosition(List positions) { positions.removeIf(position -> cursor.getMarkDeletedPosition() != null - && position.compareTo((PositionImpl) cursor.getMarkDeletedPosition()) <= 0); + && position.compareTo(cursor.getMarkDeletedPosition()) <= 0); } @Override @@ -1375,7 +1374,7 @@ public Map getSubscriptionProperties() { return subscriptionProperties; } - public PositionImpl getPositionInPendingAck(PositionImpl position) { + public Position getPositionInPendingAck(Position position) { return pendingAckHandle.getPositionInPendingAck(position); } @Override @@ -1445,11 +1444,11 @@ public PendingAckHandle getPendingAckHandle() { return pendingAckHandle; } - public void syncBatchPositionBitSetForPendingAck(PositionImpl position) { + public void syncBatchPositionBitSetForPendingAck(Position position) { this.pendingAckHandle.syncBatchPositionAckSetForTransaction(position); } - public boolean checkIsCanDeleteConsumerPendingAck(PositionImpl position) { + public boolean checkIsCanDeleteConsumerPendingAck(Position position) { return this.pendingAckHandle.checkIsCanDeleteConsumerPendingAck(position); } @@ -1477,7 +1476,7 @@ public boolean checkIfPendingAckStoreInit() { return this.pendingAckHandle.checkIfPendingAckStoreInit(); } - public PositionInPendingAckStats checkPositionInPendingAckState(PositionImpl position, Integer batchIndex) { + public PositionInPendingAckStats checkPositionInPendingAckState(Position position, Integer batchIndex) { return pendingAckHandle.checkPositionInPendingAckState(position, batchIndex); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index d9f9c4689f6ed..50129ebee0ad0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -79,11 +79,11 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.MetadataNotFoundException; import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer.CursorInfo; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.impl.ShadowManagedLedgerImpl; import org.apache.bookkeeper.mledger.util.Futures; import org.apache.bookkeeper.mledger.util.ManagedLedgerImplUtils; @@ -303,7 +303,7 @@ protected TopicStatsHelper initialValue() { "timeBasedBacklogQuotaCheckResult"); @Value private static class TimeBasedBacklogQuotaCheckResult { - PositionImpl oldestCursorMarkDeletePosition; + Position oldestCursorMarkDeletePosition; String cursorName; long positionPublishTimestampInMillis; long dataVersion; @@ -423,7 +423,7 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS } else { this.transactionBuffer = new TransactionBufferDisable(this); } - transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry(), true); + transactionBuffer.syncMaxReadPositionForNormalPublish(ledger.getLastConfirmedEntry(), true); if (ledger instanceof ShadowManagedLedgerImpl) { shadowSourceTopic = TopicName.get(ledger.getConfig().getShadowSource()); } else { @@ -681,7 +681,7 @@ private void asyncAddEntry(ByteBuf headersAndPayload, PublishContext publishCont } } - public void asyncReadEntry(PositionImpl position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { + public void asyncReadEntry(Position position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { if (ledger instanceof ManagedLedgerImpl) { ((ManagedLedgerImpl) ledger).asyncReadEntry(position, callback, ctx); } else { @@ -691,7 +691,7 @@ public void asyncReadEntry(PositionImpl position, AsyncCallbacks.ReadEntryCallba } } - public PositionImpl getPositionAfterN(PositionImpl startPosition, long n) throws ManagedLedgerException { + public Position getPositionAfterN(Position startPosition, long n) throws ManagedLedgerException { if (ledger instanceof ManagedLedgerImpl) { return ((ManagedLedgerImpl) ledger).getPositionAfterN(startPosition, n, ManagedLedgerImpl.PositionBound.startExcluded); @@ -701,7 +701,7 @@ public PositionImpl getPositionAfterN(PositionImpl startPosition, long n) throws } } - public PositionImpl getFirstPosition() throws ManagedLedgerException { + public Position getFirstPosition() throws ManagedLedgerException { if (ledger instanceof ManagedLedgerImpl) { return ((ManagedLedgerImpl) ledger).getFirstPosition(); } else { @@ -738,13 +738,13 @@ private void updateMaxReadPositionMovedForwardTimestamp() { @Override public void addComplete(Position pos, ByteBuf entryData, Object ctx) { PublishContext publishContext = (PublishContext) ctx; - PositionImpl position = (PositionImpl) pos; + Position position = pos; // Message has been successfully persisted messageDeduplication.recordMessagePersisted(publishContext, position); // in order to sync the max position when cursor read entries - transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry(), + transactionBuffer.syncMaxReadPositionForNormalPublish(ledger.getLastConfirmedEntry(), publishContext.isMarkerMessage()); publishContext.setMetadataFromEntryData(entryData); publishContext.completed(null, position.getLedgerId(), position.getEntryId()); @@ -1204,7 +1204,7 @@ private CompletableFuture getNonDurableSubscription(Stri entryId = msgId.getEntryId() - 1; } - Position startPosition = new PositionImpl(ledgerId, entryId); + Position startPosition = PositionFactory.create(ledgerId, entryId); ManagedCursor cursor = null; try { cursor = ledger.newNonDurableCursor(startPosition, subscriptionName, initialPosition, @@ -2146,7 +2146,7 @@ CompletableFuture startShadowReplicator(String shadowTopic) { String name = ShadowReplicator.getShadowReplicatorName(replicatorPrefix, shadowTopic); ManagedCursor cursor; try { - cursor = ledger.newNonDurableCursor(PositionImpl.LATEST, name); + cursor = ledger.newNonDurableCursor(PositionFactory.LATEST, name); } catch (ManagedLedgerException e) { log.error("[{}]Open non-durable cursor for shadow replicator failed, name={}", topic, name, e); return FutureUtil.failedFuture(e); @@ -3517,7 +3517,7 @@ public CompletableFuture checkTimeBacklogExceeded() { return CompletableFuture.completedFuture(false); } - PositionImpl oldestMarkDeletePosition = oldestMarkDeleteCursorInfo.getPosition(); + Position oldestMarkDeletePosition = oldestMarkDeleteCursorInfo.getPosition(); TimeBasedBacklogQuotaCheckResult lastCheckResult = timeBasedBacklogQuotaCheckResult; if (lastCheckResult != null @@ -3556,7 +3556,7 @@ public CompletableFuture checkTimeBacklogExceeded() { CompletableFuture future = new CompletableFuture<>(); // Check if first unconsumed message(first message after mark delete position) // for slowest cursor's has expired. - PositionImpl position = ((ManagedLedgerImpl) ledger).getNextValidPosition(oldestMarkDeletePosition); + Position position = ((ManagedLedgerImpl) ledger).getNextValidPosition(oldestMarkDeletePosition); ((ManagedLedgerImpl) ledger).asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { @Override @@ -3626,7 +3626,7 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { } private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaCheck( - PositionImpl markDeletePosition) + Position markDeletePosition) throws ExecutionException, InterruptedException { int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) ledger; @@ -3646,7 +3646,7 @@ private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaC // if the mark-delete position is the last entry it means all entries for // that ledger are acknowledged if (markDeletePosition.getEntryId() == markDeletePositionLedgerInfo.getEntries() - 1) { - PositionImpl positionToCheck = managedLedger.getNextValidPosition(markDeletePosition); + Position positionToCheck = managedLedger.getNextValidPosition(markDeletePosition); positionToCheckLedgerInfo = ledger.getLedgerInfo(positionToCheck.getLedgerId()).get(); } @@ -3687,7 +3687,7 @@ public void terminateComplete(Position lastCommittedPosition, Object ctx) { producers.values().forEach(Producer::disconnect); subscriptions.forEach((name, sub) -> sub.topicTerminated()); - PositionImpl lastPosition = (PositionImpl) lastCommittedPosition; + Position lastPosition = lastCommittedPosition; MessageId messageId = new MessageIdImpl(lastPosition.getLedgerId(), lastPosition.getEntryId(), -1); log.info("[{}] Topic terminated at {}", getName(), messageId); @@ -3814,7 +3814,7 @@ public CompletableFuture getLastDispatchablePosition() { } else if (md.hasTxnidMostBits() && md.hasTxnidLeastBits()) { // Filter-out transaction aborted messages. TxnID txnID = new TxnID(md.getTxnidMostBits(), md.getTxnidLeastBits()); - return !isTxnAborted(txnID, (PositionImpl) entry.getPosition()); + return !isTxnAborted(txnID, entry.getPosition()); } return true; }, getMaxReadPosition()) @@ -3838,9 +3838,8 @@ public synchronized void updateLastDispatchablePosition(Position position) { return; } - PositionImpl position0 = (PositionImpl) position; // If the position is greater than the maxReadPosition, ignore - if (position0.compareTo(getMaxReadPosition()) > 0) { + if (position.compareTo(getMaxReadPosition()) > 0) { return; } // If the lastDispatchablePosition is null, set it to the position @@ -3849,8 +3848,7 @@ public synchronized void updateLastDispatchablePosition(Position position) { return; } // If the position is greater than the lastDispatchablePosition, update it - PositionImpl lastDispatchablePosition0 = (PositionImpl) lastDispatchablePosition; - if (position0.compareTo(lastDispatchablePosition0) > 0) { + if (position.compareTo(lastDispatchablePosition) > 0) { lastDispatchablePosition = position; } } @@ -3858,7 +3856,7 @@ public synchronized void updateLastDispatchablePosition(Position position) { @Override public CompletableFuture getLastMessageId() { CompletableFuture completableFuture = new CompletableFuture<>(); - PositionImpl position = (PositionImpl) ledger.getLastConfirmedEntry(); + Position position = ledger.getLastConfirmedEntry(); String name = getName(); int partitionIndex = TopicName.getPartitionIndex(name); if (log.isDebugEnabled()) { @@ -3958,11 +3956,11 @@ public synchronized void triggerOffload(MessageIdImpl messageId) throws AlreadyR CompletableFuture promise = currentOffload = new CompletableFuture<>(); log.info("[{}] Starting offload operation at messageId {}", topic, messageId); getManagedLedger().asyncOffloadPrefix( - PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId()), + PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()), new OffloadCallback() { @Override public void offloadComplete(Position pos, Object ctx) { - PositionImpl impl = (PositionImpl) pos; + Position impl = pos; log.info("[{}] Completed successfully offload operation at messageId {}", topic, messageId); promise.complete(new MessageIdImpl(impl.getLedgerId(), impl.getEntryId(), -1)); } @@ -4179,10 +4177,10 @@ public void publishTxnMessage(TxnID txnID, ByteBuf headersAndPayload, PublishCon .thenAccept(position -> { // Message has been successfully persisted messageDeduplication.recordMessagePersisted(publishContext, - (PositionImpl) position); + position); publishContext.setProperty("txn_id", txnID.toString()); - publishContext.completed(null, ((PositionImpl) position).getLedgerId(), - ((PositionImpl) position).getEntryId()); + publishContext.completed(null, position.getLedgerId(), + position.getEntryId()); decrementPendingWriteOpsAndCheck(); }) @@ -4349,11 +4347,11 @@ public TransactionPendingAckStats getTransactionPendingAckStats(String subName, return this.subscriptions.get(subName).getTransactionPendingAckStats(lowWaterMarks); } - public PositionImpl getMaxReadPosition() { + public Position getMaxReadPosition() { return this.transactionBuffer.getMaxReadPosition(); } - public boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { + public boolean isTxnAborted(TxnID txnID, Position readPosition) { return this.transactionBuffer.isTxnAborted(txnID, readPosition); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionSnapshotCache.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionSnapshotCache.java index 63af5a1e484af..f78aabfd821c3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionSnapshotCache.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionSnapshotCache.java @@ -21,7 +21,8 @@ import java.util.NavigableMap; import java.util.TreeMap; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.common.api.proto.MarkersMessageIdData; import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshot; @@ -31,7 +32,7 @@ @Slf4j public class ReplicatedSubscriptionSnapshotCache { private final String subscription; - private final NavigableMap snapshots; + private final NavigableMap snapshots; private final int maxSnapshotToCache; public ReplicatedSubscriptionSnapshotCache(String subscription, int maxSnapshotToCache) { @@ -42,7 +43,7 @@ public ReplicatedSubscriptionSnapshotCache(String subscription, int maxSnapshotT public synchronized void addNewSnapshot(ReplicatedSubscriptionsSnapshot snapshot) { MarkersMessageIdData msgId = snapshot.getLocalMessageId(); - PositionImpl position = new PositionImpl(msgId.getLedgerId(), msgId.getEntryId()); + Position position = PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId()); if (log.isDebugEnabled()) { log.debug("[{}] Added new replicated-subscription snapshot at {} -- {}", subscription, position, @@ -61,10 +62,10 @@ public synchronized void addNewSnapshot(ReplicatedSubscriptionsSnapshot snapshot * Signal that the mark-delete position on the subscription has been advanced. If there is a snapshot that * correspond to this position, it will returned, other it will return null. */ - public synchronized ReplicatedSubscriptionsSnapshot advancedMarkDeletePosition(PositionImpl pos) { + public synchronized ReplicatedSubscriptionsSnapshot advancedMarkDeletePosition(Position pos) { ReplicatedSubscriptionsSnapshot snapshot = null; while (!snapshots.isEmpty()) { - PositionImpl first = snapshots.firstKey(); + Position first = snapshots.firstKey(); if (first.compareTo(pos) > 0) { // Snapshot is associated which an higher position, so it cannot be used now break; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java index 58f48744923d3..a8e6885525a19 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java @@ -35,7 +35,7 @@ import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.pulsar.broker.service.Replicator; import org.apache.pulsar.broker.service.Topic; @@ -137,7 +137,7 @@ private void receivedSnapshotRequest(ReplicatedSubscriptionsSnapshotRequest requ // Send response containing the current last written message id. The response // marker we're publishing locally and then replicating will have a higher // message id. - PositionImpl lastMsgId = (PositionImpl) topic.getLastPosition(); + Position lastMsgId = topic.getLastPosition(); if (log.isDebugEnabled()) { log.debug("[{}] Received snapshot request. Last msg id: {}", topic.getName(), lastMsgId); } @@ -178,7 +178,7 @@ private void receiveSubscriptionUpdated(ReplicatedSubscriptionsUpdate update) { return; } - Position pos = new PositionImpl(updatedMessageId.getLedgerId(), updatedMessageId.getEntryId()); + Position pos = PositionFactory.create(updatedMessageId.getLedgerId(), updatedMessageId.getEntryId()); if (log.isDebugEnabled()) { log.debug("[{}][{}] Received update for subscription to {}", topic, update.getSubscriptionName(), pos); @@ -288,7 +288,7 @@ public void completed(Exception e, long ledgerId, long entryId) { log.debug("[{}] Published marker at {}:{}. Exception: {}", topic.getName(), ledgerId, entryId, e); } - this.positionOfLastLocalMarker = new PositionImpl(ledgerId, entryId); + this.positionOfLastLocalMarker = PositionFactory.create(ledgerId, entryId); } PersistentTopic topic() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java index 53ba7193dc696..4eb20f02907c0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java @@ -29,7 +29,6 @@ import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.common.api.proto.MarkersMessageIdData; import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshotResponse; @@ -118,7 +117,7 @@ synchronized void receivedSnapshotResponse(Position position, ReplicatedSubscrip log.debug("[{}] Snapshot is complete {}", controller.topic().getName(), snapshotId); } // Snapshot is now complete, store it in the local topic - PositionImpl p = (PositionImpl) position; + Position p = position; controller.writeMarker( Markers.newReplicatedSubscriptionsSnapshot(snapshotId, controller.localCluster(), p.getLedgerId(), p.getEntryId(), responses)); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java index 0f06c201a81ee..b5ec70fda9774 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java @@ -19,7 +19,7 @@ package org.apache.pulsar.broker.transaction.buffer; import java.util.concurrent.CompletableFuture; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.common.policies.data.TransactionBufferStats; @@ -37,7 +37,7 @@ enum SnapshotType { * @param txnID aborted transaction ID. * @param abortedMarkerPersistentPosition the position of the abort txn marker. */ - void putAbortedTxnAndPosition(TxnID txnID, PositionImpl abortedMarkerPersistentPosition); + void putAbortedTxnAndPosition(TxnID txnID, Position abortedMarkerPersistentPosition); /** * Clean up invalid aborted transactions. @@ -56,7 +56,7 @@ enum SnapshotType { * @return a Position (startReadCursorPosition) determiner where to start to recover in the original topic. */ - CompletableFuture recoverFromSnapshot(); + CompletableFuture recoverFromSnapshot(); /** * Delete the transaction buffer aborted transaction snapshot. @@ -68,7 +68,7 @@ enum SnapshotType { * Take aborted transactions snapshot. * @return a completableFuture. */ - CompletableFuture takeAbortedTxnsSnapshot(PositionImpl maxReadPosition); + CompletableFuture takeAbortedTxnsSnapshot(Position maxReadPosition); /** * Get the lastSnapshotTimestamps. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java index 092638abf5bba..b379c4d1db10c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java @@ -23,7 +23,6 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.transaction.exception.TransactionException; import org.apache.pulsar.broker.transaction.exception.buffer.TransactionBufferException; import org.apache.pulsar.client.api.transaction.TxnID; @@ -143,20 +142,20 @@ public interface TransactionBuffer { * @param readPosition the persistent position of the txn message. * @return whether the txn is aborted. */ - boolean isTxnAborted(TxnID txnID, PositionImpl readPosition); + boolean isTxnAborted(TxnID txnID, Position readPosition); /** * Sync max read position for normal publish. - * @param position {@link PositionImpl} the position to sync. + * @param position {@link Position} the position to sync. * @param isMarkerMessage whether the message is marker message. */ - void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage); + void syncMaxReadPositionForNormalPublish(Position position, boolean isMarkerMessage); /** * Get the can read max position. * @return the stable position. */ - PositionImpl getMaxReadPosition(); + Position getMaxReadPosition(); /** * Get the snapshot type. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java index 533d0716d413c..ae755f0715ee2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java @@ -31,7 +31,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; @@ -371,12 +370,12 @@ public CompletableFuture closeAsync() { } @Override - public boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { + public boolean isTxnAborted(TxnID txnID, Position readPosition) { return false; } @Override - public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { + public void syncMaxReadPositionForNormalPublish(Position position, boolean isMarkerMessage) { if (!isMarkerMessage) { updateLastDispatchablePosition(position); if (maxReadPositionCallBack != null) { @@ -386,8 +385,8 @@ public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean i } @Override - public PositionImpl getMaxReadPosition() { - return (PositionImpl) topic.getLastPosition(); + public Position getMaxReadPosition() { + return topic.getLastPosition(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java index 967f1f16fefe4..5c9075e9a3867 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java @@ -24,8 +24,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService.ReferenceCountedWriter; @@ -49,7 +50,7 @@ public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcesso * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this * position have been deleted. */ - private final LinkedMap aborts = new LinkedMap<>(); + private final LinkedMap aborts = new LinkedMap<>(); private volatile long lastSnapshotTimestamps; @@ -68,7 +69,7 @@ public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) { } @Override - public void putAbortedTxnAndPosition(TxnID abortedTxnId, PositionImpl abortedMarkerPersistentPosition) { + public void putAbortedTxnAndPosition(TxnID abortedTxnId, Position abortedMarkerPersistentPosition) { aborts.put(abortedTxnId, abortedMarkerPersistentPosition); } @@ -96,12 +97,12 @@ private long getSystemClientOperationTimeoutMs() throws Exception { } @Override - public CompletableFuture recoverFromSnapshot() { + public CompletableFuture recoverFromSnapshot() { return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory() .getTxnBufferSnapshotService() .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> { try { - PositionImpl startReadCursorPosition = null; + Position startReadCursorPosition = null; while (reader.hasMoreEvents()) { Message message = reader.readNextAsync() .get(getSystemClientOperationTimeoutMs(), TimeUnit.MILLISECONDS); @@ -109,7 +110,7 @@ public CompletableFuture recoverFromSnapshot() { TransactionBufferSnapshot transactionBufferSnapshot = message.getValue(); if (transactionBufferSnapshot != null) { handleSnapshot(transactionBufferSnapshot); - startReadCursorPosition = PositionImpl.get( + startReadCursorPosition = PositionFactory.create( transactionBufferSnapshot.getMaxReadPositionLedgerId(), transactionBufferSnapshot.getMaxReadPositionEntryId()); } @@ -144,7 +145,7 @@ public CompletableFuture clearAbortedTxnSnapshot() { } @Override - public CompletableFuture takeAbortedTxnsSnapshot(PositionImpl maxReadPosition) { + public CompletableFuture takeAbortedTxnsSnapshot(Position maxReadPosition) { return takeSnapshotWriter.getFuture().thenCompose(writer -> { TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot(); snapshot.setTopicName(topic.getName()); @@ -202,7 +203,7 @@ private void handleSnapshot(TransactionBufferSnapshot snapshot) { snapshot.getAborts().forEach(abortTxnMetadata -> aborts.put(new TxnID(abortTxnMetadata.getTxnIdMostBits(), abortTxnMetadata.getTxnIdLeastBits()), - PositionImpl.get(abortTxnMetadata.getLedgerId(), + PositionFactory.create(abortTxnMetadata.getLedgerId(), abortTxnMetadata.getEntryId()))); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java index 385500dfbe9e7..e94e7a047797a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java @@ -36,8 +36,9 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.commons.lang3.tuple.MutablePair; @@ -80,7 +81,7 @@ public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcess /** * The map is used to clear the aborted transaction IDs persistent in the expired ledger. *

- * The key PositionImpl {@link PositionImpl} is the persistent position of + * The key Position {@link Position} is the persistent position of * the latest transaction of a segment. * The value TxnID {@link TxnID} is the latest Transaction ID in a segment. *

@@ -94,7 +95,7 @@ public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcess * the positions. *

*/ - private final LinkedMap segmentIndex = new LinkedMap<>(); + private final LinkedMap segmentIndex = new LinkedMap<>(); /** * This map is used to check whether a transaction is an aborted transaction. @@ -113,7 +114,7 @@ public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcess * indexes of the snapshot segment. *

*/ - private final LinkedMap indexes = new LinkedMap<>(); + private final LinkedMap indexes = new LinkedMap<>(); private final PersistentTopic topic; @@ -157,7 +158,7 @@ public SnapshotSegmentAbortedTxnProcessorImpl(PersistentTopic topic) { } @Override - public void putAbortedTxnAndPosition(TxnID txnID, PositionImpl position) { + public void putAbortedTxnAndPosition(TxnID txnID, Position position) { unsealedTxnIds.add(txnID); aborts.put(txnID, txnID); /* @@ -188,14 +189,14 @@ public boolean checkAbortedTransaction(TxnID txnID) { @Override public void trimExpiredAbortedTxns() { //Checking whether there are some segment expired. - List positionsNeedToDelete = new ArrayList<>(); + List positionsNeedToDelete = new ArrayList<>(); while (!segmentIndex.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger()) .ledgerExists(segmentIndex.firstKey().getLedgerId())) { if (log.isDebugEnabled()) { log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}", topic.getName(), segmentIndex.firstKey()); } - PositionImpl positionNeedToDelete = segmentIndex.firstKey(); + Position positionNeedToDelete = segmentIndex.firstKey(); positionsNeedToDelete.add(positionNeedToDelete); TxnID theLatestDeletedTxnID = segmentIndex.remove(0); @@ -216,7 +217,7 @@ private String buildKey(long sequenceId) { } @Override - public CompletableFuture takeAbortedTxnsSnapshot(PositionImpl maxReadPosition) { + public CompletableFuture takeAbortedTxnsSnapshot(Position maxReadPosition) { //Store the latest aborted transaction IDs in unsealedTxnIDs and the according the latest max read position. TransactionBufferSnapshotIndexesMetadata metadata = new TransactionBufferSnapshotIndexesMetadata( maxReadPosition.getLedgerId(), maxReadPosition.getEntryId(), @@ -226,11 +227,11 @@ public CompletableFuture takeAbortedTxnsSnapshot(PositionImpl maxReadPosit } @Override - public CompletableFuture recoverFromSnapshot() { + public CompletableFuture recoverFromSnapshot() { return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory() .getTxnBufferSnapshotIndexService() .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> { - PositionImpl startReadCursorPosition = null; + Position startReadCursorPosition = null; TransactionBufferSnapshotIndexes persistentSnapshotIndexes = null; try { /* @@ -249,7 +250,7 @@ public CompletableFuture recoverFromSnapshot() { TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue(); if (transactionBufferSnapshotIndexes != null) { persistentSnapshotIndexes = transactionBufferSnapshotIndexes; - startReadCursorPosition = PositionImpl.get( + startReadCursorPosition = PositionFactory.create( transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(), transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId()); } @@ -269,7 +270,7 @@ public CompletableFuture recoverFromSnapshot() { } finally { closeReader(reader); } - PositionImpl finalStartReadCursorPosition = startReadCursorPosition; + Position finalStartReadCursorPosition = startReadCursorPosition; TransactionBufferSnapshotIndexes finalPersistentSnapshotIndexes = persistentSnapshotIndexes; if (persistentSnapshotIndexes == null) { return recoverOldSnapshot(); @@ -290,13 +291,13 @@ public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl readOnly CompletableFuture handleSegmentFuture = new CompletableFuture<>(); completableFutures.add(handleSegmentFuture); readOnlyManagedLedger.asyncReadEntry( - new PositionImpl(index.getSegmentLedgerID(), + PositionFactory.create(index.getSegmentLedgerID(), index.getSegmentEntryID()), new AsyncCallbacks.ReadEntryCallback() { @Override public void readEntryComplete(Entry entry, Object ctx) { handleSnapshotSegmentEntry(entry); - indexes.put(new PositionImpl( + indexes.put(PositionFactory.create( index.abortedMarkLedgerID, index.abortedMarkEntryID), index); @@ -392,7 +393,7 @@ public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Ob } // This method will be deprecated and removed in version 4.x.0 - private CompletableFuture recoverOldSnapshot() { + private CompletableFuture recoverOldSnapshot() { return topic.getBrokerService().getPulsar().getPulsarResources().getTopicResources() .listPersistentTopicsAsync(NamespaceName.get(TopicName.get(topic.getName()).getNamespace())) .thenCompose(topics -> { @@ -404,7 +405,7 @@ private CompletableFuture recoverOldSnapshot() { return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory() .getTxnBufferSnapshotService() .createReader(TopicName.get(topic.getName())).thenComposeAsync(snapshotReader -> { - PositionImpl startReadCursorPositionInOldSnapshot = null; + Position startReadCursorPositionInOldSnapshot = null; try { while (snapshotReader.hasMoreEvents()) { Message message = snapshotReader.readNextAsync() @@ -414,7 +415,7 @@ private CompletableFuture recoverOldSnapshot() { message.getValue(); if (transactionBufferSnapshot != null) { handleOldSnapshot(transactionBufferSnapshot); - startReadCursorPositionInOldSnapshot = PositionImpl.get( + startReadCursorPositionInOldSnapshot = PositionFactory.create( transactionBufferSnapshot.getMaxReadPositionLedgerId(), transactionBufferSnapshot.getMaxReadPositionEntryId()); } @@ -497,7 +498,7 @@ private void handleSnapshotSegmentEntry(Entry entry) { .decode(Unpooled.wrappedBuffer(headersAndPayload).nioBuffer()); TxnIDData lastTxn = snapshotSegment.getAborts().get(snapshotSegment.getAborts().size() - 1); - segmentIndex.put(new PositionImpl(snapshotSegment.getPersistentPositionLedgerId(), + segmentIndex.put(PositionFactory.create(snapshotSegment.getPersistentPositionLedgerId(), snapshotSegment.getPersistentPositionEntryId()), new TxnID(lastTxn.getMostSigBits(), lastTxn.getLeastSigBits())); convertTypeToTxnID(snapshotSegment.getAborts()).forEach(txnID -> aborts.put(txnID, txnID)); @@ -696,7 +697,7 @@ private void executeTask() { } private CompletableFuture takeSnapshotSegmentAsync(LinkedList sealedAbortedTxnIdSegment, - PositionImpl abortedMarkerPersistentPosition) { + Position abortedMarkerPersistentPosition) { CompletableFuture res = writeSnapshotSegmentAsync(sealedAbortedTxnIdSegment, abortedMarkerPersistentPosition).thenRun(() -> { if (log.isDebugEnabled()) { @@ -720,7 +721,7 @@ private CompletableFuture takeSnapshotSegmentAsync(LinkedList seale } private CompletableFuture writeSnapshotSegmentAsync(LinkedList segment, - PositionImpl abortedMarkerPersistentPosition) { + Position abortedMarkerPersistentPosition) { TransactionBufferSnapshotSegment transactionBufferSnapshotSegment = new TransactionBufferSnapshotSegment(); transactionBufferSnapshotSegment.setAborts(convertTypeToTxnIDData(segment)); transactionBufferSnapshotSegment.setTopicName(this.topic.getName()); @@ -751,7 +752,7 @@ private CompletableFuture writeSnapshotSegmentAsync(LinkedList segm } private CompletionStage updateIndexWhenExecuteTheLatestTask() { - PositionImpl maxReadPosition = topic.getMaxReadPosition(); + Position maxReadPosition = topic.getMaxReadPosition(); List aborts = convertTypeToTxnIDData(unsealedTxnIds); if (taskQueue.size() != 1) { return CompletableFuture.completedFuture(null); @@ -762,9 +763,9 @@ private CompletionStage updateIndexWhenExecuteTheLatestTask() { } // update index after delete all segment. - private CompletableFuture deleteSnapshotSegment(List positionNeedToDeletes) { + private CompletableFuture deleteSnapshotSegment(List positionNeedToDeletes) { List> results = new ArrayList<>(); - for (PositionImpl positionNeedToDelete : positionNeedToDeletes) { + for (Position positionNeedToDelete : positionNeedToDeletes) { long sequenceIdNeedToDelete = indexes.get(positionNeedToDelete).getSequenceID(); CompletableFuture res = snapshotSegmentsWriter.getFuture() .thenCompose(writer -> writer.deleteAsync(buildKey(sequenceIdNeedToDelete), null)) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index fbd4ddf7da053..b4662e5fa83ed 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -37,8 +37,8 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceException; @@ -69,12 +69,12 @@ public class TopicTransactionBuffer extends TopicTransactionBufferState implemen private final PersistentTopic topic; - private volatile PositionImpl maxReadPosition; + private volatile Position maxReadPosition; /** * Ongoing transaction, map for remove txn stable position, linked for find max read position. */ - private final LinkedMap ongoingTxns = new LinkedMap<>(); + private final LinkedMap ongoingTxns = new LinkedMap<>(); // when change max read position, the count will +1. Take snapshot will reset the count. private final AtomicLong changeMaxReadPositionCount = new AtomicLong(); @@ -113,7 +113,7 @@ public TopicTransactionBuffer(PersistentTopic topic) { .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount(); this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar() .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis(); - this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(); + this.maxReadPosition = topic.getManagedLedger().getLastConfirmedEntry(); if (topic.getBrokerService().getPulsar().getConfiguration().isTransactionBufferSegmentedSnapshotEnabled()) { snapshotAbortedTxnProcessor = new SnapshotSegmentAbortedTxnProcessorImpl(topic); snapshotType = AbortedTxnProcessor.SnapshotType.Segment; @@ -133,7 +133,7 @@ private void recover() { public void recoverComplete() { synchronized (TopicTransactionBuffer.this) { if (ongoingTxns.isEmpty()) { - maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(); + maxReadPosition = topic.getManagedLedger().getLastConfirmedEntry(); } if (!changeToReadyState()) { log.error("[{}]Transaction buffer recover fail, current state: {}", @@ -154,7 +154,7 @@ public void recoverComplete() { @Override public void noNeedToRecover() { synchronized (TopicTransactionBuffer.this) { - maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(); + maxReadPosition = topic.getManagedLedger().getLastConfirmedEntry(); if (!changeToNoSnapshotState()) { log.error("[{}]Transaction buffer recover fail", topic.getName()); } else { @@ -171,7 +171,7 @@ public void handleTxnEntry(Entry entry) { TopicTransactionBufferRecover.SUBSCRIPTION_NAME, -1); if (msgMetadata != null && msgMetadata.hasTxnidMostBits() && msgMetadata.hasTxnidLeastBits()) { TxnID txnID = new TxnID(msgMetadata.getTxnidMostBits(), msgMetadata.getTxnidLeastBits()); - PositionImpl position = PositionImpl.get(entry.getLedgerId(), entry.getEntryId()); + Position position = PositionFactory.create(entry.getLedgerId(), entry.getEntryId()); synchronized (TopicTransactionBuffer.this) { if (Markers.isTxnMarker(msgMetadata)) { if (Markers.isTxnAbortMarker(msgMetadata)) { @@ -289,8 +289,8 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { private void handleTransactionMessage(TxnID txnId, Position position) { if (!ongoingTxns.containsKey(txnId) && !this.snapshotAbortedTxnProcessor .checkAbortedTransaction(txnId)) { - ongoingTxns.put(txnId, (PositionImpl) position); - PositionImpl firstPosition = ongoingTxns.get(ongoingTxns.firstKey()); + ongoingTxns.put(txnId, position); + Position firstPosition = ongoingTxns.get(ongoingTxns.firstKey()); // max read position is less than first ongoing transaction message position updateMaxReadPosition(((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(firstPosition), false); @@ -368,7 +368,7 @@ public CompletableFuture abortTxn(TxnID txnID, long lowWaterMark) { @Override public void addComplete(Position position, ByteBuf entryData, Object ctx) { synchronized (TopicTransactionBuffer.this) { - snapshotAbortedTxnProcessor.putAbortedTxnAndPosition(txnID, (PositionImpl) position); + snapshotAbortedTxnProcessor.putAbortedTxnAndPosition(txnID, position); removeTxnAndUpdateMaxReadPosition(txnID); snapshotAbortedTxnProcessor.trimExpiredAbortedTxns(); takeSnapshotByChangeTimes(); @@ -459,10 +459,10 @@ private void takeSnapshotByTimeout() { void removeTxnAndUpdateMaxReadPosition(TxnID txnID) { ongoingTxns.remove(txnID); if (!ongoingTxns.isEmpty()) { - PositionImpl position = ongoingTxns.get(ongoingTxns.firstKey()); + Position position = ongoingTxns.get(ongoingTxns.firstKey()); updateMaxReadPosition(((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(position), false); } else { - updateMaxReadPosition((PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(), false); + updateMaxReadPosition(topic.getManagedLedger().getLastConfirmedEntry(), false); } // Update the last dispatchable position to null if there is a TXN finished. updateLastDispatchablePosition(null); @@ -477,8 +477,8 @@ void removeTxnAndUpdateMaxReadPosition(TxnID txnID) { * @param newPosition new max read position to update. * @param disableCallback whether disable the callback. */ - void updateMaxReadPosition(PositionImpl newPosition, boolean disableCallback) { - PositionImpl preMaxReadPosition = this.maxReadPosition; + void updateMaxReadPosition(Position newPosition, boolean disableCallback) { + Position preMaxReadPosition = this.maxReadPosition; this.maxReadPosition = newPosition; if (preMaxReadPosition.compareTo(this.maxReadPosition) < 0) { if (!checkIfNoSnapshot()) { @@ -507,18 +507,18 @@ public CompletableFuture closeAsync() { } @Override - public synchronized boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { + public synchronized boolean isTxnAborted(TxnID txnID, Position readPosition) { return snapshotAbortedTxnProcessor.checkAbortedTransaction(txnID); } /** * Sync max read position for normal publish. - * @param position {@link PositionImpl} the position to sync. + * @param position {@link Position} the position to sync. * @param isMarkerMessage whether the message is marker message, in such case, we * don't need to trigger the callback to update lastMaxReadPositionMovedForwardTimestamp. */ @Override - public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { + public void syncMaxReadPositionForNormalPublish(Position position, boolean isMarkerMessage) { // when ongoing transaction is empty, proved that lastAddConfirm is can read max position, because callback // thread is the same tread, in this time the lastAddConfirm don't content transaction message. synchronized (TopicTransactionBuffer.this) { @@ -542,11 +542,11 @@ public AbortedTxnProcessor.SnapshotType getSnapshotType() { } @Override - public PositionImpl getMaxReadPosition() { + public Position getMaxReadPosition() { if (checkIfReady() || checkIfNoSnapshot()) { return this.maxReadPosition; } else { - return PositionImpl.EARLIEST; + return PositionFactory.EARLIEST; } } @@ -602,7 +602,7 @@ public static class TopicTransactionBufferRecover implements Runnable { private final TopicTransactionBufferRecoverCallBack callBack; - private Position startReadCursorPosition = PositionImpl.EARLIEST; + private Position startReadCursorPosition = PositionFactory.EARLIEST; private final SpscArrayQueue entryQueue; @@ -649,9 +649,9 @@ public void run() { log.error("[{}]Transaction buffer recover fail when open cursor!", topic.getName(), e); return; } - PositionImpl lastConfirmedEntry = - (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(); - PositionImpl currentLoadPosition = (PositionImpl) this.startReadCursorPosition; + Position lastConfirmedEntry = + topic.getManagedLedger().getLastConfirmedEntry(); + Position currentLoadPosition = this.startReadCursorPosition; FillEntryQueueCallback fillEntryQueueCallback = new FillEntryQueueCallback(entryQueue, managedCursor, TopicTransactionBufferRecover.this); if (lastConfirmedEntry.getEntryId() != -1) { @@ -660,7 +660,7 @@ public void run() { Entry entry = entryQueue.poll(); if (entry != null) { try { - currentLoadPosition = PositionImpl.get(entry.getLedgerId(), + currentLoadPosition = PositionFactory.create(entry.getLedgerId(), entry.getEntryId()); callBack.handleTxnEntry(entry); } finally { @@ -724,7 +724,7 @@ public interface MaxReadPositionCallBack { * @param oldPosition the old max read position. * @param newPosition the new max read position. */ - void maxReadPositionMovedForward(PositionImpl oldPosition, PositionImpl newPosition); + void maxReadPositionMovedForward(Position oldPosition, Position newPosition); } static class FillEntryQueueCallback implements AsyncCallbacks.ReadEntriesCallback { @@ -753,7 +753,7 @@ boolean fillQueue() { if (cursor.hasMoreEntries()) { outstandingReadsRequests.incrementAndGet(); cursor.asyncReadEntries(NUMBER_OF_PER_READ_ENTRY, - this, System.nanoTime(), PositionImpl.LATEST); + this, System.nanoTime(), PositionFactory.LATEST); } else { if (entryQueue.size() == 0) { isReadable = false; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java index 6f5dc0cd4d0dd..d0efc47c49544 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java @@ -23,7 +23,6 @@ import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -93,12 +92,12 @@ public CompletableFuture closeAsync() { } @Override - public boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { + public boolean isTxnAborted(TxnID txnID, Position readPosition) { return false; } @Override - public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { + public void syncMaxReadPositionForNormalPublish(Position position, boolean isMarkerMessage) { if (!isMarkerMessage) { updateLastDispatchablePosition(position); if (maxReadPositionCallBack != null) { @@ -108,8 +107,8 @@ public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean i } @Override - public PositionImpl getMaxReadPosition() { - return (PositionImpl) topic.getLastPosition(); + public Position getMaxReadPosition() { + return topic.getLastPosition(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckHandle.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckHandle.java index a7892a56f0bd5..168a6b1483f86 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckHandle.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckHandle.java @@ -22,7 +22,6 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; import org.apache.pulsar.broker.service.Consumer; @@ -54,9 +53,9 @@ public interface PendingAckHandle { * @return the future of this operation. * @throws TransactionConflictException if the ack with transaction is conflict with pending ack. * @throws NotAllowedException if Use this method incorrectly eg. not use - * PositionImpl or cumulative ack with a list of positions. + * Position or cumulative ack with a list of positions. */ - CompletableFuture individualAcknowledgeMessage(TxnID txnID, List individualAcknowledgeMessage(TxnID txnID, List> positions); /** * Acknowledge message(s) for an ongoing transaction. @@ -78,9 +77,9 @@ CompletableFuture individualAcknowledgeMessage(TxnID txnID, List cumulativeAcknowledgeMessage(TxnID txnID, List positions); + CompletableFuture cumulativeAcknowledgeMessage(TxnID txnID, List positions); /** * Commit a transaction. @@ -108,14 +107,14 @@ CompletableFuture individualAcknowledgeMessage(TxnID txnID, List individualAcknowledgeMessage(TxnID txnID, List individualAcknowledgeMessage(TxnID txnID, List appendIndividualAck(TxnID txnID, List> positions); + CompletableFuture appendIndividualAck(TxnID txnID, List> positions); /** * Append the cumulative pending ack operation to the ack persistent store. * * @param txnID {@link TxnID} transaction id. - * @param position {@link PositionImpl} the pending ack position. + * @param position {@link Position} the pending ack position. * @return a future represents the result of this operation */ - CompletableFuture appendCumulativeAck(TxnID txnID, PositionImpl position); + CompletableFuture appendCumulativeAck(TxnID txnID, Position position); /** * Append the pending ack commit mark to the ack persistent store. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/InMemoryPendingAckStore.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/InMemoryPendingAckStore.java index 0840e2c2f45dd..e022dba09028c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/InMemoryPendingAckStore.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/InMemoryPendingAckStore.java @@ -21,7 +21,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.transaction.pendingack.PendingAckStore; import org.apache.pulsar.client.api.transaction.TxnID; @@ -44,12 +44,12 @@ public CompletableFuture closeAsync() { @Override public CompletableFuture appendIndividualAck(TxnID txnID, - List> positions) { + List> positions) { return CompletableFuture.completedFuture(null); } @Override - public CompletableFuture appendCumulativeAck(TxnID txnID, PositionImpl position) { + public CompletableFuture appendCumulativeAck(TxnID txnID, Position position) { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckReplyCallBack.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckReplyCallBack.java index 9900d29725f21..b32dcbf3101a9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckReplyCallBack.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckReplyCallBack.java @@ -21,7 +21,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.transaction.pendingack.PendingAckReplyCallBack; @@ -87,23 +89,26 @@ public void handleMetadataEntry(PendingAckMetadataEntry pendingAckMetadataEntry) PendingAckMetadata pendingAckMetadata = pendingAckMetadataEntry.getPendingAckMetadatasList().get(0); pendingAckHandle.handleCumulativeAckRecover(txnID, - PositionImpl.get(pendingAckMetadata.getLedgerId(), pendingAckMetadata.getEntryId())); + PositionFactory.create(pendingAckMetadata.getLedgerId(), pendingAckMetadata.getEntryId())); } else { - List> positions = new ArrayList<>(); + List> positions = new ArrayList<>(); pendingAckMetadataEntry.getPendingAckMetadatasList().forEach(pendingAckMetadata -> { if (pendingAckMetadata.getAckSetsCount() == 0) { - positions.add(new MutablePair<>(PositionImpl.get(pendingAckMetadata.getLedgerId(), + positions.add(new MutablePair<>(PositionFactory.create(pendingAckMetadata.getLedgerId(), pendingAckMetadata.getEntryId()), pendingAckMetadata.getBatchSize())); } else { - PositionImpl position = - PositionImpl.get(pendingAckMetadata.getLedgerId(), pendingAckMetadata.getEntryId()); + long[] ackSets = null; if (pendingAckMetadata.getAckSetsCount() > 0) { - long[] ackSets = new long[pendingAckMetadata.getAckSetsCount()]; + ackSets = new long[pendingAckMetadata.getAckSetsCount()]; for (int i = 0; i < pendingAckMetadata.getAckSetsCount(); i++) { ackSets[i] = pendingAckMetadata.getAckSetAt(i); } - position.setAckSet(ackSets); + } else { + ackSets = new long[0]; } + Position position = + AckSetStateUtil.createPositionWithAckSet(pendingAckMetadata.getLedgerId(), + pendingAckMetadata.getEntryId(), ackSets); positions.add(new MutablePair<>(position, pendingAckMetadata.getBatchSize())); } }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java index 4dce8b9a0fcf4..f8143cfc4c125 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java @@ -42,8 +42,9 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceException; import org.apache.pulsar.broker.transaction.pendingack.PendingAckReplyCallBack; @@ -80,20 +81,20 @@ public class MLPendingAckStore implements PendingAckStore { private final SpscArrayQueue entryQueue; //this is for replay - private final PositionImpl lastConfirmedEntry; + private final Position lastConfirmedEntry; - private PositionImpl currentLoadPosition; + private Position currentLoadPosition; private final AtomicLong currentIndexLag = new AtomicLong(0); private volatile long maxIndexLag; - protected PositionImpl maxAckPosition = PositionImpl.EARLIEST; + protected Position maxAckPosition = PositionFactory.EARLIEST; private final LogIndexLagBackoff logIndexBackoff; /** - * If the Batch feature is enabled by {@link #bufferedWriter}, {@link #handleMetadataEntry(PositionImpl, List)} is + * If the Batch feature is enabled by {@link #bufferedWriter}, {@link #handleMetadataEntry(Position, List)} is * executed after all data in the batch is written, instead of - * {@link #handleMetadataEntry(PositionImpl, PendingAckMetadataEntry)} after each data is written. This is because + * {@link #handleMetadataEntry(Position, PendingAckMetadataEntry)} after each data is written. This is because * method {@link #clearUselessLogData()} deletes the data in the unit of Entry. */ private final ArrayList batchedPendingAckLogsWaitingForHandle; @@ -111,7 +112,7 @@ public class MLPendingAckStore implements PendingAckStore { * If the max position (key) is smaller than the subCursor mark delete position, * the log cursor will mark delete the position before log position (value). */ - final ConcurrentSkipListMap pendingAckLogIndex; + final ConcurrentSkipListMap pendingAckLogIndex; private final ManagedCursor subManagedCursor; @@ -123,9 +124,9 @@ public MLPendingAckStore(ManagedLedger managedLedger, ManagedCursor cursor, Timer timer, TxnLogBufferedWriterMetricsStats bufferedWriterMetrics) { this.managedLedger = managedLedger; this.cursor = cursor; - this.currentLoadPosition = (PositionImpl) this.cursor.getMarkDeletedPosition(); + this.currentLoadPosition = this.cursor.getMarkDeletedPosition(); this.entryQueue = new SpscArrayQueue<>(2000); - this.lastConfirmedEntry = (PositionImpl) managedLedger.getLastConfirmedEntry(); + this.lastConfirmedEntry = managedLedger.getLastConfirmedEntry(); this.pendingAckLogIndex = new ConcurrentSkipListMap<>(); this.subManagedCursor = subManagedCursor; this.logIndexBackoff = new LogIndexLagBackoff(transactionPendingAckLogIndexMinLag, Long.MAX_VALUE, 1); @@ -147,7 +148,7 @@ public void replayAsync(PendingAckHandleImpl pendingAckHandle, ExecutorService t //TODO can control the number of entry to read private void readAsync(int numberOfEntriesToRead, AsyncCallbacks.ReadEntriesCallback readEntriesCallback) { - cursor.asyncReadEntries(numberOfEntriesToRead, readEntriesCallback, System.nanoTime(), PositionImpl.LATEST); + cursor.asyncReadEntries(numberOfEntriesToRead, readEntriesCallback, System.nanoTime(), PositionFactory.LATEST); } @Override @@ -186,17 +187,18 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { @Override public CompletableFuture appendIndividualAck(TxnID txnID, - List> positions) { + List> positions) { PendingAckMetadataEntry pendingAckMetadataEntry = new PendingAckMetadataEntry(); pendingAckMetadataEntry.setPendingAckOp(PendingAckOp.ACK); pendingAckMetadataEntry.setAckType(AckType.Individual); List pendingAckMetadataList = new ArrayList<>(); positions.forEach(positionIntegerMutablePair -> { PendingAckMetadata pendingAckMetadata = new PendingAckMetadata(); - PositionImpl position = positionIntegerMutablePair.getLeft(); + Position position = positionIntegerMutablePair.getLeft(); int batchSize = positionIntegerMutablePair.getRight(); - if (positionIntegerMutablePair.getLeft().getAckSet() != null) { - for (long l : position.getAckSet()) { + long[] positionAckSet = AckSetStateUtil.getAckSetArrayOrNull(position); + if (positionAckSet != null) { + for (long l : positionAckSet) { pendingAckMetadata.addAckSet(l); } } @@ -210,13 +212,14 @@ public CompletableFuture appendIndividualAck(TxnID txnID, } @Override - public CompletableFuture appendCumulativeAck(TxnID txnID, PositionImpl position) { + public CompletableFuture appendCumulativeAck(TxnID txnID, Position position) { PendingAckMetadataEntry pendingAckMetadataEntry = new PendingAckMetadataEntry(); pendingAckMetadataEntry.setPendingAckOp(PendingAckOp.ACK); pendingAckMetadataEntry.setAckType(AckType.Cumulative); PendingAckMetadata pendingAckMetadata = new PendingAckMetadata(); - if (position.getAckSet() != null) { - for (long l : position.getAckSet()) { + long[] positionAckSet = AckSetStateUtil.getAckSetArrayOrNull(position); + if (positionAckSet != null) { + for (long l : positionAckSet) { pendingAckMetadata.addAckSet(l); } } @@ -257,8 +260,8 @@ public void addComplete(Position position, Object ctx) { currentIndexLag.incrementAndGet(); /** * If the Batch feature is enabled by {@link #bufferedWriter}, - * {@link #handleMetadataEntry(PositionImpl, List)} is executed after all data in the batch is written, - * instead of {@link #handleMetadataEntry(PositionImpl, PendingAckMetadataEntry)} after each data is + * {@link #handleMetadataEntry(Position, List)} is executed after all data in the batch is written, + * instead of {@link #handleMetadataEntry(Position, PendingAckMetadataEntry)} after each data is * written. This is because method {@link #clearUselessLogData()} deletes the data in the unit of Entry. * {@link TxnLogBufferedWriter.AddDataCallback#addComplete} for elements in a batch is executed * simultaneously and in strict order, so when the last element in a batch is complete, the whole @@ -267,11 +270,11 @@ public void addComplete(Position position, Object ctx) { if (position instanceof TxnBatchedPositionImpl batchedPosition){ batchedPendingAckLogsWaitingForHandle.add(pendingAckMetadataEntry); if (batchedPosition.getBatchIndex() == batchedPosition.getBatchSize() - 1){ - handleMetadataEntry((PositionImpl) position, batchedPendingAckLogsWaitingForHandle); + handleMetadataEntry(position, batchedPendingAckLogsWaitingForHandle); batchedPendingAckLogsWaitingForHandle.clear(); } } else { - handleMetadataEntry((PositionImpl) position, pendingAckMetadataEntry); + handleMetadataEntry(position, pendingAckMetadataEntry); } completableFuture.complete(null); clearUselessLogData(); @@ -301,7 +304,7 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { * @param logPosition The position of batch log Entry. * @param logList Pending ack log records in a batch log Entry. */ - private void handleMetadataEntry(PositionImpl logPosition, + private void handleMetadataEntry(Position logPosition, List logList) { Stream pendingAckMetaStream = logList.stream() .filter(log -> bothNotAbortAndCommitPredicate.test(log)) @@ -313,7 +316,7 @@ private void handleMetadataEntry(PositionImpl logPosition, pendingAckLog.getPendingAckOp() != PendingAckOp.ABORT && pendingAckLog.getPendingAckOp() != PendingAckOp.COMMIT; - private void handleMetadataEntry(PositionImpl logPosition, + private void handleMetadataEntry(Position logPosition, PendingAckMetadataEntry pendingAckMetadataEntry) { // store the persistent position in to memory // store the max position of this entry retain @@ -322,14 +325,14 @@ private void handleMetadataEntry(PositionImpl logPosition, } } - private void handleMetadataEntry(PositionImpl logPosition, Stream pendingAckListStream) { + private void handleMetadataEntry(Position logPosition, Stream pendingAckListStream) { // store the persistent position in to memory // store the max position of this entry retain Optional optional = pendingAckListStream .max((o1, o2) -> ComparisonChain.start().compare(o1.getLedgerId(), o2.getLedgerId()).compare(o1.getEntryId(), o2.getEntryId()).result()); optional.ifPresent(pendingAckMetadata -> { - PositionImpl nowPosition = PositionImpl.get(pendingAckMetadata.getLedgerId(), + Position nowPosition = PositionFactory.create(pendingAckMetadata.getLedgerId(), pendingAckMetadata.getEntryId()); if (nowPosition.compareTo(maxAckPosition) > 0) { maxAckPosition = nowPosition; @@ -346,18 +349,18 @@ private void handleMetadataEntry(PositionImpl logPosition, Stream 0 && fillEntryQueueCallback.fillQueue()) { Entry entry = entryQueue.poll(); if (entry != null) { - currentLoadPosition = PositionImpl.get(entry.getLedgerId(), entry.getEntryId()); + currentLoadPosition = PositionFactory.create(entry.getLedgerId(), entry.getEntryId()); List logs = deserializeEntry(entry); if (logs.isEmpty()){ continue; } else if (logs.size() == 1){ currentIndexLag.incrementAndGet(); PendingAckMetadataEntry log = logs.get(0); - handleMetadataEntry(new PositionImpl(entry.getLedgerId(), entry.getEntryId()), log); + handleMetadataEntry(PositionFactory.create(entry.getLedgerId(), entry.getEntryId()), log); pendingAckReplyCallBack.handleMetadataEntry(log); } else { int batchSize = logs.size(); @@ -419,7 +422,7 @@ public void run() { pendingAckReplyCallBack.handleMetadataEntry(log); } currentIndexLag.addAndGet(batchSize); - handleMetadataEntry(new PositionImpl(entry.getLedgerId(), entry.getEntryId()), logs); + handleMetadataEntry(PositionFactory.create(entry.getLedgerId(), entry.getEntryId()), logs); } entry.release(); clearUselessLogData(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleDisabled.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleDisabled.java index 0fc528f880070..4d5852ea33dc4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleDisabled.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleDisabled.java @@ -22,7 +22,6 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; import org.apache.pulsar.broker.service.Consumer; @@ -43,12 +42,12 @@ public class PendingAckHandleDisabled implements PendingAckHandle { @Override public CompletableFuture individualAcknowledgeMessage(TxnID txnID, - List> positions) { + List> positions) { return FutureUtil.failedFuture(new NotAllowedException("The transaction is disabled")); } @Override - public CompletableFuture cumulativeAcknowledgeMessage(TxnID txnID, List positions) { + public CompletableFuture cumulativeAcknowledgeMessage(TxnID txnID, List positions) { return FutureUtil.failedFuture(new NotAllowedException("The transaction is disabled")); } @@ -63,12 +62,12 @@ public CompletableFuture abortTxn(TxnID txnId, Consumer consumer, long low } @Override - public void syncBatchPositionAckSetForTransaction(PositionImpl position) { + public void syncBatchPositionAckSetForTransaction(Position position) { //no operation } @Override - public boolean checkIsCanDeleteConsumerPendingAck(PositionImpl position) { + public boolean checkIsCanDeleteConsumerPendingAck(Position position) { return false; } @@ -103,10 +102,10 @@ public boolean checkIfPendingAckStoreInit() { } @Override - public PositionImpl getPositionInPendingAck(PositionImpl position) { + public Position getPositionInPendingAck(Position position) { return null; } - public PositionInPendingAckStats checkPositionInPendingAckState(PositionImpl position, Integer batchIndex) { + public PositionInPendingAckStats checkPositionInPendingAckState(Position position, Integer batchIndex) { return null; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java index 9d07af4d26c44..98d0d3bf1b9e5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.transaction.pendingack.impl; +import static org.apache.bookkeeper.mledger.impl.AckSetStateUtil.createPositionWithAckSet; +import static org.apache.bookkeeper.mledger.impl.AckSetStateUtil.getAckSetArrayOrNull; import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.andAckSet; import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.compareToWithAckSet; import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.isAckSetOverlap; @@ -42,8 +44,9 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.AckSetState; +import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -84,7 +87,7 @@ public class PendingAckHandleImpl extends PendingAckHandleState implements Pendi * If the position is batch position and it exits the map, will do operation `and` for this * two positions bit set. */ - private LinkedMap> individualAckOfTransaction; + private LinkedMap> individualAckOfTransaction; /** * The map is for individual ack of positions for transaction. @@ -104,13 +107,13 @@ public class PendingAckHandleImpl extends PendingAckHandleState implements Pendi *

* If it does not exits the map, the position will be added to the map. */ - private ConcurrentSkipListMap> individualAckPositions; + private ConcurrentSkipListMap> individualAckPositions; /** * The map is for transaction with position witch was cumulative acked by this transaction. * Only one cumulative ack position was acked by one transaction at the same time. */ - private Pair cumulativeAckOfTransaction; + private Pair cumulativeAckOfTransaction; private final String topicName; @@ -206,12 +209,12 @@ private void initPendingAckStore() { } private void addIndividualAcknowledgeMessageRequest(TxnID txnID, - List> positions, + List> positions, CompletableFuture completableFuture) { acceptQueue.add(() -> internalIndividualAcknowledgeMessage(txnID, positions, completableFuture)); } - public void internalIndividualAcknowledgeMessage(TxnID txnID, List> positions, + public void internalIndividualAcknowledgeMessage(TxnID txnID, List> positions, CompletableFuture completableFuture) { if (txnID == null) { completableFuture.completeExceptionally(new NotAllowedException("txnID can not be null.")); @@ -226,14 +229,14 @@ public void internalIndividualAcknowledgeMessage(TxnID txnID, List pendingAckStore.appendIndividualAck(txnID, positions).thenAccept(v -> { synchronized (org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl.this) { - for (MutablePair positionIntegerMutablePair : positions) { + for (MutablePair positionIntegerMutablePair : positions) { if (log.isDebugEnabled()) { log.debug("[{}] individualAcknowledgeMessage position: [{}], " + "txnId: [{}], subName: [{}]", topicName, positionIntegerMutablePair.left, txnID, subName); } - PositionImpl position = positionIntegerMutablePair.left; + Position position = positionIntegerMutablePair.left; // If try to ack message already acked by committed transaction or // normal acknowledge,throw exception. @@ -247,12 +250,13 @@ public void internalIndividualAcknowledgeMessage(TxnID txnID, List bitSetRecyclable.size()) { bitSetRecyclable.set(positionIntegerMutablePair.right); } @@ -273,8 +277,8 @@ public void internalIndividualAcknowledgeMessage(TxnID txnID, List individualAcknowledgeMessage(TxnID txnID, - List> positions) { + List> positions) { CompletableFuture completableFuture = new CompletableFuture<>(); internalPinnedExecutor.execute(() -> { if (!checkIfReady()) { @@ -348,13 +352,13 @@ public CompletableFuture individualAcknowledgeMessage(TxnID txnID, } private void addCumulativeAcknowledgeMessageRequest(TxnID txnID, - List positions, + List positions, CompletableFuture completableFuture) { acceptQueue.add(() -> internalCumulativeAcknowledgeMessage(txnID, positions, completableFuture)); } public void internalCumulativeAcknowledgeMessage(TxnID txnID, - List positions, + List positions, CompletableFuture completableFuture) { if (txnID == null) { completableFuture.completeExceptionally(new NotAllowedException("TransactionID can not be null.")); @@ -373,7 +377,7 @@ public void internalCumulativeAcknowledgeMessage(TxnID txnID, return; } - PositionImpl position = positions.get(0); + Position position = positions.get(0); this.pendingAckStoreFuture.thenAccept(pendingAckStore -> pendingAckStore.appendCumulativeAck(txnID, position).thenAccept(v -> { @@ -382,7 +386,7 @@ public void internalCumulativeAcknowledgeMessage(TxnID txnID, + "txnID:[{}], subName: [{}].", topicName, txnID, position, subName); } - if (position.compareTo((PositionImpl) persistentSubscription.getCursor() + if (position.compareTo(persistentSubscription.getCursor() .getMarkDeletedPosition()) <= 0) { String errorMsg = "[" + topicName + "][" + subName + "] Transaction:" + txnID + " try to cumulative ack position: " + position + " within range of cursor's " @@ -420,7 +424,7 @@ public void internalCumulativeAcknowledgeMessage(TxnID txnID, } @Override - public CompletableFuture cumulativeAcknowledgeMessage(TxnID txnID, List positions) { + public CompletableFuture cumulativeAcknowledgeMessage(TxnID txnID, List positions) { CompletableFuture completableFuture = new CompletableFuture<>(); internalPinnedExecutor.execute(() -> { if (!checkIfReady()) { @@ -489,7 +493,7 @@ private void internalCommitTxn(TxnID txnID, Map properties, long l pendingAckStore.appendCommitMark(txnID, AckType.Individual).thenAccept(v -> { synchronized (PendingAckHandleImpl.this) { if (individualAckOfTransaction != null && individualAckOfTransaction.containsKey(txnID)) { - HashMap pendingAckMessageForCurrentTxn = + HashMap pendingAckMessageForCurrentTxn = individualAckOfTransaction.get(txnID); if (log.isDebugEnabled()) { log.debug("[{}] Transaction pending ack store commit txnId : " @@ -581,7 +585,7 @@ public CompletableFuture internalAbortTxn(TxnID txnId, Consumer consumer, pendingAckStoreFuture.thenAccept(pendingAckStore -> pendingAckStore.appendAbortMark(txnId, AckType.Individual).thenAccept(v -> { synchronized (PendingAckHandleImpl.this) { - HashMap pendingAckMessageForCurrentTxn = + HashMap pendingAckMessageForCurrentTxn = individualAckOfTransaction.get(txnId); if (pendingAckMessageForCurrentTxn != null) { if (log.isDebugEnabled()) { @@ -676,7 +680,7 @@ private void handleLowWaterMark(TxnID txnID, long lowWaterMark) { } @Override - public synchronized void syncBatchPositionAckSetForTransaction(PositionImpl position) { + public synchronized void syncBatchPositionAckSetForTransaction(Position position) { if (individualAckPositions == null) { individualAckPositions = new ConcurrentSkipListMap<>(); } @@ -690,13 +694,14 @@ public synchronized void syncBatchPositionAckSetForTransaction(PositionImpl posi } @Override - public synchronized boolean checkIsCanDeleteConsumerPendingAck(PositionImpl position) { + public synchronized boolean checkIsCanDeleteConsumerPendingAck(Position position) { if (!individualAckPositions.containsKey(position)) { return true; } else { position = individualAckPositions.get(position).left; - if (position.hasAckSet()) { - BitSetRecyclable bitSetRecyclable = BitSetRecyclable.valueOf(position.getAckSet()); + long[] positionAckSet = getAckSetArrayOrNull(position); + if (positionAckSet != null) { + BitSetRecyclable bitSetRecyclable = BitSetRecyclable.valueOf(positionAckSet); if (bitSetRecyclable.isEmpty()) { bitSetRecyclable.recycle(); return true; @@ -715,7 +720,7 @@ protected void handleAbort(TxnID txnID, AckType ackType) { this.cumulativeAckOfTransaction = null; } else { if (this.individualAckOfTransaction != null) { - HashMap pendingAckMessageForCurrentTxn = + HashMap pendingAckMessageForCurrentTxn = individualAckOfTransaction.get(txnID); if (pendingAckMessageForCurrentTxn != null) { individualAckAbortCommon(txnID, pendingAckMessageForCurrentTxn); @@ -724,24 +729,25 @@ protected void handleAbort(TxnID txnID, AckType ackType) { } } - private void individualAckAbortCommon(TxnID txnID, HashMap currentTxn) { - for (Map.Entry entry : + private void individualAckAbortCommon(TxnID txnID, HashMap currentTxn) { + for (Map.Entry entry : currentTxn.entrySet()) { - if (entry.getValue().hasAckSet() + long[] entryValueAckSet = getAckSetArrayOrNull(entry.getValue()); + if (entryValueAckSet != null && individualAckPositions.containsKey(entry.getValue())) { BitSetRecyclable thisBitSet = - BitSetRecyclable.valueOf(entry.getValue().getAckSet()); + BitSetRecyclable.valueOf(entryValueAckSet); int batchSize = individualAckPositions.get(entry.getValue()).right; thisBitSet.flip(0, batchSize); + AckSetState individualAckPositionAckSetState = + AckSetStateUtil.getAckSetState(individualAckPositions.get(entry.getValue()).left); BitSetRecyclable otherBitSet = - BitSetRecyclable.valueOf(individualAckPositions - .get(entry.getValue()).left.getAckSet()); + BitSetRecyclable.valueOf(individualAckPositionAckSetState.getAckSet()); otherBitSet.or(thisBitSet); if (otherBitSet.cardinality() == batchSize) { individualAckPositions.remove(entry.getValue()); } else { - individualAckPositions.get(entry.getKey()) - .left.setAckSet(otherBitSet.toLongArray()); + individualAckPositionAckSetState.setAckSet(otherBitSet.toLongArray()); } otherBitSet.recycle(); thisBitSet.recycle(); @@ -762,7 +768,7 @@ protected void handleCommit(TxnID txnID, AckType ackType, Map prop this.cumulativeAckOfTransaction = null; } else { if (this.individualAckOfTransaction != null) { - HashMap pendingAckMessageForCurrentTxn = + HashMap pendingAckMessageForCurrentTxn = individualAckOfTransaction.get(txnID); if (pendingAckMessageForCurrentTxn != null) { individualAckCommitCommon(txnID, pendingAckMessageForCurrentTxn, null); @@ -772,7 +778,7 @@ protected void handleCommit(TxnID txnID, AckType ackType, Map prop } private void individualAckCommitCommon(TxnID txnID, - HashMap currentTxn, + HashMap currentTxn, Map properties) { if (currentTxn != null) { persistentSubscription.acknowledgeMessage(new ArrayList<>(currentTxn.values()), @@ -781,7 +787,7 @@ private void individualAckCommitCommon(TxnID txnID, } } - private void handleIndividualAck(TxnID txnID, List> positions) { + private void handleIndividualAck(TxnID txnID, List> positions) { for (int i = 0; i < positions.size(); i++) { if (log.isDebugEnabled()) { log.debug("[{}][{}] TxnID:[{}] Individual acks on {}", topicName, @@ -795,11 +801,11 @@ private void handleIndividualAck(TxnID txnID, List(); } - PositionImpl position = positions.get(i).left; + Position position = positions.get(i).left; + long[] positionAckSet = getAckSetArrayOrNull(position); + if (positionAckSet != null) { - if (position.hasAckSet()) { - - HashMap pendingAckMessageForCurrentTxn = + HashMap pendingAckMessageForCurrentTxn = individualAckOfTransaction.computeIfAbsent(txnID, txn -> new HashMap<>()); if (pendingAckMessageForCurrentTxn.containsKey(position)) { @@ -818,20 +824,21 @@ private void handleIndividualAck(TxnID txnID, List positionPair = positions.get(i); - positionPair.left = PositionImpl.get(positionPair.getLeft().getLedgerId(), - positionPair.getLeft().getEntryId(), - Arrays.copyOf(positionPair.left.getAckSet(), positionPair.left.getAckSet().length)); - this.individualAckPositions.put(position, positions.get(i)); + MutablePair positionPair = positions.get(i); + long[] positionPairLeftAckSet = getAckSetArrayOrNull(positionPair.left); + positionPair.left = createPositionWithAckSet(positionPair.left.getLedgerId(), + positionPair.left.getEntryId(), + Arrays.copyOf(positionPairLeftAckSet, positionPairLeftAckSet.length)); + this.individualAckPositions.put(position, positionPair); } else { - MutablePair positionPair = + MutablePair positionPair = this.individualAckPositions.get(position); positionPair.setRight(positions.get(i).right); andAckSet(positionPair.getLeft(), position); } } else { - HashMap pendingAckMessageForCurrentTxn = + HashMap pendingAckMessageForCurrentTxn = individualAckOfTransaction.computeIfAbsent(txnID, txn -> new HashMap<>()); pendingAckMessageForCurrentTxn.put(position, position); this.individualAckPositions.putIfAbsent(position, positions.get(i)); @@ -839,7 +846,7 @@ private void handleIndividualAck(TxnID txnID, List 0 } } - protected void handleCumulativeAckRecover(TxnID txnID, PositionImpl position) { - if ((position.compareTo((PositionImpl) persistentSubscription.getCursor() + protected void handleCumulativeAckRecover(TxnID txnID, Position position) { + if ((position.compareTo(persistentSubscription.getCursor() .getMarkDeletedPosition()) > 0) && (cumulativeAckOfTransaction == null || (cumulativeAckOfTransaction.getKey().equals(txnID) && compareToWithAckSet(position, cumulativeAckOfTransaction.getValue()) > 0))) { @@ -857,9 +864,9 @@ && compareToWithAckSet(position, cumulativeAckOfTransaction.getValue()) > 0))) { } } - protected void handleIndividualAckRecover(TxnID txnID, List> positions) { - for (MutablePair positionIntegerMutablePair : positions) { - PositionImpl position = positionIntegerMutablePair.left; + protected void handleIndividualAckRecover(TxnID txnID, List> positions) { + for (MutablePair positionIntegerMutablePair : positions) { + Position position = positionIntegerMutablePair.left; // If try to ack message already acked by committed transaction or // normal acknowledge,throw exception. @@ -868,12 +875,14 @@ protected void handleIndividualAckRecover(TxnID txnID, List bitSetRecyclable.size()) { bitSetRecyclable.set(positionIntegerMutablePair.right); } @@ -887,9 +896,8 @@ protected void handleIndividualAckRecover(TxnID txnID, List getStoreManageLedger() { } @Override - public PositionInPendingAckStats checkPositionInPendingAckState(PositionImpl position, Integer batchIndex) { + public PositionInPendingAckStats checkPositionInPendingAckState(Position position, Integer batchIndex) { if (!state.equals(State.Ready)) { return new PositionInPendingAckStats(PositionInPendingAckStats.State.PendingAckNotReady); } if (persistentSubscription.getCursor().getPersistentMarkDeletedPosition() != null && position.compareTo( - (PositionImpl) persistentSubscription.getCursor().getPersistentMarkDeletedPosition()) <= 0) { + persistentSubscription.getCursor().getPersistentMarkDeletedPosition()) <= 0) { return new PositionInPendingAckStats(PositionInPendingAckStats.State.MarkDelete); } else if (individualAckPositions == null) { return new PositionInPendingAckStats(PositionInPendingAckStats.State.NotInPendingAck); } - MutablePair positionIntegerMutablePair = individualAckPositions.get(position); + MutablePair positionIntegerMutablePair = individualAckPositions.get(position); if (positionIntegerMutablePair != null) { if (batchIndex == null) { return new PositionInPendingAckStats(PositionInPendingAckStats.State.PendingAck); @@ -1079,7 +1086,7 @@ public PositionInPendingAckStats checkPositionInPendingAckState(PositionImpl pos return new PositionInPendingAckStats(PositionInPendingAckStats.State.InvalidPosition); } BitSetRecyclable bitSetRecyclable = BitSetRecyclable - .valueOf(positionIntegerMutablePair.left.getAckSet()); + .valueOf(getAckSetArrayOrNull(positionIntegerMutablePair.left)); if (bitSetRecyclable.get(batchIndex)) { bitSetRecyclable.recycle(); return new PositionInPendingAckStats(PositionInPendingAckStats.State.NotInPendingAck); @@ -1094,7 +1101,7 @@ public PositionInPendingAckStats checkPositionInPendingAckState(PositionImpl pos } @VisibleForTesting - public Map> getIndividualAckPositions() { + public Map> getIndividualAckPositions() { return individualAckPositions; } @@ -1104,9 +1111,9 @@ public boolean checkIfPendingAckStoreInit() { } @Override - public PositionImpl getPositionInPendingAck(PositionImpl position) { + public Position getPositionInPendingAck(Position position) { if (individualAckPositions != null) { - MutablePair positionPair = this.individualAckPositions.get(position); + MutablePair positionPair = this.individualAckPositions.get(position); if (positionPair != null) { return positionPair.getLeft(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java index 146ba4327d252..563b826ac74c7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java @@ -24,7 +24,6 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.Consumer; public interface CompactedTopic { @@ -35,14 +34,14 @@ public interface CompactedTopic { * Read entries from compacted topic. * * @deprecated Use {@link CompactedTopicUtils#asyncReadCompactedEntries(TopicCompactionService, ManagedCursor, - * int, long, org.apache.bookkeeper.mledger.impl.PositionImpl, boolean, ReadEntriesCallback, boolean, Consumer)} + * int, long, org.apache.bookkeeper.mledger.Position, boolean, ReadEntriesCallback, boolean, Consumer)} * instead. */ @Deprecated void asyncReadEntriesOrWait(ManagedCursor cursor, int maxEntries, long bytesToRead, - PositionImpl maxReadPosition, + Position maxReadPosition, boolean isFirstRead, ReadEntriesCallback callback, Consumer consumer); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index f67f28733bddb..baa71ffc645d6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -43,9 +43,9 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer.ReadEntriesCtx; import org.apache.pulsar.client.api.MessageId; @@ -66,7 +66,7 @@ public class CompactedTopicImpl implements CompactedTopic { private final BookKeeper bk; - private volatile PositionImpl compactionHorizon = null; + private volatile Position compactionHorizon = null; private volatile CompletableFuture compactedTopicContext = null; public CompactedTopicImpl(BookKeeper bk) { @@ -79,7 +79,7 @@ public CompletableFuture newCompactedLedger(Position p, l CompletableFuture previousContext = compactedTopicContext; compactedTopicContext = openCompactedLedger(bk, compactedLedgerId); - compactionHorizon = (PositionImpl) p; + compactionHorizon = p; // delete the ledger from the old context once the new one is open return compactedTopicContext.thenCompose( @@ -97,24 +97,24 @@ public CompletableFuture deleteCompactedLedger(long compactedLedgerId) { public void asyncReadEntriesOrWait(ManagedCursor cursor, int maxEntries, long bytesToRead, - PositionImpl maxReadPosition, + Position maxReadPosition, boolean isFirstRead, ReadEntriesCallback callback, Consumer consumer) { - PositionImpl cursorPosition; + Position cursorPosition; boolean readFromEarliest = isFirstRead && MessageId.earliest.equals(consumer.getStartMessageId()) && (!cursor.isDurable() || cursor.getName().equals(Compactor.COMPACTION_SUBSCRIPTION) || cursor.getMarkDeletedPosition() == null || cursor.getMarkDeletedPosition().getEntryId() == -1L); if (readFromEarliest){ - cursorPosition = PositionImpl.EARLIEST; + cursorPosition = PositionFactory.EARLIEST; } else { - cursorPosition = (PositionImpl) cursor.getReadPosition(); + cursorPosition = cursor.getReadPosition(); } // TODO: redeliver epoch link https://github.com/apache/pulsar/issues/13690 ReadEntriesCtx readEntriesCtx = ReadEntriesCtx.create(consumer, DEFAULT_CONSUMER_EPOCH); - final PositionImpl currentCompactionHorizon = compactionHorizon; + final Position currentCompactionHorizon = compactionHorizon; if (currentCompactionHorizon == null || currentCompactionHorizon.compareTo(cursorPosition) < 0) { @@ -166,7 +166,7 @@ public void asyncReadEntriesOrWait(ManagedCursor cursor, } } - static CompletableFuture findStartPoint(PositionImpl p, + static CompletableFuture findStartPoint(Position p, long lastEntryId, AsyncLoadingCache cache) { CompletableFuture promise = new CompletableFuture<>(); @@ -180,7 +180,7 @@ static CompletableFuture findStartPoint(PositionImpl p, } @VisibleForTesting - static void findStartPointLoop(PositionImpl p, long start, long end, + static void findStartPointLoop(Position p, long start, long end, CompletableFuture promise, AsyncLoadingCache cache) { long midpoint = start + ((end - start) / 2); @@ -374,7 +374,7 @@ private static void findFirstMatchIndexLoop(final Predicate predicate, }); } - private static int comparePositionAndMessageId(PositionImpl p, MessageIdData m) { + private static int comparePositionAndMessageId(Position p, MessageIdData m) { return ComparisonChain.start() .compare(p.getLedgerId(), m.getLedgerId()) .compare(p.getEntryId(), m.getEntryId()).result(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java index d3464d402e9c6..aae332acfcbbc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java @@ -30,8 +30,8 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.CollectionUtils; import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; @@ -42,7 +42,7 @@ public class CompactedTopicUtils { @Beta public static void asyncReadCompactedEntries(TopicCompactionService topicCompactionService, ManagedCursor cursor, int maxEntries, - long bytesToRead, PositionImpl maxReadPosition, + long bytesToRead, Position maxReadPosition, boolean readFromEarliest, AsyncCallbacks.ReadEntriesCallback callback, boolean wait, @Nullable Consumer consumer) { Objects.requireNonNull(topicCompactionService); @@ -50,11 +50,11 @@ public static void asyncReadCompactedEntries(TopicCompactionService topicCompact checkArgument(maxEntries > 0); Objects.requireNonNull(callback); - final PositionImpl readPosition; + final Position readPosition; if (readFromEarliest) { - readPosition = PositionImpl.EARLIEST; + readPosition = PositionFactory.EARLIEST; } else { - readPosition = (PositionImpl) cursor.getReadPosition(); + readPosition = cursor.getReadPosition(); } // TODO: redeliver epoch link https://github.com/apache/pulsar/issues/13690 diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PulsarTopicCompactionService.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PulsarTopicCompactionService.java index 16543bc7aa77f..27efcf9524f8f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PulsarTopicCompactionService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PulsarTopicCompactionService.java @@ -34,7 +34,6 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.common.api.proto.BrokerEntryMetadata; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.FutureUtil; @@ -75,7 +74,7 @@ public CompletableFuture> readCompactedEntries(@Nonnull Position sta CompletableFuture> resultFuture = new CompletableFuture<>(); Objects.requireNonNull(compactedTopic.getCompactedTopicContextFuture()).thenCompose( - (context) -> findStartPoint((PositionImpl) startPosition, context.ledger.getLastAddConfirmed(), + (context) -> findStartPoint(startPosition, context.ledger.getLastAddConfirmed(), context.cache).thenCompose((startPoint) -> { if (startPoint == COMPACT_LEDGER_EMPTY || startPoint == NEWER_THAN_COMPACTED) { return CompletableFuture.completedFuture(Collections.emptyList()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index 635b2c25bc1d0..1c83941d6e721 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -65,7 +65,7 @@ import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedLedgerInfo; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -3481,7 +3481,7 @@ public void testGetReadPositionWhenJoining() throws Exception { Assert.assertEquals(subStats.getConsumers().size(), 2); ConsumerStats consumerStats = subStats.getConsumers().get(0); Assert.assertEquals(consumerStats.getReadPositionWhenJoining(), - PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId() + 1).toString()); + PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId() + 1).toString()); for (Consumer consumer : consumers) { consumer.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java index 5a192d0159a42..1cc20b04c2137 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java @@ -36,7 +36,8 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import lombok.Cleanup; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.http.HttpStatus; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; @@ -49,8 +50,8 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.BatchMessageIdImpl; @@ -189,8 +190,8 @@ public void testGetTransactionInBufferStats() throws Exception { TransactionInBufferStats transactionInBufferStats = admin.transactions() .getTransactionInBufferStatsAsync(new TxnID(transaction.getTxnIdMostBits(), transaction.getTxnIdLeastBits()), topic).get(); - PositionImpl position = - PositionImpl.get(((MessageIdImpl) messageId).getLedgerId(), ((MessageIdImpl) messageId).getEntryId()); + Position position = + PositionFactory.create(((MessageIdImpl) messageId).getLedgerId(), ((MessageIdImpl) messageId).getEntryId()); assertEquals(transactionInBufferStats.startPosition, position.toString()); assertFalse(transactionInBufferStats.aborted); @@ -310,10 +311,10 @@ public void testGetTransactionMetadata() throws Exception { Map producedPartitions = transactionMetadata.producedPartitions; Map> ackedPartitions = transactionMetadata.ackedPartitions; - PositionImpl position1 = getPositionByMessageId(messageId1); - PositionImpl position2 = getPositionByMessageId(messageId2); - PositionImpl position3 = getPositionByMessageId(messageId3); - PositionImpl position4 = getPositionByMessageId(messageId4); + Position position1 = getPositionByMessageId(messageId1); + Position position2 = getPositionByMessageId(messageId2); + Position position3 = getPositionByMessageId(messageId3); + Position position4 = getPositionByMessageId(messageId4); assertFalse(producedPartitions.get(topic1).aborted); assertFalse(producedPartitions.get(topic2).aborted); @@ -375,7 +376,7 @@ public void testGetTransactionBufferStats() throws Exception { assertEquals(transactionBufferStats.state, "Ready"); assertEquals(transactionBufferStats.maxReadPosition, - PositionImpl.get(((MessageIdImpl) messageId).getLedgerId(), + PositionFactory.create(((MessageIdImpl) messageId).getLedgerId(), ((MessageIdImpl) messageId).getEntryId() + 1).toString()); assertTrue(transactionBufferStats.lastSnapshotTimestamps > currentTime); assertNull(transactionBufferStats.lowWaterMarks); @@ -509,8 +510,8 @@ public void testGetSlowTransactions() throws Exception { assertEquals(transactionMetadata.timeoutAt, 60000); } - private static PositionImpl getPositionByMessageId(MessageId messageId) { - return PositionImpl.get(((MessageIdImpl) messageId).getLedgerId(), ((MessageIdImpl) messageId).getEntryId()); + private static Position getPositionByMessageId(MessageId messageId) { + return PositionFactory.create(((MessageIdImpl) messageId).getLedgerId(), ((MessageIdImpl) messageId).getEntryId()); } @Test(timeOut = 20000) @@ -963,7 +964,7 @@ public void testPeekMessageFoReadCommittedMessages() throws Exception { } else { producer.newMessage(txn).value("msg-aborted").send(); txn.abort(); - } + } } // Then sends 1 uncommitted transactional messages. Transaction txn = pulsarClient.newTransaction().build().get(); @@ -972,7 +973,7 @@ public void testPeekMessageFoReadCommittedMessages() throws Exception { for (int i = 0; i < n - 1; i++) { producer.newMessage().value("msg-after-uncommitted").send(); } - + // peek n message, all messages value should be "msg" { List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", n, @@ -1035,11 +1036,11 @@ public void testPeekMessageForShowAllMessages() throws Exception { } else { String value = new String(peekMsg.getValue()); assertTrue(value.equals("msg") || value.equals("msg-aborted")); - } + } } for (int i = 4 * n; i < peekMsgs.size(); i++) { Message peekMsg = peekMsgs.get(i); - assertEquals(new String(peekMsg.getValue()), "msg-uncommitted"); + assertEquals(new String(peekMsg.getValue()), "msg-uncommitted"); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java index 1d166a8db5c9e..8b72411329c65 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java @@ -37,7 +37,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; @@ -82,7 +82,7 @@ public void test(DelayedDeliveryTracker tracker) throws Exception { assertEquals(tracker.getNumberOfDelayedMessages(), 5); assertTrue(tracker.hasMessageAvailable()); - Set scheduled = tracker.getScheduledMessages(10); + Set scheduled = tracker.getScheduledMessages(10); assertEquals(scheduled.size(), 1); // Move time forward diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/MockManagedCursor.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/MockManagedCursor.java index 477290fc6837a..e0e679b113f33 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/MockManagedCursor.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/MockManagedCursor.java @@ -32,7 +32,6 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; public class MockManagedCursor implements ManagedCursor { @@ -103,13 +102,13 @@ public List readEntries(int numberOfEntriesToRead) throws InterruptedExce @Override public void asyncReadEntries(int numberOfEntriesToRead, AsyncCallbacks.ReadEntriesCallback callback, Object ctx, - PositionImpl maxPosition) { + Position maxPosition) { } @Override public void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, - AsyncCallbacks.ReadEntriesCallback callback, Object ctx, PositionImpl maxPosition) { + AsyncCallbacks.ReadEntriesCallback callback, Object ctx, Position maxPosition) { } @@ -139,13 +138,13 @@ public List readEntriesOrWait(int maxEntries, long maxSizeBytes) @Override public void asyncReadEntriesOrWait(int numberOfEntriesToRead, AsyncCallbacks.ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition) { + Object ctx, Position maxPosition) { } @Override public void asyncReadEntriesOrWait(int maxEntries, long maxSizeBytes, AsyncCallbacks.ReadEntriesCallback callback, - Object ctx, PositionImpl maxPosition) { + Object ctx, Position maxPosition) { } @@ -387,7 +386,7 @@ public ManagedLedger getManagedLedger() { } @Override - public Range getLastIndividualDeletedRange() { + public Range getLastIndividualDeletedRange() { return null; } @@ -397,7 +396,7 @@ public void trimDeletedEntries(List entries) { } @Override - public long[] getDeletedBatchIndexesAsLongArray(PositionImpl position) { + public long[] getDeletedBatchIndexesAsLongArray(Position position) { return new long[0]; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java index 1e3e72aa0ec44..d09249deb5be2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java @@ -45,7 +45,8 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.mledger.ManagedCursor; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.commons.lang3.mutable.MutableLong; import org.apache.pulsar.broker.delayed.AbstractDeliveryTrackerTest; import org.apache.pulsar.broker.delayed.MockBucketSnapshotStorage; @@ -165,7 +166,7 @@ public void testContainsMessage(BucketDelayedDeliveryTracker tracker) { assertTrue(tracker.containsMessage(1, 1)); clockTime.set(20); - Set scheduledMessages = tracker.getScheduledMessages(1); + Set scheduledMessages = tracker.getScheduledMessages(1); assertEquals(scheduledMessages.stream().findFirst().get().getEntryId(), 1); tracker.addMessage(3, 3, 30); @@ -198,7 +199,7 @@ public void testRecoverSnapshot(BucketDelayedDeliveryTracker tracker) { }); assertTrue(tracker.hasMessageAvailable()); - Set scheduledMessages = new TreeSet<>(); + Set scheduledMessages = new TreeSet<>(); Awaitility.await().untilAsserted(() -> { scheduledMessages.addAll(tracker.getScheduledMessages(100)); assertEquals(scheduledMessages.size(), 1); @@ -219,7 +220,7 @@ public void testRecoverSnapshot(BucketDelayedDeliveryTracker tracker) { clockTime.set(100 * 10); assertTrue(tracker2.hasMessageAvailable()); - Set scheduledMessages2 = new TreeSet<>(); + Set scheduledMessages2 = new TreeSet<>(); Awaitility.await().untilAsserted(() -> { scheduledMessages2.addAll(tracker2.getScheduledMessages(70)); @@ -227,8 +228,8 @@ public void testRecoverSnapshot(BucketDelayedDeliveryTracker tracker) { }); int i = 31; - for (PositionImpl scheduledMessage : scheduledMessages2) { - assertEquals(scheduledMessage, PositionImpl.get(i, i)); + for (Position scheduledMessage : scheduledMessages2) { + assertEquals(scheduledMessage, PositionFactory.create(i, i)); i++; } @@ -304,14 +305,14 @@ public void testMergeSnapshot(final BucketDelayedDeliveryTracker tracker) { clockTime.set(110 * 10); - NavigableSet scheduledMessages = new TreeSet<>(); + NavigableSet scheduledMessages = new TreeSet<>(); Awaitility.await().untilAsserted(() -> { scheduledMessages.addAll(tracker2.getScheduledMessages(110)); assertEquals(scheduledMessages.size(), 110); }); for (int i = 1; i <= 110; i++) { - PositionImpl position = scheduledMessages.pollFirst(); - assertEquals(position, PositionImpl.get(i, i)); + Position position = scheduledMessages.pollFirst(); + assertEquals(position, PositionFactory.create(i, i)); } tracker2.close(); @@ -380,7 +381,7 @@ public void testWithBkException(final BucketDelayedDeliveryTracker tracker) { assertEquals(tracker2.getScheduledMessages(100).size(), 0); - Set scheduledMessages = new TreeSet<>(); + Set scheduledMessages = new TreeSet<>(); Awaitility.await().untilAsserted(() -> { scheduledMessages.addAll(tracker2.getScheduledMessages(100)); assertEquals(scheduledMessages.size(), delayedMessagesInSnapshotValue); @@ -418,10 +419,10 @@ public void testWithCreateFailDowngrade(BucketDelayedDeliveryTracker tracker) { assertEquals(6, tracker.getNumberOfDelayedMessages()); - NavigableSet scheduledMessages = tracker.getScheduledMessages(5); + NavigableSet scheduledMessages = tracker.getScheduledMessages(5); for (int i = 1; i <= 5; i++) { - PositionImpl position = scheduledMessages.pollFirst(); - assertEquals(position, PositionImpl.get(i, i)); + Position position = scheduledMessages.pollFirst(); + assertEquals(position, PositionFactory.create(i, i)); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java index 7d164b68147ab..16953d76ade45 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java @@ -18,8 +18,15 @@ */ package org.apache.pulsar.broker.intercept; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertNotNull; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import java.nio.charset.StandardCharsets; +import java.util.HashSet; +import java.util.List; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.function.Predicate; @@ -35,7 +42,6 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.OpAddEntry; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.pulsar.common.api.proto.BrokerEntryMetadata; @@ -49,15 +55,6 @@ import org.testng.Assert; import org.testng.annotations.Test; -import java.nio.charset.StandardCharsets; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotEquals; -import static org.testng.Assert.assertNotNull; - @Test(groups = "broker") public class MangedLedgerInterceptorImplTest extends MockedBookKeeperTestCase { private static final Logger log = LoggerFactory.getLogger(MangedLedgerInterceptorImplTest.class); @@ -264,9 +261,9 @@ public void testFindPositionByIndex() throws Exception { assertEquals(((ManagedLedgerInterceptorImpl) ledger.getManagedLedgerInterceptor()).getIndex(), 9); - PositionImpl position = null; + Position position = null; for (int index = 0; index <= ((ManagedLedgerInterceptorImpl) ledger.getManagedLedgerInterceptor()).getIndex(); index ++) { - position = (PositionImpl) ledger.asyncFindPosition(new IndexSearchPredicate(index)).get(); + position = ledger.asyncFindPosition(new IndexSearchPredicate(index)).get(); assertEquals(position.getEntryId(), (index % maxSequenceIdPerLedger) / MOCK_BATCH_SIZE); } @@ -279,7 +276,7 @@ public void testFindPositionByIndex() throws Exception { assertNotEquals(firstLedgerId, secondLedgerId); for (int index = 0; index <= ((ManagedLedgerInterceptorImpl) ledger.getManagedLedgerInterceptor()).getIndex(); index ++) { - position = (PositionImpl) ledger.asyncFindPosition(new IndexSearchPredicate(index)).get(); + position = ledger.asyncFindPosition(new IndexSearchPredicate(index)).get(); assertEquals(position.getEntryId(), (index % maxSequenceIdPerLedger) / MOCK_BATCH_SIZE); } @@ -298,7 +295,7 @@ public void testFindPositionByIndex() throws Exception { assertNotEquals(secondLedgerId, thirdLedgerId); for (int index = 0; index <= ((ManagedLedgerInterceptorImpl) ledger.getManagedLedgerInterceptor()).getIndex(); index ++) { - position = (PositionImpl) ledger.asyncFindPosition(new IndexSearchPredicate(index)).get(); + position = ledger.asyncFindPosition(new IndexSearchPredicate(index)).get(); assertEquals(position.getEntryId(), (index % maxSequenceIdPerLedger) / MOCK_BATCH_SIZE); } cursor.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractBaseDispatcherTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractBaseDispatcherTest.java index 0a1213ac1a860..6866e09731301 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractBaseDispatcherTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractBaseDispatcherTest.java @@ -33,8 +33,8 @@ import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.EntryImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; @@ -317,7 +317,7 @@ public void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpoc } @Override - public void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { + public void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java index 7aebf20896c2c..d20f5f0d520e9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java @@ -34,7 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; @@ -136,7 +136,7 @@ protected void setProducerAndTriggerReadEntries(Producer producer) { @Override protected Position getReplicatorReadPosition() { - return PositionImpl.EARLIEST; + return PositionFactory.EARLIEST; } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java index 8e902d5d1e700..ed7f6974dd26f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java @@ -38,8 +38,8 @@ import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -478,7 +478,7 @@ public void testUnAckMessagesWhenConcurrentDeliveryAndAck() throws Exception { ledgerId = msgId.getLedgerId(); entryId = msgId.getEntryId(); } - getCursor(topicName, subName).delete(PositionImpl.get(ledgerId, entryId, bitSetRecyclable.toLongArray())); + getCursor(topicName, subName).delete(AckSetStateUtil.createPositionWithAckSet(ledgerId, entryId, bitSetRecyclable.toLongArray())); // step 4: send messages to consumer2. receiveMessageSignal2.complete(null); // Verify: Consumer2 will get all the 100 messages, and "unAckMessages" is 100. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java index 2ce4ea9b00b2e..195d0155a31c6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java @@ -23,9 +23,9 @@ import static org.testng.Assert.assertTrue; import java.time.Duration; import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.MessageDeduplication; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -88,8 +88,8 @@ public void testNoBacklogOnDeduplication() throws Exception { producer.close(); ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); Awaitility.await().atMost(Duration.ofSeconds(deduplicationSnapshotFrequency * 3)).untilAsserted(() -> { - PositionImpl LAC = (PositionImpl) ml.getLastConfirmedEntry(); - PositionImpl cursorMD = (PositionImpl) cursor.getMarkDeletedPosition(); + Position LAC = ml.getLastConfirmedEntry(); + Position cursorMD = cursor.getMarkDeletedPosition(); assertTrue(LAC.compareTo(cursorMD) <= 0); }); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageTTLTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageTTLTest.java index 2f5ad215a1b6e..909702445f715 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageTTLTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageTTLTest.java @@ -34,7 +34,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import lombok.Cleanup; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.MessageId; @@ -108,7 +108,7 @@ public void testMessageExpiryAfterTopicUnload() throws Exception { CursorStats statsBeforeExpire = internalStatsBeforeExpire.cursors.get(subscriptionName); log.info("markDeletePosition before expire {}", statsBeforeExpire.markDeletePosition); assertEquals(statsBeforeExpire.markDeletePosition, - PositionImpl.get(firstMessageId.getLedgerId(), -1).toString()); + PositionFactory.create(firstMessageId.getLedgerId(), -1).toString()); Awaitility.await().timeout(30, TimeUnit.SECONDS) .pollDelay(3, TimeUnit.SECONDS).untilAsserted(() -> { @@ -118,7 +118,7 @@ public void testMessageExpiryAfterTopicUnload() throws Exception { PersistentTopicInternalStats internalStatsAfterExpire = admin.topics().getInternalStats(topicName); CursorStats statsAfterExpire = internalStatsAfterExpire.cursors.get(subscriptionName); log.info("markDeletePosition after expire {}", statsAfterExpire.markDeletePosition); - assertEquals(statsAfterExpire.markDeletePosition, PositionImpl.get(lastMessageId.getLedgerId(), + assertEquals(statsAfterExpire.markDeletePosition, PositionFactory.create(lastMessageId.getLedgerId(), lastMessageId.getEntryId() ).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 ba680e4bcd74c..f30ee62b64659 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 @@ -61,8 +61,8 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; @@ -202,7 +202,7 @@ void setupMLAsyncCallbackMocks() { // call addComplete on ledger asyncAddEntry doAnswer(invocationOnMock -> { ((AddEntryCallback) invocationOnMock.getArguments()[1]).addComplete( - new PositionImpl(1, 1), null, null); + PositionFactory.create(1, 1), null, null); return null; }).when(ledgerMock).asyncAddEntry(any(byte[].class), any(AddEntryCallback.class), any()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java index 0972c9098b55b..176a799292ac3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java @@ -53,9 +53,9 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.commons.lang3.reflect.FieldUtils; @@ -235,7 +235,7 @@ public void findEntryFailed(ManagedLedgerException exception, Optional PersistentTopic mock = mock(PersistentTopic.class); when(mock.getName()).thenReturn("topicname"); - when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); + when(mock.getLastPosition()).thenReturn(PositionFactory.EARLIEST); PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null); monitor.findEntryFailed(new ManagedLedgerException.ConcurrentFindCursorPositionException("failed"), @@ -406,7 +406,7 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { List ledgers = ledger.getLedgersInfoAsList(); LedgerInfo lastLedgerInfo = ledgers.get(ledgers.size() - 1); - // The `lastLedgerInfo` should be newly opened, and it does not contain any entries. + // The `lastLedgerInfo` should be newly opened, and it does not contain any entries. // Please refer to: https://github.com/apache/pulsar/pull/22034 assertEquals(lastLedgerInfo.getEntries(), 0); assertEquals(ledgers.size(), totalEntries / entriesPerLedger + 1); @@ -420,15 +420,15 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { PersistentTopic mock = mock(PersistentTopic.class); when(mock.getName()).thenReturn("topicname"); - when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); + when(mock.getLastPosition()).thenReturn(PositionFactory.EARLIEST); PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null); assertTrue(monitor.expireMessages(ttlSeconds)); Awaitility.await().untilAsserted(() -> { - PositionImpl markDeletePosition = (PositionImpl) c1.getMarkDeletedPosition(); + Position markDeletePosition = c1.getMarkDeletedPosition(); // The markDeletePosition points to the last entry of the previous ledger in lastLedgerInfo. assertEquals(markDeletePosition.getLedgerId(), lastLedgerInfo.getLedgerId() - 1); - assertEquals(markDeletePosition.getEntryId(), entriesPerLedger - 1); + assertEquals(markDeletePosition.getEntryId(), entriesPerLedger - 1); }); c1.close(); @@ -458,7 +458,7 @@ public void testIncorrectClientClock() throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), entriesNum + 1); PersistentTopic mock = mock(PersistentTopic.class); when(mock.getName()).thenReturn("topicname"); - when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); + when(mock.getLastPosition()).thenReturn(PositionFactory.EARLIEST); PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null); Thread.sleep(TimeUnit.SECONDS.toMillis(maxTTLSeconds)); monitor.expireMessages(maxTTLSeconds); @@ -481,7 +481,7 @@ public void testCheckExpiryByLedgerClosureTimeWithAckUnclosedLedger() throws Thr assertEquals(ledger.getLedgersInfoAsList().size(), 2); PersistentTopic mock = mock(PersistentTopic.class); when(mock.getName()).thenReturn("topicname"); - when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); + when(mock.getLastPosition()).thenReturn(PositionFactory.EARLIEST); PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null); AsyncCallbacks.MarkDeleteCallback markDeleteCallback = (AsyncCallbacks.MarkDeleteCallback) spy( @@ -495,7 +495,7 @@ public void testCheckExpiryByLedgerClosureTimeWithAckUnclosedLedger() throws Thr return invocation.callRealMethod(); }).when(markDeleteCallback).markDeleteFailed(any(), any()); - PositionImpl position = (PositionImpl) ledger.getLastConfirmedEntry(); + Position position = ledger.getLastConfirmedEntry(); c1.markDelete(position); Thread.sleep(TimeUnit.SECONDS.toMillis(maxTTLSeconds)); monitor.expireMessages(maxTTLSeconds); @@ -531,39 +531,39 @@ void testMessageExpiryWithPosition() throws Exception { PersistentMessageExpiryMonitor monitor = spy(new PersistentMessageExpiryMonitor(topic, cursor.getName(), cursor, subscription)); - assertEquals(cursor.getMarkDeletedPosition(), PositionImpl.get(positions.get(0).getLedgerId(), -1)); + assertEquals(cursor.getMarkDeletedPosition(), PositionFactory.create(positions.get(0).getLedgerId(), -1)); boolean issued; // Expire by position and verify mark delete position of cursor. issued = monitor.expireMessages(positions.get(15)); Awaitility.await().untilAsserted(() -> verify(monitor, times(1)).findEntryComplete(any(), any())); - assertEquals(cursor.getMarkDeletedPosition(), PositionImpl.get(positions.get(15).getLedgerId(), positions.get(15).getEntryId())); + assertEquals(cursor.getMarkDeletedPosition(), PositionFactory.create(positions.get(15).getLedgerId(), positions.get(15).getEntryId())); assertTrue(issued); clearInvocations(monitor); // Expire by position beyond last position and nothing should happen. - issued = monitor.expireMessages(PositionImpl.get(100, 100)); - assertEquals(cursor.getMarkDeletedPosition(), PositionImpl.get(positions.get(15).getLedgerId(), positions.get(15).getEntryId())); + issued = monitor.expireMessages(PositionFactory.create(100, 100)); + assertEquals(cursor.getMarkDeletedPosition(), PositionFactory.create(positions.get(15).getLedgerId(), positions.get(15).getEntryId())); assertFalse(issued); // Expire by position again and verify mark delete position of cursor didn't change. issued = monitor.expireMessages(positions.get(15)); Awaitility.await().untilAsserted(() -> verify(monitor, times(1)).findEntryComplete(any(), any())); - assertEquals(cursor.getMarkDeletedPosition(), PositionImpl.get(positions.get(15).getLedgerId(), positions.get(15).getEntryId())); + assertEquals(cursor.getMarkDeletedPosition(), PositionFactory.create(positions.get(15).getLedgerId(), positions.get(15).getEntryId())); assertTrue(issued); clearInvocations(monitor); // Expire by position before current mark delete position and verify mark delete position of cursor didn't change. issued = monitor.expireMessages(positions.get(10)); Awaitility.await().untilAsserted(() -> verify(monitor, times(1)).findEntryComplete(any(), any())); - assertEquals(cursor.getMarkDeletedPosition(), PositionImpl.get(positions.get(15).getLedgerId(), positions.get(15).getEntryId())); + assertEquals(cursor.getMarkDeletedPosition(), PositionFactory.create(positions.get(15).getLedgerId(), positions.get(15).getEntryId())); assertTrue(issued); clearInvocations(monitor); // Expire by position after current mark delete position and verify mark delete position of cursor move to new position. issued = monitor.expireMessages(positions.get(16)); Awaitility.await().untilAsserted(() -> verify(monitor, times(1)).findEntryComplete(any(), any())); - assertEquals(cursor.getMarkDeletedPosition(), PositionImpl.get(positions.get(16).getLedgerId(), positions.get(16).getEntryId())); + assertEquals(cursor.getMarkDeletedPosition(), PositionFactory.create(positions.get(16).getLedgerId(), positions.get(16).getEntryId())); assertTrue(issued); clearInvocations(monitor); 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 1118b71456e84..76f871a6c6035 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 @@ -87,9 +87,9 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; @@ -293,7 +293,7 @@ public void testPublishMessage() throws Exception { final ByteBuf payload = (ByteBuf) invocationOnMock.getArguments()[0]; final AddEntryCallback callback = (AddEntryCallback) invocationOnMock.getArguments()[1]; final Topic.PublishContext ctx = (Topic.PublishContext) invocationOnMock.getArguments()[2]; - callback.addComplete(PositionImpl.LATEST, payload, ctx); + callback.addComplete(PositionFactory.LATEST, payload, ctx); return null; }).when(ledgerMock).asyncAddEntry(any(ByteBuf.class), any(AddEntryCallback.class), any()); @@ -312,8 +312,8 @@ public void testPublishMessage() throws Exception { final Topic.PublishContext publishContext = new Topic.PublishContext() { @Override public void completed(Exception e, long ledgerId, long entryId) { - assertEquals(ledgerId, PositionImpl.LATEST.getLedgerId()); - assertEquals(entryId, PositionImpl.LATEST.getEntryId()); + assertEquals(ledgerId, PositionFactory.LATEST.getLedgerId()); + assertEquals(entryId, PositionFactory.LATEST.getEntryId()); latch.countDown(); } @@ -1461,7 +1461,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { // call addComplete on ledger asyncAddEntry doAnswer(invocationOnMock -> { - ((AddEntryCallback) invocationOnMock.getArguments()[1]).addComplete(new PositionImpl(1, 1), + ((AddEntryCallback) invocationOnMock.getArguments()[1]).addComplete(PositionFactory.create(1, 1), null, invocationOnMock.getArguments()[2]); return null; @@ -1818,7 +1818,7 @@ public void testCompactorSubscription() { PersistentSubscription sub = new PulsarCompactorSubscription(topic, compactedTopic, Compactor.COMPACTION_SUBSCRIPTION, cursorMock); - PositionImpl position = new PositionImpl(1, 1); + Position position = PositionFactory.create(1, 1); long ledgerId = 0xc0bfefeL; sub.acknowledgeMessage(Collections.singletonList(position), AckType.Cumulative, Map.of(Compactor.COMPACTED_TOPIC_LEDGER_PROPERTY, ledgerId)); @@ -1830,7 +1830,7 @@ public void testCompactorSubscription() { public void testCompactorSubscriptionUpdatedOnInit() { long ledgerId = 0xc0bfefeL; Map properties = Map.of(Compactor.COMPACTED_TOPIC_LEDGER_PROPERTY, ledgerId); - PositionImpl position = new PositionImpl(1, 1); + Position position = PositionFactory.create(1, 1); doAnswer((invokactionOnMock) -> properties).when(cursorMock).getProperties(); doAnswer((invokactionOnMock) -> position).when(cursorMock).getMarkDeletedPosition(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index b58f416ea1a57..fb92f3f47b22b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -61,7 +61,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -1626,7 +1626,7 @@ public void testReplicatorWithFailedAck() throws Exception { final ManagedCursorImpl cursor = (ManagedCursorImpl) managedLedger.openCursor("pulsar.repl.r2"); final ManagedCursorImpl spyCursor = spy(cursor); managedLedger.getCursors().removeCursor(cursor.getName()); - managedLedger.getCursors().add(spyCursor, PositionImpl.EARLIEST); + managedLedger.getCursors().add(spyCursor, PositionFactory.EARLIEST); AtomicBoolean isMakeAckFail = new AtomicBoolean(false); doAnswer(invocation -> { Position pos = (Position) invocation.getArguments()[0]; @@ -1649,7 +1649,7 @@ public void testReplicatorWithFailedAck() throws Exception { producer1.produce(2); MessageIdImpl lastMessageId = (MessageIdImpl) topic.getLastMessageId().get(); - Position lastPosition = PositionImpl.get(lastMessageId.getLedgerId(), lastMessageId.getEntryId()); + Position lastPosition = PositionFactory.create(lastMessageId.getLedgerId(), lastMessageId.getEntryId()); Awaitility.await().pollInterval(1, TimeUnit.SECONDS).timeout(30, TimeUnit.SECONDS) .ignoreExceptions() diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index 5387bc4998c6e..27afedd6b101e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -83,18 +83,18 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.TransactionMetadataStoreService; import org.apache.pulsar.broker.auth.MockAlwaysExpiredAuthenticationProvider; +import org.apache.pulsar.broker.auth.MockAuthenticationProvider; import org.apache.pulsar.broker.auth.MockAuthorizationProvider; +import org.apache.pulsar.broker.auth.MockMultiStageAuthenticationProvider; import org.apache.pulsar.broker.auth.MockMutableAuthenticationProvider; import org.apache.pulsar.broker.authentication.AuthenticationDataSubscription; -import org.apache.pulsar.broker.testcontext.PulsarTestContext; -import org.apache.pulsar.broker.TransactionMetadataStoreService; -import org.apache.pulsar.broker.auth.MockAuthenticationProvider; -import org.apache.pulsar.broker.auth.MockMultiStageAuthenticationProvider; import org.apache.pulsar.broker.authentication.AuthenticationProvider; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.authorization.AuthorizationService; @@ -104,6 +104,7 @@ import org.apache.pulsar.broker.service.ServerCnx.State; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.utils.ClientChannelHelper; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.api.ProducerAccessMode; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.util.ConsumerName; @@ -2589,7 +2590,7 @@ public void testAckCommand() throws Exception { channel.writeInbound(clientCommand); assertTrue(getResponse() instanceof CommandSuccess); - PositionImpl pos = new PositionImpl(0, 0); + Position pos = PositionFactory.create(0, 0); clientCommand = Commands.newAck(1 /* consumer id */, pos.getLedgerId(), pos.getEntryId(), null, AckType.Individual, @@ -2940,7 +2941,7 @@ private void setupMLAsyncCallbackMocks() { // call addComplete on ledger asyncAddEntry doAnswer((Answer) invocationOnMock -> { ((AddEntryCallback) invocationOnMock.getArguments()[1]).addComplete( - new PositionImpl(-1, -1), + PositionFactory.create(-1, -1), null, invocationOnMock.getArguments()[2]); return null; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMarkerDeleteTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMarkerDeleteTest.java index 72f940d238d8c..7e8454f6c7eef 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMarkerDeleteTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMarkerDeleteTest.java @@ -33,7 +33,7 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -124,7 +124,7 @@ public void testMarkerDelete() throws Exception { // maxReadPosition move to msgId1, msgId2 have not be committed assertEquals(admin.topics().getInternalStats(topicName).cursors.get(subName).markDeletePosition, - PositionImpl.get(msgId1.getLedgerId(), msgId1.getEntryId()).toString()); + PositionFactory.create(msgId1.getLedgerId(), msgId1.getEntryId()).toString()); MessageIdImpl msgId3 = (MessageIdImpl) producer.newMessage(txn3).send(); txn2.commit().get(); @@ -135,7 +135,7 @@ public void testMarkerDelete() throws Exception { // maxReadPosition move to txn1 marker, so entryId is msgId2.getEntryId() + 1, // because send msgId2 before commit txn1 assertEquals(admin.topics().getInternalStats(topicName).cursors.get(subName).markDeletePosition, - PositionImpl.get(msgId2.getLedgerId(), msgId2.getEntryId() + 1).toString()); + PositionFactory.create(msgId2.getLedgerId(), msgId2.getEntryId() + 1).toString()); MessageIdImpl msgId4 = (MessageIdImpl) producer.newMessage(txn4).send(); txn3.commit().get(); @@ -145,13 +145,13 @@ public void testMarkerDelete() throws Exception { // maxReadPosition move to txn2 marker, because msgId4 have not be committed assertEquals(admin.topics().getInternalStats(topicName).cursors.get(subName).markDeletePosition, - PositionImpl.get(msgId3.getLedgerId(), msgId3.getEntryId() + 1).toString()); + PositionFactory.create(msgId3.getLedgerId(), msgId3.getEntryId() + 1).toString()); txn4.abort().get(); // maxReadPosition move to txn4 abort marker, so entryId is msgId4.getEntryId() + 2 Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getInternalStats(topicName) - .cursors.get(subName).markDeletePosition, PositionImpl.get(msgId4.getLedgerId(), + .cursors.get(subName).markDeletePosition, PositionFactory.create(msgId4.getLedgerId(), msgId4.getEntryId() + 2).toString())); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java index f034717ccf2e3..4957cc998e327 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java @@ -44,7 +44,7 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.resources.PulsarResources; @@ -283,7 +283,7 @@ public void testIsDuplicateWithFailure() { Topic.PublishContext publishContext2 = getPublishContext(producerName2, 1); persistentTopic.publishMessage(byteBuf1, publishContext1); - persistentTopic.addComplete(new PositionImpl(0, 1), null, publishContext1); + persistentTopic.addComplete(PositionFactory.create(0, 1), null, publishContext1); verify(managedLedger, times(1)).asyncAddEntry(any(ByteBuf.class), any(), any()); Long lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName1); assertNotNull(lastSequenceIdPushed); @@ -293,7 +293,7 @@ public void testIsDuplicateWithFailure() { assertEquals(lastSequenceIdPushed.longValue(), 0); persistentTopic.publishMessage(byteBuf2, publishContext2); - persistentTopic.addComplete(new PositionImpl(0, 2), null, publishContext2); + persistentTopic.addComplete(PositionFactory.create(0, 2), null, publishContext2); verify(managedLedger, times(2)).asyncAddEntry(any(ByteBuf.class), any(), any()); lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName2); assertNotNull(lastSequenceIdPushed); @@ -305,7 +305,7 @@ public void testIsDuplicateWithFailure() { byteBuf1 = getMessage(producerName1, 1); publishContext1 = getPublishContext(producerName1, 1); persistentTopic.publishMessage(byteBuf1, publishContext1); - persistentTopic.addComplete(new PositionImpl(0, 3), null, publishContext1); + persistentTopic.addComplete(PositionFactory.create(0, 3), null, publishContext1); verify(managedLedger, times(3)).asyncAddEntry(any(ByteBuf.class), any(), any()); lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName1); assertNotNull(lastSequenceIdPushed); @@ -317,7 +317,7 @@ public void testIsDuplicateWithFailure() { byteBuf1 = getMessage(producerName1, 5); publishContext1 = getPublishContext(producerName1, 5); persistentTopic.publishMessage(byteBuf1, publishContext1); - persistentTopic.addComplete(new PositionImpl(0, 4), null, publishContext1); + persistentTopic.addComplete(PositionFactory.create(0, 4), null, publishContext1); verify(managedLedger, times(4)).asyncAddEntry(any(ByteBuf.class), any(), any()); lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName1); assertNotNull(lastSequenceIdPushed); @@ -357,7 +357,7 @@ public void testIsDuplicateWithFailure() { verify(publishContext1, times(1)).completed(any(MessageDeduplication.MessageDupUnknownException.class), eq(-1L), eq(-1L)); // complete seq 6 message eventually - persistentTopic.addComplete(new PositionImpl(0, 5), null, publishContext1); + persistentTopic.addComplete(PositionFactory.create(0, 5), null, publishContext1); // simulate failure byteBuf1 = getMessage(producerName1, 7); @@ -394,7 +394,7 @@ public void testIsDuplicateWithFailure() { publishContext1 = getPublishContext(producerName1, 8); persistentTopic.publishMessage(byteBuf1, publishContext1); verify(managedLedger, times(7)).asyncAddEntry(any(ByteBuf.class), any(), any()); - persistentTopic.addComplete(new PositionImpl(0, 5), null, publishContext1); + persistentTopic.addComplete(PositionFactory.create(0, 5), null, publishContext1); lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName1); assertNotNull(lastSequenceIdPushed); assertEquals(lastSequenceIdPushed.longValue(), 8); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryControllerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryControllerTest.java index be5294d1c0f63..2222c8156e011 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryControllerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryControllerTest.java @@ -27,10 +27,11 @@ import java.lang.reflect.Field; import java.util.Set; import java.util.TreeSet; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.util.collections.ConcurrentLongLongHashMap; -import org.apache.pulsar.utils.ConcurrentBitmapSortedLongPairSet; import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap; +import org.apache.pulsar.utils.ConcurrentBitmapSortedLongPairSet; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -224,19 +225,19 @@ public void testGetMessagesToReplayNow(boolean allowOutOfOrderDelivery) throws E if (allowOutOfOrderDelivery) { // The entries are sorted by ledger ID but not by entry ID - PositionImpl[] actual1 = controller.getMessagesToReplayNow(3).toArray(new PositionImpl[3]); - PositionImpl[] expected1 = { PositionImpl.get(1, 1), PositionImpl.get(1, 2), PositionImpl.get(1, 3) }; + Position[] actual1 = controller.getMessagesToReplayNow(3).toArray(new Position[3]); + Position[] expected1 = { PositionFactory.create(1, 1), PositionFactory.create(1, 2), PositionFactory.create(1, 3) }; assertEqualsNoOrder(actual1, expected1); } else { // The entries are completely sorted - Set actual2 = controller.getMessagesToReplayNow(6); - Set expected2 = new TreeSet<>(); - expected2.add(PositionImpl.get(1, 1)); - expected2.add(PositionImpl.get(1, 2)); - expected2.add(PositionImpl.get(1, 3)); - expected2.add(PositionImpl.get(2, 1)); - expected2.add(PositionImpl.get(2, 2)); - expected2.add(PositionImpl.get(3, 1)); + Set actual2 = controller.getMessagesToReplayNow(6); + Set expected2 = new TreeSet<>(); + expected2.add(PositionFactory.create(1, 1)); + expected2.add(PositionFactory.create(1, 2)); + expected2.add(PositionFactory.create(1, 3)); + expected2.add(PositionFactory.create(2, 1)); + expected2.add(PositionFactory.create(2, 2)); + expected2.add(PositionFactory.create(3, 1)); assertEquals(actual2, expected2); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index 03eb01e958a31..a70b3ce7a42f6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -56,7 +56,7 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerService; @@ -348,10 +348,10 @@ public void testMessageRedelivery() throws Exception { final Queue actualEntriesToConsumer2 = new ConcurrentLinkedQueue<>(); final Queue expectedEntriesToConsumer1 = new ConcurrentLinkedQueue<>(); - expectedEntriesToConsumer1.add(PositionImpl.get(1, 1)); + expectedEntriesToConsumer1.add(PositionFactory.create(1, 1)); final Queue expectedEntriesToConsumer2 = new ConcurrentLinkedQueue<>(); - expectedEntriesToConsumer2.add(PositionImpl.get(1, 2)); - expectedEntriesToConsumer2.add(PositionImpl.get(1, 3)); + expectedEntriesToConsumer2.add(PositionFactory.create(1, 2)); + expectedEntriesToConsumer2.add(PositionFactory.create(1, 3)); final AtomicInteger remainingEntriesNum = new AtomicInteger( expectedEntriesToConsumer1.size() + expectedEntriesToConsumer2.size()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java index 87408598889e7..309cd7b55ac0c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java @@ -40,10 +40,10 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.service.Consumer; @@ -110,7 +110,7 @@ public void setup() throws Exception { managedLedgerConfigMock = mock(ManagedLedgerConfig.class); doReturn(new ManagedCursorContainer()).when(ledgerMock).getCursors(); doReturn("mockCursor").when(cursorMock).getName(); - doReturn(new PositionImpl(1, 50)).when(cursorMock).getMarkDeletedPosition(); + doReturn(PositionFactory.create(1, 50)).when(cursorMock).getMarkDeletedPosition(); doReturn(ledgerMock).when(cursorMock).getManagedLedger(); doReturn(managedLedgerConfigMock).when(ledgerMock).getConfig(); doReturn(false).when(managedLedgerConfigMock).isAutoSkipNonRecoverableData(); @@ -132,10 +132,10 @@ public void teardown() throws Exception { @Test public void testCanAcknowledgeAndAbortForTransaction() throws Exception { - List> positionsPair = new ArrayList<>(); - positionsPair.add(new MutablePair<>(new PositionImpl(2, 1), 0)); - positionsPair.add(new MutablePair<>(new PositionImpl(2, 3), 0)); - positionsPair.add(new MutablePair<>(new PositionImpl(2, 5), 0)); + List> positionsPair = new ArrayList<>(); + positionsPair.add(new MutablePair<>(PositionFactory.create(2, 1), 0)); + positionsPair.add(new MutablePair<>(PositionFactory.create(2, 3), 0)); + positionsPair.add(new MutablePair<>(PositionFactory.create(2, 5), 0)); doAnswer((invocationOnMock) -> { ((AsyncCallbacks.DeleteCallback) invocationOnMock.getArguments()[1]) @@ -156,14 +156,14 @@ public void testCanAcknowledgeAndAbortForTransaction() throws Exception { // Single ack for txn1 persistentSubscription.transactionIndividualAcknowledge(txnID1, positionsPair); - List positions = new ArrayList<>(); - positions.add(new PositionImpl(1, 100)); + List positions = new ArrayList<>(); + positions.add(PositionFactory.create(1, 100)); // Cumulative ack for txn1 persistentSubscription.transactionCumulativeAcknowledge(txnID1, positions).get(); positions.clear(); - positions.add(new PositionImpl(2, 1)); + positions.add(PositionFactory.create(2, 1)); // Can not single ack message already acked. try { @@ -175,7 +175,7 @@ public void testCanAcknowledgeAndAbortForTransaction() throws Exception { } positions.clear(); - positions.add(new PositionImpl(2, 50)); + positions.add(PositionFactory.create(2, 50)); // Can not cumulative ack message for another txn. try { @@ -189,12 +189,12 @@ public void testCanAcknowledgeAndAbortForTransaction() throws Exception { } List positionList = new ArrayList<>(); - positionList.add(new PositionImpl(1, 1)); - positionList.add(new PositionImpl(1, 3)); - positionList.add(new PositionImpl(1, 5)); - positionList.add(new PositionImpl(3, 1)); - positionList.add(new PositionImpl(3, 3)); - positionList.add(new PositionImpl(3, 5)); + positionList.add(PositionFactory.create(1, 1)); + positionList.add(PositionFactory.create(1, 3)); + positionList.add(PositionFactory.create(1, 5)); + positionList.add(PositionFactory.create(3, 1)); + positionList.add(PositionFactory.create(3, 3)); + positionList.add(PositionFactory.create(3, 5)); // Acknowledge from normal consumer will succeed ignoring message acked by ongoing transaction. persistentSubscription.acknowledgeMessage(positionList, AckType.Individual, Collections.emptyMap()); @@ -203,13 +203,13 @@ public void testCanAcknowledgeAndAbortForTransaction() throws Exception { persistentSubscription.endTxn(txnID1.getMostSigBits(), txnID2.getLeastSigBits(), TxnAction.ABORT_VALUE, -1); positions.clear(); - positions.add(new PositionImpl(2, 50)); + positions.add(PositionFactory.create(2, 50)); // Retry above ack, will succeed. As abort has clear pending_ack for those messages. persistentSubscription.transactionCumulativeAcknowledge(txnID2, positions); positionsPair.clear(); - positionsPair.add(new MutablePair(new PositionImpl(2, 1), 0)); + positionsPair.add(new MutablePair(PositionFactory.create(2, 1), 0)); persistentSubscription.transactionIndividualAcknowledge(txnID2, positionsPair); } @@ -226,7 +226,7 @@ public void testAcknowledgeUpdateCursorLastActive() throws Exception { doCallRealMethod().when(cursorMock).getLastActive(); List positionList = new ArrayList<>(); - positionList.add(new PositionImpl(1, 1)); + positionList.add(PositionFactory.create(1, 1)); long beforeAcknowledgeTimestamp = System.currentTimeMillis(); Thread.sleep(1); persistentSubscription.acknowledgeMessage(positionList, AckType.Individual, Collections.emptyMap()); @@ -257,12 +257,12 @@ public CompletableFuture closeAsync() { @Override public CompletableFuture appendIndividualAck(TxnID txnID, - List> positions) { + List> positions) { return CompletableFuture.completedFuture(null); } @Override - public CompletableFuture appendCumulativeAck(TxnID txnID, PositionImpl position) { + public CompletableFuture appendCumulativeAck(TxnID txnID, Position position) { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index 5b750a0b9c2e5..070f7193874c3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -47,10 +47,10 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.Optional; -import java.util.Set; import java.util.Map; import java.util.Objects; +import java.util.Optional; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; @@ -65,10 +65,11 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerTestBase; @@ -789,13 +790,13 @@ public void testAddWaitingCursorsForNonDurable() throws Exception { assertNotNull(topic.get()); PersistentTopic persistentTopic = (PersistentTopic) topic.get(); ManagedLedgerImpl ledger = (ManagedLedgerImpl)persistentTopic.getManagedLedger(); - final ManagedCursor spyCursor= spy(ledger.newNonDurableCursor(PositionImpl.LATEST, "sub-2")); + final ManagedCursor spyCursor= spy(ledger.newNonDurableCursor(PositionFactory.LATEST, "sub-2")); doAnswer((invocation) -> { Thread.sleep(5_000); invocation.callRealMethod(); return null; }).when(spyCursor).asyncReadEntriesOrWait(any(int.class), any(long.class), - any(AsyncCallbacks.ReadEntriesCallback.class), any(Object.class), any(PositionImpl.class)); + any(AsyncCallbacks.ReadEntriesCallback.class), any(Object.class), any(Position.class)); Field cursorField = ManagedLedgerImpl.class.getDeclaredField("cursors"); cursorField.setAccessible(true); ManagedCursorContainer container = (ManagedCursorContainer) cursorField.get(ledger); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionSnapshotCacheTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionSnapshotCacheTest.java index c269b098c6b88..1587c4965c388 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionSnapshotCacheTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionSnapshotCacheTest.java @@ -22,7 +22,7 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshot; import org.testng.annotations.Test; @@ -33,8 +33,8 @@ public class ReplicatedSubscriptionSnapshotCacheTest { public void testSnapshotCache() { ReplicatedSubscriptionSnapshotCache cache = new ReplicatedSubscriptionSnapshotCache("my-subscription", 10); - assertNull(cache.advancedMarkDeletePosition(new PositionImpl(0, 0))); - assertNull(cache.advancedMarkDeletePosition(new PositionImpl(100, 0))); + assertNull(cache.advancedMarkDeletePosition(PositionFactory.create(0, 0))); + assertNull(cache.advancedMarkDeletePosition(PositionFactory.create(100, 0))); ReplicatedSubscriptionsSnapshot s1 = new ReplicatedSubscriptionsSnapshot() .setSnapshotId("snapshot-1"); @@ -57,19 +57,19 @@ public void testSnapshotCache() { cache.addNewSnapshot(s5); cache.addNewSnapshot(s7); - assertNull(cache.advancedMarkDeletePosition(new PositionImpl(0, 0))); - assertNull(cache.advancedMarkDeletePosition(new PositionImpl(1, 0))); - ReplicatedSubscriptionsSnapshot snapshot = cache.advancedMarkDeletePosition(new PositionImpl(1, 1)); + assertNull(cache.advancedMarkDeletePosition(PositionFactory.create(0, 0))); + assertNull(cache.advancedMarkDeletePosition(PositionFactory.create(1, 0))); + ReplicatedSubscriptionsSnapshot snapshot = cache.advancedMarkDeletePosition(PositionFactory.create(1, 1)); assertNotNull(snapshot); assertEquals(snapshot.getSnapshotId(), "snapshot-1"); - snapshot = cache.advancedMarkDeletePosition(new PositionImpl(5, 6)); + snapshot = cache.advancedMarkDeletePosition(PositionFactory.create(5, 6)); assertNotNull(snapshot); assertEquals(snapshot.getSnapshotId(), "snapshot-5"); // Snapshots should have been now removed - assertNull(cache.advancedMarkDeletePosition(new PositionImpl(2, 2))); - assertNull(cache.advancedMarkDeletePosition(new PositionImpl(5, 5))); + assertNull(cache.advancedMarkDeletePosition(PositionFactory.create(2, 2))); + assertNull(cache.advancedMarkDeletePosition(PositionFactory.create(5, 5))); } @Test @@ -98,12 +98,12 @@ public void testSnapshotCachePruning() { cache.addNewSnapshot(s4); // Snapshot-1 was already pruned - assertNull(cache.advancedMarkDeletePosition(new PositionImpl(1, 1))); - ReplicatedSubscriptionsSnapshot snapshot = cache.advancedMarkDeletePosition(new PositionImpl(2, 2)); + assertNull(cache.advancedMarkDeletePosition(PositionFactory.create(1, 1))); + ReplicatedSubscriptionsSnapshot snapshot = cache.advancedMarkDeletePosition(PositionFactory.create(2, 2)); assertNotNull(snapshot); assertEquals(snapshot.getSnapshotId(), "snapshot-2"); - snapshot = cache.advancedMarkDeletePosition(new PositionImpl(5, 5)); + snapshot = cache.advancedMarkDeletePosition(PositionFactory.create(5, 5)); assertNotNull(snapshot); assertEquals(snapshot.getSnapshotId(), "snapshot-4"); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilderTest.java index 4ae923bd2443c..f5c3bb9d75bbd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilderTest.java @@ -34,7 +34,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshot; import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshotRequest; @@ -100,7 +100,7 @@ public void testBuildSnapshotWith2Clusters() throws Exception { .setMessageId() .setLedgerId(11) .setEntryId(11); - builder.receivedSnapshotResponse(new PositionImpl(1, 1), response); + builder.receivedSnapshotResponse(PositionFactory.create(1, 1), response); // At this point the snapshot should be created assertEquals(markers.size(), 1); @@ -139,7 +139,7 @@ public void testBuildSnapshotWith3Clusters() throws Exception { .setMessageId() .setLedgerId(11) .setEntryId(11); - builder.receivedSnapshotResponse(new PositionImpl(1, 1), response1); + builder.receivedSnapshotResponse(PositionFactory.create(1, 1), response1); // No markers should be sent out assertTrue(markers.isEmpty()); @@ -151,7 +151,7 @@ public void testBuildSnapshotWith3Clusters() throws Exception { .setMessageId() .setLedgerId(22) .setEntryId(22); - builder.receivedSnapshotResponse(new PositionImpl(2, 2), response2); + builder.receivedSnapshotResponse(PositionFactory.create(2, 2), response2); // Since we have 2 remote clusters, a 2nd round of snapshot will be taken assertEquals(markers.size(), 1); @@ -166,7 +166,7 @@ public void testBuildSnapshotWith3Clusters() throws Exception { .setMessageId() .setLedgerId(33) .setEntryId(33); - builder.receivedSnapshotResponse(new PositionImpl(3, 3), response3); + builder.receivedSnapshotResponse(PositionFactory.create(3, 3), response3); // No markers should be sent out assertTrue(markers.isEmpty()); @@ -178,7 +178,7 @@ public void testBuildSnapshotWith3Clusters() throws Exception { .setMessageId() .setLedgerId(44) .setEntryId(44); - builder.receivedSnapshotResponse(new PositionImpl(4, 4), response4); + builder.receivedSnapshotResponse(PositionFactory.create(4, 4), response4); // At this point the snapshot should be created assertEquals(markers.size(), 1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java index 16721ca1203fd..2feaacd5b8209 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java @@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.bookkeeper.mledger.ManagedCursor; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; @@ -244,16 +244,16 @@ public void testTopicPolicyTakeSnapshot() throws Exception { countDownLatch.await(); PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); long seqId = persistentTopic.getMessageDeduplication().highestSequencedPersisted.get(producerName); - PositionImpl position = (PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor() + Position position = persistentTopic.getMessageDeduplication().getManagedCursor() .getManagedLedger().getLastConfirmedEntry(); assertEquals(seqId, msgNum - 1); assertEquals(position.getEntryId(), msgNum - 1); //The first time, use topic-leve policies, 1 second delay + 3 second interval Awaitility.await() - .until(() -> ((PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor() + .until(() -> (persistentTopic.getMessageDeduplication().getManagedCursor() .getMarkDeletedPosition()).getEntryId() == msgNum - 1); ManagedCursor managedCursor = persistentTopic.getMessageDeduplication().getManagedCursor(); - PositionImpl markDeletedPosition = (PositionImpl) managedCursor.getMarkDeletedPosition(); + Position markDeletedPosition = managedCursor.getMarkDeletedPosition(); assertEquals(position, markDeletedPosition); //remove topic-level policies, namespace-level should be used, interval becomes 5 seconds @@ -261,10 +261,10 @@ public void testTopicPolicyTakeSnapshot() throws Exception { producer.newMessage().value("msg").send(); //zk update time + 5 second interval time Awaitility.await() - .until(() -> ((PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor() + .until(() -> (persistentTopic.getMessageDeduplication().getManagedCursor() .getMarkDeletedPosition()).getEntryId() == msgNum); - markDeletedPosition = (PositionImpl) managedCursor.getMarkDeletedPosition(); - position = (PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); + markDeletedPosition = managedCursor.getMarkDeletedPosition(); + position = persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); assertEquals(msgNum, markDeletedPosition.getEntryId()); assertEquals(position, markDeletedPosition); @@ -275,17 +275,17 @@ public void testTopicPolicyTakeSnapshot() throws Exception { producer.newMessage().value("msg").send(); //ensure that the time exceeds the scheduling interval of ns and topic, but no snapshot is generated Thread.sleep(3000); - markDeletedPosition = (PositionImpl) managedCursor.getMarkDeletedPosition(); - position = (PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); + markDeletedPosition = managedCursor.getMarkDeletedPosition(); + position = persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); // broker-level interval is 7 seconds, so 3 seconds will not take a snapshot assertNotEquals(msgNum + 1, markDeletedPosition.getEntryId()); assertNotEquals(position, markDeletedPosition); // wait for scheduler Awaitility.await() - .until(() -> ((PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor() + .until(() -> (persistentTopic.getMessageDeduplication().getManagedCursor() .getMarkDeletedPosition()).getEntryId() == msgNum + 1); - markDeletedPosition = (PositionImpl) managedCursor.getMarkDeletedPosition(); - position = (PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); + markDeletedPosition = managedCursor.getMarkDeletedPosition(); + position = persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); assertEquals(msgNum + 1, markDeletedPosition.getEntryId()); assertEquals(position, markDeletedPosition); } @@ -347,13 +347,13 @@ private void testTakeSnapshot(boolean enabledSnapshot) throws Exception { countDownLatch.await(); PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); long seqId = persistentTopic.getMessageDeduplication().highestSequencedPersisted.get(producerName); - PositionImpl position = (PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); + Position position = persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); assertEquals(seqId, msgNum - 1); assertEquals(position.getEntryId(), msgNum - 1); Thread.sleep(2000); ManagedCursor managedCursor = persistentTopic.getMessageDeduplication().getManagedCursor(); - PositionImpl markDeletedPosition = (PositionImpl) managedCursor.getMarkDeletedPosition(); + Position markDeletedPosition = managedCursor.getMarkDeletedPosition(); if (enabledSnapshot) { assertEquals(position, markDeletedPosition); } else { @@ -362,14 +362,14 @@ private void testTakeSnapshot(boolean enabledSnapshot) throws Exception { } producer.newMessage().value("msg").send(); - markDeletedPosition = (PositionImpl) managedCursor.getMarkDeletedPosition(); - position = (PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); + markDeletedPosition = managedCursor.getMarkDeletedPosition(); + position = persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); assertNotEquals(msgNum, markDeletedPosition.getEntryId()); assertNotNull(position); Thread.sleep(2000); - markDeletedPosition = (PositionImpl) managedCursor.getMarkDeletedPosition(); - position = (PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); + markDeletedPosition = managedCursor.getMarkDeletedPosition(); + position = persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); if (enabledSnapshot) { assertEquals(msgNum, markDeletedPosition.getEntryId()); assertEquals(position, markDeletedPosition); @@ -424,27 +424,27 @@ public void testNamespacePolicyTakeSnapshot() throws Exception { countDownLatch.await(); PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); long seqId = persistentTopic.getMessageDeduplication().highestSequencedPersisted.get(producerName); - PositionImpl position = (PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor() + Position position = persistentTopic.getMessageDeduplication().getManagedCursor() .getManagedLedger().getLastConfirmedEntry(); assertEquals(seqId, msgNum - 1); assertEquals(position.getEntryId(), msgNum - 1); //The first time, 1 second delay + 1 second interval - Awaitility.await().until(()-> ((PositionImpl) persistentTopic + Awaitility.await().until(()-> (persistentTopic .getMessageDeduplication().getManagedCursor().getMarkDeletedPosition()).getEntryId() == msgNum -1); ManagedCursor managedCursor = persistentTopic.getMessageDeduplication().getManagedCursor(); - PositionImpl markDeletedPosition = (PositionImpl) managedCursor.getMarkDeletedPosition(); + Position markDeletedPosition = managedCursor.getMarkDeletedPosition(); assertEquals(position, markDeletedPosition); //remove namespace-level policies, broker-level should be used admin.namespaces().removeDeduplicationSnapshotInterval(myNamespace); Thread.sleep(2000); - markDeletedPosition = (PositionImpl) managedCursor.getMarkDeletedPosition(); - position = (PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); + markDeletedPosition = managedCursor.getMarkDeletedPosition(); + position = persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); assertNotEquals(msgNum - 1, markDeletedPosition.getEntryId()); assertNotEquals(position, markDeletedPosition.getEntryId()); //3 seconds total Thread.sleep(1000); - markDeletedPosition = (PositionImpl) managedCursor.getMarkDeletedPosition(); - position = (PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); + markDeletedPosition = managedCursor.getMarkDeletedPosition(); + position = persistentTopic.getMessageDeduplication().getManagedCursor().getManagedLedger().getLastConfirmedEntry(); assertEquals(msgNum - 1, markDeletedPosition.getEntryId()); assertEquals(position, markDeletedPosition); @@ -475,14 +475,14 @@ public void testDisableNamespacePolicyTakeSnapshot() throws Exception { countDownLatch.await(); PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topicName).get().get(); ManagedCursor managedCursor = persistentTopic.getMessageDeduplication().getManagedCursor(); - PositionImpl markDeletedPosition = (PositionImpl) managedCursor.getMarkDeletedPosition(); + Position markDeletedPosition = managedCursor.getMarkDeletedPosition(); long seqId = persistentTopic.getMessageDeduplication().highestSequencedPersisted.get(producerName); - PositionImpl position = (PositionImpl) persistentTopic.getMessageDeduplication().getManagedCursor() + Position position = persistentTopic.getMessageDeduplication().getManagedCursor() .getManagedLedger().getLastConfirmedEntry(); assertEquals(seqId, msgNum - 1); assertEquals(position.getEntryId(), msgNum - 1); - Awaitility.await().until(()-> ((PositionImpl) persistentTopic + Awaitility.await().until(()-> (persistentTopic .getMessageDeduplication().getManagedCursor().getMarkDeletedPosition()).getEntryId() == -1); // take snapshot is disabled, so markDeletedPosition should not change diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java index f7388ef9eb990..e5ebf5b884477 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java @@ -45,7 +45,7 @@ import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; import org.apache.commons.lang.reflect.FieldUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.service.AbstractTopic; @@ -308,7 +308,7 @@ public void testFilter() throws Exception { assertNotNull(lastMsgId); MessageIdImpl finalLastMsgId = lastMsgId; Awaitility.await().untilAsserted(() -> { - PositionImpl position = (PositionImpl) subscription.getCursor().getMarkDeletedPosition(); + Position position = subscription.getCursor().getMarkDeletedPosition(); assertEquals(position.getLedgerId(), finalLastMsgId.getLedgerId()); assertEquals(position.getEntryId(), finalLastMsgId.getEntryId()); }); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/SegmentAbortedTxnProcessorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/SegmentAbortedTxnProcessorTest.java index 7f557abe0bdad..d9ba825f02e93 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/SegmentAbortedTxnProcessorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/SegmentAbortedTxnProcessorTest.java @@ -33,8 +33,9 @@ import java.util.function.Supplier; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.commons.collections4.map.LinkedMap; import org.apache.commons.lang3.tuple.MutablePair; @@ -50,9 +51,9 @@ import org.apache.pulsar.broker.transaction.buffer.impl.SnapshotSegmentAbortedTxnProcessorImpl; import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.Transactions; import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.transaction.Transaction; @@ -60,8 +61,8 @@ import org.apache.pulsar.common.events.EventType; import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.TransactionBufferStats; import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.policies.data.TransactionBufferStats; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; import org.testng.Assert; @@ -112,20 +113,20 @@ public void testPutAbortedTxnIntoProcessor() throws Exception { //1.1 Put 10 aborted txn IDs to persistent two sealed segments. for (int i = 0; i < 10; i++) { TxnID txnID = new TxnID(0, i); - PositionImpl position = new PositionImpl(0, i); + Position position = PositionFactory.create(0, i); processor.putAbortedTxnAndPosition(txnID, position); } //1.2 Put 4 aborted txn IDs into the unsealed segment. for (int i = 10; i < 14; i++) { TxnID txnID = new TxnID(0, i); - PositionImpl position = new PositionImpl(0, i); + Position position = PositionFactory.create(0, i); processor.putAbortedTxnAndPosition(txnID, position); } //1.3 Verify the common data flow verifyAbortedTxnIDAndSegmentIndex(processor, 0, 14); //2. Take the latest snapshot and verify recover from snapshot AbortedTxnProcessor newProcessor = new SnapshotSegmentAbortedTxnProcessorImpl(persistentTopic); - PositionImpl maxReadPosition = new PositionImpl(0, 14); + Position maxReadPosition = PositionFactory.create(0, 14); //2.1 Avoid update operation being canceled. waitTaskExecuteCompletely(processor); //2.2 take the latest snapshot @@ -174,7 +175,7 @@ private void verifyAbortedTxnIDAndSegmentIndex(AbortedTxnProcessor processor, in unsealedSegmentField.setAccessible(true); indexField.setAccessible(true); LinkedList unsealedSegment = (LinkedList) unsealedSegmentField.get(processor); - LinkedMap indexes = (LinkedMap) indexField.get(processor); + LinkedMap indexes = (LinkedMap) indexField.get(processor); assertEquals(unsealedSegment.size(), txnIdSize % SEGMENT_SIZE); assertEquals(indexes.size(), txnIdSize / SEGMENT_SIZE); } @@ -197,7 +198,7 @@ public void testFuturesCanCompleteWhenItIsCanceled() throws Exception { queue.add(new MutablePair<>(SnapshotSegmentAbortedTxnProcessorImpl.PersistentWorker.OperationType.WriteSegment, new MutablePair<>(new CompletableFuture<>(), task))); try { - processor.takeAbortedTxnsSnapshot(new PositionImpl(1, 10)).get(2, TimeUnit.SECONDS); + processor.takeAbortedTxnsSnapshot(PositionFactory.create(1, 10)).get(2, TimeUnit.SECONDS); fail("The update index operation should fail."); } catch (Exception e) { Assert.assertTrue(e.getCause() instanceof BrokerServiceException.ServiceUnitNotReadyException); @@ -214,7 +215,7 @@ public void testClearSnapshotSegments() throws Exception { //1. Write two snapshot segment. for (int j = 0; j < SEGMENT_SIZE * 2; j++) { TxnID txnID = new TxnID(0, j); - PositionImpl position = new PositionImpl(0, j); + Position position = PositionFactory.create(0, j); processor.putAbortedTxnAndPosition(txnID, position); } Awaitility.await().untilAsserted(() -> verifySnapshotSegmentsSize(PROCESSOR_TOPIC, 2)); @@ -234,7 +235,7 @@ public void testClearSnapshotSegments() throws Exception { //3. Try to write a snapshot segment that will fail to update indexes. for (int j = 0; j < SEGMENT_SIZE; j++) { TxnID txnID = new TxnID(0, j); - PositionImpl position = new PositionImpl(0, j); + Position position = PositionFactory.create(0, j); processor.putAbortedTxnAndPosition(txnID, position); } //4. Wait writing segment completed. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index bde9307552f59..e4240bce700bd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -47,8 +47,9 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.commons.collections4.map.LinkedMap; @@ -491,8 +492,8 @@ private void testTopicTransactionBufferDeleteAbort(Boolean enableSnapshotSegment Field abortsField = SingleSnapshotAbortedTxnProcessorImpl.class.getDeclaredField("aborts"); abortsField.setAccessible(true); - LinkedMap linkedMap = - (LinkedMap) abortsField.get(abortedTxnProcessor); + LinkedMap linkedMap = + (LinkedMap) abortsField.get(abortedTxnProcessor); assertEquals(linkedMap.size(), 1); assertEquals(linkedMap.get(linkedMap.firstKey()).getLedgerId(), ((MessageIdImpl) message.getMessageId()).getLedgerId()); @@ -799,7 +800,7 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { @Override public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl readOnlyManagedLedger, Object ctx) { readOnlyManagedLedger.asyncReadEntry( - new PositionImpl(messageId.getLedgerId(), messageId.getEntryId()), + PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()), new AsyncCallbacks.ReadEntryCallback() { @Override public void readEntryComplete(Entry entry, Object ctx) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionConsumeTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionConsumeTest.java index 9e262d1cb5617..25479e657d456 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionConsumeTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionConsumeTest.java @@ -33,7 +33,8 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.MessageRedeliveryController; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; @@ -323,7 +324,7 @@ private List appendTransactionMessages( ByteBuf headerAndPayload = Commands.serializeMetadataAndPayload( Commands.ChecksumType.Crc32c, metadata, Unpooled.copiedBuffer(msg.getBytes(UTF_8))); - CompletableFuture completableFuture = new CompletableFuture<>(); + CompletableFuture completableFuture = new CompletableFuture<>(); topic.publishTxnMessage(txnID, headerAndPayload, new Topic.PublishContext() { @Override @@ -363,7 +364,7 @@ public long getNumberOfMessages() { @Override public void completed(Exception e, long ledgerId, long entryId) { - completableFuture.complete(PositionImpl.get(ledgerId, entryId)); + completableFuture.complete(PositionFactory.create(ledgerId, entryId)); } }); positionList.add(new MessageIdData().setLedgerId(completableFuture.get() diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java index b375ab7d95429..39f36f4d38c65 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java @@ -20,7 +20,6 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.testng.Assert.assertTrue; - import java.lang.reflect.Field; import java.util.ArrayList; import java.util.HashSet; @@ -37,8 +36,9 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ReadOnlyCursor; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.Topic; @@ -249,7 +249,7 @@ private ReadOnlyCursor getOriginTopicCursor(String topic, int partition) { } return getPulsarServiceList().get(0).getManagedLedgerFactory().openReadOnlyCursor( TopicName.get(topic).getPersistenceNamingEncoding(), - PositionImpl.EARLIEST, new ManagedLedgerConfig()); + PositionFactory.EARLIEST, new ManagedLedgerConfig()); } catch (Exception e) { log.error("Failed to get origin topic readonly cursor.", e); Assert.fail("Failed to get origin topic readonly cursor."); @@ -401,8 +401,8 @@ private int getPendingAckCount(String topic, String subscriptionName) throws Exc field = PendingAckHandleImpl.class.getDeclaredField("individualAckPositions"); field.setAccessible(true); - Map> map = - (Map>) field.get(pendingAckHandle); + Map> map = + (Map>) field.get(pendingAckHandle); if (map != null) { pendingAckCount += map.size(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index 55a3e09896557..14d4375b7bf51 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -72,11 +72,12 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerFactory; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.reflect.MethodUtils; @@ -686,7 +687,7 @@ public void testMaxReadPositionForNormalPublish() throws Exception { //test publishing normal messages will change maxReadPosition in the state of NoSnapshot. MessageIdImpl messageId = (MessageIdImpl) normalProducer.newMessage().value("normal message").send(); - PositionImpl position = topicTransactionBuffer.getMaxReadPosition(); + Position position = topicTransactionBuffer.getMaxReadPosition(); Assert.assertEquals(position.getLedgerId(), messageId.getLedgerId()); Assert.assertEquals(position.getEntryId(), messageId.getEntryId()); @@ -703,16 +704,16 @@ public void testMaxReadPositionForNormalPublish() throws Exception { Transaction transaction = pulsarClient.newTransaction() .withTransactionTimeout(5, TimeUnit.SECONDS).build().get(); MessageIdImpl messageId1 = (MessageIdImpl) txnProducer.newMessage(transaction).value("txn message").send(); - PositionImpl position1 = topicTransactionBuffer.getMaxReadPosition(); + Position position1 = topicTransactionBuffer.getMaxReadPosition(); Assert.assertEquals(position1.getLedgerId(), messageId.getLedgerId()); Assert.assertEquals(position1.getEntryId(), messageId.getEntryId()); MessageIdImpl messageId2 = (MessageIdImpl) normalProducer.newMessage().value("normal message").send(); - PositionImpl position2 = topicTransactionBuffer.getMaxReadPosition(); + Position position2 = topicTransactionBuffer.getMaxReadPosition(); Assert.assertEquals(position2.getLedgerId(), messageId.getLedgerId()); Assert.assertEquals(position2.getEntryId(), messageId.getEntryId()); transaction.commit().get(); - PositionImpl position3 = topicTransactionBuffer.getMaxReadPosition(); + Position position3 = topicTransactionBuffer.getMaxReadPosition(); Assert.assertEquals(position3.getLedgerId(), messageId2.getLedgerId()); Assert.assertEquals(position3.getEntryId(), messageId2.getEntryId() + 1); @@ -720,7 +721,7 @@ public void testMaxReadPositionForNormalPublish() throws Exception { //test publishing normal messages will change maxReadPosition if the state of TB //is Ready and ongoingTxns is empty. MessageIdImpl messageId4 = (MessageIdImpl) normalProducer.newMessage().value("normal message").send(); - PositionImpl position4 = topicTransactionBuffer.getMaxReadPosition(); + Position position4 = topicTransactionBuffer.getMaxReadPosition(); Assert.assertEquals(position4.getLedgerId(), messageId4.getLedgerId()); Assert.assertEquals(position4.getEntryId(), messageId4.getEntryId()); @@ -734,7 +735,7 @@ public void testMaxReadPositionForNormalPublish() throws Exception { maxReadPositionField.setAccessible(true); field.set(topicTransactionBuffer, TopicTransactionBufferState.State.Initializing); MessageIdImpl messageId5 = (MessageIdImpl) normalProducer.newMessage().value("normal message").send(); - PositionImpl position5 = (PositionImpl) maxReadPositionField.get(topicTransactionBuffer); + Position position5 = (Position) maxReadPositionField.get(topicTransactionBuffer); Assert.assertEquals(position5.getLedgerId(), messageId4.getLedgerId()); Assert.assertEquals(position5.getEntryId(), messageId4.getEntryId()); } @@ -829,7 +830,7 @@ public void testEndTPRecoveringWhenManagerLedgerDisReadable() throws Exception{ ManagedCursorImpl managedCursor = mock(ManagedCursorImpl.class); doReturn(true).when(managedCursor).hasMoreEntries(); doReturn(false).when(managedCursor).isClosed(); - doReturn(new PositionImpl(-1, -1)).when(managedCursor).getMarkDeletedPosition(); + doReturn(PositionFactory.create(-1, -1)).when(managedCursor).getMarkDeletedPosition(); doAnswer(invocation -> { AsyncCallbacks.ReadEntriesCallback callback = invocation.getArgument(1); callback.readEntriesFailed(new ManagedLedgerException.NonRecoverableLedgerException("No ledger exist"), @@ -1022,7 +1023,7 @@ public void testNoEntryCanBeReadWhenRecovery() throws Exception { topicTransactionBuffer.getMaxReadPosition()); completableFuture.get(); - doReturn(PositionImpl.LATEST).when(managedLedger).getLastConfirmedEntry(); + doReturn(PositionFactory.LATEST).when(managedLedger).getLastConfirmedEntry(); ManagedCursorImpl managedCursor = mock(ManagedCursorImpl.class); doReturn(false).when(managedCursor).hasMoreEntries(); doReturn(managedCursor).when(managedLedger).newNonDurableCursor(any(), any()); @@ -1119,7 +1120,7 @@ public void testNotChangeMaxReadPositionCountWhenCheckIfNoSnapshot() throws Exce }); Assert.assertEquals(changeMaxReadPositionCount.get(), 0L); - buffer.syncMaxReadPositionForNormalPublish(new PositionImpl(1, 1), false); + buffer.syncMaxReadPositionForNormalPublish(PositionFactory.create(1, 1), false); Assert.assertEquals(changeMaxReadPositionCount.get(), 0L); } @@ -1632,7 +1633,7 @@ public void testTBRecoverChangeStateError() throws InterruptedException, Timeout ManagedLedgerImpl managedLedger = mock(ManagedLedgerImpl.class); ManagedCursorContainer managedCursors = new ManagedCursorContainer(); when(managedLedger.getCursors()).thenReturn(managedCursors); - PositionImpl position = PositionImpl.EARLIEST; + Position position = PositionFactory.EARLIEST; when(managedLedger.getLastConfirmedEntry()).thenReturn(position); // Create topic. persistentTopic.set(new PersistentTopic("topic-a", managedLedger, brokerService)); @@ -2004,17 +2005,17 @@ public void testPersistentTopicGetLastDispatchablePositionWithTxn() throws Excep producer.newMessage(txn).value(UUID.randomUUID().toString()).send(); // get last dispatchable position - PositionImpl lastDispatchablePosition = (PositionImpl) persistentTopic.getLastDispatchablePosition().get(); + Position lastDispatchablePosition = persistentTopic.getLastDispatchablePosition().get(); // the last dispatchable position should be the message id of the normal message - assertEquals(lastDispatchablePosition, PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId())); + assertEquals(lastDispatchablePosition, PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId())); // abort the txn txn.abort().get(5, TimeUnit.SECONDS); // get last dispatchable position - lastDispatchablePosition = (PositionImpl) persistentTopic.getLastDispatchablePosition().get(); + lastDispatchablePosition = persistentTopic.getLastDispatchablePosition().get(); // the last dispatchable position should be the message id of the normal message - assertEquals(lastDispatchablePosition, PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId())); + assertEquals(lastDispatchablePosition, PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId())); @Cleanup diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java index f93cfbcdc50f0..af12caf1efd61 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java @@ -24,12 +24,20 @@ import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertTrue; import static org.testng.AssertJUnit.fail; +import java.time.Duration; +import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.BrokerService; @@ -60,15 +68,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.time.Duration; -import java.util.Collections; -import java.util.Map; -import java.util.Optional; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; - public class TopicTransactionBufferTest extends TransactionTestBase { @@ -234,7 +233,7 @@ public void testGetMaxPositionAfterTBReady() throws Exception { // 3. Send message and test the exception can be handled as expected. MessageIdImpl messageId = (MessageIdImpl) producer.newMessage().send(); producer.newMessage().send(); - Mockito.doReturn(new PositionImpl(messageId.getLedgerId(), messageId.getEntryId())) + Mockito.doReturn(PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId())) .when(transactionBuffer).getMaxReadPosition(); try { consumer.getLastMessageIds(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java index 6e121aca3816f..b17565cfc0dfa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java @@ -31,7 +31,7 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; import org.apache.commons.collections4.map.LinkedMap; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; @@ -211,7 +211,7 @@ public void testPendingAckLowWaterMark() throws Exception { Message message = consumer.receive(2, TimeUnit.SECONDS); assertEquals(new String(message.getData()), TEST1); consumer.acknowledgeAsync(message.getMessageId(), txn).get(); - LinkedMap> individualAckOfTransaction = null; + LinkedMap> individualAckOfTransaction = null; for (int i = 0; i < getPulsarServiceList().size(); i++) { Field field = BrokerService.class.getDeclaredField("topics"); @@ -231,7 +231,7 @@ public void testPendingAckLowWaterMark() throws Exception { field = PendingAckHandleImpl.class.getDeclaredField("individualAckOfTransaction"); field.setAccessible(true); individualAckOfTransaction = - (LinkedMap>) field.get(pendingAckHandle); + (LinkedMap>) field.get(pendingAckHandle); } } } @@ -450,8 +450,8 @@ private boolean checkTxnIsOngoingInTP(TxnID txnID, String subName) throws Except Field field2 = PendingAckHandleImpl.class.getDeclaredField("individualAckOfTransaction"); field2.setAccessible(true); - LinkedMap> individualAckOfTransaction = - (LinkedMap>) field2.get(pendingAckHandle); + LinkedMap> individualAckOfTransaction = + (LinkedMap>) field2.get(pendingAckHandle); return individualAckOfTransaction.containsKey(txnID); } @@ -465,8 +465,8 @@ private boolean checkTxnIsOngoingInTB(TxnID txnID) throws Exception { (TopicTransactionBuffer) persistentTopic.getTransactionBuffer(); Field field3 = TopicTransactionBuffer.class.getDeclaredField("ongoingTxns"); field3.setAccessible(true); - LinkedMap ongoingTxns = - (LinkedMap) field3.get(topicTransactionBuffer); + LinkedMap ongoingTxns = + (LinkedMap) field3.get(topicTransactionBuffer); return ongoingTxns.containsKey(txnID); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java index 2fdfd3a524750..eb7b24c7326dc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java @@ -28,7 +28,7 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBuffer; @@ -209,13 +209,13 @@ public void testSyncNormalPositionWhenTBRecover(boolean clientEnableTransaction, // init maxReadPosition is PositionImpl.EARLIEST Position position = topicTransactionBuffer.getMaxReadPosition(); - assertEquals(position, PositionImpl.EARLIEST); + assertEquals(position, PositionFactory.EARLIEST); MessageIdImpl messageId = (MessageIdImpl) producer.send("test".getBytes()); // send normal message can't change MaxReadPosition when state is None or Initializing position = topicTransactionBuffer.getMaxReadPosition(); - assertEquals(position, PositionImpl.EARLIEST); + assertEquals(position, PositionFactory.EARLIEST); // change to None state can recover field.set(topicTransactionBuffer, TopicTransactionBufferState.State.None); @@ -229,7 +229,7 @@ public void testSyncNormalPositionWhenTBRecover(boolean clientEnableTransaction, checkTopicTransactionBufferState(clientEnableTransaction, topicTransactionBuffer); // change MaxReadPosition to normal message position - assertEquals(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId()), + assertEquals(PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()), topicTransactionBuffer.getMaxReadPosition()); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java index 1360dd7c4442b..fd4e984b6c1fc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java @@ -30,8 +30,9 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.service.BrokerService; @@ -135,8 +136,8 @@ public void txnAckTestNoBatchAndSharedSubMemoryDeleteTest() throws Exception { PendingAckHandleImpl pendingAckHandle = (PendingAckHandleImpl) field.get(persistentSubscription); field = PendingAckHandleImpl.class.getDeclaredField("individualAckOfTransaction"); field.setAccessible(true); - LinkedMap> individualAckOfTransaction = - (LinkedMap>) field.get(pendingAckHandle); + LinkedMap> individualAckOfTransaction = + (LinkedMap>) field.get(pendingAckHandle); assertTrue(individualAckOfTransaction.isEmpty()); if (retryCnt == 0) { //one message are not ack @@ -176,7 +177,7 @@ public void txnAckTestBatchAndSharedSubMemoryDeleteTest() throws Exception { PendingAckHandleImpl pendingAckHandle = null; - LinkedMap> individualAckOfTransaction = null; + LinkedMap> individualAckOfTransaction = null; ManagedCursorImpl managedCursor = null; MessageId[] messageIds = new MessageId[2]; @@ -230,13 +231,13 @@ public void txnAckTestBatchAndSharedSubMemoryDeleteTest() throws Exception { field = PendingAckHandleImpl.class.getDeclaredField("individualAckOfTransaction"); field.setAccessible(true); individualAckOfTransaction = - (LinkedMap>) field.get(pendingAckHandle); + (LinkedMap>) field.get(pendingAckHandle); assertTrue(individualAckOfTransaction.isEmpty()); managedCursor = (ManagedCursorImpl) testPersistentSubscription.getCursor(); field = ManagedCursorImpl.class.getDeclaredField("batchDeletedIndexes"); field.setAccessible(true); - final ConcurrentSkipListMap batchDeletedIndexes = - (ConcurrentSkipListMap) field.get(managedCursor); + final ConcurrentSkipListMap batchDeletedIndexes = + (ConcurrentSkipListMap) field.get(managedCursor); if (retryCnt == 0) { //one message are not ack Awaitility.await().until(() -> { @@ -313,16 +314,16 @@ public void testPendingAckClearPositionIsSmallerThanMarkDelete() throws Exceptio .orElseThrow(); PersistentSubscription subscription = (PersistentSubscription) t.getSubscription(subscriptionName); PendingAckHandleImpl pendingAckHandle = (PendingAckHandleImpl) subscription.getPendingAckHandle(); - Map> individualAckPositions = + Map> individualAckPositions = pendingAckHandle.getIndividualAckPositions(); // one message in pending ack state assertEquals(1, individualAckPositions.size()); // put the PositionImpl.EARLIEST to the map - individualAckPositions.put(PositionImpl.EARLIEST, new MutablePair<>(PositionImpl.EARLIEST, 0)); + individualAckPositions.put(PositionFactory.EARLIEST, new MutablePair<>(PositionFactory.EARLIEST, 0)); // put the PositionImpl.LATEST to the map - individualAckPositions.put(PositionImpl.LATEST, new MutablePair<>(PositionImpl.EARLIEST, 0)); + individualAckPositions.put(PositionFactory.LATEST, new MutablePair<>(PositionFactory.EARLIEST, 0)); // three position in pending ack state assertEquals(3, individualAckPositions.size()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java index 93a2f274517d5..00cdb4162f0c4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java @@ -47,7 +47,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; import org.apache.commons.collections4.map.LinkedMap; import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.PulsarService; @@ -293,8 +293,8 @@ public void individualPendingAckReplayTest() throws Exception { // in order to check out the pending ack cursor is clear whether or not. Awaitility.await() - .until(() -> ((PositionImpl) managedCursor.getMarkDeletedPosition()) - .compareTo((PositionImpl) managedCursor.getManagedLedger().getLastConfirmedEntry()) == -1); + .until(() -> (managedCursor.getMarkDeletedPosition()) + .compareTo(managedCursor.getManagedLedger().getLastConfirmedEntry()) == -1); } @Test @@ -458,8 +458,8 @@ public void cumulativePendingAckReplayTest() throws Exception { // in order to check out the pending ack cursor is clear whether or not. Awaitility.await() - .until(() -> ((PositionImpl) managedCursor.getMarkDeletedPosition()) - .compareTo((PositionImpl) managedCursor.getManagedLedger().getLastConfirmedEntry()) == 0); + .until(() -> (managedCursor.getMarkDeletedPosition()) + .compareTo(managedCursor.getManagedLedger().getLastConfirmedEntry()) == 0); } @Test @@ -575,8 +575,8 @@ public void testDeleteUselessLogDataWhenSubCursorMoved() throws Exception { field3.setAccessible(true); field4.setAccessible(true); - ConcurrentSkipListMap pendingAckLogIndex = - (ConcurrentSkipListMap) field3.get(pendingAckStore); + ConcurrentSkipListMap pendingAckLogIndex = + (ConcurrentSkipListMap) field3.get(pendingAckStore); long maxIndexLag = (long) field4.get(pendingAckStore); Assert.assertEquals(pendingAckLogIndex.size(), 0); Assert.assertEquals(maxIndexLag, 5); @@ -718,8 +718,8 @@ public void testPendingAckLowWaterMarkRemoveFirstTxn() throws Exception { PendingAckHandleImpl oldPendingAckHandle = (PendingAckHandleImpl) field1.get(persistentSubscription); Field field2 = PendingAckHandleImpl.class.getDeclaredField("individualAckOfTransaction"); field2.setAccessible(true); - LinkedMap> oldIndividualAckOfTransaction = - (LinkedMap>) field2.get(oldPendingAckHandle); + LinkedMap> oldIndividualAckOfTransaction = + (LinkedMap>) field2.get(oldPendingAckHandle); Awaitility.await().untilAsserted(() -> Assert.assertEquals(oldIndividualAckOfTransaction.size(), 0)); PendingAckHandleImpl pendingAckHandle = new PendingAckHandleImpl(persistentSubscription); @@ -739,8 +739,8 @@ public void testPendingAckLowWaterMarkRemoveFirstTxn() throws Exception { }); - LinkedMap> individualAckOfTransaction = - (LinkedMap>) field2.get(pendingAckHandle); + LinkedMap> individualAckOfTransaction = + (LinkedMap>) field2.get(pendingAckHandle); assertFalse(individualAckOfTransaction.containsKey(transaction1.getTxnID())); assertFalse(individualAckOfTransaction.containsKey(transaction2.getTxnID())); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java index 19d6cc85c9ff6..6dd3e6e7c7822 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java @@ -18,6 +18,10 @@ */ package org.apache.pulsar.broker.transaction.pendingack.impl; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; import java.util.ArrayList; import java.util.Iterator; import java.util.LinkedHashSet; @@ -30,7 +34,8 @@ import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -42,7 +47,6 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.transaction.coordinator.impl.TxnLogBufferedWriterConfig; -import static org.mockito.Mockito.*; import org.awaitility.Awaitility; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -172,7 +176,7 @@ public void testMainProcess(boolean writeWithBatch, boolean readWithBatch) throw List> futureList = new ArrayList<>(); for (int i = 0; i < 20; i++){ TxnID txnID = new TxnID(i, i); - PositionImpl position = PositionImpl.get(i, i); + Position position = PositionFactory.create(i, i); futureList.add(mlPendingAckStoreForWrite.appendCumulativeAck(txnID, position)); } for (int i = 0; i < 10; i++){ @@ -185,7 +189,7 @@ public void testMainProcess(boolean writeWithBatch, boolean readWithBatch) throw } for (int i = 40; i < 50; i++){ TxnID txnID = new TxnID(i, i); - PositionImpl position = PositionImpl.get(i, i); + Position position = PositionFactory.create(i, i); futureList.add(mlPendingAckStoreForWrite.appendCumulativeAck(txnID, position)); } FutureUtil.waitForAll(futureList).get(); @@ -210,7 +214,7 @@ public void testMainProcess(boolean writeWithBatch, boolean readWithBatch) throw LinkedHashSet expectedPositions = calculatePendingAckIndexes(positionList, skipSet); Assert.assertEquals( mlPendingAckStoreForWrite.pendingAckLogIndex.keySet().stream() - .map(PositionImpl::getEntryId).collect(Collectors.toList()), + .map(Position::getEntryId).collect(Collectors.toList()), new ArrayList<>(expectedPositions) ); // Replay. @@ -237,19 +241,19 @@ public Object answer(InvocationOnMock invocation) throws Throwable { // Verify build sparse indexes correct after replay. Assert.assertEquals(mlPendingAckStoreForRead.pendingAckLogIndex.size(), mlPendingAckStoreForWrite.pendingAckLogIndex.size()); - Iterator> iteratorReplay = + Iterator> iteratorReplay = mlPendingAckStoreForRead.pendingAckLogIndex.entrySet().iterator(); - Iterator> iteratorWrite = + Iterator> iteratorWrite = mlPendingAckStoreForWrite.pendingAckLogIndex.entrySet().iterator(); while (iteratorReplay.hasNext()){ - Map.Entry replayEntry = iteratorReplay.next(); - Map.Entry writeEntry = iteratorWrite.next(); + Map.Entry replayEntry = iteratorReplay.next(); + Map.Entry writeEntry = iteratorWrite.next(); Assert.assertEquals(replayEntry.getKey(), writeEntry.getKey()); Assert.assertEquals(replayEntry.getValue().getLedgerId(), writeEntry.getValue().getLedgerId()); Assert.assertEquals(replayEntry.getValue().getEntryId(), writeEntry.getValue().getEntryId()); } // Verify delete correct. - when(managedCursorMock.getPersistentMarkDeletedPosition()).thenReturn(PositionImpl.get(19, 19)); + when(managedCursorMock.getPersistentMarkDeletedPosition()).thenReturn(PositionFactory.create(19, 19)); mlPendingAckStoreForWrite.clearUselessLogData(); mlPendingAckStoreForRead.clearUselessLogData(); Assert.assertTrue(mlPendingAckStoreForWrite.pendingAckLogIndex.keySet().iterator().next().getEntryId() > 19); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index 27aa98597ec12..92c51da64d39d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -51,9 +51,10 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import lombok.Cleanup; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentStickyKeyDispatcherMultipleConsumers; @@ -664,7 +665,7 @@ public void testReadAheadWhenAddingConsumers() throws Exception { PersistentSubscription sub = (PersistentSubscription) t.getSubscription("key_shared"); // We need to ensure that dispatcher does not keep to look ahead in the topic, - PositionImpl readPosition = (PositionImpl) sub.getCursor().getReadPosition(); + Position readPosition = sub.getCursor().getReadPosition(); assertTrue(readPosition.getEntryId() < 1000); } @@ -1646,8 +1647,8 @@ private AtomicInteger injectReplayReadCounter(String topicName, String cursorNam managedLedger.getCursors().removeCursor(cursor.getName()); managedLedger.getActiveCursors().removeCursor(cursor.getName()); ManagedCursorImpl spyCursor = Mockito.spy(cursor); - managedLedger.getCursors().add(spyCursor, PositionImpl.EARLIEST); - managedLedger.getActiveCursors().add(spyCursor, PositionImpl.EARLIEST); + managedLedger.getCursors().add(spyCursor, PositionFactory.EARLIEST); + managedLedger.getActiveCursors().add(spyCursor, PositionFactory.EARLIEST); AtomicInteger replyReadCounter = new AtomicInteger(); Mockito.doAnswer(invocation -> { if (!String.valueOf(invocation.getArguments()[2]).equals("Normal")) { @@ -1887,8 +1888,8 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); ManagedCursorImpl cursor = (ManagedCursorImpl) managedLedger.openCursor(subName); log.info("cursor_readPosition {}, LAC {}", cursor.getReadPosition(), managedLedger.getLastConfirmedEntry()); - assertTrue(((PositionImpl) cursor.getReadPosition()) - .compareTo((PositionImpl) managedLedger.getLastConfirmedEntry()) > 0); + assertTrue((cursor.getReadPosition()) + .compareTo(managedLedger.getLastConfirmedEntry()) > 0); // Make all consumers to start to read and acknowledge messages. // Verify: no repeated Read-and-discard. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java index 20407295ccb0e..bbac688d9224c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java @@ -32,9 +32,10 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.BrokerService; @@ -377,7 +378,7 @@ public void testInitReaderAtSpecifiedPosition() throws Exception { .receiverQueueSize(0).startMessageId(startMessageId1).create(); ManagedLedgerInternalStats.CursorStats cursor1 = admin.topics().getInternalStats(topicName).cursors.get(s1); log.info("cursor1 readPosition: {}, markDeletedPosition: {}", cursor1.readPosition, cursor1.markDeletePosition); - PositionImpl p1 = parseReadPosition(cursor1); + Position p1 = parseReadPosition(cursor1); assertEquals(p1.getLedgerId(), ledgers.get(0)); assertEquals(p1.getEntryId(), 0); reader1.close(); @@ -390,7 +391,7 @@ public void testInitReaderAtSpecifiedPosition() throws Exception { .receiverQueueSize(0).startMessageId(startMessageId2).create(); ManagedLedgerInternalStats.CursorStats cursor2 = admin.topics().getInternalStats(topicName).cursors.get(s2); log.info("cursor2 readPosition: {}, markDeletedPosition: {}", cursor2.readPosition, cursor2.markDeletePosition); - PositionImpl p2 = parseReadPosition(cursor2); + Position p2 = parseReadPosition(cursor2); assertEquals(p2.getLedgerId(), ledgers.get(0)); assertEquals(p2.getEntryId(), 0); reader2.close(); @@ -403,7 +404,7 @@ public void testInitReaderAtSpecifiedPosition() throws Exception { .receiverQueueSize(0).startMessageId(startMessageId3).create(); ManagedLedgerInternalStats.CursorStats cursor3 = admin.topics().getInternalStats(topicName).cursors.get(s3); log.info("cursor3 readPosition: {}, markDeletedPosition: {}", cursor3.readPosition, cursor3.markDeletePosition); - PositionImpl p3 = parseReadPosition(cursor3); + Position p3 = parseReadPosition(cursor3); assertEquals(p3.getLedgerId(), currentLedger.getId()); assertEquals(p3.getEntryId(), 0); reader3.close(); @@ -416,7 +417,7 @@ public void testInitReaderAtSpecifiedPosition() throws Exception { .receiverQueueSize(0).startMessageId(startMessageId4).create(); ManagedLedgerInternalStats.CursorStats cursor4 = admin.topics().getInternalStats(topicName).cursors.get(s4); log.info("cursor4 readPosition: {}, markDeletedPosition: {}", cursor4.readPosition, cursor4.markDeletePosition); - PositionImpl p4 = parseReadPosition(cursor4); + Position p4 = parseReadPosition(cursor4); assertEquals(p4.getLedgerId(), currentLedger.getId()); assertEquals(p4.getEntryId(), 0); reader4.close(); @@ -429,7 +430,7 @@ public void testInitReaderAtSpecifiedPosition() throws Exception { .receiverQueueSize(0).startMessageId(startMessageId5).create(); ManagedLedgerInternalStats.CursorStats cursor5 = admin.topics().getInternalStats(topicName).cursors.get(s5); log.info("cursor5 readPosition: {}, markDeletedPosition: {}", cursor5.readPosition, cursor5.markDeletePosition); - PositionImpl p5 = parseReadPosition(cursor5); + Position p5 = parseReadPosition(cursor5); assertEquals(p5.getLedgerId(), currentLedger.getId()); assertEquals(p5.getEntryId(), 0); reader5.close(); @@ -442,7 +443,7 @@ public void testInitReaderAtSpecifiedPosition() throws Exception { .receiverQueueSize(0).startMessageId(startMessageId6).create(); ManagedLedgerInternalStats.CursorStats cursor6 = admin.topics().getInternalStats(topicName).cursors.get(s6); log.info("cursor6 readPosition: {}, markDeletedPosition: {}", cursor6.readPosition, cursor6.markDeletePosition); - PositionImpl p6 = parseReadPosition(cursor6); + Position p6 = parseReadPosition(cursor6); assertEquals(p6.getLedgerId(), ledgers.get(ledgers.size() - 1)); assertEquals(p6.getEntryId(), 0); reader6.close(); @@ -455,7 +456,7 @@ public void testInitReaderAtSpecifiedPosition() throws Exception { .receiverQueueSize(0).startMessageId(startMessageId7).create(); ManagedLedgerInternalStats.CursorStats cursor7 = admin.topics().getInternalStats(topicName).cursors.get(s7); log.info("cursor7 readPosition: {}, markDeletedPosition: {}", cursor7.readPosition, cursor7.markDeletePosition); - PositionImpl p7 = parseReadPosition(cursor7); + Position p7 = parseReadPosition(cursor7); assertEquals(p7.getLedgerId(), currentLedger.getId()); assertEquals(p7.getEntryId(), 0); reader7.close(); @@ -468,7 +469,7 @@ public void testInitReaderAtSpecifiedPosition() throws Exception { .receiverQueueSize(0).startMessageId(startMessageId8).create(); ManagedLedgerInternalStats.CursorStats cursor8 = admin.topics().getInternalStats(topicName).cursors.get(s8); log.info("cursor8 readPosition: {}, markDeletedPosition: {}", cursor8.readPosition, cursor8.markDeletePosition); - PositionImpl p8 = parseReadPosition(cursor8); + Position p8 = parseReadPosition(cursor8); assertEquals(p8.getLedgerId(), ledgers.get(2)); assertEquals(p8.getEntryId(), 0); reader8.close(); @@ -482,7 +483,7 @@ public void testInitReaderAtSpecifiedPosition() throws Exception { ManagedLedgerInternalStats.CursorStats cursor9 = admin.topics().getInternalStats(topicName).cursors.get(s9); log.info("cursor9 readPosition: {}, markDeletedPosition: {}", cursor9.readPosition, cursor9.markDeletePosition); - PositionImpl p9 = parseReadPosition(cursor9); + Position p9 = parseReadPosition(cursor9); assertEquals(p9.getLedgerId(), ledgers.get(3)); assertEquals(p9.getEntryId(), 0); reader9.close(); @@ -495,7 +496,7 @@ public void testInitReaderAtSpecifiedPosition() throws Exception { .receiverQueueSize(0).startMessageId(startMessageId10).create(); ManagedLedgerInternalStats.CursorStats cursor10 = admin.topics().getInternalStats(topicName).cursors.get(s10); log.info("cursor10 readPosition: {}, markDeletedPosition: {}", cursor10.readPosition, cursor10.markDeletePosition); - PositionImpl p10 = parseReadPosition(cursor10); + Position p10 = parseReadPosition(cursor10); assertEquals(p10.getLedgerId(), ledgers.get(2)); assertEquals(p10.getEntryId(), 0); reader10.close(); @@ -504,9 +505,9 @@ public void testInitReaderAtSpecifiedPosition() throws Exception { admin.topics().delete(topicName, false); } - private PositionImpl parseReadPosition(ManagedLedgerInternalStats.CursorStats cursorStats) { + private Position parseReadPosition(ManagedLedgerInternalStats.CursorStats cursorStats) { String[] ledgerIdAndEntryId = cursorStats.readPosition.split(":"); - return PositionImpl.get(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); + return PositionFactory.create(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); } @Test @@ -544,10 +545,10 @@ public void testReaderInitAtDeletedPosition() throws Exception { ManagedLedgerInternalStats.CursorStats cursorStats = admin.topics().getInternalStats(topicName).cursors.get("s1"); String[] ledgerIdAndEntryId = cursorStats.markDeletePosition.split(":"); - PositionImpl actMarkDeletedPos = - PositionImpl.get(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); - PositionImpl expectedMarkDeletedPos = - PositionImpl.get(msgIdInDeletedLedger5.getLedgerId(), msgIdInDeletedLedger5.getEntryId()); + Position actMarkDeletedPos = + PositionFactory.create(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); + Position expectedMarkDeletedPos = + PositionFactory.create(msgIdInDeletedLedger5.getLedgerId(), msgIdInDeletedLedger5.getEntryId()); log.info("Expected mark deleted position: {}", expectedMarkDeletedPos); log.info("Actual mark deleted position: {}", cursorStats.markDeletePosition); assertTrue(actMarkDeletedPos.compareTo(expectedMarkDeletedPos) >= 0); @@ -605,10 +606,10 @@ public void testTrimLedgerIfNoDurableCursor() throws Exception { ManagedLedgerInternalStats.CursorStats cursorStats = admin.topics().getInternalStats(topicName).cursors.get(nonDurableCursor); String[] ledgerIdAndEntryId = cursorStats.markDeletePosition.split(":"); - PositionImpl actMarkDeletedPos = - PositionImpl.get(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); - PositionImpl expectedMarkDeletedPos = - PositionImpl.get(msgIdInDeletedLedger5.getLedgerId(), msgIdInDeletedLedger5.getEntryId()); + Position actMarkDeletedPos = + PositionFactory.create(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); + Position expectedMarkDeletedPos = + PositionFactory.create(msgIdInDeletedLedger5.getLedgerId(), msgIdInDeletedLedger5.getEntryId()); log.info("Expected mark deleted position: {}", expectedMarkDeletedPos); log.info("Actual mark deleted position: {}", cursorStats.markDeletePosition); Assert.assertTrue(actMarkDeletedPos.compareTo(expectedMarkDeletedPos) >= 0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java index da359a6aeb9c5..8df5a38bb461c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingTest.java @@ -37,8 +37,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import lombok.Cleanup; +import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.RandomUtils; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.ClientBuilder; @@ -164,7 +164,7 @@ public void testLargeMessage(boolean ackReceiptEnabled, boolean clientSizeMaxMes assertTrue(producerStats.getChunkedMessageRate() > 0); ManagedCursorImpl mcursor = (ManagedCursorImpl) topic.getManagedLedger().getCursors().iterator().next(); - PositionImpl readPosition = (PositionImpl) mcursor.getReadPosition(); + Position readPosition = mcursor.getReadPosition(); for (MessageId msgId : msgIds) { consumer.acknowledge(msgId); @@ -270,7 +270,7 @@ public void testLargeMessageAckTimeOut(boolean ackReceiptEnabled) throws Excepti } ManagedCursorImpl mcursor = (ManagedCursorImpl) topic.getManagedLedger().getCursors().iterator().next(); - PositionImpl readPosition = (PositionImpl) mcursor.getReadPosition(); + Position readPosition = mcursor.getReadPosition(); consumer.acknowledgeCumulative(lastMsgId); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageParserTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageParserTest.java index 772ddbee4e54d..7b87efa7cae39 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageParserTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageParserTest.java @@ -28,7 +28,7 @@ import lombok.Cleanup; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.CompressionType; @@ -95,7 +95,7 @@ public void testParseMessages(boolean batchEnabled, CompressionType compressionT .create(); ManagedCursor cursor = ((PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get()) - .getManagedLedger().newNonDurableCursor(PositionImpl.EARLIEST); + .getManagedLedger().newNonDurableCursor(PositionFactory.EARLIEST); if (batchEnabled) { for (int i = 0; i < n - 1; i++) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java index 4fa86c49914a4..56cf053314053 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java @@ -46,7 +46,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.TransactionMetadataStoreService; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; @@ -439,7 +438,7 @@ public void produceAbortTest() throws Exception { //when delete commit marker operation finish, it can run next delete commit marker operation //so this test may not delete all the position in this manageLedger. Position markerPosition = ((ManagedLedgerImpl) persistentSubscription.getCursor() - .getManagedLedger()).getNextValidPosition((PositionImpl) markDeletePosition); + .getManagedLedger()).getNextValidPosition(markDeletePosition); //marker is the lastConfirmedEntry, after commit the marker will only be write in if (!markerPosition.equals(lastConfirmedEntry)) { log.error("Mark delete position is not commit marker position!"); @@ -744,7 +743,7 @@ public void txnMessageAckTest() throws Exception { //when delete commit marker operation finish, it can run next delete commit marker operation //so this test may not delete all the position in this manageLedger. Position markerPosition = ((ManagedLedgerImpl) persistentSubscription.getCursor() - .getManagedLedger()).getNextValidPosition((PositionImpl) markDeletePosition); + .getManagedLedger()).getNextValidPosition(markDeletePosition); //marker is the lastConfirmedEntry, after commit the marker will only be write in if (!markerPosition.equals(lastConfirmedEntry)) { log.error("Mark delete position is not commit marker position!"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicImplTest.java index c9933b7ef8f26..9d89207d06fe9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicImplTest.java @@ -18,8 +18,10 @@ */ package org.apache.pulsar.compaction; -import static org.mockito.Mockito.*; -import static org.testng.Assert.*; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; +import static org.testng.Assert.assertEquals; import com.github.benmanes.caffeine.cache.AsyncLoadingCache; import com.github.benmanes.caffeine.cache.CacheLoader; import com.github.benmanes.caffeine.cache.Caffeine; @@ -29,7 +31,8 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import java.util.function.Supplier; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.common.api.proto.MessageIdData; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; @@ -107,7 +110,7 @@ public void testFindStartPointLoop(long start, long end, long targetMessageId) { AsyncLoadingCache cache = Caffeine.newBuilder() .buildAsync(mockCacheLoader(start, end, targetMessageId, bingoMarker)); // Do test. - PositionImpl targetPosition = PositionImpl.get(DEFAULT_LEDGER_ID, targetMessageId); + Position targetPosition = PositionFactory.create(DEFAULT_LEDGER_ID, targetMessageId); CompletableFuture promise = new CompletableFuture<>(); CompactedTopicImpl.findStartPointLoop(targetPosition, start, end, promise, cache); long result = promise.join(); @@ -137,7 +140,7 @@ public void testRecursionNumberOfFindStartPointLoop() { // executed "findStartPointLoop". Supplier loopCounter = () -> invokeCounterOfCacheGet.get() / 3; // Do test. - PositionImpl targetPosition = PositionImpl.get(DEFAULT_LEDGER_ID, targetMessageId); + Position targetPosition = PositionFactory.create(DEFAULT_LEDGER_ID, targetMessageId); CompletableFuture promise = new CompletableFuture<>(); CompactedTopicImpl.findStartPointLoop(targetPosition, start, end, promise, cacheWithCounter); // Do verify. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java index 3cca85aa2f1b6..2692e6fa19698 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java @@ -39,8 +39,8 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.lang3.tuple.Triple; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -181,18 +181,18 @@ public void testEntryLookup() throws Exception { Pair lastPosition = positions.get(positions.size() - 1); // check ids before and after ids in compacted ledger - Assert.assertEquals(CompactedTopicImpl.findStartPoint(new PositionImpl(0, 0), lastEntryId, cache).get(), + Assert.assertEquals(CompactedTopicImpl.findStartPoint(PositionFactory.create(0, 0), lastEntryId, cache).get(), Long.valueOf(0)); - Assert.assertEquals(CompactedTopicImpl.findStartPoint(new PositionImpl(Long.MAX_VALUE, 0), + Assert.assertEquals(CompactedTopicImpl.findStartPoint(PositionFactory.create(Long.MAX_VALUE, 0), lastEntryId, cache).get(), Long.valueOf(CompactedTopicImpl.NEWER_THAN_COMPACTED)); // entry 0 is never in compacted ledger due to how we generate dummy - Assert.assertEquals(CompactedTopicImpl.findStartPoint(new PositionImpl(firstPositionId.getLedgerId(), 0), + Assert.assertEquals(CompactedTopicImpl.findStartPoint(PositionFactory.create(firstPositionId.getLedgerId(), 0), lastEntryId, cache).get(), Long.valueOf(0)); // check next id after last id in compacted ledger - Assert.assertEquals(CompactedTopicImpl.findStartPoint(new PositionImpl(lastPosition.getLeft().getLedgerId(), + Assert.assertEquals(CompactedTopicImpl.findStartPoint(PositionFactory.create(lastPosition.getLeft().getLedgerId(), lastPosition.getLeft().getEntryId() + 1), lastEntryId, cache).get(), Long.valueOf(CompactedTopicImpl.NEWER_THAN_COMPACTED)); @@ -203,14 +203,14 @@ public void testEntryLookup() throws Exception { // Check ids we know are in compacted ledger for (Pair p : positions) { - PositionImpl pos = new PositionImpl(p.getLeft().getLedgerId(), p.getLeft().getEntryId()); + Position pos = PositionFactory.create(p.getLeft().getLedgerId(), p.getLeft().getEntryId()); Long got = CompactedTopicImpl.findStartPoint(pos, lastEntryId, cache).get(); Assert.assertEquals(got, p.getRight()); } // Check ids we know are in the gaps of the compacted ledger for (Pair gap : idsInGaps) { - PositionImpl pos = new PositionImpl(gap.getLeft().getLedgerId(), gap.getLeft().getEntryId()); + Position pos = PositionFactory.create(gap.getLeft().getLedgerId(), gap.getLeft().getEntryId()); Assert.assertEquals(CompactedTopicImpl.findStartPoint(pos, lastEntryId, cache).get(), gap.getRight()); } } @@ -232,7 +232,7 @@ public void testCleanupOldCompactedTopicLedger() throws Exception { // set the compacted topic ledger CompactedTopicImpl compactedTopic = new CompactedTopicImpl(bk); - compactedTopic.newCompactedLedger(new PositionImpl(1,2), oldCompactedLedger.getId()).get(); + compactedTopic.newCompactedLedger(PositionFactory.create(1,2), oldCompactedLedger.getId()).get(); // ensure both ledgers still exist, can be opened bk.openLedger(oldCompactedLedger.getId(), @@ -243,7 +243,7 @@ public void testCleanupOldCompactedTopicLedger() throws Exception { Compactor.COMPACTED_TOPIC_LEDGER_PASSWORD).close(); // update the compacted topic ledger - PositionImpl newHorizon = new PositionImpl(1,3); + Position newHorizon = PositionFactory.create(1,3); compactedTopic.newCompactedLedger(newHorizon, newCompactedLedger.getId()).get(); // Make sure the old compacted ledger still exist after the new compacted ledger created. @@ -868,7 +868,7 @@ public void testCompactWithConcurrentGetCompactionHorizonAndCompactedTopicContex CompactedTopicImpl compactedTopic = new CompactedTopicImpl(bk); - PositionImpl oldHorizon = new PositionImpl(1, 2); + Position oldHorizon = PositionFactory.create(1, 2); var future = CompletableFuture.supplyAsync(() -> { // set the compacted topic ledger return compactedTopic.newCompactedLedger(oldHorizon, oldCompactedLedger.getId()); @@ -889,7 +889,7 @@ public void testCompactWithConcurrentGetCompactionHorizonAndCompactedTopicContex future.join(); - PositionImpl newHorizon = new PositionImpl(1, 3); + Position newHorizon = PositionFactory.create(1, 3); var future2 = CompletableFuture.supplyAsync(() -> { // update the compacted topic ledger return compactedTopic.newCompactedLedger(newHorizon, newCompactedLedger.getId()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java index 2545c0362e82a..25f42b86b26ba 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java @@ -26,8 +26,9 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.Test; @@ -36,7 +37,7 @@ public class CompactedTopicUtilsTest { @Test public void testReadCompactedEntriesWithEmptyEntries() throws ExecutionException, InterruptedException { - PositionImpl lastCompactedPosition = PositionImpl.get(1, 100); + Position lastCompactedPosition = PositionFactory.create(1, 100); TopicCompactionService service = Mockito.mock(TopicCompactionService.class); Mockito.doReturn(CompletableFuture.completedFuture(Collections.emptyList())) .when(service).readCompactedEntries(Mockito.any(), Mockito.intThat(argument -> argument > 0)); @@ -44,8 +45,8 @@ public void testReadCompactedEntriesWithEmptyEntries() throws ExecutionException .getLastCompactedPosition(); - PositionImpl initPosition = PositionImpl.get(1, 90); - AtomicReference readPositionRef = new AtomicReference<>(initPosition.getNext()); + Position initPosition = PositionFactory.create(1, 90); + AtomicReference readPositionRef = new AtomicReference<>(initPosition.getNext()); ManagedCursorImpl cursor = Mockito.mock(ManagedCursorImpl.class); Mockito.doReturn(readPositionRef.get()).when(cursor).getReadPosition(); Mockito.doReturn(1).when(cursor).applyMaxSizeCap(Mockito.anyInt(), Mockito.anyLong()); @@ -70,7 +71,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { }; CompactedTopicUtils.asyncReadCompactedEntries(service, cursor, 1, 100, - PositionImpl.LATEST, false, readEntriesCallback, false, null); + PositionFactory.LATEST, false, readEntriesCallback, false, null); List entries = completableFuture.get(); Assert.assertTrue(entries.isEmpty()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java index 16945a60f5d47..1c09dc0d6434c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java @@ -47,7 +47,7 @@ import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -402,7 +402,7 @@ public void testCompactedWithConcurrentSend() throws Exception { Position lastCompactedPosition = topicCompactionService.getLastCompactedPosition().get(); Entry lastCompactedEntry = topicCompactionService.readLastCompactedEntry().get(); - Assert.assertTrue(PositionImpl.get(lastCompactedPosition.getLedgerId(), lastCompactedPosition.getEntryId()) + Assert.assertTrue(PositionFactory.create(lastCompactedPosition.getLedgerId(), lastCompactedPosition.getEntryId()) .compareTo(lastCompactedEntry.getLedgerId(), lastCompactedEntry.getEntryId()) >= 0); future.join(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/GetLastMessageIdCompactedTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/GetLastMessageIdCompactedTest.java index 6c2d848bb7c2d..a28392fb99d2a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/GetLastMessageIdCompactedTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/GetLastMessageIdCompactedTest.java @@ -21,14 +21,13 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotEquals; - import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -85,8 +84,8 @@ private void triggerCompactionAndWait(String topicName) throws Exception { (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).get().get(); persistentTopic.triggerCompaction(); Awaitility.await().untilAsserted(() -> { - PositionImpl lastConfirmPos = (PositionImpl) persistentTopic.getManagedLedger().getLastConfirmedEntry(); - PositionImpl markDeletePos = (PositionImpl) persistentTopic + Position lastConfirmPos = persistentTopic.getManagedLedger().getLastConfirmedEntry(); + Position markDeletePos = persistentTopic .getSubscription(Compactor.COMPACTION_SUBSCRIPTION).getCursor().getMarkDeletedPosition(); assertEquals(markDeletePos.getLedgerId(), lastConfirmPos.getLedgerId()); assertEquals(markDeletePos.getEntryId(), lastConfirmPos.getEntryId()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/TopicCompactionServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/TopicCompactionServiceTest.java index ba77ce5bd9d29..2aa09309d3931 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/TopicCompactionServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/TopicCompactionServiceTest.java @@ -35,7 +35,7 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; @@ -143,13 +143,13 @@ public void test() throws Exception { String markDeletePosition = admin.topics().getInternalStats(topic).cursors.get(COMPACTION_SUBSCRIPTION).markDeletePosition; String[] split = markDeletePosition.split(":"); - compactedTopic.newCompactedLedger(PositionImpl.get(Long.valueOf(split[0]), Long.valueOf(split[1])), + compactedTopic.newCompactedLedger(PositionFactory.create(Long.valueOf(split[0]), Long.valueOf(split[1])), compactedLedger).join(); Position lastCompactedPosition = service.getLastCompactedPosition().join(); assertEquals(admin.topics().getInternalStats(topic).lastConfirmedEntry, lastCompactedPosition.toString()); - List entries = service.readCompactedEntries(PositionImpl.EARLIEST, 4).join(); + List entries = service.readCompactedEntries(PositionFactory.EARLIEST, 4).join(); assertEquals(entries.size(), 3); entries.stream().map(e -> { try { @@ -170,7 +170,7 @@ public void test() throws Exception { } }); - List entries2 = service.readCompactedEntries(PositionImpl.EARLIEST, 1).join(); + List entries2 = service.readCompactedEntries(PositionFactory.EARLIEST, 1).join(); assertEquals(entries2.size(), 1); Entry entry = service.findEntryByEntryIndex(0).join(); diff --git a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionLogImpl.java b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionLogImpl.java index f2e1f60663d28..5de9b7f8fc7b4 100644 --- a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionLogImpl.java +++ b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionLogImpl.java @@ -38,8 +38,8 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerAlreadyClosedException; import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.apache.pulsar.common.naming.NamespaceName; @@ -156,7 +156,7 @@ public void replayAsync(TransactionLogReplayCallback transactionLogReplayCallbac private void readAsync(int numberOfEntriesToRead, AsyncCallbacks.ReadEntriesCallback readEntriesCallback) { - cursor.asyncReadEntries(numberOfEntriesToRead, readEntriesCallback, System.nanoTime(), PositionImpl.LATEST); + cursor.asyncReadEntries(numberOfEntriesToRead, readEntriesCallback, System.nanoTime(), PositionFactory.LATEST); } @Override @@ -264,7 +264,7 @@ public void start() { * 3. Build batched position and handle valid data. */ long[] ackSetAlreadyAck = cursor.getDeletedBatchIndexesAsLongArray( - PositionImpl.get(entry.getLedgerId(), entry.getEntryId())); + PositionFactory.create(entry.getLedgerId(), entry.getEntryId())); BitSetRecyclable bitSetAlreadyAck = null; if (ackSetAlreadyAck != null){ bitSetAlreadyAck = BitSetRecyclable.valueOf(ackSetAlreadyAck); diff --git a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnBatchedPositionImpl.java b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnBatchedPositionImpl.java index 2e897167aaf0a..158fb42cb7356 100644 --- a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnBatchedPositionImpl.java +++ b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnBatchedPositionImpl.java @@ -20,14 +20,14 @@ import lombok.Getter; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.impl.AckSetPositionImpl; import org.apache.pulsar.common.util.collections.BitSetRecyclable; /*** - * The difference with {@link PositionImpl} is that there are two more parameters: + * The difference with {@link AckSetPositionImpl} is that there are two more parameters: * {@link #batchSize}, {@link #batchIndex}. */ -public class TxnBatchedPositionImpl extends PositionImpl { +public class TxnBatchedPositionImpl extends AckSetPositionImpl { /** The data length of current batch. **/ @Getter @@ -38,7 +38,7 @@ public class TxnBatchedPositionImpl extends PositionImpl { private final int batchIndex; public TxnBatchedPositionImpl(long ledgerId, long entryId, int batchSize, int batchIndex){ - super(ledgerId, entryId); + super(ledgerId, entryId, null); this.batchIndex = batchIndex; this.batchSize = batchSize; } @@ -48,9 +48,9 @@ public TxnBatchedPositionImpl(Position position, int batchSize, int batchIndex){ } /** - * It's exactly the same as {@link PositionImpl},make sure that when {@link TxnBatchedPositionImpl} used as the key - * of map same as {@link PositionImpl}. {@link #batchSize} and {@link #batchIndex} should not be involved in - * calculate, just like {@link PositionImpl#ackSet} is not involved in calculate. + * It's exactly the same as {@link Position},make sure that when {@link TxnBatchedPositionImpl} used as the key + * of map same as {@link Position}. {@link #batchSize} and {@link #batchIndex} should not be involved in + * calculate, just like the included {@link #ackSet} is not involved in calculate. * Note: In {@link java.util.concurrent.ConcurrentSkipListMap}, it use the {@link Comparable#compareTo(Object)} to * determine whether the keys are the same. In {@link java.util.HashMap}, it use the * {@link Object#hashCode()} & {@link Object#equals(Object)} to determine whether the keys are the same. @@ -58,13 +58,12 @@ public TxnBatchedPositionImpl(Position position, int batchSize, int batchIndex){ @Override public boolean equals(Object o) { return super.equals(o); - } /** - * It's exactly the same as {@link PositionImpl},make sure that when {@link TxnBatchedPositionImpl} used as the key - * of map same as {@link PositionImpl}. {@link #batchSize} and {@link #batchIndex} should not be involved in - * calculate, just like {@link PositionImpl#ackSet} is not involved in calculate. + * It's exactly the same as {@link Position},make sure that when {@link TxnBatchedPositionImpl} used as the key + * of map same as {@link Position}. {@link #batchSize} and {@link #batchIndex} should not be involved in + * calculate, just like the included {@link #ackSet} is not involved in calculate. * Note: In {@link java.util.concurrent.ConcurrentSkipListMap}, it use the {@link Comparable#compareTo(Object)} to * determine whether the keys are the same. In {@link java.util.HashMap}, it use the * {@link Object#hashCode()} & {@link Object#equals(Object)} to determine whether the keys are the same. @@ -75,14 +74,14 @@ public int hashCode() { } /** - * It's exactly the same as {@link PositionImpl},to make sure that when compare to the "markDeletePosition", it - * looks like {@link PositionImpl}. {@link #batchSize} and {@link #batchIndex} should not be involved in calculate, - * just like {@link PositionImpl#ackSet} is not involved in calculate. + * It's exactly the same as {@link Position},to make sure that when compare to the "markDeletePosition", it + * looks like {@link Position}. {@link #batchSize} and {@link #batchIndex} should not be involved in calculate, + * just like the included {@link #ackSet} is not involved in calculate. * Note: In {@link java.util.concurrent.ConcurrentSkipListMap}, it use the {@link Comparable#compareTo(Object)} to * determine whether the keys are the same. In {@link java.util.HashMap}, it use the * {@link Object#hashCode()} & {@link Object#equals(Object)} to determine whether the keys are the same. */ - public int compareTo(PositionImpl that) { + public int compareTo(Position that) { return super.compareTo(that); } diff --git a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionLogImplTest.java b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionLogImplTest.java index 4790b063e70f4..1d4e5dd2d0413 100644 --- a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionLogImplTest.java +++ b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionLogImplTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.transaction.coordinator.impl; +import static org.apache.pulsar.transaction.coordinator.impl.DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS; +import static org.mockito.Mockito.mock; import com.google.common.collect.ComparisonChain; import io.netty.util.HashedWheelTimer; import io.netty.util.concurrent.DefaultThreadFactory; @@ -34,8 +36,8 @@ import java.util.stream.Collectors; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.api.proto.Subscription; import org.apache.pulsar.common.util.FutureUtil; @@ -46,8 +48,6 @@ import org.apache.pulsar.transaction.coordinator.proto.TransactionMetadataEntry; import org.apache.pulsar.transaction.coordinator.proto.TxnStatus; import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase; -import static org.apache.pulsar.transaction.coordinator.impl.DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS; -import static org.mockito.Mockito.*; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.DataProvider; @@ -269,10 +269,10 @@ factory, new ManagedLedgerConfig(), bufferedWriterConfigForRecover, transactionT .result(); } }).collect(Collectors.toList()); - PositionImpl markDeletedPosition = null; - LinkedHashMap batchIndexes = null; + Position markDeletedPosition = null; + LinkedHashMap batchIndexes = null; if (expectedDeletedPositions.get(0) instanceof TxnBatchedPositionImpl){ - Pair> pair = + Pair> pair = calculateBatchIndexes( expectedDeletedPositions.stream() .map(p -> (TxnBatchedPositionImpl)p) @@ -283,7 +283,7 @@ factory, new ManagedLedgerConfig(), bufferedWriterConfigForRecover, transactionT } else { markDeletedPosition = calculateMarkDeletedPosition(expectedDeletedPositions); } - final PositionImpl markDeletedPosition_final = markDeletedPosition; + final Position markDeletedPosition_final = markDeletedPosition; // Assert mark deleted position correct. Awaitility.await().atMost(2, TimeUnit.SECONDS).until(() -> { Position actualMarkDeletedPosition = managedCursor.getMarkDeletedPosition(); @@ -293,7 +293,7 @@ factory, new ManagedLedgerConfig(), bufferedWriterConfigForRecover, transactionT // Assert batchIndexes correct. if (batchIndexes != null){ // calculate last deleted position. - Map.Entry + Map.Entry lastOne = batchIndexes.entrySet().stream().reduce((a, b) -> b).get(); // Wait last one has been deleted from cursor. Awaitility.await().atMost(2, TimeUnit.SECONDS).until(() -> { @@ -301,8 +301,8 @@ factory, new ManagedLedgerConfig(), bufferedWriterConfigForRecover, transactionT return Arrays.equals(lastOne.getValue().toLongArray(), ls); }); // Verify batch indexes. - for (Map.Entry entry : batchIndexes.entrySet()){ - PositionImpl p = entry.getKey(); + for (Map.Entry entry : batchIndexes.entrySet()){ + Position p = entry.getKey(); long[] actualAckSet = managedCursor.getBatchPositionAckSet(p); Assert.assertEquals(actualAckSet, entry.getValue().toLongArray()); entry.getValue().recycle(); @@ -320,7 +320,7 @@ factory, new ManagedLedgerConfig(), bufferedWriterConfigForRecover, transactionT /*** * Calculate markDeletedPosition by {@param sortedDeletedPositions}. */ - private PositionImpl calculateMarkDeletedPosition(Collection sortedDeletedPositions){ + private Position calculateMarkDeletedPosition(Collection sortedDeletedPositions){ Position markDeletedPosition = null; for (Position position : sortedDeletedPositions){ if (markDeletedPosition == null){ @@ -338,19 +338,19 @@ private PositionImpl calculateMarkDeletedPosition(Collection sortedDel if (markDeletedPosition == null) { return null; } - return PositionImpl.get(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId()); + return PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId()); } /*** * Calculate markDeletedPosition and batchIndexes by {@param sortedDeletedPositions}. */ - private Pair> calculateBatchIndexes( + private Pair> calculateBatchIndexes( List sortedDeletedPositions){ // build batchIndexes. - LinkedHashMap batchIndexes = new LinkedHashMap<>(); + LinkedHashMap batchIndexes = new LinkedHashMap<>(); for (TxnBatchedPositionImpl batchedPosition : sortedDeletedPositions){ batchedPosition.setAckSetByIndex(); - PositionImpl k = PositionImpl.get(batchedPosition.getLedgerId(), batchedPosition.getEntryId()); + Position k = PositionFactory.create(batchedPosition.getLedgerId(), batchedPosition.getEntryId()); BitSetRecyclable bitSetRecyclable = batchIndexes.get(k); if (bitSetRecyclable == null){ bitSetRecyclable = BitSetRecyclable.valueOf(batchedPosition.getAckSet()); @@ -360,8 +360,8 @@ private Pair> calcul } // calculate markDeletedPosition. Position markDeletedPosition = null; - for (Map.Entry entry : batchIndexes.entrySet()){ - PositionImpl position = entry.getKey(); + for (Map.Entry entry : batchIndexes.entrySet()){ + Position position = entry.getKey(); BitSetRecyclable bitSetRecyclable = entry.getValue(); if (!bitSetRecyclable.isEmpty()){ break; @@ -380,7 +380,7 @@ private Pair> calcul } // remove empty bitSet. List shouldRemoveFromMap = new ArrayList<>(); - for (Map.Entry entry : batchIndexes.entrySet()) { + for (Map.Entry entry : batchIndexes.entrySet()) { BitSetRecyclable bitSetRecyclable = entry.getValue(); if (bitSetRecyclable.isEmpty()) { shouldRemoveFromMap.add(entry.getKey()); @@ -390,7 +390,7 @@ private Pair> calcul BitSetRecyclable bitSetRecyclable = batchIndexes.remove(position); bitSetRecyclable.recycle(); } - return Pair.of(PositionImpl.get(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId()), + return Pair.of(PositionFactory.create(markDeletedPosition.getLedgerId(), markDeletedPosition.getEntryId()), batchIndexes); } } \ No newline at end of file diff --git a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnBatchedPositionImplTest.java b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnBatchedPositionImplTest.java index 0905fdad72d40..a1263ae71d299 100644 --- a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnBatchedPositionImplTest.java +++ b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnBatchedPositionImplTest.java @@ -24,7 +24,8 @@ import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListMap; -import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.testng.Assert; import org.testng.annotations.DataProvider; @@ -74,7 +75,7 @@ public Object[][] testHashcodeAndEqualsData(){ /** * Why is this test needed ? - * {@link org.apache.bookkeeper.mledger.impl.ManagedCursorImpl} maintains batchIndexes, use {@link PositionImpl} or + * {@link org.apache.bookkeeper.mledger.impl.ManagedCursorImpl} maintains batchIndexes, use {@link Position} or * {@link TxnBatchedPositionImpl} as the key. However, different maps may use "param-key.equals(key-in-map)" to * determine the contains, or use "key-in-map.equals(param-key)" or use "param-key.compareTo(key-in-map)" or use * "key-in-map.compareTo(param-key)" to determine the {@link Map#containsKey(Object)}, the these approaches may @@ -88,48 +89,48 @@ public void testKeyInMap(long ledgerId, long entryId, int batchSize, int batchIn // build data. Random random = new Random(); int v = random.nextInt(); - PositionImpl position = new PositionImpl(ledgerId, entryId); + Position position = PositionFactory.create(ledgerId, entryId); TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize, batchIndex); // ConcurrentSkipListMap. - ConcurrentSkipListMap map1 = new ConcurrentSkipListMap<>(); + ConcurrentSkipListMap map1 = new ConcurrentSkipListMap<>(); map1.put(position, v); Assert.assertTrue(map1.containsKey(txnBatchedPosition)); - ConcurrentSkipListMap map2 = new ConcurrentSkipListMap<>(); + ConcurrentSkipListMap map2 = new ConcurrentSkipListMap<>(); map2.put(txnBatchedPosition, v); Assert.assertTrue(map2.containsKey(position)); // HashMap. - HashMap map3 = new HashMap<>(); + HashMap map3 = new HashMap<>(); map3.put(position, v); Assert.assertTrue(map3.containsKey(txnBatchedPosition)); - HashMap map4 = new HashMap<>(); + HashMap map4 = new HashMap<>(); map4.put(txnBatchedPosition, v); Assert.assertTrue(map4.containsKey(position)); // ConcurrentHashMap. - ConcurrentHashMap map5 = new ConcurrentHashMap<>(); + ConcurrentHashMap map5 = new ConcurrentHashMap<>(); map5.put(position, v); Assert.assertTrue(map5.containsKey(txnBatchedPosition)); - ConcurrentHashMap map6 = new ConcurrentHashMap<>(); + ConcurrentHashMap map6 = new ConcurrentHashMap<>(); map6.put(txnBatchedPosition, v); Assert.assertTrue(map6.containsKey(position)); // LinkedHashMap. - LinkedHashMap map7 = new LinkedHashMap<>(); + LinkedHashMap map7 = new LinkedHashMap<>(); map7.put(position, v); Assert.assertTrue(map7.containsKey(txnBatchedPosition)); - LinkedHashMap map8 = new LinkedHashMap<>(); + LinkedHashMap map8 = new LinkedHashMap<>(); map8.put(txnBatchedPosition, v); Assert.assertTrue(map8.containsKey(position)); } /** * Why is this test needed ? - * Make sure that when compare to the "markDeletePosition", it looks like {@link PositionImpl} + * Make sure that when compare to the "markDeletePosition", it looks like {@link Position} * Note: In {@link java.util.concurrent.ConcurrentSkipListMap}, it use the {@link Comparable#compareTo(Object)} to * determine whether the keys are the same. In {@link java.util.HashMap}, it use the * {@link Object#hashCode()} & {@link Object#equals(Object)} to determine whether the keys are the same. */ @Test public void testCompareTo(){ - PositionImpl position = new PositionImpl(1, 1); + Position position = PositionFactory.create(1, 1); TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, 2, 0); Assert.assertEquals(position.compareTo(txnBatchedPosition), 0); Assert.assertEquals(txnBatchedPosition.compareTo(position), 0); diff --git a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java index 8b496af6aa718..3147279477843 100644 --- a/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java +++ b/pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java @@ -18,6 +18,9 @@ */ package org.apache.pulsar.transaction.coordinator.impl; +import static org.apache.pulsar.transaction.coordinator.impl.DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; import com.fasterxml.jackson.databind.ObjectMapper; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; @@ -53,8 +56,8 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.CollectionUtils; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase; @@ -62,8 +65,6 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import static org.apache.pulsar.transaction.coordinator.impl.DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS; -import static org.testng.Assert.*; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -194,7 +195,7 @@ public void testMainProcess(int batchedWriteMaxRecords, int batchedWriteMaxSize, // Store the param-context, param-position, param-exception of callback function and complete-count for verify. List contextArrayOfCallback = Collections.synchronizedList(new ArrayList<>()); Map exceptionArrayOfCallback = new ConcurrentHashMap<>(); - Map> positionsOfCallback = Collections.synchronizedMap(new LinkedHashMap<>()); + Map> positionsOfCallback = Collections.synchronizedMap(new LinkedHashMap<>()); AtomicBoolean anyFlushCompleted = new AtomicBoolean(); TxnLogBufferedWriter.AddDataCallback callback = new TxnLogBufferedWriter.AddDataCallback(){ @Override @@ -204,7 +205,7 @@ public void addComplete(Position position, Object ctx) { return; } contextArrayOfCallback.add((int)ctx); - PositionImpl lightPosition = PositionImpl.get(position.getLedgerId(), position.getEntryId()); + Position lightPosition = PositionFactory.create(position.getLedgerId(), position.getEntryId()); positionsOfCallback.computeIfAbsent(lightPosition, p -> Collections.synchronizedList(new ArrayList<>())); positionsOfCallback.get(lightPosition).add(position); @@ -299,7 +300,7 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { * Note2: Verify that all entry was written in strict order. */ if (BookieErrorType.NO_ERROR == bookieErrorType) { - Iterator callbackPositionIterator = positionsOfCallback.keySet().iterator(); + Iterator callbackPositionIterator = positionsOfCallback.keySet().iterator(); List dataArrayWrite = dataSerializer.getGeneratedJsonArray(); int entryCounter = 0; while (managedCursor.hasMoreEntries()) { @@ -311,7 +312,7 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { // Get data read. Entry entry = entries.get(m); // Assert the position of the read matches the position of the callback. - PositionImpl callbackPosition = callbackPositionIterator.next(); + Position callbackPosition = callbackPositionIterator.next(); assertEquals(entry.getLedgerId(), callbackPosition.getLedgerId()); assertEquals(entry.getEntryId(), callbackPosition.getEntryId()); if (exactlyBatched) { @@ -394,7 +395,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { dataArrayFlushedToBookie.add(byteBuf.readInt()); AsyncCallbacks.AddEntryCallback callback = (AsyncCallbacks.AddEntryCallback) invocation.getArguments()[1]; - callback.addComplete(PositionImpl.get(1,1), byteBuf, + callback.addComplete(PositionFactory.create(1,1), byteBuf, invocation.getArguments()[2]); return null; } @@ -1022,7 +1023,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { writeCounter.incrementAndGet(); AsyncCallbacks.AddEntryCallback callback = (AsyncCallbacks.AddEntryCallback) invocation.getArguments()[1]; - callback.addComplete(PositionImpl.get(1,1), (ByteBuf)invocation.getArguments()[0], + callback.addComplete(PositionFactory.create(1,1), (ByteBuf)invocation.getArguments()[0], invocation.getArguments()[2]); return null; } diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java index 9f89bd52a8626..b4ed940c9cdca 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java @@ -51,9 +51,9 @@ import org.apache.bookkeeper.mledger.OffloadedLedgerMetadata; import org.apache.bookkeeper.mledger.OffloadedLedgerMetadataConsumer; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.OffloadSegmentInfoImpl; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.offload.jcloud.BlockAwareSegmentInputStream; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlock; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlock.IndexInputStream; @@ -111,7 +111,7 @@ public class BlobStoreManagedLedgerOffloader implements LedgerOffloader { private final OffsetsCache entryOffsetsCache; private final ConcurrentLinkedQueue offloadBuffer = new ConcurrentLinkedQueue<>(); private CompletableFuture offloadResult; - private volatile PositionImpl lastOfferedPosition = PositionImpl.LATEST; + private volatile Position lastOfferedPosition = PositionFactory.LATEST; private final Duration maxSegmentCloseTime; private final long minSegmentCloseTimeMillis; private final long segmentBeginTimeMillis; @@ -525,7 +525,7 @@ private synchronized boolean closeSegment() { return result; } - private PositionImpl lastOffered() { + private Position lastOffered() { return lastOfferedPosition; } From c2702e9bc46c444cbc99f4b64cb453c622b56c26 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Thu, 13 Jun 2024 12:26:40 -0700 Subject: [PATCH 300/580] [fix][broker] Asynchronously return brokerRegistry.lookupAsync when checking if broker is active(ExtensibleLoadManagerImpl only) (#22899) --- .../channel/ServiceUnitStateChannelImpl.java | 84 +++++++++++-------- .../channel/ServiceUnitStateChannelTest.java | 51 +++++++++-- 2 files changed, 89 insertions(+), 46 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index 069ac51655141..f04734c4ad9bb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -484,7 +484,7 @@ private CompletableFuture> getActiveOwnerAsync( String serviceUnit, ServiceUnitState state, Optional owner) { - return deferGetOwnerRequest(serviceUnit) + return dedupeGetOwnerRequest(serviceUnit) .thenCompose(newOwner -> { if (newOwner == null) { return CompletableFuture.completedFuture(null); @@ -622,7 +622,7 @@ public CompletableFuture publishAssignEventAsync(String serviceUnit, Str } EventType eventType = Assign; eventCounters.get(eventType).getTotal().incrementAndGet(); - CompletableFuture getOwnerRequest = deferGetOwnerRequest(serviceUnit); + CompletableFuture getOwnerRequest = dedupeGetOwnerRequest(serviceUnit); pubAsync(serviceUnit, new ServiceUnitStateData(Assigning, broker, getNextVersionId(serviceUnit))) .whenComplete((__, ex) -> { @@ -932,44 +932,54 @@ private boolean isTargetBroker(String broker) { return broker.equals(brokerId); } - private CompletableFuture deferGetOwnerRequest(String serviceUnit) { + private CompletableFuture deferGetOwner(String serviceUnit) { + var future = new CompletableFuture().orTimeout(inFlightStateWaitingTimeInMillis, + TimeUnit.MILLISECONDS) + .exceptionally(e -> { + var ownerAfter = getOwner(serviceUnit); + log.warn("{} failed to wait for owner for serviceUnit:{}; Trying to " + + "return the current owner:{}", + brokerId, serviceUnit, ownerAfter, e); + if (ownerAfter == null) { + throw new IllegalStateException(e); + } + return ownerAfter.orElse(null); + }); + if (debug()) { + log.info("{} is waiting for owner for serviceUnit:{}", brokerId, serviceUnit); + } + return future; + } + + private CompletableFuture dedupeGetOwnerRequest(String serviceUnit) { var requested = new MutableObject>(); try { - return getOwnerRequests - .computeIfAbsent(serviceUnit, k -> { - var ownerBefore = getOwner(serviceUnit); - if (ownerBefore != null && ownerBefore.isPresent()) { - // Here, we do a quick active check first with the computeIfAbsent lock - brokerRegistry.lookupAsync(ownerBefore.get()).getNow(Optional.empty()) - .ifPresent(__ -> requested.setValue( - CompletableFuture.completedFuture(ownerBefore.get()))); - - if (requested.getValue() != null) { - return requested.getValue(); - } - } - - - CompletableFuture future = - new CompletableFuture().orTimeout(inFlightStateWaitingTimeInMillis, - TimeUnit.MILLISECONDS) - .exceptionally(e -> { - var ownerAfter = getOwner(serviceUnit); - log.warn("{} failed to wait for owner for serviceUnit:{}; Trying to " - + "return the current owner:{}", - brokerId, serviceUnit, ownerAfter, e); - if (ownerAfter == null) { - throw new IllegalStateException(e); - } - return ownerAfter.orElse(null); - }); - if (debug()) { - log.info("{} is waiting for owner for serviceUnit:{}", brokerId, serviceUnit); - } - requested.setValue(future); - return future; - }); + return getOwnerRequests.computeIfAbsent(serviceUnit, k -> { + var ownerBefore = getOwner(serviceUnit); + if (ownerBefore != null && ownerBefore.isPresent()) { + // Here, we do the broker active check first with the computeIfAbsent lock + requested.setValue(brokerRegistry.lookupAsync(ownerBefore.get()) + .thenCompose(brokerLookupData -> { + if (brokerLookupData.isPresent()) { + // The owner broker is active. + // Immediately return the request. + return CompletableFuture.completedFuture(ownerBefore.get()); + } else { + // The owner broker is inactive. + // The leader broker should be cleaning up the orphan service units. + // Defer this request til the leader notifies the new ownerships. + return deferGetOwner(serviceUnit); + } + })); + } else { + // The owner broker has not been declared yet. + // The ownership should be in the middle of transferring or assigning. + // Defer this request til the inflight ownership change is complete. + requested.setValue(deferGetOwner(serviceUnit)); + } + return requested.getValue(); + }); } finally { var future = requested.getValue(); if (future != null) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java index c0fdd95a6a3db..837aceca1416f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java @@ -1620,32 +1620,63 @@ public void testOverrideOrphanStateData() @Test(priority = 19) public void testActiveGetOwner() throws Exception { - - // set the bundle owner is the broker + // case 1: the bundle owner is empty String broker = brokerId2; String bundle = "public/owned/0xfffffff0_0xffffffff"; + overrideTableViews(bundle, null); + assertEquals(Optional.empty(), channel1.getOwnerAsync(bundle).get()); + + // case 2: the bundle ownership is transferring, and the dst broker is not the channel owner + overrideTableViews(bundle, + new ServiceUnitStateData(Releasing, broker, brokerId1, 1)); + assertEquals(Optional.of(broker), channel1.getOwnerAsync(bundle).get()); + + + // case 3: the bundle ownership is transferring, and the dst broker is the channel owner + overrideTableViews(bundle, + new ServiceUnitStateData(Assigning, brokerId1, brokerId2, 1)); + assertTrue(!channel1.getOwnerAsync(bundle).isDone()); + + // case 4: the bundle ownership is found overrideTableViews(bundle, new ServiceUnitStateData(Owned, broker, null, 1)); var owner = channel1.getOwnerAsync(bundle).get(5, TimeUnit.SECONDS).get(); assertEquals(owner, broker); - // simulate the owner is inactive + // case 5: the owner lookup gets delayed var spyRegistry = spy(new BrokerRegistryImpl(pulsar)); - doReturn(CompletableFuture.completedFuture(Optional.empty())) - .when(spyRegistry).lookupAsync(eq(broker)); FieldUtils.writeDeclaredField(channel1, "brokerRegistry", spyRegistry , true); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1000, true); + var delayedFuture = new CompletableFuture(); + doReturn(delayedFuture).when(spyRegistry).lookupAsync(eq(broker)); + CompletableFuture.runAsync(() -> { + try { + Thread.sleep(500); + } catch (InterruptedException e) { + Thread.currentThread().interrupt();; + } + delayedFuture.complete(Optional.of(broker)); + }); - - // verify getOwnerAsync times out because the owner is inactive now. + // verify the owner eventually returns in inFlightStateWaitingTimeInMillis. long start = System.currentTimeMillis(); + assertEquals(broker, channel1.getOwnerAsync(bundle).get().get()); + long elapsed = System.currentTimeMillis() - start; + assertTrue(elapsed < 1000); + + // case 6: the owner is inactive + doReturn(CompletableFuture.completedFuture(Optional.empty())) + .when(spyRegistry).lookupAsync(eq(broker)); + + // verify getOwnerAsync times out + start = System.currentTimeMillis(); var ex = expectThrows(ExecutionException.class, () -> channel1.getOwnerAsync(bundle).get()); assertTrue(ex.getCause() instanceof IllegalStateException); assertTrue(System.currentTimeMillis() - start >= 1000); - // simulate ownership cleanup(no selected owner) by the leader channel + // case 7: the ownership cleanup(no new owner) by the leader channel doReturn(CompletableFuture.completedFuture(Optional.empty())) .when(loadManager).selectAsync(any(), any(), any()); var leaderChannel = channel1; @@ -1669,7 +1700,8 @@ public void testActiveGetOwner() throws Exception { waitUntilState(channel2, bundle, Init); assertTrue(System.currentTimeMillis() - start < 20_000); - // simulate ownership cleanup(brokerId1 selected owner) by the leader channel + + // case 8: simulate ownership cleanup(brokerId1 as the new owner) by the leader channel overrideTableViews(bundle, new ServiceUnitStateData(Owned, broker, null, 1)); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) @@ -1694,6 +1726,7 @@ public void testActiveGetOwner() throws Exception { } + private static ConcurrentHashMap>> getOwnerRequests( ServiceUnitStateChannel channel) throws IllegalAccessException { return (ConcurrentHashMap>>) From f7d35e5ddbfb96ef4eda636ba7808868dc56017f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 14 Jun 2024 04:24:07 +0300 Subject: [PATCH 301/580] [improve][meta] Fix invalid use of drain API and race condition in closing metadata store (#22585) Co-authored-by: Matteo Merli --- .../AbstractBatchedMetadataStore.java | 29 ++++++++++++++++--- 1 file changed, 25 insertions(+), 4 deletions(-) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java index 5b45530d2e20e..4fa1c6aca0fee 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java @@ -86,9 +86,13 @@ public void close() throws Exception { // Fail all the pending items MetadataStoreException ex = new MetadataStoreException.AlreadyClosedException("Metadata store is getting closed"); - readOps.drain(op -> op.getFuture().completeExceptionally(ex)); - writeOps.drain(op -> op.getFuture().completeExceptionally(ex)); - + MetadataOp op; + while ((op = readOps.poll()) != null) { + op.getFuture().completeExceptionally(ex); + } + while ((op = writeOps.poll()) != null) { + op.getFuture().completeExceptionally(ex); + } scheduledTask.cancel(true); } super.close(); @@ -98,7 +102,13 @@ public void close() throws Exception { private void flush() { while (!readOps.isEmpty()) { List ops = new ArrayList<>(); - readOps.drain(ops::add, maxOperations); + for (int i = 0; i < maxOperations; i++) { + MetadataOp op = readOps.poll(); + if (op == null) { + break; + } + ops.add(op); + } internalBatchOperation(ops); } @@ -167,6 +177,11 @@ public void updateMetadataEventSynchronizer(MetadataEventSynchronizer synchroniz } private void enqueue(MessagePassingQueue queue, MetadataOp op) { + if (isClosed()) { + MetadataStoreException ex = new MetadataStoreException.AlreadyClosedException(); + op.getFuture().completeExceptionally(ex); + return; + } if (enabled) { if (!queue.offer(op)) { // Execute individually if we're failing to enqueue @@ -182,6 +197,12 @@ private void enqueue(MessagePassingQueue queue, MetadataOp op) { } private void internalBatchOperation(List ops) { + if (isClosed()) { + MetadataStoreException ex = + new MetadataStoreException.AlreadyClosedException(); + ops.forEach(op -> op.getFuture().completeExceptionally(ex)); + return; + } long now = System.currentTimeMillis(); for (MetadataOp op : ops) { this.batchMetadataStoreStats.recordOpWaiting(now - op.created()); From 7a21918cb70e6da33e1829d1f28d21bdd03be799 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Fri, 14 Jun 2024 17:02:33 +0800 Subject: [PATCH 302/580] [fix][cli] Fix the pulsar-daemon parameter passthrough syntax (#22905) Co-authored-by: Lari Hotari --- bin/pulsar-daemon | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/bin/pulsar-daemon b/bin/pulsar-daemon index 210162b6a2190..2c05cb5c49dab 100755 --- a/bin/pulsar-daemon +++ b/bin/pulsar-daemon @@ -157,7 +157,7 @@ start () echo starting $command, logging to $logfile echo Note: Set immediateFlush to true in conf/log4j2.yaml will guarantee the logging event is flushing to disk immediately. The default behavior is switched off due to performance considerations. pulsar=$PULSAR_HOME/bin/pulsar - nohup $pulsar $command "$1" > "$out" 2>&1 < /dev/null & + nohup $pulsar $command "$@" > "$out" 2>&1 < /dev/null & echo $! > $pid sleep 1; head $out sleep 2; @@ -216,7 +216,7 @@ stop () case $startStop in (start) - start "$*" + start "$@" ;; (stop) @@ -224,21 +224,20 @@ case $startStop in ;; (restart) - forceStopFlag=$(echo "$*"|grep "\-force") - if [[ "$forceStopFlag" != "" ]] + if [[ "$1" == "-force" ]] then - stop "-force" + stop -force + # remove "-force" from the arguments + shift else stop fi if [ "$?" == 0 ] then - sleep 3 - paramaters="$*" - startParamaters=${paramaters//-force/} - start "$startParamaters" + sleep 3 + start "$@" else - echo "WARNNING : $command failed restart, for $command is not stopped completely." + echo "WARNNING : $command failed restart, for $command is not stopped completely." fi ;; From 6831231e7aeffa39c4d79f5983ef9dc7ba25c449 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Fri, 14 Jun 2024 20:05:09 +0800 Subject: [PATCH 303/580] [fix][broker] Fix topic status for oldestBacklogMessageAgeSeconds continuously increases even when there is no backlog. (#22907) --- .../service/persistent/PersistentTopic.java | 24 ++- .../service/BacklogQuotaManagerTest.java | 180 +++++++++++++++++- 2 files changed, 194 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 50129ebee0ad0..711e1d93f742f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -1467,7 +1467,7 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, return FutureUtil.failedFuture( new TopicBusyException("Topic has subscriptions: " + subscriptions.keys())); } else if (failIfHasBacklogs) { - if (hasBacklogs()) { + if (hasBacklogs(false)) { List backlogSubs = subscriptions.values().stream() .filter(sub -> sub.getNumberOfEntriesInBacklog(false) > 0) @@ -2638,12 +2638,9 @@ public CompletableFuture asyncGetStats(GetStatsOptions stats.backlogQuotaLimitTime = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); TimeBasedBacklogQuotaCheckResult backlogQuotaCheckResult = timeBasedBacklogQuotaCheckResult; - stats.oldestBacklogMessageAgeSeconds = (backlogQuotaCheckResult == null) - ? (long) -1 - : TimeUnit.MILLISECONDS.toSeconds( - Clock.systemUTC().millis() - backlogQuotaCheckResult.getPositionPublishTimestampInMillis()); - + stats.oldestBacklogMessageAgeSeconds = getBestEffortOldestUnacknowledgedMessageAgeSeconds(); stats.oldestBacklogMessageSubscriptionName = (backlogQuotaCheckResult == null) + || !hasBacklogs(getStatsOptions.isGetPreciseBacklog()) ? null : backlogQuotaCheckResult.getCursorName(); @@ -2906,7 +2903,7 @@ public boolean isActive(InactiveTopicDeleteMode deleteMode) { } break; case delete_when_subscriptions_caught_up: - if (hasBacklogs()) { + if (hasBacklogs(false)) { return true; } break; @@ -2919,8 +2916,8 @@ public boolean isActive(InactiveTopicDeleteMode deleteMode) { } } - private boolean hasBacklogs() { - return subscriptions.values().stream().anyMatch(sub -> sub.getNumberOfEntriesInBacklog(false) > 0); + private boolean hasBacklogs(boolean getPreciseBacklog) { + return subscriptions.values().stream().anyMatch(sub -> sub.getNumberOfEntriesInBacklog(getPreciseBacklog) > 0); } @Override @@ -3466,6 +3463,9 @@ public boolean isSizeBacklogExceeded() { @Override public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { + if (!hasBacklogs(false)) { + return 0; + } TimeBasedBacklogQuotaCheckResult result = timeBasedBacklogQuotaCheckResult; if (result == null) { return -1; @@ -3553,6 +3553,9 @@ public CompletableFuture checkTimeBacklogExceeded() { } if (brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()) { + if (!hasBacklogs(true)) { + return CompletableFuture.completedFuture(false); + } CompletableFuture future = new CompletableFuture<>(); // Check if first unconsumed message(first message after mark delete position) // for slowest cursor's has expired. @@ -3606,6 +3609,9 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { return future; } else { try { + if (!hasBacklogs(false)) { + return CompletableFuture.completedFuture(false); + } EstimateTimeBasedBacklogQuotaCheckResult checkResult = estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); if (checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp() != null) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java index 6be7023b161f1..56f9f4f91246e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BacklogQuotaManagerTest.java @@ -29,6 +29,7 @@ import static org.assertj.core.api.AssertionsForClassTypes.within; 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 com.google.common.collect.Sets; @@ -296,8 +297,12 @@ public void testBacklogQuotaWithReader() throws Exception { } private TopicStats getTopicStats(String topic1) throws PulsarAdminException { + return getTopicStats(topic1, true); + } + + private TopicStats getTopicStats(String topic1, boolean getPreciseBacklog) throws PulsarAdminException { TopicStats stats = - admin.topics().getStats(topic1, GetStatsOptions.builder().getPreciseBacklog(true).build()); + admin.topics().getStats(topic1, GetStatsOptions.builder().getPreciseBacklog(getPreciseBacklog).build()); return stats; } @@ -502,9 +507,117 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce // Cache should be used, since position hasn't changed assertThat(getReadEntries(topic1)).isEqualTo(readEntries); + + // Move subscription 1 and 2 to end + Message msg = consumer1.receive(); + consumer1.acknowledge(msg); + consumer2.acknowledge(secondOldestMessage); + for (int i = 0; i < 2; i++) { + Message message = consumer2.receive(); + log.info("Subscription 2 about to ack message ID {}", message.getMessageId()); + consumer2.acknowledge(message); + } + + log.info("Subscription 1 and 2 moved to end. Now should not backlog"); + waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + topicStats = getTopicStats(topic1); + assertThat(topicStats.getBacklogSize()).isEqualTo(0); + assertThat(topicStats.getSubscriptions().get(subName1).getMsgBacklog()).isEqualTo(0); + assertThat(topicStats.getSubscriptions().get(subName2).getMsgBacklog()).isEqualTo(0); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(0); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); + + metrics = prometheusMetricsClient.getMetrics(); + backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); + assertThat(backlogAgeMetric.tags).containsExactly( + entry("cluster", CLUSTER_NAME), + entry("namespace", namespace), + entry("topic", topic1)); + assertThat((long) backlogAgeMetric.value).isEqualTo(0); + + // producer should create success. + Producer producer2 = createProducer(client, topic1); + assertNotNull(producer2); } } + @Test + public void backlogsStatsPreciseWithNoBacklog() throws PulsarAdminException, PulsarClientException, InterruptedException { + config.setPreciseTimeBasedBacklogQuotaCheck(true); + config.setExposePreciseBacklogInPrometheus(true); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + final int timeLimitSeconds = 2; + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitTime(timeLimitSeconds) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + message_age); + + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) + .maxBackoffInterval(5, SECONDS) + .statsInterval(0, SECONDS).build()) { + final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); + + final String subName1 = "c1"; + final int numMsgs = 4; + + Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1) + .acknowledgmentGroupTime(0, SECONDS) + .subscribe(); + Producer producer = createProducer(client, topic1); + + byte[] content = new byte[1024]; + for (int i = 0; i < numMsgs; i++) { + MessageId send = producer.send(content); + System.out.println(i + ":msg:" + MILLISECONDS.toSeconds(System.currentTimeMillis())); + } + + String c1MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName1).markDeletePosition; + + // Move subscription 1 to end + for (int i = 0; i < numMsgs; i++) { + Message message1 = consumer1.receive(); + consumer1.acknowledge(message1); + } + + // This code will wait about 4~5 Seconds, to make sure the oldest message is 4~5 seconds old + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + Metrics metrics = prometheusMetricsClient.getMetrics(); + TopicStats topicStats = getTopicStats(topic1); + + assertThat(topicStats.getBacklogQuotaLimitTime()).isEqualTo(timeLimitSeconds); + assertThat(topicStats.getBacklogSize()).isEqualTo(0); + assertThat(topicStats.getSubscriptions().get(subName1).getMsgBacklog()).isEqualTo(0); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(0); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); + + Metric backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); + assertThat(backlogAgeMetric.tags).containsExactly( + entry("cluster", CLUSTER_NAME), + entry("namespace", namespace), + entry("topic", topic1)); + assertThat((long) backlogAgeMetric.value).isEqualTo(0); + + // producer should create success. + Producer producer2 = createProducer(client, topic1); + assertNotNull(producer2); + } + config.setPreciseTimeBasedBacklogQuotaCheck(false); + config.setExposePreciseBacklogInPrometheus(false); + } + private long getReadEntries(String topic1) { return ((PersistentTopic) pulsar.getBrokerService().getTopicReference(topic1).get()) .getManagedLedger().getStats().getEntriesReadTotalCount(); @@ -609,6 +722,71 @@ public void backlogsStatsNotPrecise() throws PulsarAdminException, PulsarClientE } } + @Test + public void backlogsStatsNotPreciseWithNoBacklog() throws PulsarAdminException, PulsarClientException, InterruptedException { + config.setPreciseTimeBasedBacklogQuotaCheck(false); + config.setExposePreciseBacklogInPrometheus(false); + config.setManagedLedgerMaxEntriesPerLedger(6); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + final int timeLimitSeconds = 2; + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitTime(timeLimitSeconds) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + message_age); + + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) + .maxBackoffInterval(3, SECONDS) + .statsInterval(0, SECONDS).build()) { + final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); + + final String subName1 = "brandNewC1"; + final int numMsgs = 5; + + Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1) + .acknowledgmentGroupTime(0, SECONDS) + .isAckReceiptEnabled(true) + .subscribe(); + Producer producer = createProducer(client, topic1); + + byte[] content = new byte[1024]; + for (int i = 0; i < numMsgs; i++) { + producer.send(content); + } + + String c1MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName1).markDeletePosition; + + log.info("Moved subscription 1 to end"); + for (int i = 0; i < numMsgs; i++) { + consumer1.acknowledge(consumer1.receive()); + } + + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + // backlog and backlogAceSeconds should be 0 + TopicStats topicStats = getTopicStats(topic1, false); + Metrics metrics = prometheusMetricsClient.getMetrics(); + assertEquals(topicStats.getSubscriptions().get(subName1).getMsgBacklog(), 0); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(0); + Metric backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); + assertThat(backlogAgeMetric.value).isEqualTo(0); + + // producer should create success. + Producer producer2 = createProducer(client, topic1); + assertNotNull(producer2); + + config.setManagedLedgerMaxEntriesPerLedger(MAX_ENTRIES_PER_LEDGER); + } + } + private void unloadAndLoadTopic(String topic, Producer producer) throws PulsarAdminException, PulsarClientException { admin.topics().unload(topic); From f1228177727a4033057a14c3d799474d871f5391 Mon Sep 17 00:00:00 2001 From: corey <145762140+richiefanfan@users.noreply.github.com> Date: Fri, 14 Jun 2024 23:47:34 +0800 Subject: [PATCH 304/580] [fix] [broker] Fix typo in PersistentTopicsBase (#22904) --- .../apache/pulsar/broker/admin/impl/PersistentTopicsBase.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index ff764b368eb83..2f2a899950a1e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -4480,11 +4480,11 @@ private CompletableFuture validateNonPartitionTopicNameAsync(String topicN // Partition topic index is 0 to (number of partition - 1) if (metadata.partitions > 0 && suffix >= (long) metadata.partitions) { log.warn("[{}] Can't create topic {} with \"-partition-\" followed by" - + " a number smaller then number of partition of partitioned topic {}.", + + " a number smaller than number of partition of partitioned topic {}.", clientAppId(), topicName, partitionTopicName.getLocalName()); throw new RestException(Status.PRECONDITION_FAILED, "Can't create topic " + topicName + " with \"-partition-\" followed by" - + " a number smaller then number of partition of partitioned topic " + + " a number smaller than number of partition of partitioned topic " + partitionTopicName.getLocalName()); } else if (metadata.partitions == 0) { log.warn("[{}] Can't create topic {} with \"-partition-\" followed by" From f83dbe9cf77a0ca64e0ebc0f2dc75994372c69a7 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 14 Jun 2024 17:34:43 -0700 Subject: [PATCH 305/580] [feat][broker] PIP-264: Add OpenTelemetry producer metrics (#22882) --- .../apache/pulsar/broker/PulsarService.java | 7 ++ .../pulsar/broker/service/Producer.java | 86 +++++++++-------- .../stats/OpenTelemetryProducerStats.java | 95 +++++++++++++++++++ .../stats/OpenTelemetryProducerStatsTest.java | 88 +++++++++++++++++ .../client/api/BrokerServiceLookupTest.java | 1 + .../client/api/NonPersistentTopicTest.java | 31 +++++- .../NonPersistentPublisherStatsImpl.java | 20 ++++ .../data/stats/PublisherStatsImpl.java | 39 ++++++++ .../org/apache/pulsar/common/stats/Rate.java | 7 ++ .../OpenTelemetryAttributes.java | 15 +++ 10 files changed, 348 insertions(+), 41 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryProducerStats.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryProducerStatsTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 6cbc99e2cf4d4..65dd90f7a1235 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -112,6 +112,7 @@ import org.apache.pulsar.broker.service.schema.SchemaStorageFactory; import org.apache.pulsar.broker.stats.MetricsGenerator; import org.apache.pulsar.broker.stats.OpenTelemetryConsumerStats; +import org.apache.pulsar.broker.stats.OpenTelemetryProducerStats; import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; import org.apache.pulsar.broker.stats.PulsarBrokerOpenTelemetry; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; @@ -258,6 +259,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private final PulsarBrokerOpenTelemetry openTelemetry; private OpenTelemetryTopicStats openTelemetryTopicStats; private OpenTelemetryConsumerStats openTelemetryConsumerStats; + private OpenTelemetryProducerStats openTelemetryProducerStats; private TransactionMetadataStoreService transactionMetadataStoreService; private TransactionBufferProvider transactionBufferProvider; @@ -676,6 +678,10 @@ public CompletableFuture closeAsync() { brokerClientSharedTimer.stop(); monotonicSnapshotClock.close(); + if (openTelemetryProducerStats != null) { + openTelemetryProducerStats.close(); + openTelemetryProducerStats = null; + } if (openTelemetryConsumerStats != null) { openTelemetryConsumerStats.close(); openTelemetryConsumerStats = null; @@ -827,6 +833,7 @@ public void start() throws PulsarServerException { openTelemetryTopicStats = new OpenTelemetryTopicStats(this); openTelemetryConsumerStats = new OpenTelemetryConsumerStats(this); + openTelemetryProducerStats = new OpenTelemetryProducerStats(this); localMetadataSynchronizer = StringUtils.isNotBlank(config.getMetadataSyncEventTopic()) ? new PulsarMetadataEventSynchronizer(this, config.getMetadataSyncEventTopic()) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java index cf54ffea7db66..b4578711027ef 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java @@ -23,10 +23,12 @@ import static org.apache.pulsar.common.protocol.Commands.hasChecksum; import static org.apache.pulsar.common.protocol.Commands.readChecksum; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.CaseFormat; import com.google.common.base.MoreObjects; import io.netty.buffer.ByteBuf; import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; +import io.opentelemetry.api.common.Attributes; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -36,6 +38,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.intercept.BrokerInterceptor; @@ -57,8 +60,8 @@ import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.schema.SchemaVersion; -import org.apache.pulsar.common.stats.Rate; import org.apache.pulsar.common.util.DateFormatter; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,10 +77,6 @@ public class Producer { private final long producerId; private final String appId; private final BrokerInterceptor brokerInterceptor; - private Rate msgIn; - private Rate chunkedMessageRate; - // it records msg-drop rate only for non-persistent topic - private final Rate msgDrop; private volatile long pendingPublishAcks = 0; private static final AtomicLongFieldUpdater pendingPublishAcksUpdater = AtomicLongFieldUpdater @@ -87,6 +86,10 @@ public class Producer { private final CompletableFuture closeFuture; private final PublisherStatsImpl stats; + private volatile Attributes attributes = null; + private static final AtomicReferenceFieldUpdater ATTRIBUTES_FIELD_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(Producer.class, Attributes.class, "attributes"); + private final boolean isRemote; private final String remoteCluster; private final boolean isNonPersistentTopic; @@ -118,10 +121,7 @@ public Producer(Topic topic, TransportCnx cnx, long producerId, String producerN this.epoch = epoch; this.closeFuture = new CompletableFuture<>(); this.appId = appId; - this.msgIn = new Rate(); - this.chunkedMessageRate = new Rate(); this.isNonPersistentTopic = topic instanceof NonPersistentTopic; - this.msgDrop = this.isNonPersistentTopic ? new Rate() : null; this.isShadowTopic = topic instanceof PersistentTopic && ((PersistentTopic) topic).getShadowSourceTopic().isPresent(); @@ -270,7 +270,7 @@ public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf he private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, long batchSize, boolean isChunked, boolean isMarker, Position position) { MessagePublishContext messagePublishContext = - MessagePublishContext.get(this, sequenceId, msgIn, headersAndPayload.readableBytes(), + MessagePublishContext.get(this, sequenceId, headersAndPayload.readableBytes(), batchSize, isChunked, System.nanoTime(), isMarker, position); if (brokerInterceptor != null) { brokerInterceptor @@ -282,7 +282,7 @@ private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, l private void publishMessageToTopic(ByteBuf headersAndPayload, long lowestSequenceId, long highestSequenceId, long batchSize, boolean isChunked, boolean isMarker, Position position) { MessagePublishContext messagePublishContext = MessagePublishContext.get(this, lowestSequenceId, - highestSequenceId, msgIn, headersAndPayload.readableBytes(), batchSize, + highestSequenceId, headersAndPayload.readableBytes(), batchSize, isChunked, System.nanoTime(), isMarker, position); if (brokerInterceptor != null) { brokerInterceptor @@ -339,8 +339,8 @@ private void publishOperationCompleted() { } public void recordMessageDrop(int batchSize) { - if (this.isNonPersistentTopic) { - msgDrop.recordEvent(batchSize); + if (stats instanceof NonPersistentPublisherStatsImpl nonPersistentPublisherStats) { + nonPersistentPublisherStats.recordMsgDrop(batchSize); } } @@ -374,7 +374,6 @@ private static final class MessagePublishContext implements PublishContext, Runn private long sequenceId; private long ledgerId; private long entryId; - private Rate rateIn; private int msgSize; private long batchSize; private boolean chunked; @@ -536,13 +535,13 @@ public void run() { } // stats - rateIn.recordMultipleEvents(batchSize, msgSize); + producer.stats.recordMsgIn(batchSize, msgSize); producer.topic.recordAddLatency(System.nanoTime() - startTimeNs, TimeUnit.NANOSECONDS); producer.cnx.getCommandSender().sendSendReceiptResponse(producer.producerId, sequenceId, highestSequenceId, ledgerId, entryId); producer.cnx.completedSendOperation(producer.isNonPersistentTopic, msgSize); if (this.chunked) { - producer.chunkedMessageRate.recordEvent(); + producer.stats.recordChunkedMsgIn(); } producer.publishOperationCompleted(); if (producer.brokerInterceptor != null) { @@ -552,12 +551,11 @@ public void run() { recycle(); } - static MessagePublishContext get(Producer producer, long sequenceId, Rate rateIn, int msgSize, - long batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) { + static MessagePublishContext get(Producer producer, long sequenceId, int msgSize, long batchSize, + boolean chunked, long startTimeNs, boolean isMarker, Position position) { MessagePublishContext callback = RECYCLER.get(); callback.producer = producer; callback.sequenceId = sequenceId; - callback.rateIn = rateIn; callback.msgSize = msgSize; callback.batchSize = batchSize; callback.chunked = chunked; @@ -573,13 +571,12 @@ static MessagePublishContext get(Producer producer, long sequenceId, Rate rateIn return callback; } - static MessagePublishContext get(Producer producer, long lowestSequenceId, long highestSequenceId, Rate rateIn, - int msgSize, long batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) { + static MessagePublishContext get(Producer producer, long lowestSequenceId, long highestSequenceId, int msgSize, + long batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) { MessagePublishContext callback = RECYCLER.get(); callback.producer = producer; callback.sequenceId = lowestSequenceId; callback.highestSequenceId = highestSequenceId; - callback.rateIn = rateIn; callback.msgSize = msgSize; callback.batchSize = batchSize; callback.originalProducerName = null; @@ -628,7 +625,6 @@ public void recycle() { highestSequenceId = -1L; originalSequenceId = -1L; originalHighestSequenceId = -1L; - rateIn = null; msgSize = 0; ledgerId = -1L; entryId = -1L; @@ -733,25 +729,12 @@ public void topicMigrated(Optional clusterUrl) { } public void updateRates() { - msgIn.calculateRate(); - chunkedMessageRate.calculateRate(); - stats.msgRateIn = msgIn.getRate(); - stats.msgThroughputIn = msgIn.getValueRate(); - stats.averageMsgSize = msgIn.getAverageValue(); - stats.chunkedMessageRate = chunkedMessageRate.getRate(); - if (chunkedMessageRate.getCount() > 0 && this.topic instanceof PersistentTopic) { - ((PersistentTopic) this.topic).msgChunkPublished = true; - } - if (this.isNonPersistentTopic) { - msgDrop.calculateRate(); - ((NonPersistentPublisherStatsImpl) stats).msgDropRate = msgDrop.getValueRate(); + stats.calculateRates(); + if (stats.getMsgChunkIn().getCount() > 0 && topic instanceof PersistentTopic persistentTopic) { + persistentTopic.msgChunkPublished = true; } } - public void updateRates(int numOfMessages, long msgSizeInBytes) { - msgIn.recordMultipleEvents(numOfMessages, msgSizeInBytes); - } - public boolean isRemote() { return isRemote; } @@ -817,7 +800,7 @@ public void publishTxnMessage(TxnID txnID, long producerId, long sequenceId, lon return; } MessagePublishContext messagePublishContext = - MessagePublishContext.get(this, sequenceId, highSequenceId, msgIn, + MessagePublishContext.get(this, sequenceId, highSequenceId, headersAndPayload.readableBytes(), batchSize, isChunked, System.nanoTime(), isMarker, null); if (brokerInterceptor != null) { brokerInterceptor @@ -871,4 +854,29 @@ public void incrementThrottleCount() { public void decrementThrottleCount() { cnx.decrementThrottleCount(); } + + public Attributes getOpenTelemetryAttributes() { + if (attributes != null) { + return attributes; + } + return ATTRIBUTES_FIELD_UPDATER.updateAndGet(this, old -> { + if (old != null) { + return old; + } + var topicName = TopicName.get(topic.getName()); + var builder = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_PRODUCER_NAME, producerName) + .put(OpenTelemetryAttributes.PULSAR_PRODUCER_ID, producerId) + .put(OpenTelemetryAttributes.PULSAR_PRODUCER_ACCESS_MODE, + CaseFormat.UPPER_CAMEL.to(CaseFormat.LOWER_UNDERSCORE, accessMode.name())) + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, topicName.getDomain().toString()) + .put(OpenTelemetryAttributes.PULSAR_TENANT, topicName.getTenant()) + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, topicName.getNamespace()) + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName.getPartitionedTopicName()); + if (topicName.isPartitioned()) { + builder.put(OpenTelemetryAttributes.PULSAR_PARTITION_INDEX, topicName.getPartitionIndex()); + } + return builder.build(); + }); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryProducerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryProducerStats.java new file mode 100644 index 0000000000000..9c09804554c31 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryProducerStats.java @@ -0,0 +1,95 @@ +/* + * 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.stats; + +import io.opentelemetry.api.metrics.BatchCallback; +import io.opentelemetry.api.metrics.ObservableLongMeasurement; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.Producer; +import org.apache.pulsar.common.policies.data.stats.NonPersistentPublisherStatsImpl; + +public class OpenTelemetryProducerStats implements AutoCloseable { + + // Replaces pulsar_producer_msg_rate_in + public static final String MESSAGE_IN_COUNTER = "pulsar.broker.producer.message.incoming.count"; + private final ObservableLongMeasurement messageInCounter; + + // Replaces pulsar_producer_msg_throughput_in + public static final String BYTES_IN_COUNTER = "pulsar.broker.producer.message.incoming.size"; + private final ObservableLongMeasurement bytesInCounter; + + public static final String MESSAGE_DROP_COUNTER = "pulsar.broker.producer.message.drop.count"; + private final ObservableLongMeasurement messageDropCounter; + + private final BatchCallback batchCallback; + + public OpenTelemetryProducerStats(PulsarService pulsar) { + var meter = pulsar.getOpenTelemetry().getMeter(); + + messageInCounter = meter + .counterBuilder(MESSAGE_IN_COUNTER) + .setUnit("{message}") + .setDescription("The total number of messages received from this producer.") + .buildObserver(); + + bytesInCounter = meter + .counterBuilder(BYTES_IN_COUNTER) + .setUnit("By") + .setDescription("The total number of messages bytes received from this producer.") + .buildObserver(); + + messageDropCounter = meter + .counterBuilder(MESSAGE_DROP_COUNTER) + .setUnit("{message}") + .setDescription("The total number of messages dropped from this producer.") + .buildObserver(); + + batchCallback = meter.batchCallback(() -> pulsar.getBrokerService() + .getTopics() + .values() + .stream() + .filter(future -> future.isDone() && !future.isCompletedExceptionally()) + .map(CompletableFuture::join) + .filter(Optional::isPresent) + .flatMap(topic -> topic.get().getProducers().values().stream()) + .forEach(this::recordMetricsForProducer), + messageInCounter, + bytesInCounter, + messageDropCounter); + } + + @Override + public void close() { + batchCallback.close(); + } + + private void recordMetricsForProducer(Producer producer) { + var attributes = producer.getOpenTelemetryAttributes(); + var stats = producer.getStats(); + + messageInCounter.record(stats.getMsgInCounter(), attributes); + bytesInCounter.record(stats.getBytesInCounter(), attributes); + + if (stats instanceof NonPersistentPublisherStatsImpl nonPersistentStats) { + messageDropCounter.record(nonPersistentStats.getMsgDropCount(), attributes); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryProducerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryProducerStatsTest.java new file mode 100644 index 0000000000000..e273ac4446141 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryProducerStatsTest.java @@ -0,0 +1,88 @@ +/* + * 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.stats; + +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; +import static org.assertj.core.api.Assertions.assertThat; +import io.opentelemetry.api.common.Attributes; +import lombok.Cleanup; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class OpenTelemetryProducerStatsTest extends BrokerTestBase { + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.baseSetup(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder builder) { + super.customizeMainPulsarTestContextBuilder(builder); + builder.enableOpenTelemetry(true); + } + + + @Test(timeOut = 30_000) + public void testMessagingMetrics() throws Exception { + var topicName = BrokerTestUtil.newUniqueName("persistent://prop/ns-abc/testProducerMessagingMetrics"); + admin.topics().createNonPartitionedTopic(topicName); + + var messageCount = 5; + var producerName = BrokerTestUtil.newUniqueName("testProducerName"); + + @Cleanup + var producer = pulsarClient.newProducer() + .producerName(producerName) + .topic(topicName) + .create(); + for (int i = 0; i < messageCount; i++) { + producer.send(String.format("msg-%d", i).getBytes()); + } + + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, "persistent") + .put(OpenTelemetryAttributes.PULSAR_TENANT, "prop") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "prop/ns-abc") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName) + .put(OpenTelemetryAttributes.PULSAR_PRODUCER_NAME, producerName) + .put(OpenTelemetryAttributes.PULSAR_PRODUCER_ID, 0) + .put(OpenTelemetryAttributes.PULSAR_PRODUCER_ACCESS_MODE, "shared") + .build(); + + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + + assertMetricLongSumValue(metrics, OpenTelemetryProducerStats.MESSAGE_IN_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + assertMetricLongSumValue(metrics, OpenTelemetryProducerStats.BYTES_IN_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index 336728f279eda..e99802a5bc5c4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -192,6 +192,7 @@ public void testMultipleBrokerLookup() throws Exception { // Disable collecting topic stats during this test, as it deadlocks on access to map BrokerService.topics. pulsar2.getOpenTelemetryTopicStats().close(); pulsar2.getOpenTelemetryConsumerStats().close(); + pulsar2.getOpenTelemetryProducerStats().close(); var metricReader = pulsarTestContext.getOpenTelemetryMetricReader(); var lookupRequestSemaphoreField = BrokerService.class.getDeclaredField("lookupRequestSemaphore"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java index 4f64c4271fe89..e5c992ec6f858 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.client.api; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; +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.assertNotEquals; @@ -27,6 +29,7 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.google.common.collect.Sets; +import io.opentelemetry.api.common.Attributes; import java.net.URL; import java.util.HashSet; import java.util.Optional; @@ -50,6 +53,8 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentReplicator; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; +import org.apache.pulsar.broker.stats.OpenTelemetryProducerStats; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.MessageIdImpl; @@ -65,6 +70,7 @@ import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.apache.pulsar.zookeeper.ZookeeperServerTest; import org.awaitility.Awaitility; @@ -105,6 +111,12 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder pulsarTestContextBuilder) { + super.customizeMainPulsarTestContextBuilder(pulsarTestContextBuilder); + pulsarTestContextBuilder.enableOpenTelemetry(true); + } + @Test(timeOut = 90000 /* 1.5mn */) public void testNonPersistentPartitionsAreNotAutoCreatedWhenThePartitionedTopicDoesNotExist() throws Exception { final boolean defaultAllowAutoTopicCreation = conf.isAllowAutoTopicCreation(); @@ -357,9 +369,12 @@ public void testProducerRateLimit() throws Exception { @Cleanup("shutdownNow") ExecutorService executor = Executors.newFixedThreadPool(5); AtomicBoolean failed = new AtomicBoolean(false); + @Cleanup Consumer consumer = pulsarClient.newConsumer().topic(topic).subscriptionName("subscriber-1") .subscribe(); - Producer producer = pulsarClient.newProducer().topic(topic).create(); + var producerName = BrokerTestUtil.newUniqueName("testProducerRateLimit"); + @Cleanup + Producer producer = pulsarClient.newProducer().topic(topic).producerName(producerName).create(); byte[] msgData = "testData".getBytes(); final int totalProduceMessages = 10; CountDownLatch latch = new CountDownLatch(totalProduceMessages); @@ -392,7 +407,19 @@ public void testProducerRateLimit() throws Exception { // but as message should be dropped at broker: broker should not receive the message assertNotEquals(messageSet.size(), totalProduceMessages); - producer.close(); + // Verify the corresponding metric is updated + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_PRODUCER_NAME, producerName) + .put(OpenTelemetryAttributes.PULSAR_PRODUCER_ID, 0) + .put(OpenTelemetryAttributes.PULSAR_PRODUCER_ACCESS_MODE, "shared") + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, "non-persistent") + .put(OpenTelemetryAttributes.PULSAR_TENANT, "my-property") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "my-property/my-ns") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topic) + .build(); + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, OpenTelemetryProducerStats.MESSAGE_DROP_COUNTER, attributes, + value -> assertThat(value).isPositive()); } finally { conf.setMaxConcurrentNonPersistentMessagePerConnection(defaultNonPersistentMessageRate); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentPublisherStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentPublisherStatsImpl.java index adf3f92ae71fc..d62e9b8dbbeae 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentPublisherStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentPublisherStatsImpl.java @@ -18,10 +18,12 @@ */ package org.apache.pulsar.common.policies.data.stats; +import com.fasterxml.jackson.annotation.JsonIgnore; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.Objects; import lombok.Getter; import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats; +import org.apache.pulsar.common.stats.Rate; /** * Non-persistent publisher statistics. @@ -35,10 +37,28 @@ public class NonPersistentPublisherStatsImpl extends PublisherStatsImpl implemen @Getter public double msgDropRate; + @JsonIgnore + private final Rate msgDrop = new Rate(); + public NonPersistentPublisherStatsImpl add(NonPersistentPublisherStatsImpl stats) { Objects.requireNonNull(stats); super.add(stats); this.msgDropRate += stats.msgDropRate; return this; } + + public void calculateRates() { + super.calculateRates(); + msgDrop.calculateRate(); + msgDropRate = msgDrop.getRate(); + } + + public void recordMsgDrop(long numMessages) { + msgDrop.recordEvent(numMessages); + } + + @JsonIgnore + public long getMsgDropCount() { + return msgDrop.getTotalCount(); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PublisherStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PublisherStatsImpl.java index 304361bb2daec..3f9067eba0b25 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PublisherStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PublisherStatsImpl.java @@ -23,6 +23,7 @@ import lombok.Data; import org.apache.pulsar.client.api.ProducerAccessMode; import org.apache.pulsar.common.policies.data.PublisherStats; +import org.apache.pulsar.common.stats.Rate; /** * Statistics about a publisher. @@ -64,6 +65,11 @@ public class PublisherStatsImpl implements PublisherStats { /** Metadata (key/value strings) associated with this publisher. */ public Map metadata; + @JsonIgnore + private final Rate msgIn = new Rate(); + @JsonIgnore + private final Rate msgChunkIn = new Rate(); + public PublisherStatsImpl add(PublisherStatsImpl stats) { if (stats == null) { throw new IllegalArgumentException("stats can't be null"); @@ -107,4 +113,37 @@ public String getClientVersion() { public void setClientVersion(String clientVersion) { this.clientVersion = clientVersion; } + + public void calculateRates() { + msgIn.calculateRate(); + msgChunkIn.calculateRate(); + + msgRateIn = msgIn.getRate(); + msgThroughputIn = msgIn.getValueRate(); + averageMsgSize = msgIn.getAverageValue(); + chunkedMessageRate = msgChunkIn.getRate(); + } + + public void recordMsgIn(long messageCount, long byteCount) { + msgIn.recordMultipleEvents(messageCount, byteCount); + } + + @JsonIgnore + public long getMsgInCounter() { + return msgIn.getTotalCount(); + } + + @JsonIgnore + public long getBytesInCounter() { + return msgIn.getTotalValue(); + } + + public void recordChunkedMsgIn() { + msgChunkIn.recordEvent(); + } + + @JsonIgnore + public long getChunkedMsgInCounter() { + return msgChunkIn.getTotalCount(); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/Rate.java b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/Rate.java index 886e31ab71216..936962d8ee544 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/stats/Rate.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/stats/Rate.java @@ -28,6 +28,7 @@ public class Rate { private final LongAdder valueAdder = new LongAdder(); private final LongAdder countAdder = new LongAdder(); private final LongAdder totalCountAdder = new LongAdder(); + private final LongAdder totalValueAdder = new LongAdder(); // Computed stats private long count = 0L; @@ -43,12 +44,14 @@ public void recordEvent() { public void recordEvent(long value) { valueAdder.add(value); + totalValueAdder.add(value); countAdder.increment(); totalCountAdder.increment(); } public void recordMultipleEvents(long events, long totalValue) { valueAdder.add(totalValue); + totalValueAdder.add(totalValue); countAdder.add(events); totalCountAdder.add(events); } @@ -88,4 +91,8 @@ public double getValueRate() { public long getTotalCount() { return this.totalCountAdder.longValue(); } + + public long getTotalValue() { + return this.totalValueAdder.sum(); + } } diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index 9783f0e754f63..004741b6dfb55 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -87,6 +87,21 @@ public interface OpenTelemetryAttributes { */ AttributeKey PULSAR_CONSUMER_CONNECTED_SINCE = AttributeKey.longKey("pulsar.consumer.connected_since"); + /** + * The name of the Pulsar producer. + */ + AttributeKey PULSAR_PRODUCER_NAME = AttributeKey.stringKey("pulsar.producer.name"); + + /** + * The ID of the Pulsar producer. + */ + AttributeKey PULSAR_PRODUCER_ID = AttributeKey.longKey("pulsar.producer.id"); + + /** + * The access mode of the Pulsar producer. + */ + AttributeKey PULSAR_PRODUCER_ACCESS_MODE = AttributeKey.stringKey("pulsar.producer.access_mode"); + /** * The address of the Pulsar client. */ From 1dcd07be0e62ef40eb077d3a521eab49ce2e0966 Mon Sep 17 00:00:00 2001 From: jito Date: Mon, 17 Jun 2024 01:01:16 +0900 Subject: [PATCH 306/580] [fix][misc] Correct the comment of tlsAllowInsecureConnection in ClusterDataImpl class. (#22919) Signed-off-by: jitokim --- conf/client.conf | 2 +- deployment/terraform-ansible/templates/client.conf | 2 +- .../org/apache/pulsar/common/policies/data/ClusterDataImpl.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/conf/client.conf b/conf/client.conf index 8a485e5676c7b..25d65c3947e39 100644 --- a/conf/client.conf +++ b/conf/client.conf @@ -41,7 +41,7 @@ authPlugin= # authParams=tlsCertFile:/path/to/client-cert.pem,tlsKeyFile:/path/to/client-key.pem authParams= -# Allow TLS connections to servers whose certificate cannot be +# Allow TLS connections to servers whose certificate cannot # be verified to have been signed by a trusted certificate # authority. tlsAllowInsecureConnection=false diff --git a/deployment/terraform-ansible/templates/client.conf b/deployment/terraform-ansible/templates/client.conf index ba1d396bf8423..755577cf38e03 100644 --- a/deployment/terraform-ansible/templates/client.conf +++ b/deployment/terraform-ansible/templates/client.conf @@ -41,7 +41,7 @@ authPlugin= # authParams=tlsCertFile:/path/to/client-cert.pem,tlsKeyFile:/path/to/client-key.pem authParams= -# Allow TLS connections to servers whose certificate cannot be +# Allow TLS connections to servers whose certificate cannot # be verified to have been signed by a trusted certificate # authority. tlsAllowInsecureConnection=false diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterDataImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterDataImpl.java index fffe87a300562..6cb5a0034e938 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterDataImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterDataImpl.java @@ -107,7 +107,7 @@ public final class ClusterDataImpl implements ClusterData, Cloneable { private boolean brokerClientTlsEnabled; @ApiModelProperty( name = "tlsAllowInsecureConnection", - value = "Allow TLS connections to servers whose certificate cannot be" + value = "Allow TLS connections to servers whose certificate cannot" + " be verified to have been signed by a trusted certificate" + " authority." ) From 1a6254a5a69e03bccd82a4a456d9be9a4b9795a7 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Sun, 16 Jun 2024 13:01:54 -0700 Subject: [PATCH 307/580] [feat][broker] PIP-264: Enable OpenTelemetry reusable data memory mode (#22914) --- .../apache/pulsar/opentelemetry/OpenTelemetryService.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java index b5610fc485b3c..b32d353eb5ae7 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java @@ -25,6 +25,7 @@ import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; +import io.opentelemetry.sdk.common.export.MemoryMode; import io.opentelemetry.sdk.resources.Resource; import io.opentelemetry.semconv.ResourceAttributes; import java.io.Closeable; @@ -72,7 +73,9 @@ public OpenTelemetryService(String clusterName, sdkBuilder.addPropertiesSupplier(() -> Map.of( OTEL_SDK_DISABLED_KEY, "true", // Cardinality limit includes the overflow attribute set, so we need to add 1. - "otel.experimental.metrics.cardinality.limit", Integer.toString(MAX_CARDINALITY_LIMIT + 1) + "otel.experimental.metrics.cardinality.limit", Integer.toString(MAX_CARDINALITY_LIMIT + 1), + // Reduce number of allocations by using reusable data mode. + "otel.java.experimental.exporter.memory_mode", MemoryMode.REUSABLE_DATA.name() )); sdkBuilder.addResourceCustomizer( From fa745384c2c3ea8c16e6c0cd078328a653fa3073 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Mon, 17 Jun 2024 11:25:47 +0800 Subject: [PATCH 308/580] [fix] Revert "[fix][cli] Fix the shell script parameter passthrough syntax (#22867)" (#22921) --- bin/bookkeeper | 12 ++++++------ bin/pulsar | 38 +++++++++++++++++++------------------- 2 files changed, 25 insertions(+), 25 deletions(-) diff --git a/bin/bookkeeper b/bin/bookkeeper index 13d092f4c99a6..0cc07dd49aba5 100755 --- a/bin/bookkeeper +++ b/bin/bookkeeper @@ -214,20 +214,20 @@ OPTS="$OPTS $BK_METADATA_OPTIONS" #Change to BK_HOME to support relative paths cd "$BK_HOME" if [ $COMMAND == "bookie" ]; then - exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.server.Main --conf $BOOKIE_CONF "$@" + exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.server.Main --conf $BOOKIE_CONF $@ elif [ $COMMAND == "autorecovery" ]; then - exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.replication.AutoRecoveryMain --conf $BOOKIE_CONF "$@" + exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.replication.AutoRecoveryMain --conf $BOOKIE_CONF $@ elif [ $COMMAND == "localbookie" ]; then NUMBER=$1 shift - exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.util.LocalBookKeeper $NUMBER $BOOKIE_CONF "$@" + exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.util.LocalBookKeeper $NUMBER $BOOKIE_CONF $@ elif [ $COMMAND == "upgrade" ]; then - exec $JAVA $OPTS org.apache.bookkeeper.bookie.FileSystemUpgrade --conf $BOOKIE_CONF "$@" + exec $JAVA $OPTS org.apache.bookkeeper.bookie.FileSystemUpgrade --conf $BOOKIE_CONF $@ elif [ $COMMAND == "shell" ]; then ENTRY_FORMATTER_ARG="-DentryFormatterClass=${ENTRY_FORMATTER_CLASS:-org.apache.bookkeeper.util.StringEntryFormatter}" - exec $JAVA $OPTS $ENTRY_FORMATTER_ARG org.apache.bookkeeper.bookie.BookieShell -conf $BOOKIE_CONF "$@" + exec $JAVA $OPTS $ENTRY_FORMATTER_ARG org.apache.bookkeeper.bookie.BookieShell -conf $BOOKIE_CONF $@ elif [ $COMMAND == "help" -o $COMMAND == "--help" -o $COMMAND == "-h" ]; then bookkeeper_help; else - exec $JAVA $OPTS $COMMAND "$@" + exec $JAVA $OPTS $COMMAND $@ fi diff --git a/bin/pulsar b/bin/pulsar index f6061601d88b1..ab0029af5b0da 100755 --- a/bin/pulsar +++ b/bin/pulsar @@ -329,56 +329,56 @@ fi cd "$PULSAR_HOME" if [ $COMMAND == "broker" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-broker.log"} - exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.PulsarBrokerStarter --broker-conf $PULSAR_BROKER_CONF "$@" + exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.PulsarBrokerStarter --broker-conf $PULSAR_BROKER_CONF $@ elif [ $COMMAND == "bookie" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"bookkeeper.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.server.Main --conf $PULSAR_BOOKKEEPER_CONF "$@" + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.server.Main --conf $PULSAR_BOOKKEEPER_CONF $@ elif [ $COMMAND == "zookeeper" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"zookeeper.log"} - exec $JAVA ${ZK_OPTS} $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_ZK_CONF "$@" + exec $JAVA ${ZK_OPTS} $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_ZK_CONF $@ elif [ $COMMAND == "global-zookeeper" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"global-zookeeper.log"} # Allow global ZK to turn into read-only mode when it cannot reach the quorum OPTS="${OPTS} ${ZK_OPTS} -Dreadonlymode.enabled=true" - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_GLOBAL_ZK_CONF "$@" + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_GLOBAL_ZK_CONF $@ elif [ $COMMAND == "configuration-store" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"configuration-store.log"} # Allow global ZK to turn into read-only mode when it cannot reach the quorum OPTS="${OPTS} ${ZK_OPTS} -Dreadonlymode.enabled=true" - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_CONFIGURATION_STORE_CONF "$@" + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_CONFIGURATION_STORE_CONF $@ elif [ $COMMAND == "proxy" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-proxy.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.proxy.server.ProxyServiceStarter --config $PULSAR_PROXY_CONF "$@" + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.proxy.server.ProxyServiceStarter --config $PULSAR_PROXY_CONF $@ elif [ $COMMAND == "websocket" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-websocket.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.websocket.service.WebSocketServiceStarter $PULSAR_WEBSOCKET_CONF "$@" + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.websocket.service.WebSocketServiceStarter $PULSAR_WEBSOCKET_CONF $@ elif [ $COMMAND == "functions-worker" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-functions-worker.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.functions.worker.FunctionWorkerStarter -c $PULSAR_WORKER_CONF "$@" + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.functions.worker.FunctionWorkerStarter -c $PULSAR_WORKER_CONF $@ elif [ $COMMAND == "standalone" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-standalone.log"} - exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS ${ZK_OPTS} -Dpulsar.log.file=$PULSAR_LOG_FILE -Dpulsar.config.file=$PULSAR_STANDALONE_CONF org.apache.pulsar.PulsarStandaloneStarter "$@" + exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS ${ZK_OPTS} -Dpulsar.log.file=$PULSAR_LOG_FILE -Dpulsar.config.file=$PULSAR_STANDALONE_CONF org.apache.pulsar.PulsarStandaloneStarter $@ elif [ ${COMMAND} == "autorecovery" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-autorecovery.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.replication.AutoRecoveryMain --conf $PULSAR_BOOKKEEPER_CONF "$@" + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.replication.AutoRecoveryMain --conf $PULSAR_BOOKKEEPER_CONF $@ elif [ $COMMAND == "initialize-cluster-metadata" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataSetup "$@" + exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataSetup $@ elif [ $COMMAND == "delete-cluster-metadata" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataTeardown "$@" + exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataTeardown $@ elif [ $COMMAND == "initialize-transaction-coordinator-metadata" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarTransactionCoordinatorMetadataSetup "$@" + exec $JAVA $OPTS org.apache.pulsar.PulsarTransactionCoordinatorMetadataSetup $@ elif [ $COMMAND == "initialize-namespace" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarInitialNamespaceSetup "$@" + exec $JAVA $OPTS org.apache.pulsar.PulsarInitialNamespaceSetup $@ elif [ $COMMAND == "zookeeper-shell" ]; then - exec $JAVA $OPTS org.apache.zookeeper.ZooKeeperMain "$@" + exec $JAVA $OPTS org.apache.zookeeper.ZooKeeperMain $@ elif [ $COMMAND == "broker-tool" ]; then - exec $JAVA $OPTS org.apache.pulsar.broker.tools.BrokerTool "$@" + exec $JAVA $OPTS org.apache.pulsar.broker.tools.BrokerTool $@ elif [ $COMMAND == "compact-topic" ]; then - exec $JAVA $OPTS org.apache.pulsar.compaction.CompactorTool --broker-conf $PULSAR_BROKER_CONF "$@" + exec $JAVA $OPTS org.apache.pulsar.compaction.CompactorTool --broker-conf $PULSAR_BROKER_CONF $@ elif [ $COMMAND == "tokens" ]; then - exec $JAVA $OPTS org.apache.pulsar.utils.auth.tokens.TokensCliUtils "$@" + exec $JAVA $OPTS org.apache.pulsar.utils.auth.tokens.TokensCliUtils $@ elif [ $COMMAND == "version" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarVersionStarter "$@" + exec $JAVA $OPTS org.apache.pulsar.PulsarVersionStarter $@ elif [ $COMMAND == "help" -o $COMMAND == "--help" -o $COMMAND == "-h" ]; then pulsar_help; else From 9f8adc5962f2733bd192ebd3b58978ed9cebab14 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Mon, 17 Jun 2024 12:43:23 +0800 Subject: [PATCH 309/580] [improve] [pip] PIP-358: let resource weight work for OverloadShedder, LeastLongTermMessageRate, ModularLoadManagerImpl. (#22889) Implementation PR: https://github.com/apache/pulsar/pull/22888 ### Motivation Initially, we introduce `loadBalancerCPUResourceWeight`, `loadBalancerBandwidthInResourceWeight`, `loadBalancerBandwidthOutResourceWeight`, `loadBalancerMemoryResourceWeight`, `loadBalancerDirectMemoryResourceWeight` in `ThresholdShedder` to control the resource weight for different resources when calculating the load of the broker. Then we let it work for `LeastResourceUsageWithWeight` for better bundle placement policy. But https://github.com/apache/pulsar/pull/19559 and https://github.com/apache/pulsar/pull/21168 have point out that the actual load of the broker is not related to the memory usage and direct memory usage, thus we have changed the default value of `loadBalancerMemoryResourceWeight`, `loadBalancerDirectMemoryResourceWeight` to 0.0. There are still some places where memory usage and direct memory usage are used to calculate the load of the broker, such as `OverloadShedder`, `LeastLongTermMessageRate`, `ModularLoadManagerImpl`. We should let the resource weight work for these places so that we can set the resource weight to 0.0 to avoid the impact of memory usage and direct memory usage on the load of the broker. ### Modifications - Let resource weight work for `OverloadShedder`, `LeastLongTermMessageRate`, `ModularLoadManagerImpl`. --- pip/pip-358.md | 38 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 pip/pip-358.md diff --git a/pip/pip-358.md b/pip/pip-358.md new file mode 100644 index 0000000000000..cd5397309851a --- /dev/null +++ b/pip/pip-358.md @@ -0,0 +1,38 @@ + +# PIP-358: let resource weight work for OverloadShedder, LeastLongTermMessageRate, ModularLoadManagerImpl. + +# Background knowledge + +Initially, we introduce `loadBalancerCPUResourceWeight`, `loadBalancerBandwidthInResourceWeight`, `loadBalancerBandwidthOutResourceWeight`, +`loadBalancerMemoryResourceWeight`, `loadBalancerDirectMemoryResourceWeight` in `ThresholdShedder` to control the resource weight for +different resources when calculating the load of the broker. +Then we let it work for `LeastResourceUsageWithWeight` for better bundle placement policy. + +But https://github.com/apache/pulsar/pull/19559 and https://github.com/apache/pulsar/pull/21168 have pointed out that the actual load +of the broker is not related to the memory usage and direct memory usage, thus we have changed the default value of +`loadBalancerMemoryResourceWeight`, `loadBalancerDirectMemoryResourceWeight` to 0.0. + +There are still some places where memory usage and direct memory usage are used to calculate the load of the broker, such as +`OverloadShedder`, `LeastLongTermMessageRate`, `ModularLoadManagerImpl`. We should let the resource weight work for these places +so that we can set the resource weight to 0.0 to avoid the impact of memory usage and direct memory usage on the load of the broker. + +# Motivation + +The actual load of the broker is not related to the memory usage and direct memory usage, thus we should let the resource weight work for +`OverloadShedder`, `LeastLongTermMessageRate`, `ModularLoadManagerImpl` so that we can set the resource weight to 0.0 to avoid the impact of +memory usage and direct memory usage on the load of the broker. + + +# Detailed Design + +Let resource weight work for `OverloadShedder`, `LeastLongTermMessageRate`, `ModularLoadManagerImpl`. +- For `OverloadShedder`, `LeastLongTermMessageRate`, we replace `getMaxResourceUsage()` with `getMaxResourceUsageWithWeight()` in the calculation of the load of the broker. +- For `ModularLoadManagerImpl`, we replace `getMaxResourceUsage()` with `getMaxResourceUsageWithWeight()` when checking if the broker is overloaded and decide whether to update the broker data to metadata store. + +# Backward & Forward Compatibility + + +# Links + +* Mailing List discussion thread: https://lists.apache.org/thread/lj34s3vmjbzlwmy8d66d0bsb25vnq9ky +* Mailing List voting thread: https://lists.apache.org/thread/b7dzm0yz6l40pkxmxhto5mro7brmz57r From dfbf05a5d512a8643eb03f9422bfc8d8f42db23c Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Mon, 17 Jun 2024 12:43:53 +0800 Subject: [PATCH 310/580] [fix] [broker] fix unload bundle count metric. (#22895) ### Motivation Those bundles that are filtered when try to unload them should not be included in the indicator. ### Modifications Increment the metric only when the bundle are unloaded. --- .../impl/ModularLoadManagerImpl.java | 16 ++--- .../impl/ModularLoadManagerImplTest.java | 59 +++++++++++++++++++ 2 files changed, 68 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index 764580e9b6d95..e1259e97aa3e1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -38,6 +38,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -625,6 +626,7 @@ public synchronized void doLoadShedding() { final Multimap bundlesToUnload = loadSheddingStrategy.findBundlesForUnloading(loadData, conf); bundlesToUnload.asMap().forEach((broker, bundles) -> { + AtomicBoolean unloadBundleForBroker = new AtomicBoolean(false); bundles.forEach(bundle -> { final String namespaceName = LoadManagerShared.getNamespaceNameFromBundleName(bundle); final String bundleRange = LoadManagerShared.getBundleRangeFromBundleName(bundle); @@ -654,24 +656,24 @@ public synchronized void doLoadShedding() { pulsar.getAdminClient().namespaces() .unloadNamespaceBundle(namespaceName, bundleRange, destBroker.get()); loadData.getRecentlyUnloadedBundles().put(bundle, System.currentTimeMillis()); + unloadBundleCount++; + unloadBundleForBroker.set(true); } catch (PulsarServerException | PulsarAdminException e) { log.warn("Error when trying to perform load shedding on {} for broker {}", bundle, broker, e); } }); + if (unloadBundleForBroker.get()) { + unloadBrokerCount++; + } }); - updateBundleUnloadingMetrics(bundlesToUnload); + updateBundleUnloadingMetrics(); } /** * As leader broker, update bundle unloading metrics. - * - * @param bundlesToUnload */ - private void updateBundleUnloadingMetrics(Multimap bundlesToUnload) { - unloadBrokerCount += bundlesToUnload.keySet().size(); - unloadBundleCount += bundlesToUnload.values().size(); - + private void updateBundleUnloadingMetrics() { List metrics = new ArrayList<>(); Map dimensions = new HashMap<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java index 20a33a70bfa40..6ae491c55b845 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java @@ -23,6 +23,7 @@ import static org.apache.pulsar.broker.resources.LoadBalanceResources.BUNDLE_DATA_BASE_PATH; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -563,6 +564,64 @@ public void testLoadShedding() throws Exception { .unloadNamespaceBundle(Mockito.anyString(), Mockito.anyString(), Mockito.anyString()); } + @Test + public void testUnloadBundleMetric() throws Exception { + final NamespaceBundleStats stats1 = new NamespaceBundleStats(); + final NamespaceBundleStats stats2 = new NamespaceBundleStats(); + stats1.msgRateIn = 100; + stats2.msgRateIn = 200; + final Map statsMap = new ConcurrentHashMap<>(); + statsMap.put(mockBundleName(1), stats1); + statsMap.put(mockBundleName(2), stats2); + final LocalBrokerData localBrokerData = new LocalBrokerData(); + localBrokerData.update(new SystemResourceUsage(), statsMap); + final Namespaces namespacesSpy1 = spy(pulsar1.getAdminClient().namespaces()); + doNothing().when(namespacesSpy1).unloadNamespaceBundle(Mockito.anyString(), Mockito.anyString(), Mockito.anyString()); + setField(pulsar1.getAdminClient(), "namespaces", namespacesSpy1); + ModularLoadManagerImpl primaryLoadManagerSpy = spy(primaryLoadManager); + + pulsar1.getConfiguration().setLoadBalancerEnabled(true); + final LoadData loadData = (LoadData) getField(primaryLoadManagerSpy, "loadData"); + + final Map brokerDataMap = loadData.getBrokerData(); + final BrokerData brokerDataSpy1 = spy(brokerDataMap.get(primaryBrokerId)); + when(brokerDataSpy1.getLocalData()).thenReturn(localBrokerData); + brokerDataMap.put(primaryBrokerId, brokerDataSpy1); + // Need to update all the bundle data for the shredder to see the spy. + primaryLoadManagerSpy.handleDataNotification(new Notification(NotificationType.Created, LoadManager.LOADBALANCE_BROKERS_ROOT + "/broker:8080")); + + sleep(100); + + // Most expensive bundle will be unloaded. + localBrokerData.setCpu(new ResourceUsage(90, 100)); + primaryLoadManagerSpy.doLoadShedding(); + assertEquals(getField(primaryLoadManagerSpy, "unloadBundleCount"), 1l); + assertEquals(getField(primaryLoadManagerSpy, "unloadBrokerCount"), 1l); + + // Now less expensive bundle will be unloaded + primaryLoadManagerSpy.doLoadShedding(); + assertEquals(getField(primaryLoadManagerSpy, "unloadBundleCount"), 2l); + assertEquals(getField(primaryLoadManagerSpy, "unloadBrokerCount"), 2l); + + // Now both are in grace period: neither should be unloaded. + primaryLoadManagerSpy.doLoadShedding(); + assertEquals(getField(primaryLoadManagerSpy, "unloadBundleCount"), 2l); + assertEquals(getField(primaryLoadManagerSpy, "unloadBrokerCount"), 2l); + + // clear the recently unloaded bundles to avoid the grace period + loadData.getRecentlyUnloadedBundles().clear(); + + // Test bundle to be unloaded is filtered. + doAnswer(invocation -> { + // return empty broker to avoid unloading the bundle + return Optional.empty(); + }).when(primaryLoadManagerSpy).selectBroker(any()); + primaryLoadManagerSpy.doLoadShedding(); + + assertEquals(getField(primaryLoadManagerSpy, "unloadBundleCount"), 2l); + assertEquals(getField(primaryLoadManagerSpy, "unloadBrokerCount"), 2l); + } + // Test that ModularLoadManagerImpl will determine that writing local data to ZooKeeper is necessary if certain // metrics change by a percentage threshold. From 7020ea2f2c968b2e50e4f2feb90b16d5d61280eb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Mon, 17 Jun 2024 13:31:52 +0800 Subject: [PATCH 311/580] [improve][misc] Bump RoaringBitmap version to 1.0.6 (#22920) --- distribution/server/src/assemble/LICENSE.bin.txt | 3 +-- pom.xml | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 1a66ab6d70a2f..9f5209bc7fd8d 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -514,8 +514,7 @@ The Apache Software License, Version 2.0 * RxJava - io.reactivex.rxjava3-rxjava-3.0.1.jar * RoaringBitmap - - org.roaringbitmap-RoaringBitmap-0.9.44.jar - - org.roaringbitmap-shims-0.9.44.jar + - org.roaringbitmap-RoaringBitmap-1.0.6.jar * OpenTelemetry - io.opentelemetry-opentelemetry-api-1.38.0.jar - io.opentelemetry-opentelemetry-api-incubator-1.38.0-alpha.jar diff --git a/pom.xml b/pom.xml index 71562619c18d5..62644d38d167c 100644 --- a/pom.xml +++ b/pom.xml @@ -317,7 +317,7 @@ flexible messaging model and an intuitive client API. 1.3 0.4 9.1.0 - 0.9.44 + 1.0.6 1.6.1 6.4.0 3.33.0 From f3d4d5ac0442eed2b538b8587186cdc0b8df9987 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Jun 2024 09:26:28 +0300 Subject: [PATCH 312/580] [fix][fn] Enable optimized Netty direct byte buffer support for Pulsar Function runtimes (#22910) --- .../functions/runtime/RuntimeUtils.java | 18 ++++++++-- .../kubernetes/KubernetesRuntimeTest.java | 36 ++++++++++--------- .../runtime/process/ProcessRuntimeTest.java | 16 +++++---- 3 files changed, 46 insertions(+), 24 deletions(-) diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/RuntimeUtils.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/RuntimeUtils.java index 6160626c958ef..49a5dd40fa271 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/RuntimeUtils.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/RuntimeUtils.java @@ -367,12 +367,26 @@ public static List getCmd(InstanceConfig instanceConfig, instanceConfig.getFunctionDetails().getName(), shardId)); + // Needed for optimized Netty direct byte buffer support args.add("-Dio.netty.tryReflectionSetAccessible=true"); + // Handle possible shaded Netty versions + args.add("-Dorg.apache.pulsar.shade.io.netty.tryReflectionSetAccessible=true"); + args.add("-Dio.grpc.netty.shaded.io.netty.tryReflectionSetAccessible=true"); + + if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_11)) { + // Needed for optimized Netty direct byte buffer support + args.add("--add-opens"); + args.add("java.base/java.nio=ALL-UNNAMED"); + args.add("--add-opens"); + args.add("java.base/jdk.internal.misc=ALL-UNNAMED"); + } - // Needed for netty.DnsResolverUtil on JDK9+ if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) { + // Needed for optimized checksum calculation when com.scurrilous.circe.checksum.Java9IntHash + // is used. That gets used when the native library libcirce-checksum is not available or cannot + // be loaded. args.add("--add-opens"); - args.add("java.base/sun.net=ALL-UNNAMED"); + args.add("java.base/java.util.zip=ALL-UNNAMED"); } if (instanceConfig.getAdditionalJavaRuntimeArguments() != null) { diff --git a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java index 980f763f7c303..bf73f0a9d34a2 100644 --- a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java +++ b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java @@ -441,14 +441,14 @@ private void verifyJavaInstance(InstanceConfig config, String depsDir, boolean s if (null != depsDir) { extraDepsEnv = " -Dpulsar.functions.extra.dependencies.dir=" + depsDir; classpath = classpath + ":" + depsDir + "/*"; - totalArgs = 46; - portArg = 33; - metricsPortArg = 35; + totalArgs = 52; + portArg = 39; + metricsPortArg = 41; } else { extraDepsEnv = ""; - portArg = 32; - metricsPortArg = 34; - totalArgs = 45; + portArg = 38; + metricsPortArg = 40; + totalArgs = 51; } if (secretsAttached) { totalArgs += 4; @@ -479,7 +479,11 @@ private void verifyJavaInstance(InstanceConfig config, String depsDir, boolean s + "-Dpulsar.function.log.dir=" + logDirectory + "/" + FunctionCommon.getFullyQualifiedName(config.getFunctionDetails()) + " -Dpulsar.function.log.file=" + config.getFunctionDetails().getName() + "-$SHARD_ID" + " -Dio.netty.tryReflectionSetAccessible=true" - + " --add-opens java.base/sun.net=ALL-UNNAMED" + + " -Dorg.apache.pulsar.shade.io.netty.tryReflectionSetAccessible=true" + + " -Dio.grpc.netty.shaded.io.netty.tryReflectionSetAccessible=true" + + " --add-opens java.base/java.nio=ALL-UNNAMED" + + " --add-opens java.base/jdk.internal.misc=ALL-UNNAMED" + + " --add-opens java.base/java.util.zip=ALL-UNNAMED" + " -Xmx" + RESOURCES.getRam() + " org.apache.pulsar.functions.instance.JavaInstanceMain" + " --jar " + jarLocation @@ -1314,7 +1318,7 @@ private void assertMetricsPortConfigured(Map functionRuntimeFact .contains("--metrics_port 0")); } } - + @Test public void testDeleteStatefulSetWithTranslatedKubernetesLabelChars() throws Exception { InstanceConfig config = createJavaInstanceConfig(FunctionDetails.Runtime.JAVA, false); @@ -1323,22 +1327,22 @@ public void testDeleteStatefulSetWithTranslatedKubernetesLabelChars() throws Exc CoreV1Api coreApi = mock(CoreV1Api.class); AppsV1Api appsApi = mock(AppsV1Api.class); - + Call successfulCall = mock(Call.class); Response okResponse = mock(Response.class); when(okResponse.code()).thenReturn(HttpURLConnection.HTTP_OK); when(okResponse.isSuccessful()).thenReturn(true); when(okResponse.message()).thenReturn(""); when(successfulCall.execute()).thenReturn(okResponse); - + final String expectedFunctionNamePrefix = String.format("pf-%s-%s-%s", "c-tenant", "c-ns", "c-fn"); - + factory = createKubernetesRuntimeFactory(null, 10, 1.0, 1.0); factory.setCoreClient(coreApi); factory.setAppsClient(appsApi); ArgumentMatcher hasTranslatedFunctionName = (String t) -> t.startsWith(expectedFunctionNamePrefix); - + when(appsApi.deleteNamespacedStatefulSetCall( argThat(hasTranslatedFunctionName), anyString(), isNull(), isNull(), anyInt(), isNull(), anyString(), any(), isNull())).thenReturn(successfulCall); @@ -1350,14 +1354,14 @@ public void testDeleteStatefulSetWithTranslatedKubernetesLabelChars() throws Exc V1PodList podList = mock(V1PodList.class); when(podList.getItems()).thenReturn(Collections.emptyList()); - + String expectedLabels = String.format("tenant=%s,namespace=%s,name=%s", "c-tenant", "c-ns", "c-fn"); - + when(coreApi.listNamespacedPod(anyString(), isNull(), isNull(), isNull(), isNull(), eq(expectedLabels), isNull(), isNull(), isNull(), isNull(), isNull())).thenReturn(podList); - KubernetesRuntime kr = factory.createContainer(config, "/test/code", "code.yml", "/test/transforms", "transform.yml", Long.MIN_VALUE); + KubernetesRuntime kr = factory.createContainer(config, "/test/code", "code.yml", "/test/transforms", "transform.yml", Long.MIN_VALUE); kr.deleteStatefulSet(); - + verify(coreApi).listNamespacedPod(anyString(), isNull(), isNull(), isNull(), isNull(), eq(expectedLabels), isNull(), isNull(), isNull(), isNull(), isNull()); } diff --git a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/process/ProcessRuntimeTest.java b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/process/ProcessRuntimeTest.java index f63f24dc25624..365704ea0b4ed 100644 --- a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/process/ProcessRuntimeTest.java +++ b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/process/ProcessRuntimeTest.java @@ -297,7 +297,7 @@ private void verifyJavaInstance(InstanceConfig config, Path depsDir, String webS String extraDepsEnv; int portArg; int metricsPortArg; - int totalArgCount = 48; + int totalArgCount = 54; if (webServiceUrl != null && config.isExposePulsarAdminClientEnabled()) { totalArgCount += 3; } @@ -305,13 +305,13 @@ private void verifyJavaInstance(InstanceConfig config, Path depsDir, String webS assertEquals(args.size(), totalArgCount); extraDepsEnv = " -Dpulsar.functions.extra.dependencies.dir=" + depsDir; classpath = classpath + ":" + depsDir + "/*"; - portArg = 31; - metricsPortArg = 33; + portArg = 37; + metricsPortArg = 39; } else { assertEquals(args.size(), totalArgCount-1); extraDepsEnv = ""; - portArg = 30; - metricsPortArg = 32; + portArg = 36; + metricsPortArg = 38; } if (webServiceUrl != null && config.isExposePulsarAdminClientEnabled()) { portArg += 3; @@ -328,7 +328,11 @@ private void verifyJavaInstance(InstanceConfig config, Path depsDir, String webS + "-Dpulsar.function.log.dir=" + logDirectory + "/functions/" + FunctionCommon.getFullyQualifiedName(config.getFunctionDetails()) + " -Dpulsar.function.log.file=" + config.getFunctionDetails().getName() + "-" + config.getInstanceId() + " -Dio.netty.tryReflectionSetAccessible=true" - + " --add-opens java.base/sun.net=ALL-UNNAMED" + + " -Dorg.apache.pulsar.shade.io.netty.tryReflectionSetAccessible=true" + + " -Dio.grpc.netty.shaded.io.netty.tryReflectionSetAccessible=true" + + " --add-opens java.base/java.nio=ALL-UNNAMED" + + " --add-opens java.base/jdk.internal.misc=ALL-UNNAMED" + + " --add-opens java.base/java.util.zip=ALL-UNNAMED" + " org.apache.pulsar.functions.instance.JavaInstanceMain" + " --jar " + userJarFile + " --transform_function_jar " + userJarFile From 9aed73653e1f706e3517072cce4a352d0838f8d7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 17 Jun 2024 23:39:08 +0800 Subject: [PATCH 313/580] [fix] [broker] response not-found error if topic does not exist when calling getPartitionedTopicMetadata (#22838) --- .../admin/impl/PersistentTopicsBase.java | 21 +- .../broker/admin/v2/NonPersistentTopics.java | 16 +- .../pulsar/broker/lookup/TopicLookupBase.java | 22 +- .../broker/namespace/NamespaceService.java | 101 ++- .../broker/namespace/TopicExistsInfo.java | 82 +++ .../pulsar/broker/service/BrokerService.java | 117 ++-- .../pulsar/broker/service/ServerCnx.java | 81 +-- .../GetPartitionMetadataMultiBrokerTest.java | 222 +++++++ .../admin/GetPartitionMetadataTest.java | 608 ++++++++++-------- .../pulsar/broker/admin/TopicsTest.java | 13 +- .../lookup/http/HttpTopicLookupv2Test.java | 19 +- .../namespace/NamespaceServiceTest.java | 7 +- .../pulsar/broker/service/TopicGCTest.java | 2 + .../client/impl/ConsumerBuilderImpl.java | 37 +- 14 files changed, 899 insertions(+), 449 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/TopicExistsInfo.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 2f2a899950a1e..beb8ecc8d799b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -561,13 +561,13 @@ protected CompletableFuture internalGetPartitionedMeta // is a non-partitioned topic so we shouldn't check if the topic exists. return pulsar().getBrokerService().isAllowAutoTopicCreationAsync(topicName) .thenCompose(brokerAllowAutoTopicCreation -> { - if (checkAllowAutoCreation) { + if (checkAllowAutoCreation && brokerAllowAutoTopicCreation) { // Whether it exists or not, auto create a non-partitioned topic by client. return CompletableFuture.completedFuture(metadata); } else { // If it does not exist, response a Not Found error. // Otherwise, response a non-partitioned metadata. - return internalCheckTopicExists(topicName).thenApply(__ -> metadata); + return internalCheckNonPartitionedTopicExists(topicName).thenApply(__ -> metadata); } }); } @@ -715,6 +715,17 @@ public void updatePropertiesFailed(ManagedLedgerException exception, Object ctx) protected CompletableFuture internalCheckTopicExists(TopicName topicName) { return pulsar().getNamespaceService().checkTopicExists(topicName) + .thenAccept(info -> { + boolean exists = info.isExists(); + info.recycle(); + if (!exists) { + throw new RestException(Status.NOT_FOUND, getTopicNotFoundErrorMessage(topicName.toString())); + } + }); + } + + protected CompletableFuture internalCheckNonPartitionedTopicExists(TopicName topicName) { + return pulsar().getNamespaceService().checkNonPartitionedTopicExists(topicName) .thenAccept(exist -> { if (!exist) { throw new RestException(Status.NOT_FOUND, getTopicNotFoundErrorMessage(topicName.toString())); @@ -5338,8 +5349,10 @@ protected CompletableFuture validateShadowTopics(List shadowTopics "Only persistent topic can be set as shadow topic")); } futures.add(pulsar().getNamespaceService().checkTopicExists(shadowTopicName) - .thenAccept(isExists -> { - if (!isExists) { + .thenAccept(info -> { + boolean exists = info.isExists(); + info.recycle(); + if (!exists) { throw new RestException(Status.PRECONDITION_FAILED, "Shadow topic [" + shadowTopic + "] not exists."); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index 5a7ea1b7632c8..9f58aa4ca9d44 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -98,8 +98,20 @@ public void getPartitionedMetadata( @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "Is check configuration required to automatically create topic") @QueryParam("checkAllowAutoCreation") @DefaultValue("false") boolean checkAllowAutoCreation) { - super.getPartitionedMetadata(asyncResponse, tenant, namespace, encodedTopic, authoritative, - checkAllowAutoCreation); + validateTopicName(tenant, namespace, encodedTopic); + validateTopicOwnershipAsync(topicName, authoritative).whenComplete((__, ex) -> { + if (ex != null) { + Throwable actEx = FutureUtil.unwrapCompletionException(ex); + if (isNot307And404Exception(actEx)) { + log.error("[{}] Failed to get internal stats for topic {}", clientAppId(), topicName, ex); + } + resumeAsyncResponseExceptionally(asyncResponse, actEx); + } else { + // "super.getPartitionedMetadata" will handle error itself. + super.getPartitionedMetadata(asyncResponse, tenant, namespace, encodedTopic, authoritative, + checkAllowAutoCreation); + } + }); } @GET diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java index 7b2c777414884..9a05c3d992aaf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java @@ -67,16 +67,22 @@ protected CompletableFuture internalLookupTopicAsync(final TopicName .thenCompose(__ -> validateGlobalNamespaceOwnershipAsync(topicName.getNamespaceObject())) .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.LOOKUP, null)) .thenCompose(__ -> { + // Case-1: Non-persistent topic. // Currently, it's hard to check the non-persistent-non-partitioned topic, because it only exists // in the broker, it doesn't have metadata. If the topic is non-persistent and non-partitioned, - // we'll return the true flag. - CompletableFuture existFuture = (!topicName.isPersistent() && !topicName.isPartitioned()) - ? CompletableFuture.completedFuture(true) - : pulsar().getNamespaceService().checkTopicExists(topicName) - .thenCompose(exists -> exists ? CompletableFuture.completedFuture(true) - : pulsar().getBrokerService().isAllowAutoTopicCreationAsync(topicName)); - - return existFuture; + // we'll return the true flag. So either it is a partitioned topic or not, the result will be true. + if (!topicName.isPersistent()) { + return CompletableFuture.completedFuture(true); + } + // Case-2: Persistent topic. + return pulsar().getNamespaceService().checkTopicExists(topicName).thenCompose(info -> { + boolean exists = info.isExists(); + info.recycle(); + if (exists) { + return CompletableFuture.completedFuture(true); + } + return pulsar().getBrokerService().isAllowAutoTopicCreationAsync(topicName); + }); }) .thenCompose(exist -> { if (!exist) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 80559b736c6ca..9df2b09204c15 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -51,6 +51,7 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; import javax.annotation.Nullable; +import lombok.extern.slf4j.Slf4j; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.ListUtils; import org.apache.commons.lang3.StringUtils; @@ -72,6 +73,7 @@ import org.apache.pulsar.broker.stats.prometheus.metrics.Summary; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -123,6 +125,7 @@ * * @see org.apache.pulsar.broker.PulsarService */ +@Slf4j public class NamespaceService implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(NamespaceService.class); @@ -1400,40 +1403,86 @@ public CompletableFuture> getOwnedTopicListForNamespaceBundle(Names }); } - public CompletableFuture checkTopicExists(TopicName topic) { - CompletableFuture future; - // If the topic is persistent and the name includes `-partition-`, find the topic from the managed/ledger. - if (topic.isPersistent() && topic.isPartitioned()) { - future = pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); + /*** + * Check topic exists( partitioned or non-partitioned ). + */ + public CompletableFuture checkTopicExists(TopicName topic) { + return pulsar.getBrokerService() + .fetchPartitionedTopicMetadataAsync(TopicName.get(topic.toString())) + .thenCompose(metadata -> { + if (metadata.partitions > 0) { + return CompletableFuture.completedFuture( + TopicExistsInfo.newPartitionedTopicExists(metadata.partitions)); + } + return checkNonPartitionedTopicExists(topic) + .thenApply(b -> b ? TopicExistsInfo.newNonPartitionedTopicExists() + : TopicExistsInfo.newTopicNotExists()); + }); + } + + /*** + * Check non-partitioned topic exists. + */ + public CompletableFuture checkNonPartitionedTopicExists(TopicName topic) { + if (topic.isPersistent()) { + return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); } else { - future = CompletableFuture.completedFuture(false); + return checkNonPersistentNonPartitionedTopicExists(topic.toString()); } + } - return future.thenCompose(found -> { - if (found != null && found) { - return CompletableFuture.completedFuture(true); + /** + * Regarding non-persistent topic, we do not know whether it exists or not. Redirect the request to the ownership + * broker of this topic. HTTP API has implemented the mechanism that redirect to ownership broker, so just call + * HTTP API here. + */ + public CompletableFuture checkNonPersistentNonPartitionedTopicExists(String topic) { + TopicName topicName = TopicName.get(topic); + // "non-partitioned & non-persistent" topics only exist on the owner broker. + return checkTopicOwnership(TopicName.get(topic)).thenCompose(isOwned -> { + // The current broker is the owner. + if (isOwned) { + CompletableFuture> nonPersistentTopicFuture = pulsar.getBrokerService() + .getTopic(topic, false); + if (nonPersistentTopicFuture != null) { + return nonPersistentTopicFuture.thenApply(Optional::isPresent); + } else { + return CompletableFuture.completedFuture(false); + } } - return pulsar.getBrokerService() - .fetchPartitionedTopicMetadataAsync(TopicName.get(topic.getPartitionedTopicName())) - .thenCompose(metadata -> { - if (metadata.partitions > 0) { - return CompletableFuture.completedFuture(true); - } - - if (topic.isPersistent()) { - return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); - } else { - // The non-partitioned non-persistent topic only exist in the broker topics. - CompletableFuture> nonPersistentTopicFuture = - pulsar.getBrokerService().getTopics().get(topic.toString()); - if (nonPersistentTopicFuture == null) { + // Forward to the owner broker. + PulsarClientImpl pulsarClient; + try { + pulsarClient = (PulsarClientImpl) pulsar.getClient(); + } catch (Exception ex) { + // This error will never occur. + log.error("{} Failed to get partition metadata due to create internal admin client fails", topic, ex); + return FutureUtil.failedFuture(ex); + } + LookupOptions lookupOptions = LookupOptions.builder().readOnly(false).authoritative(true).build(); + return getBrokerServiceUrlAsync(TopicName.get(topic), lookupOptions) + .thenCompose(lookupResult -> { + if (!lookupResult.isPresent()) { + log.error("{} Failed to get partition metadata due can not find the owner broker", topic); + return FutureUtil.failedFuture(new ServiceUnitNotReadyException( + "No broker was available to own " + topicName)); + } + return pulsarClient.getLookup(lookupResult.get().getLookupData().getBrokerUrl()) + .getPartitionedTopicMetadata(topicName, false) + .thenApply(metadata -> true) + .exceptionallyCompose(ex -> { + Throwable actEx = FutureUtil.unwrapCompletionException(ex); + if (actEx instanceof PulsarClientException.NotFoundException + || actEx instanceof PulsarClientException.TopicDoesNotExistException + || actEx instanceof PulsarAdminException.NotFoundException) { return CompletableFuture.completedFuture(false); } else { - return nonPersistentTopicFuture.thenApply(Optional::isPresent); + log.error("{} Failed to get partition metadata due to redirecting fails", topic, ex); + return CompletableFuture.failedFuture(ex); } - } - }); + }); + }); }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/TopicExistsInfo.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/TopicExistsInfo.java new file mode 100644 index 0000000000000..1c3f117719e8e --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/TopicExistsInfo.java @@ -0,0 +1,82 @@ +/* + * 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.namespace; + +import io.netty.util.Recycler; +import lombok.Getter; +import org.apache.pulsar.common.policies.data.TopicType; + +public class TopicExistsInfo { + + private static final Recycler RECYCLER = new Recycler<>() { + @Override + protected TopicExistsInfo newObject(Handle handle) { + return new TopicExistsInfo(handle); + } + }; + + private static TopicExistsInfo nonPartitionedExists = new TopicExistsInfo(true, 0); + + private static TopicExistsInfo notExists = new TopicExistsInfo(false, 0); + + public static TopicExistsInfo newPartitionedTopicExists(Integer partitions){ + TopicExistsInfo info = RECYCLER.get(); + info.exists = true; + info.partitions = partitions.intValue(); + return info; + } + + public static TopicExistsInfo newNonPartitionedTopicExists(){ + return nonPartitionedExists; + } + + public static TopicExistsInfo newTopicNotExists(){ + return notExists; + } + + private final Recycler.Handle handle; + + @Getter + private int partitions; + @Getter + private boolean exists; + + private TopicExistsInfo(Recycler.Handle handle) { + this.handle = handle; + } + + private TopicExistsInfo(boolean exists, int partitions) { + this.handle = null; + this.partitions = partitions; + this.exists = exists; + } + + public void recycle() { + if (this == notExists || this == nonPartitionedExists || this.handle == null) { + return; + } + this.exists = false; + this.partitions = 0; + this.handle.recycle(this); + } + + public TopicType getTopicType() { + return this.partitions > 0 ? TopicType.PARTITIONED : TopicType.NON_PARTITIONED; + } +} 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 82d7fad38740e..6ecd0a1ba6075 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 @@ -3178,65 +3178,66 @@ public CompletableFuture fetchPartitionedTopicMetadata if (pulsar.getNamespaceService() == null) { return FutureUtil.failedFuture(new NamingException("namespace service is not ready")); } - return pulsar.getPulsarResources().getNamespaceResources().getPoliciesAsync(topicName.getNamespaceObject()) - .thenCompose(policies -> pulsar.getNamespaceService().checkTopicExists(topicName) - .thenCompose(topicExists -> fetchPartitionedTopicMetadataAsync(topicName) - .thenCompose(metadata -> { - CompletableFuture future = new CompletableFuture<>(); - - // There are a couple of potentially blocking calls, which we cannot make from the - // MetadataStore callback thread. - pulsar.getExecutor().execute(() -> { - // If topic is already exist, creating partitioned topic is not allowed. - - if (metadata.partitions == 0 - && !topicExists - && !topicName.isPartitioned() - && pulsar.getBrokerService() - .isDefaultTopicTypePartitioned(topicName, policies)) { - isAllowAutoTopicCreationAsync(topicName, policies).thenAccept(allowed -> { - if (allowed) { - pulsar.getBrokerService() - .createDefaultPartitionedTopicAsync(topicName, policies) - .thenAccept(md -> future.complete(md)) - .exceptionally(ex -> { - if (ex.getCause() - instanceof MetadataStoreException - .AlreadyExistsException) { - log.info("[{}] The partitioned topic is already" - + " created, try to refresh the cache and read" - + " again.", topicName); - // The partitioned topic might be created concurrently - fetchPartitionedTopicMetadataAsync(topicName, true) - .whenComplete((metadata2, ex2) -> { - if (ex2 == null) { - future.complete(metadata2); - } else { - future.completeExceptionally(ex2); - } - }); - } else { - log.error("[{}] operation of creating partitioned" - + " topic metadata failed", - topicName, ex); - future.completeExceptionally(ex); - } - return null; - }); - } else { - future.complete(metadata); - } - }).exceptionally(ex -> { - future.completeExceptionally(ex); - return null; - }); - } else { - future.complete(metadata); - } - }); + return pulsar.getNamespaceService().checkTopicExists(topicName).thenComposeAsync(topicExistsInfo -> { + final boolean topicExists = topicExistsInfo.isExists(); + final TopicType topicType = topicExistsInfo.getTopicType(); + final Integer partitions = topicExistsInfo.getPartitions(); + topicExistsInfo.recycle(); + + // Topic exists. + if (topicExists) { + if (topicType.equals(TopicType.PARTITIONED)) { + return CompletableFuture.completedFuture(new PartitionedTopicMetadata(partitions)); + } + return CompletableFuture.completedFuture(new PartitionedTopicMetadata(0)); + } - return future; - }))); + // Try created if allowed to create a partitioned topic automatically. + return pulsar.getPulsarResources().getNamespaceResources().getPoliciesAsync(topicName.getNamespaceObject()) + .thenComposeAsync(policies -> { + return isAllowAutoTopicCreationAsync(topicName, policies).thenComposeAsync(allowed -> { + // Not Allow auto-creation. + if (!allowed) { + // Do not change the original behavior, or default return a non-partitioned topic. + return CompletableFuture.completedFuture(new PartitionedTopicMetadata(0)); + } + + // Allow auto create non-partitioned topic. + boolean autoCreatePartitionedTopic = pulsar.getBrokerService() + .isDefaultTopicTypePartitioned(topicName, policies); + if (!autoCreatePartitionedTopic || topicName.isPartitioned()) { + return CompletableFuture.completedFuture(new PartitionedTopicMetadata(0)); + } + + // Create partitioned metadata. + return pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName, policies) + .exceptionallyCompose(ex -> { + // The partitioned topic might be created concurrently. + if (ex.getCause() instanceof MetadataStoreException.AlreadyExistsException) { + log.info("[{}] The partitioned topic is already created, try to refresh the cache" + + " and read again.", topicName); + CompletableFuture recheckFuture = + fetchPartitionedTopicMetadataAsync(topicName, true); + recheckFuture.exceptionally(ex2 -> { + // Just for printing a log if error occurs. + log.error("[{}] Fetch partitioned topic metadata failed", topicName, ex); + return null; + }); + return recheckFuture; + } else { + log.error("[{}] operation of creating partitioned topic metadata failed", + topicName, ex); + return CompletableFuture.failedFuture(ex); + } + }); + }, pulsar.getExecutor()).exceptionallyCompose(ex -> { + log.error("[{}] operation of get partitioned metadata failed due to calling" + + " isAllowAutoTopicCreationAsync failed", + topicName, ex); + return CompletableFuture.failedFuture(ex); + }); + }, pulsar.getExecutor()); + }, pulsar.getExecutor()); } @SuppressWarnings("deprecation") 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 6901097bbbb27..b184f79494998 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 @@ -84,8 +84,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.namespace.LookupOptions; -import org.apache.pulsar.broker.resources.NamespaceResources; -import org.apache.pulsar.broker.resources.TopicResources; +import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; import org.apache.pulsar.broker.service.BrokerServiceException.ServerMetadataException; import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException; @@ -161,6 +160,7 @@ import org.apache.pulsar.common.policies.data.ClusterPolicies.ClusterUrl; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.TopicOperation; +import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.protocol.ByteBufPair; import org.apache.pulsar.common.protocol.CommandUtils; @@ -614,58 +614,33 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa if (isAuthorized) { // Get if exists, respond not found error if not exists. getBrokerService().isAllowAutoTopicCreationAsync(topicName).thenAccept(brokerAllowAutoCreate -> { - boolean autoCreateIfNotExist = partitionMetadata.isMetadataAutoCreationEnabled(); + boolean autoCreateIfNotExist = partitionMetadata.isMetadataAutoCreationEnabled() + && brokerAllowAutoCreate; if (!autoCreateIfNotExist) { - final NamespaceResources namespaceResources = getBrokerService().pulsar() - .getPulsarResources().getNamespaceResources(); - final TopicResources topicResources = getBrokerService().pulsar().getPulsarResources() - .getTopicResources(); - namespaceResources.getPartitionedTopicResources() - .getPartitionedTopicMetadataAsync(topicName, false) - .handle((metadata, getMetadataEx) -> { - if (getMetadataEx != null) { - log.error("{} {} Failed to get partition metadata", topicName, - ServerCnx.this.toString(), getMetadataEx); - writeAndFlush( - Commands.newPartitionMetadataResponse(ServerError.MetadataError, - "Failed to get partition metadata", - requestId)); - } else if (metadata.isPresent()) { - commandSender.sendPartitionMetadataResponse(metadata.get().partitions, - requestId); - } else if (topicName.isPersistent()) { - topicResources.persistentTopicExists(topicName).thenAccept(exists -> { - if (exists) { - commandSender.sendPartitionMetadataResponse(0, requestId); - return; - } - writeAndFlush(Commands.newPartitionMetadataResponse( - ServerError.TopicNotFound, "", requestId)); - }).exceptionally(ex -> { - log.error("{} {} Failed to get partition metadata", topicName, - ServerCnx.this.toString(), ex); - writeAndFlush( - Commands.newPartitionMetadataResponse(ServerError.MetadataError, - "Failed to check partition metadata", - requestId)); - return null; - }); - } else { - // Regarding non-persistent topic, we do not know whether it exists or not. - // Just return a non-partitioned metadata if partitioned metadata does not - // exist. - // Broker will respond a not found error when doing subscribing or producing if - // broker not allow to auto create topics. - commandSender.sendPartitionMetadataResponse(0, requestId); - } - return null; - }).whenComplete((ignore, ignoreEx) -> { - lookupSemaphore.release(); - if (ignoreEx != null) { - log.error("{} {} Failed to handle partition metadata request", topicName, - ServerCnx.this.toString(), ignoreEx); - } - }); + NamespaceService namespaceService = getBrokerService().getPulsar().getNamespaceService(); + namespaceService.checkTopicExists(topicName).thenAccept(topicExistsInfo -> { + lookupSemaphore.release(); + if (!topicExistsInfo.isExists()) { + writeAndFlush(Commands.newPartitionMetadataResponse( + ServerError.TopicNotFound, "", requestId)); + } else if (topicExistsInfo.getTopicType().equals(TopicType.PARTITIONED)) { + commandSender.sendPartitionMetadataResponse(topicExistsInfo.getPartitions(), + requestId); + } else { + commandSender.sendPartitionMetadataResponse(0, requestId); + } + // release resources. + topicExistsInfo.recycle(); + }).exceptionally(ex -> { + lookupSemaphore.release(); + log.error("{} {} Failed to get partition metadata", topicName, + ServerCnx.this.toString(), ex); + writeAndFlush( + Commands.newPartitionMetadataResponse(ServerError.MetadataError, + "Failed to get partition metadata", + requestId)); + return null; + }); } else { // Get if exists, create a new one if not exists. unsafeGetPartitionedTopicMetadataAsync(getBrokerService().pulsar(), topicName) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java new file mode 100644 index 0000000000000..28cf91ee165e2 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java @@ -0,0 +1,222 @@ +/* + * 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.admin; + +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import java.net.URL; +import java.util.List; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.TopicType; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Test(groups = "broker-admin") +@Slf4j +public class GetPartitionMetadataMultiBrokerTest extends GetPartitionMetadataTest { + + private PulsarService pulsar2; + private URL url2; + private PulsarAdmin admin2; + private PulsarClientImpl clientWithHttpLookup2; + private PulsarClientImpl clientWitBinaryLookup2; + + @BeforeClass(alwaysRun = true) + protected void setup() throws Exception { + super.setup(); + } + + @Override + @AfterClass(alwaysRun = true) + protected void cleanup() throws Exception { + super.cleanup(); + } + + @Override + protected void cleanupBrokers() throws Exception { + // Cleanup broker2. + if (clientWithHttpLookup2 != null) { + clientWithHttpLookup2.close(); + clientWithHttpLookup2 = null; + } + if (clientWitBinaryLookup2 != null) { + clientWitBinaryLookup2.close(); + clientWitBinaryLookup2 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } + if (pulsar2 != null) { + pulsar2.close(); + pulsar2 = null; + } + + // Super cleanup. + super.cleanupBrokers(); + } + + @Override + protected void setupBrokers() throws Exception { + super.setupBrokers(); + doInitConf(); + pulsar2 = new PulsarService(conf); + pulsar2.start(); + url2 = new URL(pulsar2.getWebServiceAddress()); + admin2 = PulsarAdmin.builder().serviceHttpUrl(url2.toString()).build(); + clientWithHttpLookup2 = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar2.getWebServiceAddress()).build(); + clientWitBinaryLookup2 = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar2.getBrokerServiceUrl()).build(); + } + + @Override + protected PulsarClientImpl[] getClientsToTest() { + return new PulsarClientImpl[] {clientWithHttpLookup1, clientWitBinaryLookup1, + clientWithHttpLookup2, clientWitBinaryLookup2}; + } + + protected PulsarClientImpl[] getClientsToTest(boolean isUsingHttpLookup) { + if (isUsingHttpLookup) { + return new PulsarClientImpl[]{clientWithHttpLookup1, clientWithHttpLookup2}; + } else { + return new PulsarClientImpl[]{clientWitBinaryLookup1, clientWitBinaryLookup2}; + } + } + + @Override + protected int getLookupRequestPermits() { + return pulsar1.getBrokerService().getLookupRequestSemaphore().availablePermits() + + pulsar2.getBrokerService().getLookupRequestSemaphore().availablePermits(); + } + + protected void verifyPartitionsNeverCreated(String topicNameStr) throws Exception { + TopicName topicName = TopicName.get(topicNameStr); + try { + List topicList = admin1.topics().getList("public/default"); + for (int i = 0; i < 3; i++) { + assertFalse(topicList.contains(topicName.getPartition(i))); + } + } catch (Exception ex) { + // If the namespace bundle has not been loaded yet, it means no non-persistent topic was created. So + // this behavior is also correct. + // This error is not expected, a seperated PR is needed to fix this issue. + assertTrue(ex.getMessage().contains("Failed to find ownership for")); + } + } + + protected void verifyNonPartitionedTopicNeverCreated(String topicNameStr) throws Exception { + TopicName topicName = TopicName.get(topicNameStr); + try { + List topicList = admin1.topics().getList("public/default"); + assertFalse(topicList.contains(topicName.getPartitionedTopicName())); + } catch (Exception ex) { + // If the namespace bundle has not been loaded yet, it means no non-persistent topic was created. So + // this behavior is also correct. + // This error is not expected, a seperated PR is needed to fix this issue. + assertTrue(ex.getMessage().contains("Failed to find ownership for")); + } + } + + protected void modifyTopicAutoCreation(boolean allowAutoTopicCreation, + TopicType allowAutoTopicCreationType, + int defaultNumPartitions) throws Exception { + doModifyTopicAutoCreation(admin1, pulsar1, allowAutoTopicCreation, allowAutoTopicCreationType, + defaultNumPartitions); + doModifyTopicAutoCreation(admin2, pulsar2, allowAutoTopicCreation, allowAutoTopicCreationType, + defaultNumPartitions); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "topicDomains") + public void testAutoCreatingMetadataWhenCallingOldAPI(TopicDomain topicDomain) throws Exception { + super.testAutoCreatingMetadataWhenCallingOldAPI(topicDomain); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "autoCreationParamsAll", enabled = false) + public void testGetMetadataIfNonPartitionedTopicExists(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup, + TopicDomain topicDomain) throws Exception { + super.testGetMetadataIfNonPartitionedTopicExists(configAllowAutoTopicCreation, paramMetadataAutoCreationEnabled, + isUsingHttpLookup, topicDomain); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "autoCreationParamsAll") + public void testGetMetadataIfPartitionedTopicExists(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup, + TopicDomain topicDomain) throws Exception { + super.testGetMetadataIfNonPartitionedTopicExists(configAllowAutoTopicCreation, paramMetadataAutoCreationEnabled, + isUsingHttpLookup, topicDomain); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "clients") + public void testAutoCreatePartitionedTopic(boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { + super.testAutoCreatePartitionedTopic(isUsingHttpLookup, topicDomain); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "clients") + public void testAutoCreateNonPartitionedTopic(boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { + super.testAutoCreateNonPartitionedTopic(isUsingHttpLookup, topicDomain); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "autoCreationParamsNotAllow") + public void testGetMetadataIfNotAllowedCreate(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup) throws Exception { + super.testGetMetadataIfNotAllowedCreate(configAllowAutoTopicCreation, paramMetadataAutoCreationEnabled, + isUsingHttpLookup); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "autoCreationParamsNotAllow") + public void testGetMetadataIfNotAllowedCreateOfNonPersistentTopic(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup) throws Exception { + super.testGetMetadataIfNotAllowedCreateOfNonPersistentTopic(configAllowAutoTopicCreation, + paramMetadataAutoCreationEnabled, isUsingHttpLookup); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java index 51f643d2b7823..bf99b172829a7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java @@ -22,70 +22,150 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.google.common.collect.Sets; +import java.net.URL; +import java.util.Collections; import java.util.List; import java.util.Optional; -import java.util.concurrent.Semaphore; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.impl.LookupService; import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.awaitility.Awaitility; -import org.testng.Assert; -import org.testng.annotations.AfterMethod; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Test(groups = "broker-admin") @Slf4j -public class GetPartitionMetadataTest extends ProducerConsumerBase { +public class GetPartitionMetadataTest { - private static final String DEFAULT_NS = "public/default"; + protected static final String DEFAULT_NS = "public/default"; - private PulsarClientImpl clientWithHttpLookup; - private PulsarClientImpl clientWitBinaryLookup; + protected String clusterName = "c1"; - @Override + protected LocalBookkeeperEnsemble bkEnsemble; + + protected ServiceConfiguration conf = new ServiceConfiguration(); + + protected PulsarService pulsar1; + protected URL url1; + protected PulsarAdmin admin1; + protected PulsarClientImpl clientWithHttpLookup1; + protected PulsarClientImpl clientWitBinaryLookup1; + + @BeforeClass(alwaysRun = true) protected void setup() throws Exception { - super.internalSetup(); - super.producerBaseSetup(); - clientWithHttpLookup = - (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar.getWebServiceAddress()).build(); - clientWitBinaryLookup = - (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); + bkEnsemble = new LocalBookkeeperEnsemble(3, 0, () -> 0); + bkEnsemble.start(); + // Start broker. + setupBrokers(); + // Create default NS. + admin1.clusters().createCluster(clusterName, new ClusterDataImpl()); + admin1.tenants().createTenant(NamespaceName.get(DEFAULT_NS).getTenant(), + new TenantInfoImpl(Collections.emptySet(), Sets.newHashSet(clusterName))); + admin1.namespaces().createNamespace(DEFAULT_NS); } - @Override - @AfterMethod(alwaysRun = true) + @AfterClass(alwaysRun = true) protected void cleanup() throws Exception { - super.internalCleanup(); - if (clientWithHttpLookup != null) { - clientWithHttpLookup.close(); + cleanupBrokers(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } + } + + protected void cleanupBrokers() throws Exception { + // Cleanup broker2. + if (clientWithHttpLookup1 != null) { + clientWithHttpLookup1.close(); + clientWithHttpLookup1 = null; + } + if (clientWitBinaryLookup1 != null) { + clientWitBinaryLookup1.close(); + clientWitBinaryLookup1 = null; } - if (clientWitBinaryLookup != null) { - clientWitBinaryLookup.close(); + if (admin1 != null) { + admin1.close(); + admin1 = null; } + if (pulsar1 != null) { + pulsar1.close(); + pulsar1 = null; + } + // Reset configs. + conf = new ServiceConfiguration(); + } + + protected void setupBrokers() throws Exception { + doInitConf(); + // Start broker. + pulsar1 = new PulsarService(conf); + pulsar1.start(); + url1 = new URL(pulsar1.getWebServiceAddress()); + admin1 = PulsarAdmin.builder().serviceHttpUrl(url1.toString()).build(); + clientWithHttpLookup1 = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar1.getWebServiceAddress()).build(); + clientWitBinaryLookup1 = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar1.getBrokerServiceUrl()).build(); } - @Override - protected void doInitConf() throws Exception { - super.doInitConf(); + protected void doInitConf() { + conf.setClusterName(clusterName); + conf.setAdvertisedAddress("localhost"); + conf.setBrokerServicePort(Optional.of(0)); + conf.setWebServicePort(Optional.of(0)); + conf.setMetadataStoreUrl("zk:127.0.0.1:" + bkEnsemble.getZookeeperPort()); + conf.setConfigurationMetadataStoreUrl("zk:127.0.0.1:" + bkEnsemble.getZookeeperPort() + "/foo"); + conf.setBrokerDeleteInactiveTopicsEnabled(false); + conf.setBrokerShutdownTimeoutMs(0L); + conf.setLoadBalancerSheddingEnabled(false); } - private LookupService getLookupService(boolean isUsingHttpLookup) { + protected PulsarClientImpl[] getClientsToTest() { + return new PulsarClientImpl[] {clientWithHttpLookup1, clientWitBinaryLookup1}; + } + + protected PulsarClientImpl[] getClientsToTest(boolean isUsingHttpLookup) { if (isUsingHttpLookup) { - return clientWithHttpLookup.getLookup(); + return new PulsarClientImpl[] {clientWithHttpLookup1}; } else { - return clientWitBinaryLookup.getLookup(); + return new PulsarClientImpl[] {clientWitBinaryLookup1}; } + + } + + protected int getLookupRequestPermits() { + return pulsar1.getBrokerService().getLookupRequestSemaphore().availablePermits(); + } + + protected void verifyPartitionsNeverCreated(String topicNameStr) throws Exception { + TopicName topicName = TopicName.get(topicNameStr); + List topicList = admin1.topics().getList("public/default"); + for (int i = 0; i < 3; i++) { + assertFalse(topicList.contains(topicName.getPartition(i))); + } + } + + protected void verifyNonPartitionedTopicNeverCreated(String topicNameStr) throws Exception { + TopicName topicName = TopicName.get(topicNameStr); + List topicList = admin1.topics().getList("public/default"); + assertFalse(topicList.contains(topicName.getPartitionedTopicName())); } @DataProvider(name = "topicDomains") @@ -96,43 +176,53 @@ public Object[][] topicDomains() { }; } - @Test(dataProvider = "topicDomains") - public void testAutoCreatingMetadataWhenCallingOldAPI(TopicDomain topicDomain) throws Exception { - conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); - conf.setDefaultNumPartitions(3); - conf.setAllowAutoTopicCreation(true); - setup(); - - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); - - // HTTP client. - final String tp1 = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); - clientWithHttpLookup.getPartitionsForTopic(tp1).join(); - Optional metadata1 = pulsar.getPulsarResources().getNamespaceResources() - .getPartitionedTopicResources() - .getPartitionedTopicMetadataAsync(TopicName.get(tp1), true).join(); - assertTrue(metadata1.isPresent()); - assertEquals(metadata1.get().partitions, 3); - - // Binary client. - final String tp2 = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); - clientWitBinaryLookup.getPartitionsForTopic(tp2).join(); - Optional metadata2 = pulsar.getPulsarResources().getNamespaceResources() - .getPartitionedTopicResources() - .getPartitionedTopicMetadataAsync(TopicName.get(tp2), true).join(); - assertTrue(metadata2.isPresent()); - assertEquals(metadata2.get().partitions, 3); - - // Verify: lookup semaphore has been releases. + protected static void doModifyTopicAutoCreation(PulsarAdmin admin1, PulsarService pulsar1, + boolean allowAutoTopicCreation, TopicType allowAutoTopicCreationType, + int defaultNumPartitions) throws Exception { + admin1.brokers().updateDynamicConfiguration( + "allowAutoTopicCreation", allowAutoTopicCreation + ""); + admin1.brokers().updateDynamicConfiguration( + "allowAutoTopicCreationType", allowAutoTopicCreationType + ""); + admin1.brokers().updateDynamicConfiguration( + "defaultNumPartitions", defaultNumPartitions + ""); Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); + assertEquals(pulsar1.getConfiguration().isAllowAutoTopicCreation(), allowAutoTopicCreation); + assertEquals(pulsar1.getConfiguration().getAllowAutoTopicCreationType(), allowAutoTopicCreationType); + assertEquals(pulsar1.getConfiguration().getDefaultNumPartitions(), defaultNumPartitions); }); + } - // Cleanup. - admin.topics().deletePartitionedTopic(tp1, false); - admin.topics().deletePartitionedTopic(tp2, false); + protected void modifyTopicAutoCreation(boolean allowAutoTopicCreation, + TopicType allowAutoTopicCreationType, + int defaultNumPartitions) throws Exception { + doModifyTopicAutoCreation(admin1, pulsar1, allowAutoTopicCreation, allowAutoTopicCreationType, + defaultNumPartitions); + } + + @Test(dataProvider = "topicDomains") + public void testAutoCreatingMetadataWhenCallingOldAPI(TopicDomain topicDomain) throws Exception { + modifyTopicAutoCreation(true, TopicType.PARTITIONED, 3); + + int lookupPermitsBefore = getLookupRequestPermits(); + + for (PulsarClientImpl client : getClientsToTest()) { + // Verify: the behavior of topic creation. + final String tp = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + client.getPartitionsForTopic(tp).join(); + Optional metadata1 = pulsar1.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources() + .getPartitionedTopicMetadataAsync(TopicName.get(tp), true).join(); + assertTrue(metadata1.isPresent()); + assertEquals(metadata1.get().partitions, 3); + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + + // Cleanup. + admin1.topics().deletePartitionedTopic(tp, false); + } } @DataProvider(name = "autoCreationParamsAll") @@ -163,40 +253,32 @@ public void testGetMetadataIfNonPartitionedTopicExists(boolean configAllowAutoTo boolean paramMetadataAutoCreationEnabled, boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { - conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); - conf.setDefaultNumPartitions(3); - conf.setAllowAutoTopicCreation(configAllowAutoTopicCreation); - setup(); + modifyTopicAutoCreation(configAllowAutoTopicCreation, TopicType.PARTITIONED, 3); - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); + int lookupPermitsBefore = getLookupRequestPermits(); - LookupService lookup = getLookupService(isUsingHttpLookup); // Create topic. - final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); - final TopicName topicName = TopicName.get(topicNameStr); - admin.topics().createNonPartitionedTopic(topicNameStr); - // Verify. - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); - PartitionedTopicMetadata response = - lookup.getPartitionedTopicMetadata(topicName, paramMetadataAutoCreationEnabled).join(); - assertEquals(response.partitions, 0); - List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); - assertFalse(partitionedTopics.contains(topicNameStr)); - List topicList = admin.topics().getList("public/default"); - for (int i = 0; i < 3; i++) { - assertFalse(topicList.contains(topicName.getPartition(i))); - } + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp_"); + admin1.topics().createNonPartitionedTopic(topicNameStr); + + PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); + for (PulsarClientImpl client : clientArray) { + // Verify: the result of get partitioned topic metadata. + PartitionedTopicMetadata response = + client.getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled).join(); + assertEquals(response.partitions, 0); + List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); + assertFalse(partitionedTopics.contains(topicNameStr)); + verifyPartitionsNeverCreated(topicNameStr); - // Verify: lookup semaphore has been releases. - Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); - }); + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + } // Cleanup. - client.close(); - admin.topics().delete(topicNameStr, false); + admin1.topics().delete(topicNameStr, false); } @Test(dataProvider = "autoCreationParamsAll") @@ -204,36 +286,30 @@ public void testGetMetadataIfPartitionedTopicExists(boolean configAllowAutoTopic boolean paramMetadataAutoCreationEnabled, boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { - conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); - conf.setDefaultNumPartitions(3); - conf.setAllowAutoTopicCreation(configAllowAutoTopicCreation); - setup(); + modifyTopicAutoCreation(configAllowAutoTopicCreation, TopicType.PARTITIONED, 3); - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); + int lookupPermitsBefore = getLookupRequestPermits(); - LookupService lookup = getLookupService(isUsingHttpLookup); // Create topic. final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); - final TopicName topicName = TopicName.get(topicNameStr); - admin.topics().createPartitionedTopic(topicNameStr, 3); - // Verify. - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); - PartitionedTopicMetadata response = - lookup.getPartitionedTopicMetadata(topicName, paramMetadataAutoCreationEnabled).join(); - assertEquals(response.partitions, 3); - List topicList = admin.topics().getList("public/default"); - assertFalse(topicList.contains(topicNameStr)); - - // Verify: lookup semaphore has been releases. - Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); - }); + admin1.topics().createPartitionedTopic(topicNameStr, 3); + + PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); + for (PulsarClientImpl client : clientArray) { + // Verify: the result of get partitioned topic metadata. + PartitionedTopicMetadata response = + client.getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled).join(); + assertEquals(response.partitions, 3); + verifyNonPartitionedTopicNeverCreated(topicNameStr); + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + } // Cleanup. - client.close(); - admin.topics().deletePartitionedTopic(topicNameStr, false); + admin1.topics().deletePartitionedTopic(topicNameStr, false); } @DataProvider(name = "clients") @@ -247,76 +323,96 @@ public Object[][] clients(){ @Test(dataProvider = "clients") public void testAutoCreatePartitionedTopic(boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { - conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); - conf.setDefaultNumPartitions(3); - conf.setAllowAutoTopicCreation(true); - setup(); - - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); - - LookupService lookup = getLookupService(isUsingHttpLookup); - // Create topic. - final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); - final TopicName topicName = TopicName.get(topicNameStr); - // Verify. - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); - PartitionedTopicMetadata response = lookup.getPartitionedTopicMetadata(topicName, true).join(); - assertEquals(response.partitions, 3); - List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); - assertTrue(partitionedTopics.contains(topicNameStr)); - List topicList = admin.topics().getList("public/default"); - assertFalse(topicList.contains(topicNameStr)); - for (int i = 0; i < 3; i++) { + modifyTopicAutoCreation(true, TopicType.PARTITIONED, 3); + + int lookupPermitsBefore = getLookupRequestPermits(); + + PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); + for (PulsarClientImpl client : clientArray) { + // Case-1: normal topic. + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + // Verify: the result of get partitioned topic metadata. + PartitionedTopicMetadata response = client.getPartitionedTopicMetadata(topicNameStr, true).join(); + assertEquals(response.partitions, 3); + // Verify: the behavior of topic creation. + List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); + assertTrue(partitionedTopics.contains(topicNameStr)); + verifyNonPartitionedTopicNeverCreated(topicNameStr); // The API "getPartitionedTopicMetadata" only creates the partitioned metadata, it will not create the // partitions. - assertFalse(topicList.contains(topicName.getPartition(i))); + verifyPartitionsNeverCreated(topicNameStr); + + // Case-2: topic with suffix "-partition-1". + final String topicNameStrWithSuffix = BrokerTestUtil.newUniqueName( + topicDomain.value() + "://" + DEFAULT_NS + "/tp") + "-partition-1"; + // Verify: the result of get partitioned topic metadata. + PartitionedTopicMetadata response2 = + client.getPartitionedTopicMetadata(topicNameStrWithSuffix, true).join(); + assertEquals(response2.partitions, 0); + // Verify: the behavior of topic creation. + List partitionedTopics2 = + admin1.topics().getPartitionedTopicList("public/default"); + assertFalse(partitionedTopics2.contains(topicNameStrWithSuffix)); + assertFalse(partitionedTopics2.contains( + TopicName.get(topicNameStrWithSuffix).getPartitionedTopicName())); + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + // Cleanup. + admin1.topics().deletePartitionedTopic(topicNameStr, false); + try { + admin1.topics().delete(topicNameStrWithSuffix, false); + } catch (Exception ex) {} } - // Verify: lookup semaphore has been releases. - Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); - }); - - // Cleanup. - client.close(); - admin.topics().deletePartitionedTopic(topicNameStr, false); } @Test(dataProvider = "clients") public void testAutoCreateNonPartitionedTopic(boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { - conf.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); - conf.setAllowAutoTopicCreation(true); - setup(); - - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); - - LookupService lookup = getLookupService(isUsingHttpLookup); - // Create topic. - final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); - final TopicName topicName = TopicName.get(topicNameStr); - // Verify. - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); - PartitionedTopicMetadata response = lookup.getPartitionedTopicMetadata(topicName, true).join(); - assertEquals(response.partitions, 0); - List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); - assertFalse(partitionedTopics.contains(topicNameStr)); - List topicList = admin.topics().getList("public/default"); - assertFalse(topicList.contains(topicNameStr)); - - // Verify: lookup semaphore has been releases. - Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); - }); - - // Cleanup. - client.close(); - try { - admin.topics().delete(topicNameStr, false); - } catch (Exception ex) {} + modifyTopicAutoCreation(true, TopicType.NON_PARTITIONED, 3); + + int lookupPermitsBefore = getLookupRequestPermits(); + + PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); + for (PulsarClientImpl client : clientArray) { + // Case 1: normal topic. + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + // Verify: the result of get partitioned topic metadata. + PartitionedTopicMetadata response = client.getPartitionedTopicMetadata(topicNameStr, true).join(); + assertEquals(response.partitions, 0); + // Verify: the behavior of topic creation. + List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); + assertFalse(partitionedTopics.contains(topicNameStr)); + verifyPartitionsNeverCreated(topicNameStr); + + // Case-2: topic with suffix "-partition-1". + final String topicNameStrWithSuffix = BrokerTestUtil.newUniqueName( + topicDomain.value() + "://" + DEFAULT_NS + "/tp") + "-partition-1"; + // Verify: the result of get partitioned topic metadata. + PartitionedTopicMetadata response2 = + client.getPartitionedTopicMetadata(topicNameStrWithSuffix, true).join(); + assertEquals(response2.partitions, 0); + // Verify: the behavior of topic creation. + List partitionedTopics2 = + admin1.topics().getPartitionedTopicList("public/default"); + assertFalse(partitionedTopics2.contains(topicNameStrWithSuffix)); + assertFalse(partitionedTopics2.contains( + TopicName.get(topicNameStrWithSuffix).getPartitionedTopicName())); + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + // Cleanup. + try { + admin1.topics().delete(topicNameStr, false); + } catch (Exception ex) {} + try { + admin1.topics().delete(topicNameStrWithSuffix, false); + } catch (Exception ex) {} + } } @DataProvider(name = "autoCreationParamsNotAllow") @@ -336,64 +432,38 @@ public Object[][] autoCreationParamsNotAllow(){ public void testGetMetadataIfNotAllowedCreate(boolean configAllowAutoTopicCreation, boolean paramMetadataAutoCreationEnabled, boolean isUsingHttpLookup) throws Exception { - if (!configAllowAutoTopicCreation && paramMetadataAutoCreationEnabled) { - // These test cases are for the following PR. - // Which was described in the Motivation of https://github.com/apache/pulsar/pull/22206. - return; - } - conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); - conf.setDefaultNumPartitions(3); - conf.setAllowAutoTopicCreation(configAllowAutoTopicCreation); - setup(); - - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); - - LookupService lookup = getLookupService(isUsingHttpLookup); - // Define topic. - final String topicNameStr = BrokerTestUtil.newUniqueName("persistent://" + DEFAULT_NS + "/tp"); - final TopicName topicName = TopicName.get(topicNameStr); - // Verify. - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); - try { - lookup.getPartitionedTopicMetadata(TopicName.get(topicNameStr), paramMetadataAutoCreationEnabled).join(); - fail("Expect a not found exception"); - } catch (Exception e) { - log.warn("", e); - Throwable unwrapEx = FutureUtil.unwrapCompletionException(e); - assertTrue(unwrapEx instanceof PulsarClientException.TopicDoesNotExistException - || unwrapEx instanceof PulsarClientException.NotFoundException); - } + modifyTopicAutoCreation(configAllowAutoTopicCreation, TopicType.PARTITIONED, 3); - List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); - pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources().partitionedTopicExists(topicName); - assertFalse(partitionedTopics.contains(topicNameStr)); - List topicList = admin.topics().getList("public/default"); - assertFalse(topicList.contains(topicNameStr)); - for (int i = 0; i < 3; i++) { - assertFalse(topicList.contains(topicName.getPartition(i))); - } + int lookupPermitsBefore = getLookupRequestPermits(); - // Verify: lookup semaphore has been releases. - Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); - }); - - // Cleanup. - client.close(); - } + PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); + for (PulsarClientImpl client : clientArray) { + // Define topic. + final String topicNameStr = BrokerTestUtil.newUniqueName("persistent://" + DEFAULT_NS + "/tp"); + final TopicName topicName = TopicName.get(topicNameStr); + // Verify: the result of get partitioned topic metadata. + try { + client.getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled) + .join(); + fail("Expect a not found exception"); + } catch (Exception e) { + Throwable unwrapEx = FutureUtil.unwrapCompletionException(e); + assertTrue(unwrapEx instanceof PulsarClientException.TopicDoesNotExistException + || unwrapEx instanceof PulsarClientException.NotFoundException); + } + // Verify: the behavior of topic creation. + List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); + pulsar1.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(topicName); + assertFalse(partitionedTopics.contains(topicNameStr)); + verifyNonPartitionedTopicNeverCreated(topicNameStr); + verifyPartitionsNeverCreated(topicNameStr); - @DataProvider(name = "autoCreationParamsForNonPersistentTopic") - public Object[][] autoCreationParamsForNonPersistentTopic(){ - return new Object[][]{ - // configAllowAutoTopicCreation, paramCreateIfAutoCreationEnabled, isUsingHttpLookup. - {true, true, true}, - {true, true, false}, - {false, true, true}, - {false, true, false}, - {false, false, true} - }; + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + } } /** @@ -408,66 +478,46 @@ public Object[][] autoCreationParamsForNonPersistentTopic(){ * param-auto-create = false * HTTP API: not found error * binary API: not support - * This test only guarantees that the behavior is the same as before. The following separated PR will fix the - * incorrect behavior. + * After PIP-344, the behavior will be the same as persistent topics, which was described in PIP-344. */ - @Test(dataProvider = "autoCreationParamsForNonPersistentTopic") - public void testGetNonPersistentMetadataIfNotAllowedCreate(boolean configAllowAutoTopicCreation, + @Test(dataProvider = "autoCreationParamsNotAllow") + public void testGetMetadataIfNotAllowedCreateOfNonPersistentTopic(boolean configAllowAutoTopicCreation, boolean paramMetadataAutoCreationEnabled, boolean isUsingHttpLookup) throws Exception { - conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); - conf.setDefaultNumPartitions(3); - conf.setAllowAutoTopicCreation(configAllowAutoTopicCreation); - setup(); - - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); - - LookupService lookup = getLookupService(isUsingHttpLookup); - // Define topic. - final String topicNameStr = BrokerTestUtil.newUniqueName("non-persistent://" + DEFAULT_NS + "/tp"); - final TopicName topicName = TopicName.get(topicNameStr); - // Verify. - // Regarding non-persistent topic, we do not know whether it exists or not. - // Broker will return a non-partitioned metadata if partitioned metadata does not exist. - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); - - if (!configAllowAutoTopicCreation && !paramMetadataAutoCreationEnabled && isUsingHttpLookup) { + modifyTopicAutoCreation(configAllowAutoTopicCreation, TopicType.PARTITIONED, 3); + + int lookupPermitsBefore = getLookupRequestPermits(); + + PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); + for (PulsarClientImpl client : clientArray) { + // Define topic. + final String topicNameStr = BrokerTestUtil.newUniqueName("non-persistent://" + DEFAULT_NS + "/tp"); + final TopicName topicName = TopicName.get(topicNameStr); + // Verify: the result of get partitioned topic metadata. try { - lookup.getPartitionedTopicMetadata(TopicName.get(topicNameStr), paramMetadataAutoCreationEnabled) + PartitionedTopicMetadata topicMetadata = client + .getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled) .join(); - Assert.fail("Expected a not found ex"); + log.info("Get topic metadata: {}", topicMetadata.partitions); + fail("Expected a not found ex"); } catch (Exception ex) { - // Cleanup. - client.close(); - return; + Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex); + assertTrue(unwrapEx instanceof PulsarClientException.TopicDoesNotExistException + || unwrapEx instanceof PulsarClientException.NotFoundException); } - } - PartitionedTopicMetadata metadata = lookup - .getPartitionedTopicMetadata(TopicName.get(topicNameStr), paramMetadataAutoCreationEnabled).join(); - if (configAllowAutoTopicCreation && paramMetadataAutoCreationEnabled) { - assertEquals(metadata.partitions, 3); - } else { - assertEquals(metadata.partitions, 0); - } - - List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); - pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() - .partitionedTopicExists(topicName); - if (configAllowAutoTopicCreation && paramMetadataAutoCreationEnabled) { - assertTrue(partitionedTopics.contains(topicNameStr)); - } else { + // Verify: the behavior of topic creation. + List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); + pulsar1.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(topicName); assertFalse(partitionedTopics.contains(topicNameStr)); + verifyNonPartitionedTopicNeverCreated(topicNameStr); + verifyPartitionsNeverCreated(topicNameStr); } // Verify: lookup semaphore has been releases. Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); }); - - // Cleanup. - client.close(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsTest.java index 9aa29f08c5ce8..c9457e1a8883f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsTest.java @@ -56,6 +56,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.namespace.TopicExistsInfo; import org.apache.pulsar.broker.rest.Topics; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -357,9 +358,12 @@ public void testLookUpWithException() throws Exception { CompletableFuture future = new CompletableFuture(); future.completeExceptionally(new BrokerServiceException("Fake Exception")); CompletableFuture existFuture = new CompletableFuture(); - existFuture.complete(true); + existFuture.complete(TopicExistsInfo.newNonPartitionedTopicExists()); doReturn(future).when(nameSpaceService).getBrokerServiceUrlAsync(any(), any()); doReturn(existFuture).when(nameSpaceService).checkTopicExists(any()); + CompletableFuture existBooleanFuture = new CompletableFuture(); + existBooleanFuture.complete(false); + doReturn(existBooleanFuture).when(nameSpaceService).checkNonPartitionedTopicExists(any()); doReturn(nameSpaceService).when(pulsar).getNamespaceService(); AsyncResponse asyncResponse = mock(AsyncResponse.class); ProducerMessages producerMessages = new ProducerMessages(); @@ -370,7 +374,7 @@ public void testLookUpWithException() throws Exception { topics.produceOnPersistentTopic(asyncResponse, testTenant, testNamespace, testTopicName, false, producerMessages); ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(RestException.class); verify(asyncResponse, timeout(5000).times(1)).resume(responseCaptor.capture()); - Assert.assertEquals(responseCaptor.getValue().getMessage(), "Can't find owner of given topic."); + Assert.assertTrue(responseCaptor.getValue().getMessage().contains(topicName + " not found")); } @Test @@ -378,8 +382,11 @@ public void testLookUpTopicNotExist() throws Exception { String topicName = "persistent://" + testTenant + "/" + testNamespace + "/" + testTopicName; NamespaceService nameSpaceService = mock(NamespaceService.class); CompletableFuture existFuture = new CompletableFuture(); - existFuture.complete(false); + existFuture.complete(TopicExistsInfo.newTopicNotExists()); + CompletableFuture existBooleanFuture = new CompletableFuture(); + existBooleanFuture.complete(false); doReturn(existFuture).when(nameSpaceService).checkTopicExists(any()); + doReturn(existBooleanFuture).when(nameSpaceService).checkNonPartitionedTopicExists(any()); doReturn(nameSpaceService).when(pulsar).getNamespaceService(); AsyncResponse asyncResponse = mock(AsyncResponse.class); ProducerMessages producerMessages = new ProducerMessages(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java index 7004eae29b5ac..ab492de055ba5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java @@ -44,6 +44,7 @@ import org.apache.pulsar.broker.lookup.RedirectData; import org.apache.pulsar.broker.lookup.v1.TopicLookup; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.namespace.TopicExistsInfo; import org.apache.pulsar.broker.resources.ClusterResources; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.resources.PulsarResources; @@ -149,9 +150,12 @@ public void testLookupTopicNotExist() throws Exception { config.setAuthorizationEnabled(true); NamespaceService namespaceService = pulsar.getNamespaceService(); - CompletableFuture future = new CompletableFuture<>(); - future.complete(false); + CompletableFuture future = new CompletableFuture<>(); + future.complete(TopicExistsInfo.newTopicNotExists()); doReturn(future).when(namespaceService).checkTopicExists(any(TopicName.class)); + CompletableFuture booleanFuture = new CompletableFuture<>(); + booleanFuture.complete(false); + doReturn(booleanFuture).when(namespaceService).checkNonPartitionedTopicExists(any(TopicName.class)); AsyncResponse asyncResponse1 = mock(AsyncResponse.class); destLookup.lookupTopicAsync(asyncResponse1, TopicDomain.persistent.value(), "myprop", "usc", "ns2", "topic_not_exist", false, null, null); @@ -260,9 +264,12 @@ public void testValidateReplicationSettingsOnNamespace() throws Exception { policies3Future.complete(Optional.of(policies3)); doReturn(policies3Future).when(namespaceResources).getPoliciesAsync(namespaceName2); NamespaceService namespaceService = pulsar.getNamespaceService(); - CompletableFuture future = new CompletableFuture<>(); - future.complete(false); + CompletableFuture future = new CompletableFuture<>(); + future.complete(TopicExistsInfo.newTopicNotExists()); doReturn(future).when(namespaceService).checkTopicExists(any(TopicName.class)); + CompletableFuture booleanFuture = new CompletableFuture<>(); + booleanFuture.complete(false); + doReturn(future).when(namespaceService).checkNonPartitionedTopicExists(any(TopicName.class)); destLookup.lookupTopicAsync(asyncResponse, TopicDomain.persistent.value(), property, cluster, ns2, "invalid-localCluster", false, null, null); verify(asyncResponse).resume(arg.capture()); @@ -294,8 +301,8 @@ public void topicNotFound() throws Exception { doReturn(uri).when(uriInfo).getRequestUri(); config.setAuthorizationEnabled(true); NamespaceService namespaceService = pulsar.getNamespaceService(); - CompletableFuture future = new CompletableFuture<>(); - future.complete(false); + CompletableFuture future = new CompletableFuture<>(); + future.complete(TopicExistsInfo.newTopicNotExists()); doReturn(future).when(namespaceService).checkTopicExists(any(TopicName.class)); // Get the current semaphore first diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java index a0313ef743667..0b0d38a071e9b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java @@ -815,14 +815,15 @@ public void testCheckTopicExists(String topicDomain) throws Exception { String topic = topicDomain + "://prop/ns-abc/" + UUID.randomUUID(); admin.topics().createNonPartitionedTopic(topic); Awaitility.await().untilAsserted(() -> { - assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(topic)).get()); + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(topic)).get().isExists()); }); String partitionedTopic = topicDomain + "://prop/ns-abc/" + UUID.randomUUID(); admin.topics().createPartitionedTopic(partitionedTopic, 5); Awaitility.await().untilAsserted(() -> { - assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(partitionedTopic)).get()); - assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(partitionedTopic + "-partition-2")).get()); + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(partitionedTopic)).get().isExists()); + assertTrue(pulsar.getNamespaceService() + .checkTopicExists(TopicName.get(partitionedTopic + "-partition-2")).get().isExists()); }); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java index 7790940c1327f..8fdf0723ea8d1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -99,6 +100,7 @@ public void testCreateConsumerAfterOnePartDeleted() throws Exception { Consumer consumerAllPartition = pulsarClient.newConsumer(Schema.STRING).topic(topic) .subscriptionName(subscription).isAckReceiptEnabled(true).subscribe(); Message msg = consumerAllPartition.receive(2, TimeUnit.SECONDS); + assertNotNull(msg); String receivedMsgValue = msg.getValue(); log.info("received msg: {}", receivedMsgValue); consumerAllPartition.acknowledge(msg); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java index 7735f66e7838a..4d6cf96a01068 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java @@ -32,6 +32,7 @@ import lombok.Getter; import lombok.NonNull; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.BatchReceivePolicy; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -58,7 +59,6 @@ import org.apache.pulsar.client.impl.conf.TopicConsumerConfigurationData; import org.apache.pulsar.client.util.RetryMessageUtil; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.util.FutureUtil; @Getter(AccessLevel.PUBLIC) @@ -104,6 +104,31 @@ public Consumer subscribe() throws PulsarClientException { } } + private CompletableFuture checkDlqAlreadyExists(String topic) { + CompletableFuture existsFuture = new CompletableFuture<>(); + client.getPartitionedTopicMetadata(topic, false).thenAccept(metadata -> { + TopicName topicName = TopicName.get(topic); + if (topicName.isPersistent()) { + // Either partitioned or non-partitioned, it exists. + existsFuture.complete(true); + } else { + // If it is a non-persistent topic, return true only it is a partitioned topic. + existsFuture.complete(metadata != null && metadata.partitions > 0); + } + }).exceptionally(ex -> { + Throwable actEx = FutureUtil.unwrapCompletionException(ex); + if (actEx instanceof PulsarClientException.NotFoundException + || actEx instanceof PulsarClientException.TopicDoesNotExistException + || actEx instanceof PulsarAdminException.NotFoundException) { + existsFuture.complete(false); + } else { + existsFuture.completeExceptionally(ex); + } + return null; + }); + return existsFuture; + } + @Override public CompletableFuture> subscribeAsync() { if (conf.getTopicNames().isEmpty() && conf.getTopicsPattern() == null) { @@ -135,20 +160,18 @@ public CompletableFuture> subscribeAsync() { DeadLetterPolicy deadLetterPolicy = conf.getDeadLetterPolicy(); if (deadLetterPolicy == null || StringUtils.isBlank(deadLetterPolicy.getRetryLetterTopic()) || StringUtils.isBlank(deadLetterPolicy.getDeadLetterTopic())) { - CompletableFuture retryLetterTopicMetadata = - client.getPartitionedTopicMetadata(oldRetryLetterTopic, true); - CompletableFuture deadLetterTopicMetadata = - client.getPartitionedTopicMetadata(oldDeadLetterTopic, true); + CompletableFuture retryLetterTopicMetadata = checkDlqAlreadyExists(oldRetryLetterTopic); + CompletableFuture deadLetterTopicMetadata = checkDlqAlreadyExists(oldDeadLetterTopic); applyDLQConfig = CompletableFuture.allOf(retryLetterTopicMetadata, deadLetterTopicMetadata) .thenAccept(__ -> { String retryLetterTopic = topicFirst + "-" + conf.getSubscriptionName() + RetryMessageUtil.RETRY_GROUP_TOPIC_SUFFIX; String deadLetterTopic = topicFirst + "-" + conf.getSubscriptionName() + RetryMessageUtil.DLQ_GROUP_TOPIC_SUFFIX; - if (retryLetterTopicMetadata.join().partitions > 0) { + if (retryLetterTopicMetadata.join()) { retryLetterTopic = oldRetryLetterTopic; } - if (deadLetterTopicMetadata.join().partitions > 0) { + if (deadLetterTopicMetadata.join()) { deadLetterTopic = oldDeadLetterTopic; } if (deadLetterPolicy == null) { From 2dc0d96fa0da696949414d86fb11a62beca7cb3f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Jun 2024 21:13:10 +0300 Subject: [PATCH 314/580] [fix][test] Fix TableViewBuilderImplTest NPE and infinite loop (#22924) --- .../client/impl/TableViewBuilderImplTest.java | 33 ++++++++++++------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewBuilderImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewBuilderImplTest.java index eee8ba4e8f41a..01353e47cd0cb 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewBuilderImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewBuilderImplTest.java @@ -18,6 +18,14 @@ */ package org.apache.pulsar.client.impl; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertNotNull; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.PulsarClientException; @@ -25,32 +33,25 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.client.impl.conf.ReaderConfigurationData; +import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; - -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.testng.Assert.assertNotNull; - /** - * Unit tests of {@link TablewViewBuilderImpl}. + * Unit tests of {@link TableViewBuilderImpl}. */ public class TableViewBuilderImplTest { private static final String TOPIC_NAME = "testTopicName"; private PulsarClientImpl client; private TableViewBuilderImpl tableViewBuilderImpl; + private CompletableFuture readNextFuture; @BeforeClass(alwaysRun = true) public void setup() { Reader reader = mock(Reader.class); - when(reader.readNextAsync()).thenReturn(CompletableFuture.allOf()); + readNextFuture = new CompletableFuture(); + when(reader.readNextAsync()).thenReturn(readNextFuture); client = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); when(client.getCnxPool()).thenReturn(connectionPool); @@ -61,6 +62,14 @@ public void setup() { tableViewBuilderImpl = new TableViewBuilderImpl(client, Schema.BYTES); } + @AfterClass(alwaysRun = true) + public void cleanup() { + if (readNextFuture != null) { + readNextFuture.completeExceptionally(new PulsarClientException.AlreadyClosedException("Closing test case")); + readNextFuture = null; + } + } + @Test public void testTableViewBuilderImpl() throws PulsarClientException { TableView tableView = tableViewBuilderImpl.topic(TOPIC_NAME) From bc3dc7727b132dd88aa84f6befef42ea0646ec50 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 18 Jun 2024 14:33:33 +0800 Subject: [PATCH 315/580] [fix] [client] Fix resource leak in Pulsar Client since HttpLookupService doesn't get closed (#22858) --- .../PulsarClientImplMultiBrokersTest.java | 79 +++++++++++++++++++ .../pulsar/client/impl/PulsarClientImpl.java | 22 ++++++ 2 files changed, 101 insertions(+) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PulsarClientImplMultiBrokersTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PulsarClientImplMultiBrokersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PulsarClientImplMultiBrokersTest.java new file mode 100644 index 0000000000000..29604d0440b05 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PulsarClientImplMultiBrokersTest.java @@ -0,0 +1,79 @@ +/* + * 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.admin; + +import static org.testng.Assert.fail; +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertTrue; +import java.util.Map; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.MultiBrokerBaseTest; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.client.impl.LookupService; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.Test; + +/** + * Test multi-broker admin api. + */ +@Slf4j +@Test(groups = "broker-admin") +public class PulsarClientImplMultiBrokersTest extends MultiBrokerBaseTest { + @Override + protected int numberOfAdditionalBrokers() { + return 3; + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + this.conf.setManagedLedgerMaxEntriesPerLedger(10); + } + + @Override + protected void onCleanup() { + super.onCleanup(); + } + + @Test(timeOut = 30 * 1000) + public void testReleaseUrlLookupServices() throws Exception { + PulsarClientImpl pulsarClient = (PulsarClientImpl) additionalBrokerClients.get(0); + Map urlLookupMap = WhiteboxImpl.getInternalState(pulsarClient, "urlLookupMap"); + assertEquals(urlLookupMap.size(), 0); + for (PulsarService pulsar : additionalBrokers) { + pulsarClient.getLookup(pulsar.getBrokerServiceUrl()); + pulsarClient.getLookup(pulsar.getWebServiceAddress()); + } + assertEquals(urlLookupMap.size(), additionalBrokers.size() * 2); + // Verify: lookup services will be release. + pulsarClient.close(); + assertEquals(urlLookupMap.size(), 0); + try { + for (PulsarService pulsar : additionalBrokers) { + pulsarClient.getLookup(pulsar.getBrokerServiceUrl()); + pulsarClient.getLookup(pulsar.getWebServiceAddress()); + } + fail("Expected a error when calling pulsarClient.getLookup if getLookup was closed"); + } catch (IllegalStateException illegalArgumentException) { + assertTrue(illegalArgumentException.getMessage().contains("has been closed")); + } + assertEquals(urlLookupMap.size(), 0); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index e8107efe98ec0..f4afb2931cc9e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -33,6 +33,7 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -744,6 +745,21 @@ public void close() throws PulsarClientException { } } + private void closeUrlLookupMap() { + Map closedUrlLookupServices = new HashMap(urlLookupMap.size()); + urlLookupMap.entrySet().forEach(e -> { + try { + e.getValue().close(); + } catch (Exception ex) { + log.error("Error closing lookup service {}", e.getKey(), ex); + } + closedUrlLookupServices.put(e.getKey(), e.getValue()); + }); + closedUrlLookupServices.entrySet().forEach(e -> { + urlLookupMap.remove(e.getKey(), e.getValue()); + }); + } + @Override public CompletableFuture closeAsync() { log.info("Client closing. URL: {}", lookup.getServiceUrl()); @@ -754,6 +770,8 @@ public CompletableFuture closeAsync() { final CompletableFuture closeFuture = new CompletableFuture<>(); List> futures = new ArrayList<>(); + closeUrlLookupMap(); + producers.forEach(p -> futures.add(p.closeAsync().handle((__, t) -> { if (t != null) { log.error("Error closing producer {}", p, t); @@ -982,6 +1000,10 @@ public CompletableFuture getConnection(final String topic, final Stri public LookupService getLookup(String serviceUrl) { return urlLookupMap.computeIfAbsent(serviceUrl, url -> { + if (isClosed()) { + throw new IllegalStateException("Pulsar client has been closed, can not build LookupService when" + + " calling get lookup with an url"); + } try { return createLookup(serviceUrl); } catch (PulsarClientException e) { From aa8f696b8e17a49d1a7ff6cdc25f1d86e7c4a8ed Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Tue, 18 Jun 2024 08:43:40 -0700 Subject: [PATCH 316/580] [fix][broker] Update init and shutdown time and other minor logic (ExtensibleLoadManagerImpl only) (#22930) --- .../pulsar/PulsarClusterMetadataSetup.java | 4 +-- .../extensions/ExtensibleLoadManagerImpl.java | 6 +++-- .../channel/ServiceUnitStateChannelImpl.java | 26 ++++++++++++++----- .../store/TableViewLoadDataStoreImpl.java | 12 +++++---- .../broker/namespace/NamespaceService.java | 5 ++-- .../PulsarClientBasedHandlerTest.java | 3 +-- 6 files changed, 36 insertions(+), 20 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java index d5b8df43a4737..04a66ff022e2a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java @@ -375,8 +375,8 @@ public static void createTenantIfAbsent(PulsarResources resources, String tenant } } - static void createNamespaceIfAbsent(PulsarResources resources, NamespaceName namespaceName, - String cluster, int bundleNumber) throws IOException { + public static void createNamespaceIfAbsent(PulsarResources resources, NamespaceName namespaceName, + String cluster, int bundleNumber) throws IOException { NamespaceResources namespaceResources = resources.getNamespaceResources(); if (!namespaceResources.namespaceExists(namespaceName)) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 1e519b3284fbd..92dcf8001ada5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -668,7 +668,9 @@ public CompletableFuture> getOwnershipWithLookupDataA public CompletableFuture unloadNamespaceBundleAsync(ServiceUnitId bundle, Optional destinationBroker, - boolean force) { + boolean force, + long timeout, + TimeUnit timeoutUnit) { if (NamespaceService.isSLAOrHeartbeatNamespace(bundle.getNamespaceObject().toString())) { log.info("Skip unloading namespace bundle: {}.", bundle); return CompletableFuture.completedFuture(null); @@ -691,7 +693,7 @@ public CompletableFuture unloadNamespaceBundleAsync(ServiceUnitId bundle, UnloadDecision unloadDecision = new UnloadDecision(unload, UnloadDecision.Label.Success, UnloadDecision.Reason.Admin); return unloadAsync(unloadDecision, - conf.getNamespaceBundleUnloadingTimeoutMs(), TimeUnit.MILLISECONDS); + timeout, timeoutUnit); }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index f04734c4ad9bb..1688a892e237f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -114,7 +114,6 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { public static final CompressionType MSG_COMPRESSION_TYPE = CompressionType.ZSTD; private static final int OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS = 5000; private static final int OWNERSHIP_CLEAN_UP_WAIT_RETRY_DELAY_IN_MILLIS = 100; - public static final int OWNERSHIP_CLEAN_UP_CONVERGENCE_DELAY_IN_MILLIS = 3000; public static final long VERSION_ID_INIT = 1; // initial versionId public static final long MAX_CLEAN_UP_DELAY_TIME_IN_SECS = 3 * 60; // 3 mins private static final long MIN_CLEAN_UP_DELAY_TIME_IN_SECS = 0; // 0 secs to clean immediately @@ -298,7 +297,8 @@ public synchronized void start() throws PulsarServerException { (pulsar.getPulsarResources(), SYSTEM_NAMESPACE.getTenant(), config.getClusterName()); PulsarClusterMetadataSetup.createNamespaceIfAbsent - (pulsar.getPulsarResources(), SYSTEM_NAMESPACE, config.getClusterName()); + (pulsar.getPulsarResources(), SYSTEM_NAMESPACE, config.getClusterName(), + config.getDefaultNumberOfNamespaceBundles()); ExtensibleLoadManagerImpl.createSystemTopic(pulsar, TOPIC); @@ -1018,6 +1018,9 @@ private CompletableFuture closeServiceUnit(String serviceUnit, boolean if (ex != null) { log.error("Failed to close topics under bundle:{} in {} ms", bundle.toString(), unloadBundleTime, ex); + if (!disconnectClients) { + pulsar.getBrokerService().cleanUnloadedTopicFromCache(bundle); + } } else { log.info("Unloading bundle:{} with {} topics completed in {} ms", bundle, unloadedTopics, unloadBundleTime); @@ -1342,11 +1345,6 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max } } if (cleaned) { - try { - MILLISECONDS.sleep(OWNERSHIP_CLEAN_UP_CONVERGENCE_DELAY_IN_MILLIS); - } catch (InterruptedException e) { - log.warn("Interrupted while gracefully waiting for the cleanup convergence."); - } break; } else { try { @@ -1357,9 +1355,23 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max } } } + log.info("Finished cleanup waiting for orphan broker:{}. Elapsed {} ms", brokerId, + System.currentTimeMillis() - started); } private synchronized void doCleanup(String broker) { + try { + if (getChannelOwnerAsync().get(MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS, TimeUnit.SECONDS) + .isEmpty()) { + log.error("Found the channel owner is empty. Skip the inactive broker:{}'s orphan bundle cleanup", + broker); + return; + } + } catch (Exception e) { + log.error("Failed to find the channel owner. Skip the inactive broker:{}'s orphan bundle cleanup", broker); + return; + } + long startTime = System.nanoTime(); log.info("Started ownership cleanup for the inactive broker:{}", broker); int orphanServiceUnitCleanupCnt = 0; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java index 81cf33b4a55d2..e9289d3ccdac2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java @@ -31,7 +31,6 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TableView; @@ -44,6 +43,7 @@ public class TableViewLoadDataStoreImpl implements LoadDataStore { private static final long LOAD_DATA_REPORT_UPDATE_MAX_INTERVAL_MULTIPLIER_BEFORE_RESTART = 2; + private static final long INIT_TIMEOUT_IN_SECS = 5; private volatile TableView tableView; private volatile long tableViewLastUpdateTimestamp; @@ -123,10 +123,11 @@ public synchronized void start() throws LoadDataStoreException { public synchronized void startTableView() throws LoadDataStoreException { if (tableView == null) { try { - tableView = client.newTableViewBuilder(Schema.JSON(clazz)).topic(topic).create(); + tableView = client.newTableViewBuilder(Schema.JSON(clazz)).topic(topic).createAsync() + .get(INIT_TIMEOUT_IN_SECS, TimeUnit.SECONDS); tableView.forEachAndListen((k, v) -> tableViewLastUpdateTimestamp = System.currentTimeMillis()); - } catch (PulsarClientException e) { + } catch (Exception e) { tableView = null; throw new LoadDataStoreException(e); } @@ -137,8 +138,9 @@ public synchronized void startTableView() throws LoadDataStoreException { public synchronized void startProducer() throws LoadDataStoreException { if (producer == null) { try { - producer = client.newProducer(Schema.JSON(clazz)).topic(topic).create(); - } catch (PulsarClientException e) { + producer = client.newProducer(Schema.JSON(clazz)).topic(topic).createAsync() + .get(INIT_TIMEOUT_IN_SECS, TimeUnit.SECONDS); + } catch (Exception e) { producer = null; throw new LoadDataStoreException(e); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 9df2b09204c15..df6a141ddcf1a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -840,7 +840,7 @@ public CompletableFuture unloadNamespaceBundle(NamespaceBundle bundle, boolean closeWithoutWaitingClientDisconnect) { if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return ExtensibleLoadManagerImpl.get(loadManager.get()) - .unloadNamespaceBundleAsync(bundle, destinationBroker, false); + .unloadNamespaceBundleAsync(bundle, destinationBroker, false, timeout, timeoutUnit); } // unload namespace bundle OwnedBundle ob = ownershipCache.getOwnedBundle(bundle); @@ -1290,7 +1290,8 @@ public CompletableFuture removeOwnedServiceUnitAsync(NamespaceBundle nsBun if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); future = extensibleLoadManager.unloadNamespaceBundleAsync( - nsBundle, Optional.empty(), true); + nsBundle, Optional.empty(), true, + pulsar.getConfig().getNamespaceBundleUnloadingTimeoutMs(), TimeUnit.MILLISECONDS); } else { future = ownershipCache.removeOwnership(nsBundle); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/protocol/PulsarClientBasedHandlerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/protocol/PulsarClientBasedHandlerTest.java index 9cc20cf7b9def..bdaddf9afb1da 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/protocol/PulsarClientBasedHandlerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/protocol/PulsarClientBasedHandlerTest.java @@ -27,7 +27,6 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -73,7 +72,7 @@ public void testStopBroker() throws PulsarServerException { pulsar.close(); final var elapsedMs = System.currentTimeMillis() - beforeStop; log.info("It spends {} ms to stop the broker ({} for protocol handler)", elapsedMs, handler.closeTimeMs); - Assert.assertTrue(elapsedMs < ServiceUnitStateChannelImpl.OWNERSHIP_CLEAN_UP_CONVERGENCE_DELAY_IN_MILLIS + Assert.assertTrue(elapsedMs < + handler.closeTimeMs + shutdownTimeoutMs + 1000); // tolerate 1 more second for other processes } From 5ed07c1eba7d05a095f5a165742ca5275d6c673f Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Wed, 19 Jun 2024 02:23:50 +0800 Subject: [PATCH 317/580] [improve][cli] Use LoadManagerReport instead of Object (#22850) --- .../loadbalancer/LocalBrokerDataTest.java | 21 ++++++++--- .../pulsar/testclient/BrokerMonitor.java | 36 +++++++++++-------- 2 files changed, 38 insertions(+), 19 deletions(-) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerDataTest.java b/pulsar-common/src/test/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerDataTest.java index db55ecfe5035a..b5d7e3c355a50 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerDataTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/policies/data/loadbalancer/LocalBrokerDataTest.java @@ -20,7 +20,8 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; +import com.fasterxml.jackson.databind.ObjectReader; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.testng.Assert; import org.testng.annotations.Test; @@ -30,14 +31,26 @@ public class LocalBrokerDataTest { @Test - public void testLocalBrokerDataDeserialization() { + public void testLocalBrokerDataDeserialization() throws JsonProcessingException { + ObjectReader LOAD_REPORT_READER = ObjectMapperFactory.getMapper().reader() + .forType(LoadManagerReport.class); String data = "{\"webServiceUrl\":\"http://10.244.2.23:8080\",\"webServiceUrlTls\":\"https://10.244.2.23:8081\",\"pulsarServiceUrlTls\":\"pulsar+ssl://10.244.2.23:6651\",\"persistentTopicsEnabled\":true,\"nonPersistentTopicsEnabled\":false,\"cpu\":{\"usage\":3.1577712104798255,\"limit\":100.0},\"memory\":{\"usage\":614.0,\"limit\":1228.0},\"directMemory\":{\"usage\":32.0,\"limit\":1228.0},\"bandwidthIn\":{\"usage\":0.0,\"limit\":0.0},\"bandwidthOut\":{\"usage\":0.0,\"limit\":0.0},\"msgThroughputIn\":0.0,\"msgThroughputOut\":0.0,\"msgRateIn\":0.0,\"msgRateOut\":0.0,\"lastUpdate\":1650886425227,\"lastStats\":{\"pulsar/pulsar/10.244.2.23:8080/0x00000000_0xffffffff\":{\"msgRateIn\":0.0,\"msgThroughputIn\":0.0,\"msgRateOut\":0.0,\"msgThroughputOut\":0.0,\"consumerCount\":0,\"producerCount\":0,\"topics\":1,\"cacheSize\":0}},\"numTopics\":1,\"numBundles\":1,\"numConsumers\":0,\"numProducers\":0,\"bundles\":[\"pulsar/pulsar/10.244.2.23:8080/0x00000000_0xffffffff\"],\"lastBundleGains\":[],\"lastBundleLosses\":[],\"brokerVersionString\":\"2.11.0-hw-0.0.4-SNAPSHOT\",\"protocols\":{},\"advertisedListeners\":{},\"bundleStats\":{\"pulsar/pulsar/10.244.2.23:8080/0x00000000_0xffffffff\":{\"msgRateIn\":0.0,\"msgThroughputIn\":0.0,\"msgRateOut\":0.0,\"msgThroughputOut\":0.0,\"consumerCount\":0,\"producerCount\":0,\"topics\":1,\"cacheSize\":0}},\"maxResourceUsage\":0.49645519256591797,\"loadReportType\":\"LocalBrokerData\"}"; - Gson gson = new Gson(); - LocalBrokerData localBrokerData = gson.fromJson(data, LocalBrokerData.class); + LoadManagerReport localBrokerData = LOAD_REPORT_READER.readValue(data); Assert.assertEquals(localBrokerData.getMemory().limit, 1228.0d, 0.0001f); Assert.assertEquals(localBrokerData.getMemory().usage, 614.0d, 0.0001f); Assert.assertEquals(localBrokerData.getMemory().percentUsage(), ((float) localBrokerData.getMemory().usage) / ((float) localBrokerData.getMemory().limit) * 100, 0.0001f); } + @Test + public void testTimeAverageBrokerDataDataDeserialization() throws JsonProcessingException { + ObjectReader TIME_AVERAGE_READER = ObjectMapperFactory.getMapper().reader() + .forType(TimeAverageBrokerData.class); + String data = "{\"shortTermMsgThroughputIn\":100,\"shortTermMsgThroughputOut\":200,\"shortTermMsgRateIn\":300,\"shortTermMsgRateOut\":400,\"longTermMsgThroughputIn\":567.891,\"longTermMsgThroughputOut\":678.912,\"longTermMsgRateIn\":789.123,\"longTermMsgRateOut\":890.123}"; + TimeAverageBrokerData timeAverageBrokerData = TIME_AVERAGE_READER.readValue(data); + assertEquals(timeAverageBrokerData.getShortTermMsgThroughputIn(), 100.00); + assertEquals(timeAverageBrokerData.getShortTermMsgThroughputOut(), 200.00); + assertEquals(timeAverageBrokerData.getShortTermMsgRateIn(), 300.00); + assertEquals(timeAverageBrokerData.getShortTermMsgRateOut(), 400.00); + } @Test public void testMaxResourceUsage() { diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/BrokerMonitor.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/BrokerMonitor.java index a2f5b382c7b8f..6af4925a7c664 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/BrokerMonitor.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/BrokerMonitor.java @@ -20,7 +20,7 @@ import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.BROKER_LOAD_DATA_STORE_TOPIC; import static org.apache.pulsar.broker.resources.LoadBalanceResources.BROKER_TIME_AVERAGE_BASE_PATH; -import com.google.gson.Gson; +import com.fasterxml.jackson.databind.ObjectReader; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -35,6 +35,8 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SizeUnit; import org.apache.pulsar.client.api.TableView; +import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; import org.apache.pulsar.policies.data.loadbalancer.LoadReport; import org.apache.pulsar.policies.data.loadbalancer.LocalBrokerData; import org.apache.pulsar.policies.data.loadbalancer.ResourceUsage; @@ -63,7 +65,11 @@ public class BrokerMonitor extends CmdBase { private static final int ZOOKEEPER_TIMEOUT_MILLIS = 30000; private static final int GLOBAL_STATS_PRINT_PERIOD_MILLIS = 60000; private ZooKeeper zkClient; - private static final Gson gson = new Gson(); + private static final ObjectReader LOAD_REPORT_READER = ObjectMapperFactory.getMapper().reader() + .forType(LoadManagerReport.class); + + private static final ObjectReader TIME_AVERAGE_READER = ObjectMapperFactory.getMapper().reader() + .forType(TimeAverageBrokerData.class); // Fields common for message rows. private static final List MESSAGE_FIELDS = Arrays.asList("MSG/S IN", "MSG/S OUT", "TOTAL", "KB/S IN", @@ -85,7 +91,7 @@ public class BrokerMonitor extends CmdBase { private static final Object[] ALLOC_MESSAGE_ROW = makeMessageRow("ALLOC MSG"); private static final Object[] GLOBAL_HEADER = { "BROKER", "BUNDLE", "MSG/S", "LONG/S", "KB/S", "MAX %" }; - private Map loadData; + private Map loadData; private static final FixedColumnLengthTableMaker localTableMaker = new FixedColumnLengthTableMaker(); @@ -146,9 +152,9 @@ private void printGlobalData() { double totalLongTermMessageRate = 0; double maxMaxUsage = 0; int i = 1; - for (final Map.Entry entry : loadData.entrySet()) { + for (final Map.Entry entry : loadData.entrySet()) { final String broker = entry.getKey(); - final Object data = entry.getValue(); + final LoadManagerReport data = entry.getValue(); rows[i] = new Object[GLOBAL_HEADER.length]; rows[i][0] = broker; int numBundles; @@ -177,9 +183,8 @@ private void printGlobalData() { messageRate = localData.getMsgRateIn() + localData.getMsgRateOut(); final String timeAveragePath = BROKER_TIME_AVERAGE_BASE_PATH + "/" + broker; try { - final TimeAverageBrokerData timeAverageData = gson.fromJson( - new String(zkClient.getData(timeAveragePath, false, null)), - TimeAverageBrokerData.class); + final TimeAverageBrokerData timeAverageData = TIME_AVERAGE_READER.readValue( + new String(zkClient.getData(timeAveragePath, false, null))); longTermMessageRate = timeAverageData.getLongTermMsgRateIn() + timeAverageData.getLongTermMsgRateOut(); } catch (Exception x) { @@ -307,20 +312,21 @@ private double percentUsage(final double usage, final double limit) { private synchronized void printData(final String path) { final String broker = brokerNameFromPath(path); String jsonString; + LoadManagerReport loadManagerReport; try { jsonString = new String(zkClient.getData(path, this, null)); + loadManagerReport = LOAD_REPORT_READER.readValue(jsonString); } catch (Exception ex) { throw new RuntimeException(ex); } - // Use presence of the String "allocated" to determine if this is using SimpleLoadManagerImpl. - if (jsonString.contains("allocated")) { - printLoadReport(broker, gson.fromJson(jsonString, LoadReport.class)); - } else { - final LocalBrokerData localBrokerData = gson.fromJson(jsonString, LocalBrokerData.class); + if (loadManagerReport instanceof LoadReport) { + printLoadReport(broker, (LoadReport) loadManagerReport); + } else { + final LocalBrokerData localBrokerData = (LocalBrokerData) loadManagerReport; final String timeAveragePath = BROKER_TIME_AVERAGE_BASE_PATH + "/" + broker; try { - final TimeAverageBrokerData timeAverageData = gson.fromJson( - new String(zkClient.getData(timeAveragePath, false, null)), TimeAverageBrokerData.class); + final TimeAverageBrokerData timeAverageData = TIME_AVERAGE_READER.readValue( + new String(zkClient.getData(timeAveragePath, false, null))); printBrokerData(broker, localBrokerData, timeAverageData); } catch (Exception e) { throw new RuntimeException(e); From 6a1bbe6ba092336ff66658f985a25de901687683 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 18 Jun 2024 17:30:08 -0700 Subject: [PATCH 318/580] [feat][broker] PIP-264: Add OpenTelemetry broker connection metrics (#22931) --- .../pulsar/broker/service/PulsarStats.java | 3 +- .../stats/BrokerOperabilityMetrics.java | 57 ++++++++-- ...enTelemetryBrokerOperabilityStatsTest.java | 104 ++++++++++++++++++ .../OpenTelemetryAttributes.java | 16 +++ 4 files changed, 170 insertions(+), 10 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryBrokerOperabilityStatsTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java index db14892d26663..7ffc7818d4c2d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java @@ -78,8 +78,7 @@ public PulsarStats(PulsarService pulsar) { this.bundleStats = new ConcurrentHashMap<>(); this.tempMetricsCollection = new ArrayList<>(); this.metricsCollection = new ArrayList<>(); - this.brokerOperabilityMetrics = new BrokerOperabilityMetrics(pulsar.getConfiguration().getClusterName(), - pulsar.getAdvertisedAddress()); + this.brokerOperabilityMetrics = new BrokerOperabilityMetrics(pulsar); this.tempNonPersistentTopics = new ArrayList<>(); this.exposePublisherStats = pulsar.getConfiguration().isExposePublisherStats(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BrokerOperabilityMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BrokerOperabilityMetrics.java index b6379d381c6f3..3f991be8184ab 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BrokerOperabilityMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BrokerOperabilityMetrics.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.stats; +import io.opentelemetry.api.metrics.ObservableLongCounter; import io.prometheus.client.Counter; import java.util.ArrayList; import java.util.HashMap; @@ -25,32 +26,72 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.LongAdder; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.common.stats.Metrics; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.ConnectionCreateStatus; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.ConnectionStatus; /** */ -public class BrokerOperabilityMetrics { +public class BrokerOperabilityMetrics implements AutoCloseable { private static final Counter TOPIC_LOAD_FAILED = Counter.build("topic_load_failed", "-").register(); private final List metricsList; private final String localCluster; private final DimensionStats topicLoadStats; private final String brokerName; private final LongAdder connectionTotalCreatedCount; - private final LongAdder connectionCreateSuccessCount; - private final LongAdder connectionCreateFailCount; private final LongAdder connectionTotalClosedCount; private final LongAdder connectionActive; - public BrokerOperabilityMetrics(String localCluster, String brokerName) { + private final LongAdder connectionCreateSuccessCount; + private final LongAdder connectionCreateFailCount; + + public static final String CONNECTION_COUNTER_METRIC_NAME = "pulsar.broker.connection.count"; + private final ObservableLongCounter connectionCounter; + + public static final String CONNECTION_CREATE_COUNTER_METRIC_NAME = + "pulsar.broker.connection.create.operation.count"; + private final ObservableLongCounter connectionCreateCounter; + + public BrokerOperabilityMetrics(PulsarService pulsar) { this.metricsList = new ArrayList<>(); - this.localCluster = localCluster; + this.localCluster = pulsar.getConfiguration().getClusterName(); this.topicLoadStats = new DimensionStats("pulsar_topic_load_times", 60); - this.brokerName = brokerName; + this.brokerName = pulsar.getAdvertisedAddress(); this.connectionTotalCreatedCount = new LongAdder(); - this.connectionCreateSuccessCount = new LongAdder(); - this.connectionCreateFailCount = new LongAdder(); this.connectionTotalClosedCount = new LongAdder(); this.connectionActive = new LongAdder(); + + this.connectionCreateSuccessCount = new LongAdder(); + this.connectionCreateFailCount = new LongAdder(); + + connectionCounter = pulsar.getOpenTelemetry().getMeter() + .counterBuilder(CONNECTION_COUNTER_METRIC_NAME) + .setDescription("The number of connections.") + .setUnit("{connection}") + .buildWithCallback(measurement -> { + var closedConnections = connectionTotalClosedCount.sum(); + var openedConnections = connectionTotalCreatedCount.sum(); + var activeConnections = openedConnections - closedConnections; + measurement.record(activeConnections, ConnectionStatus.ACTIVE.attributes); + measurement.record(openedConnections, ConnectionStatus.OPEN.attributes); + measurement.record(closedConnections, ConnectionStatus.CLOSE.attributes); + }); + + connectionCreateCounter = pulsar.getOpenTelemetry().getMeter() + .counterBuilder(CONNECTION_CREATE_COUNTER_METRIC_NAME) + .setDescription("The number of connection create operations.") + .setUnit("{operation}") + .buildWithCallback(measurement -> { + measurement.record(connectionCreateSuccessCount.sum(), ConnectionCreateStatus.SUCCESS.attributes); + measurement.record(connectionCreateFailCount.sum(), ConnectionCreateStatus.FAILURE.attributes); + }); + } + + @Override + public void close() throws Exception { + connectionCounter.close(); + connectionCreateCounter.close(); } public List getMetrics() { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryBrokerOperabilityStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryBrokerOperabilityStatsTest.java new file mode 100644 index 0000000000000..4378e6b05b3ee --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryBrokerOperabilityStatsTest.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.broker.stats; + +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.ConnectionCreateStatus; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class OpenTelemetryBrokerOperabilityStatsTest extends BrokerTestBase { + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.baseSetup(); + } + + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder pulsarTestContextBuilder) { + super.customizeMainPulsarTestContextBuilder(pulsarTestContextBuilder); + pulsarTestContextBuilder.enableOpenTelemetry(true); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testBrokerConnection() throws Exception { + var topicName = BrokerTestUtil.newUniqueName("persistent://my-namespace/use/my-ns/testBrokerConnection"); + + @Cleanup + var producer = pulsarClient.newProducer().topic(topicName).create(); + + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, BrokerOperabilityMetrics.CONNECTION_COUNTER_METRIC_NAME, + OpenTelemetryAttributes.ConnectionStatus.OPEN.attributes, 1); + assertMetricLongSumValue(metrics, BrokerOperabilityMetrics.CONNECTION_COUNTER_METRIC_NAME, + OpenTelemetryAttributes.ConnectionStatus.CLOSE.attributes, 0); + assertMetricLongSumValue(metrics, BrokerOperabilityMetrics.CONNECTION_COUNTER_METRIC_NAME, + OpenTelemetryAttributes.ConnectionStatus.ACTIVE.attributes, 1); + + assertMetricLongSumValue(metrics, BrokerOperabilityMetrics.CONNECTION_CREATE_COUNTER_METRIC_NAME, + ConnectionCreateStatus.SUCCESS.attributes, 1); + assertMetricLongSumValue(metrics, BrokerOperabilityMetrics.CONNECTION_CREATE_COUNTER_METRIC_NAME, + ConnectionCreateStatus.FAILURE.attributes, 0); + + pulsarClient.close(); + + metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, BrokerOperabilityMetrics.CONNECTION_COUNTER_METRIC_NAME, + OpenTelemetryAttributes.ConnectionStatus.CLOSE.attributes, 1); + + pulsar.getConfiguration().setAuthenticationEnabled(true); + + replacePulsarClient(PulsarClient.builder() + .serviceUrl(lookupUrl.toString()) + .operationTimeout(1, TimeUnit.MILLISECONDS)); + assertThatThrownBy(() -> pulsarClient.newProducer().topic(topicName).create()) + .isInstanceOf(PulsarClientException.AuthenticationException.class); + pulsarClient.close(); + + metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, BrokerOperabilityMetrics.CONNECTION_COUNTER_METRIC_NAME, + OpenTelemetryAttributes.ConnectionStatus.OPEN.attributes, 2); + assertMetricLongSumValue(metrics, BrokerOperabilityMetrics.CONNECTION_COUNTER_METRIC_NAME, + OpenTelemetryAttributes.ConnectionStatus.CLOSE.attributes, 2); + assertMetricLongSumValue(metrics, BrokerOperabilityMetrics.CONNECTION_COUNTER_METRIC_NAME, + OpenTelemetryAttributes.ConnectionStatus.ACTIVE.attributes, 0); + + assertMetricLongSumValue(metrics, BrokerOperabilityMetrics.CONNECTION_CREATE_COUNTER_METRIC_NAME, + ConnectionCreateStatus.SUCCESS.attributes, 1); + assertMetricLongSumValue(metrics, BrokerOperabilityMetrics.CONNECTION_CREATE_COUNTER_METRIC_NAME, + ConnectionCreateStatus.FAILURE.attributes, 1); + } +} diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index 004741b6dfb55..6639cd68b398e 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -142,4 +142,20 @@ enum BacklogQuotaType { TIME; public final Attributes attributes = Attributes.of(PULSAR_BACKLOG_QUOTA_TYPE, name().toLowerCase()); } + + AttributeKey PULSAR_CONNECTION_STATUS = AttributeKey.stringKey("pulsar.connection.status"); + enum ConnectionStatus { + ACTIVE, + OPEN, + CLOSE; + public final Attributes attributes = Attributes.of(PULSAR_CONNECTION_STATUS, name().toLowerCase()); + } + + AttributeKey PULSAR_CONNECTION_CREATE_STATUS = + AttributeKey.stringKey("pulsar.connection.create.operation.status"); + enum ConnectionCreateStatus { + SUCCESS, + FAILURE; + public final Attributes attributes = Attributes.of(PULSAR_CONNECTION_CREATE_STATUS, name().toLowerCase()); + } } From bacb162049f8f3daa125d69db28630fdd3ceac45 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 19 Jun 2024 12:56:30 +0800 Subject: [PATCH 319/580] [cleanup][client] Remove unneeded sync scope in TransactionImpl (#22932) --- .../impl/transaction/TransactionImpl.java | 44 +++++++++---------- 1 file changed, 20 insertions(+), 24 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionImpl.java index d1260ba045e6d..a88d65fce3100 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionImpl.java @@ -106,18 +106,16 @@ public void run(Timeout timeout) throws Exception { public CompletableFuture registerProducedTopic(String topic) { CompletableFuture completableFuture = new CompletableFuture<>(); if (checkIfOpen(completableFuture)) { - synchronized (TransactionImpl.this) { - // we need to issue the request to TC to register the produced topic - return registerPartitionMap.compute(topic, (key, future) -> { - if (future != null) { - return future.thenCompose(ignored -> CompletableFuture.completedFuture(null)); - } else { - return tcClient.addPublishPartitionToTxnAsync( - txnId, Lists.newArrayList(topic)) - .thenCompose(ignored -> CompletableFuture.completedFuture(null)); - } - }); - } + // we need to issue the request to TC to register the produced topic + return registerPartitionMap.compute(topic, (key, future) -> { + if (future != null) { + return future.thenCompose(ignored -> CompletableFuture.completedFuture(null)); + } else { + return tcClient.addPublishPartitionToTxnAsync( + txnId, Lists.newArrayList(topic)) + .thenCompose(ignored -> CompletableFuture.completedFuture(null)); + } + }); } return completableFuture; } @@ -147,18 +145,16 @@ public void registerSendOp(CompletableFuture newSendFuture) { public CompletableFuture registerAckedTopic(String topic, String subscription) { CompletableFuture completableFuture = new CompletableFuture<>(); if (checkIfOpen(completableFuture)) { - synchronized (TransactionImpl.this) { - // we need to issue the request to TC to register the acked topic - return registerSubscriptionMap.compute(Pair.of(topic, subscription), (key, future) -> { - if (future != null) { - return future.thenCompose(ignored -> CompletableFuture.completedFuture(null)); - } else { - return tcClient.addSubscriptionToTxnAsync( - txnId, topic, subscription) - .thenCompose(ignored -> CompletableFuture.completedFuture(null)); - } - }); - } + // we need to issue the request to TC to register the acked topic + return registerSubscriptionMap.compute(Pair.of(topic, subscription), (key, future) -> { + if (future != null) { + return future.thenCompose(ignored -> CompletableFuture.completedFuture(null)); + } else { + return tcClient.addSubscriptionToTxnAsync( + txnId, topic, subscription) + .thenCompose(ignored -> CompletableFuture.completedFuture(null)); + } + }); } return completableFuture; } From feae58988d672767c076daa0c7caa5613cbba36e Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 19 Jun 2024 15:13:57 +0800 Subject: [PATCH 320/580] [fix] [broker] Messages lost on the remote cluster when using topic level replication (#22890) --- .../service/persistent/PersistentTopic.java | 49 ++++----- .../broker/service/OneWayReplicatorTest.java | 103 ++++++++++++++++++ .../service/OneWayReplicatorTestBase.java | 22 ++++ .../OneWayReplicatorUsingGlobalZKTest.java | 5 + 4 files changed, 153 insertions(+), 26 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 711e1d93f742f..630712f536874 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -439,13 +439,6 @@ public CompletableFuture initialize() { this.createPersistentSubscriptions(); })); - for (ManagedCursor cursor : ledger.getCursors()) { - if (cursor.getName().startsWith(replicatorPrefix)) { - String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); - String remoteCluster = PersistentReplicator.getRemoteCluster(cursor.getName()); - futures.add(addReplicationCluster(remoteCluster, cursor, localCluster)); - } - } return FutureUtil.waitForAll(futures).thenCompose(__ -> brokerService.pulsar().getPulsarResources().getNamespaceResources() .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) @@ -476,6 +469,7 @@ public CompletableFuture initialize() { isAllowAutoUpdateSchema = policies.is_allow_auto_update_schema; }, getOrderedExecutor()) .thenCompose(ignore -> initTopicPolicy()) + .thenCompose(ignore -> removeOrphanReplicationCursors()) .exceptionally(ex -> { log.warn("[{}] Error getting policies {} and isEncryptionRequired will be set to false", topic, ex.getMessage()); @@ -553,6 +547,21 @@ private void createPersistentSubscriptions() { checkReplicatedSubscriptionControllerState(); } + private CompletableFuture removeOrphanReplicationCursors() { + List> futures = new ArrayList<>(); + List replicationClusters = topicPolicies.getReplicationClusters().get(); + for (ManagedCursor cursor : ledger.getCursors()) { + if (cursor.getName().startsWith(replicatorPrefix)) { + String remoteCluster = PersistentReplicator.getRemoteCluster(cursor.getName()); + if (!replicationClusters.contains(remoteCluster)) { + log.warn("Remove the orphan replicator because the cluster '{}' does not exist", remoteCluster); + futures.add(removeReplicator(remoteCluster)); + } + } + } + return FutureUtil.waitForAll(futures); + } + /** * Unload a subscriber. * @throws SubscriptionNotFoundException If subscription not founded. @@ -2055,30 +2064,18 @@ public void openCursorFailed(ManagedLedgerException exception, Object ctx) { return future; } - private CompletableFuture checkReplicationCluster(String remoteCluster) { - return brokerService.getPulsar().getPulsarResources().getNamespaceResources() - .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) - .thenApply(optPolicies -> optPolicies.map(policies -> policies.replication_clusters) - .orElse(Collections.emptySet()).contains(remoteCluster) - || topicPolicies.getReplicationClusters().get().contains(remoteCluster)); - } - protected CompletableFuture addReplicationCluster(String remoteCluster, ManagedCursor cursor, String localCluster) { return AbstractReplicator.validatePartitionedTopicAsync(PersistentTopic.this.getName(), brokerService) - .thenCompose(__ -> checkReplicationCluster(remoteCluster)) - .thenCompose(clusterExists -> { - if (!clusterExists) { - log.warn("Remove the replicator because the cluster '{}' does not exist", remoteCluster); - return removeReplicator(remoteCluster).thenApply(__ -> null); - } - return brokerService.pulsar().getPulsarResources().getClusterResources() - .getClusterAsync(remoteCluster) - .thenApply(clusterData -> - brokerService.getReplicationClient(remoteCluster, clusterData)); - }) + .thenCompose(__ -> brokerService.pulsar().getPulsarResources().getClusterResources() + .getClusterAsync(remoteCluster) + .thenApply(clusterData -> + brokerService.getReplicationClient(remoteCluster, clusterData))) .thenAccept(replicationClient -> { if (replicationClient == null) { + log.error("[{}] Can not create replicator because the remote client can not be created." + + " remote cluster: {}. State of transferring : {}", + topic, remoteCluster, transferring); return; } lock.readLock().lock(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index 3dcd787a0cd5e..c9b23c6437a22 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -46,6 +46,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; +import java.util.function.Supplier; import lombok.AllArgsConstructor; import lombok.Data; import lombok.SneakyThrows; @@ -58,7 +59,10 @@ import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; @@ -78,6 +82,7 @@ import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -809,4 +814,102 @@ public void testExpandTopicPartitionsOnNamespaceLevelReplication() throws Except admin2.topics().deletePartitionedTopic(topicName, false); }); } + + private String getTheLatestMessage(String topic, PulsarClient client, PulsarAdmin admin) throws Exception { + String dummySubscription = "s_" + UUID.randomUUID().toString().replace("-", ""); + admin.topics().createSubscription(topic, dummySubscription, MessageId.earliest); + Consumer c = client.newConsumer(Schema.STRING).topic(topic).subscriptionName(dummySubscription) + .subscribe(); + String lastMsgValue = null; + while (true) { + Message msg = c.receive(2, TimeUnit.SECONDS); + if (msg == null) { + break; + } + lastMsgValue = msg.getValue(); + } + c.unsubscribe(); + return lastMsgValue; + } + + enum ReplicationLevel { + TOPIC_LEVEL, + NAMESPACE_LEVEL; + } + + @DataProvider(name = "replicationLevels") + public Object[][] replicationLevels() { + return new Object[][]{ + {ReplicationLevel.TOPIC_LEVEL}, + {ReplicationLevel.NAMESPACE_LEVEL} + }; + } + + @Test(dataProvider = "replicationLevels") + public void testReloadWithTopicLevelGeoReplication(ReplicationLevel replicationLevel) throws Exception { + final String topicName = ((Supplier) () -> { + if (replicationLevel.equals(ReplicationLevel.TOPIC_LEVEL)) { + return BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); + } else { + return BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + } + }).get(); + admin1.topics().createNonPartitionedTopic(topicName); + admin2.topics().createNonPartitionedTopic(topicName); + admin2.topics().createSubscription(topicName, "s1", MessageId.earliest); + if (replicationLevel.equals(ReplicationLevel.TOPIC_LEVEL)) { + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); + } else { + pulsar1.getConfig().setTopicLevelPoliciesEnabled(false); + } + verifyReplicationWorks(topicName); + + /** + * Verify: + * 1. Inject an error to make the replicator is not able to work. + * 2. Send one message, since the replicator does not work anymore, this message will not be replicated. + * 3. Unload topic, the replicator will be re-created. + * 4. Verify: the message can be replicated to the remote cluster. + */ + // Step 1: Inject an error to make the replicator is not able to work. + Replicator replicator = broker1.getTopic(topicName, false).join().get().getReplicators().get(cluster2); + replicator.terminate(); + + // Step 2: Send one message, since the replicator does not work anymore, this message will not be replicated. + String msg = UUID.randomUUID().toString(); + Producer p1 = client1.newProducer(Schema.STRING).topic(topicName).create(); + p1.send(msg); + p1.close(); + // The result of "peek message" will be the messages generated, so it is not the same as the message just sent. + Thread.sleep(3000); + assertNotEquals(getTheLatestMessage(topicName, client2, admin2), msg); + assertEquals(admin1.topics().getStats(topicName).getReplication().get(cluster2).getReplicationBacklog(), 1); + + // Step 3: Unload topic, the replicator will be re-created. + admin1.topics().unload(topicName); + + // Step 4. Verify: the message can be replicated to the remote cluster. + Awaitility.await().atMost(Duration.ofSeconds(300)).untilAsserted(() -> { + log.info("replication backlog: {}", + admin1.topics().getStats(topicName).getReplication().get(cluster2).getReplicationBacklog()); + assertEquals(admin1.topics().getStats(topicName).getReplication().get(cluster2).getReplicationBacklog(), 0); + assertEquals(getTheLatestMessage(topicName, client2, admin2), msg); + }); + + // Cleanup. + if (replicationLevel.equals(ReplicationLevel.TOPIC_LEVEL)) { + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); + Awaitility.await().untilAsserted(() -> { + assertEquals(broker1.getTopic(topicName, false).join().get().getReplicators().size(), 0); + }); + admin1.topics().delete(topicName, false); + admin2.topics().delete(topicName, false); + } else { + pulsar1.getConfig().setTopicLevelPoliciesEnabled(true); + cleanupTopics(() -> { + admin1.topics().delete(topicName); + admin2.topics().delete(topicName); + }); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index 7372b2e478475..ffe6147412e56 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -350,6 +350,28 @@ protected PulsarClient initClient(ClientBuilder clientBuilder) throws Exception } protected void verifyReplicationWorks(String topic) throws Exception { + // Wait for replicator starting. + Awaitility.await().until(() -> { + try { + PersistentTopic persistentTopic = (PersistentTopic) pulsar1.getBrokerService() + .getTopic(topic, false).join().get(); + if (persistentTopic.getReplicators().size() > 0) { + return true; + } + } catch (Exception ex) {} + + try { + String partition0 = TopicName.get(topic).getPartition(0).toString(); + PersistentTopic persistentTopic = (PersistentTopic) pulsar1.getBrokerService() + .getTopic(partition0, false).join().get(); + if (persistentTopic.getReplicators().size() > 0) { + return true; + } + } catch (Exception ex) {} + + return false; + }); + // Verify: pub & sub. final String subscription = "__subscribe_1"; final String msgValue = "__msg1"; Producer producer1 = client1.newProducer(Schema.STRING).topic(topic).create(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java index b4747a8bd0e47..b8f8edce2477e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java @@ -104,4 +104,9 @@ public void testNoExpandTopicPartitionsWhenDisableTopicLevelReplication() throws public void testExpandTopicPartitionsOnNamespaceLevelReplication() throws Exception { super.testExpandTopicPartitionsOnNamespaceLevelReplication(); } + + @Test(enabled = false) + public void testReloadWithTopicLevelGeoReplication(ReplicationLevel replicationLevel) throws Exception { + super.testReloadWithTopicLevelGeoReplication(replicationLevel); + } } From 5dc030431a60b49e81d577cd06a1ae63dbee0293 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Wed, 19 Jun 2024 17:40:50 +0800 Subject: [PATCH 321/580] [improve] [pip] PIP-358: let resource weight work for OverloadShedder, LeastLongTermMessageRate, ModularLoadManagerImpl. (#22888) --- conf/broker.conf | 5 --- conf/standalone.conf | 5 --- .../impl/LeastLongTermMessageRate.java | 6 ++- .../impl/ModularLoadManagerImpl.java | 28 +++++++++----- .../loadbalance/impl/OverloadShedder.java | 9 ++++- .../ModularLoadManagerStrategyTest.java | 3 ++ .../impl/ModularLoadManagerImplTest.java | 4 ++ .../loadbalance/impl/OverloadShedderTest.java | 38 +++++++++++++++++++ 8 files changed, 76 insertions(+), 22 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 8fd266d609cf4..5c5d8d42817e9 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1394,19 +1394,15 @@ loadBalancerMsgThroughputMultiplierDifferenceShedderThreshold=4 loadBalancerHistoryResourcePercentage=0.9 # The BandWidthIn usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. loadBalancerBandwidthInResourceWeight=1.0 # The BandWidthOut usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. loadBalancerBandwidthOutResourceWeight=1.0 # The CPU usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. loadBalancerCPUResourceWeight=1.0 # The direct memory usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. # Direct memory usage cannot accurately reflect the machine's load, # and it is not recommended to use it to score the machine's load. loadBalancerDirectMemoryResourceWeight=0 @@ -1814,7 +1810,6 @@ strictBookieAffinityEnabled=false # These settings are left here for compatibility # The heap memory usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. # Deprecated: Memory is no longer used as a load balancing item loadBalancerMemoryResourceWeight=0 diff --git a/conf/standalone.conf b/conf/standalone.conf index 6b261ce11c6cd..635b31ac38def 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -911,23 +911,18 @@ loadBalancerBrokerThresholdShedderPercentage=10 loadBalancerHistoryResourcePercentage=0.9 # The BandWidthIn usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. loadBalancerBandwidthInResourceWeight=1.0 # The BandWidthOut usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. loadBalancerBandwidthOutResourceWeight=1.0 # The CPU usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. loadBalancerCPUResourceWeight=1.0 # The heap memory usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. loadBalancerMemoryResourceWeight=0 # The direct memory usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. loadBalancerDirectMemoryResourceWeight=0 # Bundle unload minimum throughput threshold (MB), avoiding bundle unload frequently. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LeastLongTermMessageRate.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LeastLongTermMessageRate.java index fe161467338ff..f51ca797f0edb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LeastLongTermMessageRate.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LeastLongTermMessageRate.java @@ -52,7 +52,11 @@ public LeastLongTermMessageRate() { // Any broker at (or above) the overload threshold will have a score of POSITIVE_INFINITY. private static double getScore(final BrokerData brokerData, final ServiceConfiguration conf) { final double overloadThreshold = conf.getLoadBalancerBrokerOverloadedThresholdPercentage() / 100.0; - final double maxUsage = brokerData.getLocalData().getMaxResourceUsage(); + final double maxUsage = brokerData.getLocalData().getMaxResourceUsageWithWeight( + conf.getLoadBalancerCPUResourceWeight(), + conf.getLoadBalancerDirectMemoryResourceWeight(), + conf.getLoadBalancerBandwidthInResourceWeight(), + conf.getLoadBalancerBandwidthOutResourceWeight()); if (maxUsage > overloadThreshold) { log.warn("Broker {} is overloaded: max usage={}", brokerData.getLocalData().getWebServiceUrl(), maxUsage); return Double.POSITIVE_INFINITY; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index e1259e97aa3e1..3af372607cb16 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -419,6 +419,14 @@ private double percentChange(final double oldValue, final double newValue) { return 100 * Math.abs((oldValue - newValue) / oldValue); } + private double getMaxResourceUsageWithWeight(LocalBrokerData localBrokerData, ServiceConfiguration conf) { + return localBrokerData.getMaxResourceUsageWithWeight( + conf.getLoadBalancerCPUResourceWeight(), + conf.getLoadBalancerDirectMemoryResourceWeight(), + conf.getLoadBalancerBandwidthInResourceWeight(), + conf.getLoadBalancerBandwidthOutResourceWeight()); + } + // Determine if the broker data requires an update by delegating to the update condition. private boolean needBrokerDataUpdate() { final long updateMaxIntervalMillis = TimeUnit.MINUTES @@ -431,14 +439,15 @@ private boolean needBrokerDataUpdate() { // Always update after surpassing the maximum interval. return true; } - final double maxChange = Math - .max(100.0 * (Math.abs(lastData.getMaxResourceUsage() - localData.getMaxResourceUsage())), - Math.max(percentChange(lastData.getMsgRateIn() + lastData.getMsgRateOut(), - localData.getMsgRateIn() + localData.getMsgRateOut()), - Math.max( - percentChange(lastData.getMsgThroughputIn() + lastData.getMsgThroughputOut(), - localData.getMsgThroughputIn() + localData.getMsgThroughputOut()), - percentChange(lastData.getNumBundles(), localData.getNumBundles())))); + final double maxChange = LocalBrokerData.max( + percentChange(lastData.getMsgRateIn() + lastData.getMsgRateOut(), + localData.getMsgRateIn() + localData.getMsgRateOut()), + percentChange(lastData.getMsgThroughputIn() + lastData.getMsgThroughputOut(), + localData.getMsgThroughputIn() + localData.getMsgThroughputOut()), + percentChange(lastData.getNumBundles(), localData.getNumBundles()), + 100.0 * Math.abs(getMaxResourceUsageWithWeight(lastData, conf) + - getMaxResourceUsageWithWeight(localData, conf)) + ); if (maxChange > conf.getLoadBalancerReportUpdateThresholdPercentage()) { log.info("Writing local data to metadata store because maximum change {}% exceeded threshold {}%; " + "time since last report written is {} seconds", maxChange, @@ -927,7 +936,8 @@ Optional selectBroker(final ServiceUnitId serviceUnit) { } final double overloadThreshold = conf.getLoadBalancerBrokerOverloadedThresholdPercentage() / 100.0; - final double maxUsage = loadData.getBrokerData().get(broker.get()).getLocalData().getMaxResourceUsage(); + final double maxUsage = getMaxResourceUsageWithWeight( + loadData.getBrokerData().get(broker.get()).getLocalData(), conf); if (maxUsage > overloadThreshold) { // All brokers that were in the filtered list were overloaded, so check if there is a better broker LoadManagerShared.applyNamespacePolicies(serviceUnit, policies, brokerCandidateCache, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/OverloadShedder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/OverloadShedder.java index a4eb5077224ce..fb31548227b31 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/OverloadShedder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/OverloadShedder.java @@ -36,7 +36,8 @@ /** * Load shedding strategy which will attempt to shed exactly one bundle on brokers which are overloaded, that is, whose * maximum system resource usage exceeds loadBalancerBrokerOverloadedThresholdPercentage. To see which resources are - * considered when determining the maximum system resource, see {@link LocalBrokerData#getMaxResourceUsage()}. A bundle + * considered when determining the maximum system resource, see + * {@link LocalBrokerData#getMaxResourceUsageWithWeight(double, double, double, double)}. A bundle * is recommended for unloading off that broker if and only if the following conditions hold: The broker has at * least two bundles assigned and the broker has at least one bundle that has not been unloaded recently according to * LoadBalancerSheddingGracePeriodMinutes. The unloaded bundle will be the most expensive bundle in terms of message @@ -71,7 +72,11 @@ public Multimap findBundlesForUnloading(final LoadData loadData, loadData.getBrokerData().forEach((broker, brokerData) -> { final LocalBrokerData localData = brokerData.getLocalData(); - final double currentUsage = localData.getMaxResourceUsage(); + final double currentUsage = localData.getMaxResourceUsageWithWeight( + conf.getLoadBalancerCPUResourceWeight(), + conf.getLoadBalancerDirectMemoryResourceWeight(), + conf.getLoadBalancerBandwidthInResourceWeight(), + conf.getLoadBalancerBandwidthOutResourceWeight()); if (currentUsage < overloadThreshold) { if (log.isDebugEnabled()) { log.debug("[{}] Broker is not overloaded, ignoring at this point ({})", broker, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java index 2b6bfb742eb04..f5bd0f46a5ec1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java @@ -68,6 +68,9 @@ public void testLeastLongTermMessageRate() { assertEquals(strategy.selectBroker(brokerDataMap.keySet(), bundleData, loadData, conf), Optional.of("2")); brokerData2.getLocalData().setCpu(new ResourceUsage(90, 100)); assertEquals(strategy.selectBroker(brokerDataMap.keySet(), bundleData, loadData, conf), Optional.of("3")); + // disable considering cpu usage to avoid broker2 being overloaded. + conf.setLoadBalancerCPUResourceWeight(0); + assertEquals(strategy.selectBroker(brokerDataMap.keySet(), bundleData, loadData, conf), Optional.of("2")); } // Test that least resource usage with weight works correctly. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java index 6ae491c55b845..68bef8b241c7b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java @@ -684,6 +684,10 @@ public void testNeedBrokerDataUpdate() throws Exception { currentData.setCpu(new ResourceUsage(206, 1000)); assert (needUpdate.get()); + // set the resource weight of cpu to 0, so that it should not trigger an update + conf.setLoadBalancerCPUResourceWeight(0); + assert (!needUpdate.get()); + lastData.setCpu(new ResourceUsage()); currentData.setCpu(new ResourceUsage()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/OverloadShedderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/OverloadShedderTest.java index 1f0962f2e44c3..a05c5d4a9e0c2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/OverloadShedderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/OverloadShedderTest.java @@ -149,6 +149,44 @@ public void testBrokerWithMultipleBundles() { assertEquals(bundlesToUnload.get("broker-1"), List.of("bundle-10", "bundle-9")); } + @Test + public void testBrokerWithResourceWeight() { + int numBundles = 10; + LoadData loadData = new LoadData(); + LocalBrokerData broker1 = new LocalBrokerData(); + + double brokerThroghput = 0; + for (int i = 1; i <= numBundles; i++) { + broker1.getBundles().add("bundle-" + i); + BundleData bundle = new BundleData(); + TimeAverageMessageData db = new TimeAverageMessageData(); + double throughput = i * 1024 * 1024; + db.setMsgThroughputIn(throughput); + db.setMsgThroughputOut(throughput); + bundle.setShortTermData(db); + loadData.getBundleData().put("bundle-" + i, bundle); + brokerThroghput += throughput; + } + broker1.setMsgThroughputIn(brokerThroghput); + broker1.setMsgThroughputOut(brokerThroghput); + + loadData.getBrokerData().put("broker-1", new BrokerData(broker1)); + + // set bandwidth usage to 99.9% so that it is considered overloaded + broker1.setBandwidthIn(new ResourceUsage(999, 1000)); + broker1.setBandwidthOut(new ResourceUsage(999, 1000)); + assertFalse(os.findBundlesForUnloading(loadData, conf).isEmpty()); + + // set bandwidth resource weight to 0 so that it is not considered overloaded + conf.setLoadBalancerBandwidthInResourceWeight(0); + conf.setLoadBalancerBandwidthOutResourceWeight(0); + assertTrue(os.findBundlesForUnloading(loadData, conf).isEmpty()); + + // set bandwidth resource weight back to 1, or it will affect other tests + conf.setLoadBalancerBandwidthInResourceWeight(1); + conf.setLoadBalancerBandwidthOutResourceWeight(1); + } + @Test public void testFilterRecentlyUnloaded() { int numBundles = 10; From e0f545ab0583daf83430ef7a8c6d619c70fe1642 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 19 Jun 2024 17:05:39 +0300 Subject: [PATCH 322/580] [fix][cli] Refactor scripts to detect major Java version and pass correct add-opens parameters (#22927) --- bin/bookkeeper | 40 +++++++++++++++------- bin/function-localrunner | 51 +++++++++++++++++++--------- bin/pulsar | 55 ++++++++++++++++++++----------- bin/pulsar-admin-common.cmd | 46 ++++++++++++++++---------- bin/pulsar-admin-common.sh | 32 +++++++++++++++--- bin/pulsar-admin.cmd | 5 +-- bin/pulsar-client.cmd | 5 +-- bin/pulsar-perf | 17 +++++++--- bin/pulsar-perf.cmd | 66 +++++-------------------------------- bin/pulsar-shell.cmd | 6 ++-- 10 files changed, 188 insertions(+), 135 deletions(-) diff --git a/bin/bookkeeper b/bin/bookkeeper index 0cc07dd49aba5..668c5d4db70a8 100755 --- a/bin/bookkeeper +++ b/bin/bookkeeper @@ -69,6 +69,29 @@ else JAVA=$JAVA_HOME/bin/java fi +# JAVA_MAJOR_VERSION should get set by conf/bkenv.sh, just in case it's not +if [[ -z $JAVA_MAJOR_VERSION ]]; then + for token in $("$JAVA" -version 2>&1 | grep 'version "'); do + if [[ $token =~ \"([[:digit:]]+)\.([[:digit:]]+)(.*)\" ]]; then + if [[ ${BASH_REMATCH[1]} == "1" ]]; then + JAVA_MAJOR_VERSION=${BASH_REMATCH[2]} + else + JAVA_MAJOR_VERSION=${BASH_REMATCH[1]} + fi + break + elif [[ $token =~ \"([[:digit:]]+)(.*)\" ]]; then + # Process the java versions without dots, such as `17-internal`. + JAVA_MAJOR_VERSION=${BASH_REMATCH[1]} + break + fi + done +fi + +if [[ $JAVA_MAJOR_VERSION -lt 17 ]]; then + echo "Error: Bookkeeper included in Pulsar requires Java 17 or later." 1>&2 + exit 1 +fi + # exclude tests jar RELEASE_JAR=`ls $BK_HOME/bookkeeper-server-*.jar 2> /dev/null | grep -v tests | tail -1` if [ $? == 0 ]; then @@ -168,17 +191,12 @@ OPTS="$OPTS -Dlog4j.configurationFile=`basename $BOOKIE_LOG_CONF`" # Allow Netty to use reflection access OPTS="$OPTS -Dio.netty.tryReflectionSetAccessible=true" -IS_JAVA_8=$( $JAVA -version 2>&1 | grep version | grep '"1\.8' ) -# Start --add-opens options -# '--add-opens' option is not supported in jdk8 -if [[ -z "$IS_JAVA_8" ]]; then - # BookKeeper: enable posix_fadvise usage and DirectMemoryCRC32Digest (https://github.com/apache/bookkeeper/pull/3234) - OPTS="$OPTS --add-opens java.base/java.io=ALL-UNNAMED --add-opens java.base/java.util.zip=ALL-UNNAMED" - # Netty: enable java.nio.DirectByteBuffer - # https://github.com/netty/netty/blob/4.1/common/src/main/java/io/netty/util/internal/PlatformDependent0.java - # https://github.com/netty/netty/issues/12265 - OPTS="$OPTS --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.misc=ALL-UNNAMED" -fi +# BookKeeper: enable posix_fadvise usage and DirectMemoryCRC32Digest (https://github.com/apache/bookkeeper/pull/3234) +OPTS="$OPTS --add-opens java.base/java.io=ALL-UNNAMED --add-opens java.base/java.util.zip=ALL-UNNAMED" +# Netty: enable java.nio.DirectByteBuffer +# https://github.com/netty/netty/blob/4.1/common/src/main/java/io/netty/util/internal/PlatformDependent0.java +# https://github.com/netty/netty/issues/12265 +OPTS="$OPTS --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.misc=ALL-UNNAMED" OPTS="-cp $BOOKIE_CLASSPATH $OPTS" diff --git a/bin/function-localrunner b/bin/function-localrunner index 2e0aa0f6dffe2..b2405db724e72 100755 --- a/bin/function-localrunner +++ b/bin/function-localrunner @@ -34,21 +34,39 @@ else JAVA=$JAVA_HOME/bin/java fi +for token in $("$JAVA" -version 2>&1 | grep 'version "'); do + if [[ $token =~ \"([[:digit:]]+)\.([[:digit:]]+)(.*)\" ]]; then + if [[ ${BASH_REMATCH[1]} == "1" ]]; then + JAVA_MAJOR_VERSION=${BASH_REMATCH[2]} + else + JAVA_MAJOR_VERSION=${BASH_REMATCH[1]} + fi + break + elif [[ $token =~ \"([[:digit:]]+)(.*)\" ]]; then + # Process the java versions without dots, such as `17-internal`. + JAVA_MAJOR_VERSION=${BASH_REMATCH[1]} + break + fi +done + PULSAR_MEM=${PULSAR_MEM:-"-Xmx128m -XX:MaxDirectMemorySize=128m"} # Garbage collection options PULSAR_GC=${PULSAR_GC:-"-XX:+UseZGC -XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch"} # Garbage collection log. -IS_JAVA_8=$( $JAVA -version 2>&1 | grep version | grep '"1\.8' ) -if [[ -z "$IS_JAVA_8" ]]; then - # >= JDK 9 - PULSAR_GC_LOG=${PULSAR_GC_LOG:-"-Xlog:gc:logs/pulsar_gc_%p.log:time,uptime:filecount=10,filesize=20M"} - # '--add-opens' option is not supported in JDK 1.8 - OPTS="$OPTS --add-opens java.base/sun.net=ALL-UNNAMED --add-opens java.base/java.lang=ALL-UNNAMED" -else - # == JDK 1.8 - PULSAR_GC_LOG=${PULSAR_GC_LOG:-"-Xloggc:logs/pulsar_gc_%p.log -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=20M"} +PULSAR_GC_LOG_DIR=${PULSAR_GC_LOG_DIR:-logs} +if [[ -z "$PULSAR_GC_LOG" ]]; then + if [[ $JAVA_MAJOR_VERSION -gt 8 ]]; then + PULSAR_GC_LOG="-Xlog:gc*,safepoint:${PULSAR_GC_LOG_DIR}/pulsar_gc_%p.log:time,uptime,tags:filecount=10,filesize=20M" + if [[ $JAVA_MAJOR_VERSION -ge 17 ]]; then + # Use async logging on Java 17+ https://bugs.openjdk.java.net/browse/JDK-8264323 + PULSAR_GC_LOG="-Xlog:async ${PULSAR_GC_LOG}" + fi + else + # Java 8 gc log options + PULSAR_GC_LOG="-Xloggc:${PULSAR_GC_LOG_DIR}/pulsar_gc_%p.log -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCApplicationStoppedTime -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=20M" + fi fi # Extra options to be passed to the jvm @@ -90,13 +108,16 @@ OPTS="$OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF`" # Allow Netty to use reflection access OPTS="$OPTS -Dio.netty.tryReflectionSetAccessible=true" +OPTS="$OPTS -Dorg.apache.pulsar.shade.io.netty.tryReflectionSetAccessible=true" + +if [[ $JAVA_MAJOR_VERSION -gt 8 ]]; then + # Required by Pulsar client optimized checksum calculation on other than Linux x86_64 platforms + # reflection access to java.util.zip.CRC32C + OPTS="$OPTS --add-opens java.base/java.util.zip=ALL-UNNAMED" +fi -# Start --add-opens options -# '--add-opens' option is not supported in jdk8 -if [[ -z "$IS_JAVA_8" ]]; then - # Netty: enable java.nio.DirectByteBuffer - # https://github.com/netty/netty/blob/4.1/common/src/main/java/io/netty/util/internal/PlatformDependent0.java - # https://github.com/netty/netty/issues/12265 +if [[ $JAVA_MAJOR_VERSION -ge 11 ]]; then + # Required by Netty for optimized direct byte buffer access OPTS="$OPTS --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.misc=ALL-UNNAMED" fi diff --git a/bin/pulsar b/bin/pulsar index ab0029af5b0da..09be2ac50e279 100755 --- a/bin/pulsar +++ b/bin/pulsar @@ -129,6 +129,29 @@ else JAVA=$JAVA_HOME/bin/java fi +# JAVA_MAJOR_VERSION should get set by conf/pulsar_env.sh, just in case it's not +if [[ -z $JAVA_MAJOR_VERSION ]]; then + for token in $("$JAVA" -version 2>&1 | grep 'version "'); do + if [[ $token =~ \"([[:digit:]]+)\.([[:digit:]]+)(.*)\" ]]; then + if [[ ${BASH_REMATCH[1]} == "1" ]]; then + JAVA_MAJOR_VERSION=${BASH_REMATCH[2]} + else + JAVA_MAJOR_VERSION=${BASH_REMATCH[1]} + fi + break + elif [[ $token =~ \"([[:digit:]]+)(.*)\" ]]; then + # Process the java versions without dots, such as `17-internal`. + JAVA_MAJOR_VERSION=${BASH_REMATCH[1]} + break + fi + done +fi + +if [[ $JAVA_MAJOR_VERSION -lt 17 ]]; then + echo "Error: Pulsar requires Java 17 or later." 1>&2 + exit 1 +fi + # exclude tests jar RELEASE_JAR=`ls $PULSAR_HOME/pulsar-*.jar 2> /dev/null | grep -v tests | tail -1` if [ $? == 0 ]; then @@ -254,27 +277,21 @@ OPTS="$OPTS -Djute.maxbuffer=10485760 -Djava.net.preferIPv4Stack=true" # Enable TCP keepalive for all Zookeeper client connections OPTS="$OPTS -Dzookeeper.clientTcpKeepAlive=true" +# BookKeeper: enable posix_fadvise usage and DirectMemoryCRC32Digest (https://github.com/apache/bookkeeper/pull/3234) +OPTS="$OPTS --add-opens java.base/java.io=ALL-UNNAMED --add-opens java.base/java.util.zip=ALL-UNNAMED" +# Required by JvmDefaultGCMetricsLogger & MBeanStatsGenerator +OPTS="$OPTS --add-opens java.management/sun.management=ALL-UNNAMED" +# Required by MBeanStatsGenerator +OPTS="$OPTS --add-opens jdk.management/com.sun.management.internal=ALL-UNNAMED" # Allow Netty to use reflection access OPTS="$OPTS -Dio.netty.tryReflectionSetAccessible=true" -IS_JAVA_8=$( $JAVA -version 2>&1 | grep version | grep '"1\.8' ) -# Start --add-opens options -# '--add-opens' option is not supported in jdk8 -if [[ -z "$IS_JAVA_8" ]]; then - # BookKeeper: enable posix_fadvise usage and DirectMemoryCRC32Digest (https://github.com/apache/bookkeeper/pull/3234) - OPTS="$OPTS --add-opens java.base/java.io=ALL-UNNAMED --add-opens java.base/java.util.zip=ALL-UNNAMED" - # Netty: enable java.nio.DirectByteBuffer - # https://github.com/netty/netty/blob/4.1/common/src/main/java/io/netty/util/internal/PlatformDependent0.java - # https://github.com/netty/netty/issues/12265 - OPTS="$OPTS --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.misc=ALL-UNNAMED" - # netty.DnsResolverUtil - OPTS="$OPTS --add-opens java.base/sun.net=ALL-UNNAMED" - # JvmDefaultGCMetricsLogger & MBeanStatsGenerator - OPTS="$OPTS --add-opens java.management/sun.management=ALL-UNNAMED" - # MBeanStatsGenerator - OPTS="$OPTS --add-opens jdk.management/com.sun.management.internal=ALL-UNNAMED" - # LinuxInfoUtils - OPTS="$OPTS --add-opens java.base/jdk.internal.platform=ALL-UNNAMED" -fi +OPTS="$OPTS -Dorg.apache.pulsar.shade.io.netty.tryReflectionSetAccessible=true" +# Netty: enable java.nio.DirectByteBuffer +# https://github.com/netty/netty/blob/4.1/common/src/main/java/io/netty/util/internal/PlatformDependent0.java +# https://github.com/netty/netty/issues/12265 +OPTS="$OPTS --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.misc=ALL-UNNAMED" +# Required by LinuxInfoUtils +OPTS="$OPTS --add-opens java.base/jdk.internal.platform=ALL-UNNAMED" OPTS="-cp $PULSAR_CLASSPATH $OPTS" diff --git a/bin/pulsar-admin-common.cmd b/bin/pulsar-admin-common.cmd index c52bc1389f68a..c59f0e9b424d3 100644 --- a/bin/pulsar-admin-common.cmd +++ b/bin/pulsar-admin-common.cmd @@ -19,7 +19,7 @@ @echo off -if "%JAVA_HOME%" == "" ( +if not defined JAVA_HOME ( for %%i in (java.exe) do set "JAVACMD=%%~$PATH:i" ) else ( set "JAVACMD=%JAVA_HOME%\bin\java.exe" @@ -28,16 +28,28 @@ if "%JAVA_HOME%" == "" ( if not exist "%JAVACMD%" ( echo The JAVA_HOME environment variable is not defined correctly, so Pulsar CLI cannot be started. >&2 echo JAVA_HOME is set to "%JAVA_HOME%", but "%JAVACMD%" does not exist. >&2 - goto error + exit /b 1 ) +set JAVA_MAJOR_VERSION=0 +REM Requires "setlocal enabledelayedexpansion" to work +for /f tokens^=3 %%g in ('"!JAVACMD!" -version 2^>^&1 ^| findstr /i version') do ( + set JAVA_MAJOR_VERSION=%%g +) +set JAVA_MAJOR_VERSION=%JAVA_MAJOR_VERSION:"=% +for /f "delims=.-_ tokens=1-2" %%v in ("%JAVA_MAJOR_VERSION%") do ( + if /I "%%v" EQU "1" ( + set JAVA_MAJOR_VERSION=%%w + ) else ( + set JAVA_MAJOR_VERSION=%%v + ) +) for %%i in ("%~dp0.") do SET "SCRIPT_PATH=%%~fi" set "PULSAR_HOME_DIR=%SCRIPT_PATH%\..\" for %%i in ("%PULSAR_HOME_DIR%.") do SET "PULSAR_HOME=%%~fi" set "PULSAR_CLASSPATH=%PULSAR_CLASSPATH%;%PULSAR_HOME%\lib\*" - if "%PULSAR_CLIENT_CONF%" == "" set "PULSAR_CLIENT_CONF=%PULSAR_HOME%\conf\client.conf" if "%PULSAR_LOG_CONF%" == "" set "PULSAR_LOG_CONF=%PULSAR_HOME%\conf\log4j2.yaml" @@ -50,18 +62,21 @@ set "PULSAR_CLASSPATH=%PULSAR_CLASSPATH%;%PULSAR_LOG_CONF_DIR%" set "OPTS=%OPTS% -Dlog4j.configurationFile="%PULSAR_LOG_CONF_BASENAME%"" set "OPTS=%OPTS% -Djava.net.preferIPv4Stack=true" -set "isjava8=false" -FOR /F "tokens=*" %%g IN ('"java -version 2>&1"') do ( - echo %%g|find "version" >nul - if errorlevel 0 ( - echo %%g|find "1.8" >nul - if errorlevel 0 ( - set "isjava8=true" - ) - ) +REM Allow Netty to use reflection access +set "OPTS=%OPTS% -Dio.netty.tryReflectionSetAccessible=true" +set "OPTS=%OPTS% -Dorg.apache.pulsar.shade.io.netty.tryReflectionSetAccessible=true" + +if %JAVA_MAJOR_VERSION% GTR 8 ( + set "OPTS=%OPTS% --add-opens java.base/sun.net=ALL-UNNAMED --add-opens java.base/java.lang=ALL-UNNAMED" + REM Required by Pulsar client optimized checksum calculation on other than Linux x86_64 platforms + REM reflection access to java.util.zip.CRC32C + set "OPTS=%OPTS% --add-opens java.base/java.util.zip=ALL-UNNAMED" ) -if "%isjava8%" == "false" set "OPTS=%OPTS% --add-opens java.base/sun.net=ALL-UNNAMED" +if %JAVA_MAJOR_VERSION% GEQ 11 ( + REM Required by Netty for optimized direct byte buffer access + set "OPTS=%OPTS% --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.misc=ALL-UNNAMED" +) set "OPTS=-cp "%PULSAR_CLASSPATH%" %OPTS%" set "OPTS=%OPTS% %PULSAR_EXTRA_OPTS%" @@ -78,7 +93,4 @@ set "OPTS=%OPTS% -Dpulsar.log.dir=%PULSAR_LOG_DIR%" set "OPTS=%OPTS% -Dpulsar.log.level=%PULSAR_LOG_LEVEL%" set "OPTS=%OPTS% -Dpulsar.log.root.level=%PULSAR_LOG_ROOT_LEVEL%" set "OPTS=%OPTS% -Dpulsar.log.immediateFlush=%PULSAR_LOG_IMMEDIATE_FLUSH%" -set "OPTS=%OPTS% -Dpulsar.routing.appender.default=%PULSAR_ROUTING_APPENDER_DEFAULT%" - -:error -exit /b 1 +set "OPTS=%OPTS% -Dpulsar.routing.appender.default=%PULSAR_ROUTING_APPENDER_DEFAULT%" \ No newline at end of file diff --git a/bin/pulsar-admin-common.sh b/bin/pulsar-admin-common.sh index 8aa21c00f634d..336ff43c1a861 100755 --- a/bin/pulsar-admin-common.sh +++ b/bin/pulsar-admin-common.sh @@ -37,6 +37,21 @@ else JAVA=$JAVA_HOME/bin/java fi +for token in $("$JAVA" -version 2>&1 | grep 'version "'); do + if [[ $token =~ \"([[:digit:]]+)\.([[:digit:]]+)(.*)\" ]]; then + if [[ ${BASH_REMATCH[1]} == "1" ]]; then + JAVA_MAJOR_VERSION=${BASH_REMATCH[2]} + else + JAVA_MAJOR_VERSION=${BASH_REMATCH[1]} + fi + break + elif [[ $token =~ \"([[:digit:]]+)(.*)\" ]]; then + # Process the java versions without dots, such as `17-internal`. + JAVA_MAJOR_VERSION=${BASH_REMATCH[1]} + break + fi +done + # exclude tests jar RELEASE_JAR=`ls $PULSAR_HOME/pulsar-*.jar 2> /dev/null | grep -v tests | tail -1` if [ $? == 0 ]; then @@ -91,11 +106,20 @@ PULSAR_CLASSPATH="`dirname $PULSAR_LOG_CONF`:$PULSAR_CLASSPATH" OPTS="$OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF`" OPTS="$OPTS -Djava.net.preferIPv4Stack=true" -IS_JAVA_8=$( $JAVA -version 2>&1 | grep version | grep '"1\.8' ) -# Start --add-opens options -# '--add-opens' option is not supported in jdk8 -if [[ -z "$IS_JAVA_8" ]]; then +# Allow Netty to use reflection access +OPTS="$OPTS -Dio.netty.tryReflectionSetAccessible=true" +OPTS="$OPTS -Dorg.apache.pulsar.shade.io.netty.tryReflectionSetAccessible=true" + +if [[ $JAVA_MAJOR_VERSION -gt 8 ]]; then OPTS="$OPTS --add-opens java.base/sun.net=ALL-UNNAMED --add-opens java.base/java.lang=ALL-UNNAMED" + # Required by Pulsar client optimized checksum calculation on other than Linux x86_64 platforms + # reflection access to java.util.zip.CRC32C + OPTS="$OPTS --add-opens java.base/java.util.zip=ALL-UNNAMED" +fi + +if [[ $JAVA_MAJOR_VERSION -ge 11 ]]; then + # Required by Netty for optimized direct byte buffer access + OPTS="$OPTS --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.misc=ALL-UNNAMED" fi OPTS="-cp $PULSAR_CLASSPATH $OPTS" diff --git a/bin/pulsar-admin.cmd b/bin/pulsar-admin.cmd index 45bd8d4541fed..e29d804d70c45 100644 --- a/bin/pulsar-admin.cmd +++ b/bin/pulsar-admin.cmd @@ -18,7 +18,7 @@ @REM @echo off - +setlocal enabledelayedexpansion for %%i in ("%~dp0.") do SET "SCRIPT_PATH=%%~fi" set "PULSAR_HOME_DIR=%SCRIPT_PATH%\..\" for %%i in ("%PULSAR_HOME_DIR%.") do SET "PULSAR_HOME=%%~fi" @@ -27,4 +27,5 @@ if ERRORLEVEL 1 ( exit /b 1 ) cd "%PULSAR_HOME%" -"%JAVACMD%" %OPTS% org.apache.pulsar.admin.cli.PulsarAdminTool %PULSAR_CLIENT_CONF% %* \ No newline at end of file +"%JAVACMD%" %OPTS% org.apache.pulsar.admin.cli.PulsarAdminTool %PULSAR_CLIENT_CONF% %* +endlocal \ No newline at end of file diff --git a/bin/pulsar-client.cmd b/bin/pulsar-client.cmd index 9e3cef45a25a7..9cacf71cc3a79 100644 --- a/bin/pulsar-client.cmd +++ b/bin/pulsar-client.cmd @@ -18,7 +18,7 @@ @REM @echo off - +setlocal enabledelayedexpansion for %%i in ("%~dp0.") do SET "SCRIPT_PATH=%%~fi" set "PULSAR_HOME_DIR=%SCRIPT_PATH%\..\" for %%i in ("%PULSAR_HOME_DIR%.") do SET "PULSAR_HOME=%%~fi" @@ -27,4 +27,5 @@ if ERRORLEVEL 1 ( exit /b 1 ) cd "%PULSAR_HOME%" -"%JAVACMD%" %OPTS% org.apache.pulsar.client.cli.PulsarClientTool %PULSAR_CLIENT_CONF% %* \ No newline at end of file +"%JAVACMD%" %OPTS% org.apache.pulsar.client.cli.PulsarClientTool %PULSAR_CLIENT_CONF% %* +endlocal \ No newline at end of file diff --git a/bin/pulsar-perf b/bin/pulsar-perf index 1f6ce97476b4e..9108a42ef994f 100755 --- a/bin/pulsar-perf +++ b/bin/pulsar-perf @@ -103,11 +103,20 @@ PULSAR_CLASSPATH="$PULSAR_JAR:$PULSAR_CLASSPATH:$PULSAR_EXTRA_CLASSPATH" PULSAR_CLASSPATH="`dirname $PULSAR_LOG_CONF`:$PULSAR_CLASSPATH" OPTS="$OPTS -Dlog4j.configurationFile=`basename $PULSAR_LOG_CONF` -Djava.net.preferIPv4Stack=true" -IS_JAVA_8=$( $JAVA -version 2>&1 | grep version | grep '"1\.8' ) -# Start --add-opens options -# '--add-opens' option is not supported in jdk8 -if [[ -z "$IS_JAVA_8" ]]; then +# Allow Netty to use reflection access +OPTS="$OPTS -Dio.netty.tryReflectionSetAccessible=true" +OPTS="$OPTS -Dorg.apache.pulsar.shade.io.netty.tryReflectionSetAccessible=true" + +if [[ $JAVA_MAJOR_VERSION -gt 8 ]]; then OPTS="$OPTS --add-opens java.base/sun.net=ALL-UNNAMED --add-opens java.base/java.lang=ALL-UNNAMED" + # Required by Pulsar client optimized checksum calculation on other than Linux x86_64 platforms + # reflection access to java.util.zip.CRC32C + OPTS="$OPTS --add-opens java.base/java.util.zip=ALL-UNNAMED" +fi + +if [[ $JAVA_MAJOR_VERSION -ge 11 ]]; then + # Required by Netty for optimized direct byte buffer access + OPTS="$OPTS --add-opens java.base/java.nio=ALL-UNNAMED --add-opens java.base/jdk.internal.misc=ALL-UNNAMED" fi OPTS="-cp $PULSAR_CLASSPATH $OPTS" diff --git a/bin/pulsar-perf.cmd b/bin/pulsar-perf.cmd index f2b33ef6eb88e..aaeaa7a21856b 100644 --- a/bin/pulsar-perf.cmd +++ b/bin/pulsar-perf.cmd @@ -18,67 +18,17 @@ @REM @echo off - -if "%JAVA_HOME%" == "" ( - for %%i in (java.exe) do set "JAVACMD=%%~$PATH:i" -) else ( - set "JAVACMD=%JAVA_HOME%\bin\java.exe" -) - -if not exist "%JAVACMD%" ( - echo The JAVA_HOME environment variable is not defined correctly, so Pulsar CLI cannot be started. >&2 - echo JAVA_HOME is set to "%JAVA_HOME%", but "%JAVACMD%" does not exist. >&2 - exit /B 1 -) - +setlocal enabledelayedexpansion for %%i in ("%~dp0.") do SET "SCRIPT_PATH=%%~fi" set "PULSAR_HOME_DIR=%SCRIPT_PATH%\..\" for %%i in ("%PULSAR_HOME_DIR%.") do SET "PULSAR_HOME=%%~fi" -set "PULSAR_CLASSPATH=%PULSAR_CLASSPATH%;%PULSAR_HOME%\lib\*" - - -if "%PULSAR_CLIENT_CONF%" == "" set "PULSAR_CLIENT_CONF=%PULSAR_HOME%\conf\client.conf" -if "%PULSAR_LOG_CONF%" == "" set "PULSAR_LOG_CONF=%PULSAR_HOME%\conf\log4j2.yaml" - -set "PULSAR_LOG_CONF_DIR1=%PULSAR_LOG_CONF%\..\" -for %%i in ("%PULSAR_LOG_CONF_DIR1%.") do SET "PULSAR_LOG_CONF_DIR=%%~fi" -for %%a in ("%PULSAR_LOG_CONF%") do SET "PULSAR_LOG_CONF_BASENAME=%%~nxa" - -set "PULSAR_CLASSPATH=%PULSAR_CLASSPATH%;%PULSAR_LOG_CONF_DIR%" -if not "%PULSAR_EXTRA_CLASSPATH%" == "" set "PULSAR_CLASSPATH=%PULSAR_CLASSPATH%;%PULSAR_EXTRA_CLASSPATH%" - - -if "%PULSAR_PERFTEST_CONF%" == "" set "PULSAR_PERFTEST_CONF=%PULSAR_CLIENT_CONF%" - - -set "OPTS=%OPTS% -Dlog4j.configurationFile="%PULSAR_LOG_CONF_BASENAME%"" -set "OPTS=%OPTS% -Djava.net.preferIPv4Stack=true" - - -set "OPTS=-cp "%PULSAR_CLASSPATH%" %OPTS%" -set "OPTS=%OPTS% %PULSAR_EXTRA_OPTS%" - -if "%PULSAR_LOG_DIR%" == "" set "PULSAR_LOG_DIR=%PULSAR_HOME%\logs" -if "%PULSAR_LOG_FILE%" == "" set "PULSAR_LOG_FILE=pulsar-perftest.log" if "%PULSAR_LOG_APPENDER%" == "" set "PULSAR_LOG_APPENDER=Console" -if "%PULSAR_LOG_LEVEL%" == "" set "PULSAR_LOG_LEVEL=info" -if "%PULSAR_LOG_ROOT_LEVEL%" == "" set "PULSAR_LOG_ROOT_LEVEL=%PULSAR_LOG_LEVEL%" -if "%PULSAR_LOG_IMMEDIATE_FLUSH%" == "" set "PULSAR_LOG_IMMEDIATE_FLUSH=false" - - -set "OPTS=%OPTS% -Dpulsar.log.appender=%PULSAR_LOG_APPENDER%" -set "OPTS=%OPTS% -Dpulsar.log.dir=%PULSAR_LOG_DIR%" -set "OPTS=%OPTS% -Dpulsar.log.level=%PULSAR_LOG_LEVEL%" -set "OPTS=%OPTS% -Dpulsar.log.root.level=%PULSAR_LOG_ROOT_LEVEL%" -set "OPTS=%OPTS% -Dpulsar.log.immediateFlush=%PULSAR_LOG_IMMEDIATE_FLUSH%" - +if "%PULSAR_LOG_FILE%" == "" set "PULSAR_LOG_FILE=pulsar-perftest.log" +call "%PULSAR_HOME%\bin\pulsar-admin-common.cmd" +if ERRORLEVEL 1 ( + exit /b 1 +) +if "%PULSAR_PERFTEST_CONF%" == "" set "PULSAR_PERFTEST_CONF=%PULSAR_CLIENT_CONF%" "%JAVACMD%" %OPTS% org.apache.pulsar.testclient.PulsarPerfTestTool "%PULSAR_PERFTEST_CONF%" %* - - - -:error -set ERROR_CODE=1 -goto :eof - - +endlocal diff --git a/bin/pulsar-shell.cmd b/bin/pulsar-shell.cmd index c339d34289572..615408f9c7a6e 100644 --- a/bin/pulsar-shell.cmd +++ b/bin/pulsar-shell.cmd @@ -18,7 +18,7 @@ @REM @echo off - +setlocal enabledelayedexpansion for %%i in ("%~dp0.") do SET "SCRIPT_PATH=%%~fi" set "PULSAR_HOME_DIR=%SCRIPT_PATH%\..\" for %%i in ("%PULSAR_HOME_DIR%.") do SET "PULSAR_HOME=%%~fi" @@ -26,9 +26,9 @@ call "%PULSAR_HOME%\bin\pulsar-admin-common.cmd" if ERRORLEVEL 1 ( exit /b 1 ) - set "OPTS=%OPTS% -Dorg.jline.terminal.jansi=false" set "OPTS=%OPTS% -Dpulsar.shell.config.default=%cd%" set "DEFAULT_CONFIG=-Dpulsar.shell.config.default="%PULSAR_CLIENT_CONF%"" cd "%PULSAR_HOME%" -"%JAVACMD%" %OPTS% %DEFAULT_CONFIG% org.apache.pulsar.shell.PulsarShell %* \ No newline at end of file +"%JAVACMD%" %OPTS% %DEFAULT_CONFIG% org.apache.pulsar.shell.PulsarShell %* +endlocal \ No newline at end of file From 5fc0eafab9ea2a4ece7b87218404489c270b64e6 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 19 Jun 2024 22:29:17 +0800 Subject: [PATCH 323/580] [improve] [broker] PIP-356 Support Geo-Replication starts at earliest position (#22856) --- .../pulsar/broker/ServiceConfiguration.java | 6 ++ .../service/persistent/PersistentTopic.java | 9 +- .../broker/service/OneWayReplicatorTest.java | 99 +++++++++++++++++++ .../OneWayReplicatorUsingGlobalZKTest.java | 52 ++++++++++ 4 files changed, 165 insertions(+), 1 deletion(-) 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 63ef6f3efe6d0..73bf2316b8287 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 @@ -1345,6 +1345,12 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "Max number of snapshot to be cached per subscription.") private int replicatedSubscriptionsSnapshotMaxCachedPerSubscription = 10; + @FieldContext( + category = CATEGORY_SERVER, + dynamic = true, + doc = "The position that replication task start at, it can be set to earliest or latest (default).") + private String replicationStartAt = "latest"; + @FieldContext( category = CATEGORY_SERVER, dynamic = true, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 630712f536874..f3121ec152642 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -2041,7 +2041,14 @@ CompletableFuture startReplicator(String remoteCluster) { final CompletableFuture future = new CompletableFuture<>(); String name = PersistentReplicator.getReplicatorName(replicatorPrefix, remoteCluster); - ledger.asyncOpenCursor(name, new OpenCursorCallback() { + final InitialPosition initialPosition; + if (MessageId.earliest.toString() + .equalsIgnoreCase(getBrokerService().getPulsar().getConfiguration().getReplicationStartAt())) { + initialPosition = InitialPosition.Earliest; + } else { + initialPosition = InitialPosition.Latest; + } + ledger.asyncOpenCursor(name, initialPosition, new OpenCursorCallback() { @Override public void openCursorComplete(ManagedCursor cursor, Object ctx) { String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index c9b23c6437a22..e686cd2c94f62 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -25,6 +25,7 @@ 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.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -36,6 +37,7 @@ import java.lang.reflect.Method; import java.time.Duration; import java.util.Arrays; +import java.util.Collections; import java.util.Iterator; import java.util.Optional; import java.util.UUID; @@ -71,6 +73,7 @@ import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; +import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.naming.TopicName; @@ -912,4 +915,100 @@ public void testReloadWithTopicLevelGeoReplication(ReplicationLevel replicationL }); } } + + protected void enableReplication(String topic) throws Exception { + admin1.topics().setReplicationClusters(topic, Arrays.asList(cluster1, cluster2)); + } + + protected void disableReplication(String topic) throws Exception { + admin1.topics().setReplicationClusters(topic, Arrays.asList(cluster1, cluster2)); + } + + @Test + public void testConfigReplicationStartAt() throws Exception { + // Initialize. + String ns1 = defaultTenant + "/ns_" + UUID.randomUUID().toString().replace("-", ""); + String subscription1 = "s1"; + admin1.namespaces().createNamespace(ns1); + if (!usingGlobalZK) { + admin2.namespaces().createNamespace(ns1); + } + + RetentionPolicies retentionPolicies = new RetentionPolicies(60 * 24, 1024); + admin1.namespaces().setRetention(ns1, retentionPolicies); + admin2.namespaces().setRetention(ns1, retentionPolicies); + + // 1. default config. + // Enable replication for topic1. + final String topic1 = BrokerTestUtil.newUniqueName("persistent://" + ns1 + "/tp_"); + admin1.topics().createNonPartitionedTopicAsync(topic1); + admin1.topics().createSubscription(topic1, subscription1, MessageId.earliest); + Producer p1 = client1.newProducer(Schema.STRING).topic(topic1).create(); + p1.send("msg-1"); + p1.close(); + enableReplication(topic1); + // Verify: since the replication was started at latest, there is no message to consume. + Consumer c1 = client2.newConsumer(Schema.STRING).topic(topic1).subscriptionName(subscription1) + .subscribe(); + Message msg1 = c1.receive(2, TimeUnit.SECONDS); + assertNull(msg1); + c1.close(); + disableReplication(topic1); + + // 2.Update config: start at "earliest". + admin1.brokers().updateDynamicConfiguration("replicationStartAt", MessageId.earliest.toString()); + Awaitility.await().untilAsserted(() -> { + pulsar1.getConfiguration().getReplicationStartAt().equalsIgnoreCase("earliest"); + }); + + final String topic2 = BrokerTestUtil.newUniqueName("persistent://" + ns1 + "/tp_"); + admin1.topics().createNonPartitionedTopicAsync(topic2); + admin1.topics().createSubscription(topic2, subscription1, MessageId.earliest); + Producer p2 = client1.newProducer(Schema.STRING).topic(topic2).create(); + p2.send("msg-1"); + p2.close(); + enableReplication(topic2); + // Verify: since the replication was started at earliest, there is one message to consume. + Consumer c2 = client2.newConsumer(Schema.STRING).topic(topic2).subscriptionName(subscription1) + .subscribe(); + Message msg2 = c2.receive(2, TimeUnit.SECONDS); + assertNotNull(msg2); + assertEquals(msg2.getValue(), "msg-1"); + c2.close(); + disableReplication(topic2); + + // 2.Update config: start at "latest". + admin1.brokers().updateDynamicConfiguration("replicationStartAt", MessageId.latest.toString()); + Awaitility.await().untilAsserted(() -> { + pulsar1.getConfiguration().getReplicationStartAt().equalsIgnoreCase("latest"); + }); + + final String topic3 = BrokerTestUtil.newUniqueName("persistent://" + ns1 + "/tp_"); + admin1.topics().createNonPartitionedTopicAsync(topic3); + admin1.topics().createSubscription(topic3, subscription1, MessageId.earliest); + Producer p3 = client1.newProducer(Schema.STRING).topic(topic3).create(); + p3.send("msg-1"); + p3.close(); + enableReplication(topic3); + // Verify: since the replication was started at latest, there is no message to consume. + Consumer c3 = client2.newConsumer(Schema.STRING).topic(topic3).subscriptionName(subscription1) + .subscribe(); + Message msg3 = c3.receive(2, TimeUnit.SECONDS); + assertNull(msg3); + c3.close(); + disableReplication(topic3); + + // cleanup. + // There is no good way to delete topics when using global ZK, skip cleanup. + admin1.namespaces().setNamespaceReplicationClusters(ns1, Collections.singleton(cluster1)); + admin1.namespaces().unload(ns1); + admin2.namespaces().setNamespaceReplicationClusters(ns1, Collections.singleton(cluster2)); + admin2.namespaces().unload(ns1); + admin1.topics().delete(topic1, false); + admin2.topics().delete(topic1, false); + admin1.topics().delete(topic2, false); + admin2.topics().delete(topic2, false); + admin1.topics().delete(topic3, false); + admin2.topics().delete(topic3, false); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java index b8f8edce2477e..31e94f435f0f6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java @@ -18,7 +18,19 @@ */ package org.apache.pulsar.broker.service; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import java.util.Arrays; +import java.util.HashSet; +import java.util.UUID; +import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -109,4 +121,44 @@ public void testExpandTopicPartitionsOnNamespaceLevelReplication() throws Except public void testReloadWithTopicLevelGeoReplication(ReplicationLevel replicationLevel) throws Exception { super.testReloadWithTopicLevelGeoReplication(replicationLevel); } + + @Test + @Override + public void testConfigReplicationStartAt() throws Exception { + // Initialize. + String ns1 = defaultTenant + "/ns_" + UUID.randomUUID().toString().replace("-", ""); + String subscription1 = "s1"; + admin1.namespaces().createNamespace(ns1); + RetentionPolicies retentionPolicies = new RetentionPolicies(60 * 24, 1024); + admin1.namespaces().setRetention(ns1, retentionPolicies); + admin2.namespaces().setRetention(ns1, retentionPolicies); + + // Update config: start at "earliest". + admin1.brokers().updateDynamicConfiguration("replicationStartAt", MessageId.earliest.toString()); + Awaitility.await().untilAsserted(() -> { + pulsar1.getConfiguration().getReplicationStartAt().equalsIgnoreCase("earliest"); + }); + + // Verify: since the replication was started at earliest, there is one message to consume. + final String topic1 = BrokerTestUtil.newUniqueName("persistent://" + ns1 + "/tp_"); + admin1.topics().createNonPartitionedTopicAsync(topic1); + admin1.topics().createSubscription(topic1, subscription1, MessageId.earliest); + org.apache.pulsar.client.api.Producer p1 = client1.newProducer(Schema.STRING).topic(topic1).create(); + p1.send("msg-1"); + p1.close(); + + admin1.namespaces().setNamespaceReplicationClusters(ns1, new HashSet<>(Arrays.asList(cluster1, cluster2))); + org.apache.pulsar.client.api.Consumer c1 = client2.newConsumer(Schema.STRING).topic(topic1) + .subscriptionName(subscription1).subscribe(); + Message msg2 = c1.receive(2, TimeUnit.SECONDS); + assertNotNull(msg2); + assertEquals(msg2.getValue(), "msg-1"); + c1.close(); + + // cleanup. + admin1.brokers().updateDynamicConfiguration("replicationStartAt", MessageId.latest.toString()); + Awaitility.await().untilAsserted(() -> { + pulsar1.getConfiguration().getReplicationStartAt().equalsIgnoreCase("latest"); + }); + } } From 5b1f653e65ccd967fd9642e6d6959de4b1b01a63 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Thu, 20 Jun 2024 15:18:03 +0800 Subject: [PATCH 324/580] [improve][broker] Optimize `ConcurrentOpenLongPairRangeSet` by RoaringBitmap (#22908) --- .../server/src/assemble/LICENSE.bin.txt | 2 +- .../shell/src/assemble/LICENSE.bin.txt | 2 + pom.xml | 2 +- pulsar-common/pom.xml | 5 + .../ConcurrentOpenLongPairRangeSet.java | 12 +- .../collections/ConcurrentRoaringBitSet.java | 439 ++++++++++++++++++ 6 files changed, 453 insertions(+), 9 deletions(-) create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 9f5209bc7fd8d..3b30a40ff83e9 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -514,7 +514,7 @@ The Apache Software License, Version 2.0 * RxJava - io.reactivex.rxjava3-rxjava-3.0.1.jar * RoaringBitmap - - org.roaringbitmap-RoaringBitmap-1.0.6.jar + - org.roaringbitmap-RoaringBitmap-1.1.0.jar * OpenTelemetry - io.opentelemetry-opentelemetry-api-1.38.0.jar - io.opentelemetry-opentelemetry-api-incubator-1.38.0-alpha.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index ff590023ff3a5..6cefa42bed85a 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -383,6 +383,8 @@ The Apache Software License, Version 2.0 - simpleclient_tracer_common-0.16.0.jar - simpleclient_tracer_otel-0.16.0.jar - simpleclient_tracer_otel_agent-0.16.0.jar + * RoaringBitmap + - RoaringBitmap-1.1.0.jar * Log4J - log4j-api-2.23.1.jar - log4j-core-2.23.1.jar diff --git a/pom.xml b/pom.xml index 62644d38d167c..8325336aa9684 100644 --- a/pom.xml +++ b/pom.xml @@ -317,7 +317,7 @@ flexible messaging model and an intuitive client API. 1.3 0.4 9.1.0 - 1.0.6 + 1.1.0 1.6.1 6.4.0 3.33.0 diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 62e7bde25603c..d29ce3126635f 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -253,6 +253,11 @@ awaitility test + + + org.roaringbitmap + RoaringBitmap + diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java index 72215d7296cc3..b5ad89d1695d4 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java @@ -29,6 +29,7 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.lang.mutable.MutableInt; +import org.roaringbitmap.RoaringBitSet; /** * A Concurrent set comprising zero or more ranges of type {@link LongPair}. This can be alternative of @@ -44,7 +45,7 @@ public class ConcurrentOpenLongPairRangeSet> implements LongPairRangeSet { protected final NavigableMap rangeBitSetMap = new ConcurrentSkipListMap<>(); - private boolean threadSafe = true; + private final boolean threadSafe; private final int bitSetSize; private final LongPairConsumer consumer; @@ -95,9 +96,7 @@ public void addOpenClosed(long lowerKey, long lowerValueOpen, long upperKey, lon // (2) set 0th-index to upper-index in upperRange.getKey() if (isValid(upperKey, upperValue)) { BitSet rangeBitSet = rangeBitSetMap.computeIfAbsent(upperKey, (key) -> createNewBitSet()); - if (rangeBitSet != null) { - rangeBitSet.set(0, (int) upperValue + 1); - } + rangeBitSet.set(0, (int) upperValue + 1); } // No-op if values are not valid eg: if lower == LongPair.earliest or upper == LongPair.latest then nothing // to set @@ -414,7 +413,6 @@ private int getSafeEntry(long value) { } private BitSet createNewBitSet() { - return this.threadSafe ? new ConcurrentBitSet(bitSetSize) : new BitSet(bitSetSize); + return this.threadSafe ? new ConcurrentRoaringBitSet() : new RoaringBitSet(); } - -} +} \ No newline at end of file diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java new file mode 100644 index 0000000000000..814e58400993b --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java @@ -0,0 +1,439 @@ +/* + * 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.collections; + +import java.util.BitSet; +import java.util.concurrent.locks.StampedLock; +import java.util.stream.IntStream; +import org.roaringbitmap.RoaringBitSet; + +public class ConcurrentRoaringBitSet extends RoaringBitSet { + private final StampedLock rwLock = new StampedLock(); + + public ConcurrentRoaringBitSet() { + super(); + } + + @Override + public boolean get(int bitIndex) { + long stamp = rwLock.tryOptimisticRead(); + boolean isSet = super.get(bitIndex); + if (!rwLock.validate(stamp)) { + stamp = rwLock.readLock(); + try { + isSet = super.get(bitIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return isSet; + } + + @Override + public void set(int bitIndex) { + long stamp = rwLock.writeLock(); + try { + super.set(bitIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void clear(int bitIndex) { + long stamp = rwLock.writeLock(); + try { + super.clear(bitIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void set(int fromIndex, int toIndex) { + long stamp = rwLock.writeLock(); + try { + super.set(fromIndex, toIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void clear(int fromIndex, int toIndex) { + long stamp = rwLock.writeLock(); + try { + super.clear(fromIndex, toIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void clear() { + long stamp = rwLock.writeLock(); + try { + super.clear(); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public int nextSetBit(int fromIndex) { + long stamp = rwLock.tryOptimisticRead(); + int nextSetBit = super.nextSetBit(fromIndex); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + nextSetBit = super.nextSetBit(fromIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return nextSetBit; + } + + @Override + public int nextClearBit(int fromIndex) { + long stamp = rwLock.tryOptimisticRead(); + int nextClearBit = super.nextClearBit(fromIndex); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + nextClearBit = super.nextClearBit(fromIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return nextClearBit; + } + + @Override + public int previousSetBit(int fromIndex) { + long stamp = rwLock.tryOptimisticRead(); + int previousSetBit = super.previousSetBit(fromIndex); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + previousSetBit = super.previousSetBit(fromIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return previousSetBit; + } + + @Override + public int previousClearBit(int fromIndex) { + long stamp = rwLock.tryOptimisticRead(); + int previousClearBit = super.previousClearBit(fromIndex); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + previousClearBit = super.previousClearBit(fromIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return previousClearBit; + } + + @Override + public int length() { + long stamp = rwLock.tryOptimisticRead(); + int length = super.length(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + length = super.length(); + } finally { + rwLock.unlockRead(stamp); + } + } + return length; + } + + @Override + public boolean isEmpty() { + long stamp = rwLock.tryOptimisticRead(); + boolean isEmpty = super.isEmpty(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + isEmpty = super.isEmpty(); + } finally { + rwLock.unlockRead(stamp); + } + } + return isEmpty; + } + + @Override + public int cardinality() { + long stamp = rwLock.tryOptimisticRead(); + int cardinality = super.cardinality(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + cardinality = super.cardinality(); + } finally { + rwLock.unlockRead(stamp); + } + } + return cardinality; + } + + @Override + public int size() { + long stamp = rwLock.tryOptimisticRead(); + int size = super.size(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + size = super.size(); + } finally { + rwLock.unlockRead(stamp); + } + } + return size; + } + + @Override + public byte[] toByteArray() { + long stamp = rwLock.tryOptimisticRead(); + byte[] byteArray = super.toByteArray(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + byteArray = super.toByteArray(); + } finally { + rwLock.unlockRead(stamp); + } + } + return byteArray; + } + + @Override + public long[] toLongArray() { + long stamp = rwLock.tryOptimisticRead(); + long[] longArray = super.toLongArray(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + longArray = super.toLongArray(); + } finally { + rwLock.unlockRead(stamp); + } + } + return longArray; + } + + @Override + public void flip(int bitIndex) { + long stamp = rwLock.writeLock(); + try { + super.flip(bitIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void flip(int fromIndex, int toIndex) { + long stamp = rwLock.writeLock(); + try { + super.flip(fromIndex, toIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void set(int bitIndex, boolean value) { + long stamp = rwLock.writeLock(); + try { + super.set(bitIndex, value); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void set(int fromIndex, int toIndex, boolean value) { + long stamp = rwLock.writeLock(); + try { + super.set(fromIndex, toIndex, value); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public BitSet get(int fromIndex, int toIndex) { + long stamp = rwLock.tryOptimisticRead(); + BitSet bitSet = super.get(fromIndex, toIndex); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + bitSet = super.get(fromIndex, toIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return bitSet; + } + + @Override + public boolean intersects(BitSet set) { + long stamp = rwLock.writeLock(); + try { + return super.intersects(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void and(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.and(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void or(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.or(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void xor(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.xor(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void andNot(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.andNot(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + /** + * Returns the clone of the internal wrapped {@code BitSet}. + * This won't be a clone of the {@code ConcurrentBitSet} object. + * + * @return a clone of the internal wrapped {@code BitSet} + */ + @Override + public Object clone() { + long stamp = rwLock.tryOptimisticRead(); + RoaringBitSet clone = (RoaringBitSet) super.clone(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + clone = (RoaringBitSet) super.clone(); + } finally { + rwLock.unlockRead(stamp); + } + } + return clone; + } + + @Override + public String toString() { + long stamp = rwLock.tryOptimisticRead(); + String str = super.toString(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + str = super.toString(); + } finally { + rwLock.unlockRead(stamp); + } + } + return str; + } + + /** + * This operation is not supported on {@code ConcurrentBitSet}. + */ + @Override + public IntStream stream() { + throw new UnsupportedOperationException("stream is not supported"); + } + + public boolean equals(final Object o) { + long stamp = rwLock.tryOptimisticRead(); + boolean isEqual = super.equals(o); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + isEqual = super.equals(o); + } finally { + rwLock.unlockRead(stamp); + } + } + return isEqual; + } + + public int hashCode() { + long stamp = rwLock.tryOptimisticRead(); + int hashCode = super.hashCode(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + hashCode = super.hashCode(); + } finally { + rwLock.unlockRead(stamp); + } + } + return hashCode; + } +} From ca6450598469f158d8fa4cc942fb51e12ed1b609 Mon Sep 17 00:00:00 2001 From: yangyijun <1012293987@qq.com> Date: Thu, 20 Jun 2024 18:43:43 +0800 Subject: [PATCH 325/580] [fix] [broker] broker log a full thread dump when a deadlock is detected in healthcheck every time (#22916) --- .../java/org/apache/pulsar/broker/admin/impl/BrokersBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index 9db17f76a8dbe..4d0b598a8e4f1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -88,7 +88,7 @@ public class BrokersBase extends AdminResource { private static final Duration HEALTH_CHECK_READ_TIMEOUT = Duration.ofSeconds(58); private static final TimeoutException HEALTH_CHECK_TIMEOUT_EXCEPTION = FutureUtil.createTimeoutException("Timeout", BrokersBase.class, "healthCheckRecursiveReadNext(...)"); - private volatile long threadDumpLoggedTimestamp; + private static volatile long threadDumpLoggedTimestamp; @GET @Path("/{cluster}") From 36bae695fb07f3ee790bee603149c4c2712187e0 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Thu, 20 Jun 2024 20:22:28 +0800 Subject: [PATCH 326/580] [feat][broker] PIP-321 Introduce allowed-cluster at the namespace level (#22378) --- .../broker/admin/impl/NamespacesBase.java | 79 ++++++++++- .../pulsar/broker/admin/v2/Namespaces.java | 47 +++++++ .../service/persistent/PersistentTopic.java | 90 ++++++++----- .../pulsar/broker/web/PulsarWebResource.java | 15 ++- .../namespace/NamespaceServiceTest.java | 127 ++++++++++++++++++ .../pulsar/broker/service/ReplicatorTest.java | 56 ++++++++ .../pulsar/client/admin/Namespaces.java | 83 ++++++++++++ .../pulsar/common/policies/data/Policies.java | 5 +- .../client/admin/internal/NamespacesImpl.java | 24 ++++ .../pulsar/admin/cli/PulsarAdminToolTest.java | 10 +- .../pulsar/admin/cli/CmdNamespaces.java | 32 +++++ .../common/policies/data/PolicyName.java | 3 +- 12 files changed, 527 insertions(+), 44 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 afcf4e646fa2c..4d26fe2a4c35b 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 @@ -703,9 +703,21 @@ protected CompletableFuture internalSetNamespaceReplicationClusters(List - validateClusterForTenantAsync( - namespaceName.getTenant(), clusterId)); + .thenCompose(__ -> getNamespacePoliciesAsync(this.namespaceName) + .thenCompose(nsPolicies -> { + if (nsPolicies.allowed_clusters.isEmpty()) { + return validateClusterForTenantAsync( + namespaceName.getTenant(), clusterId); + } + if (!nsPolicies.allowed_clusters.contains(clusterId)) { + String msg = String.format("Cluster [%s] is not in the " + + "list of allowed clusters list for namespace " + + "[%s]", clusterId, namespaceName.toString()); + log.info(msg); + throw new RestException(Status.FORBIDDEN, msg); + } + return CompletableFuture.completedFuture(null); + })); }).collect(Collectors.toList()); return FutureUtil.waitForAll(futures).thenApply(__ -> replicationClusterSet); })) @@ -2722,4 +2734,65 @@ protected CompletableFuture internalGetDispatcherPauseOnAckStatePersist return policiesOpt.map(p -> p.dispatcherPauseOnAckStatePersistentEnabled).orElse(false); }); } + + protected CompletableFuture internalSetNamespaceAllowedClusters(List clusterIds) { + return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.ALLOW_CLUSTERS, PolicyOperation.WRITE) + .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + // Allowed clusters in the namespace policy should be included in the allowed clusters in the tenant + // policy. + .thenCompose(__ -> FutureUtil.waitForAll(clusterIds.stream().map(clusterId -> + validateClusterForTenantAsync(namespaceName.getTenant(), clusterId)) + .collect(Collectors.toList()))) + // Allowed clusters should include all the existed replication clusters and could not contain global + // cluster. + .thenCompose(__ -> { + checkNotNull(clusterIds, "ClusterIds should not be null"); + if (clusterIds.contains("global")) { + throw new RestException(Status.PRECONDITION_FAILED, + "Cannot specify global in the list of allowed clusters"); + } + return getNamespacePoliciesAsync(this.namespaceName).thenApply(namespacePolicies -> { + namespacePolicies.replication_clusters.forEach(replicationCluster -> { + if (!clusterIds.contains(replicationCluster)) { + throw new RestException(Status.BAD_REQUEST, + String.format("Allowed clusters do not contain the replication cluster %s. " + + "Please remove the replication cluster if the cluster is not allowed " + + "for this namespace", replicationCluster)); + } + }); + return Sets.newHashSet(clusterIds); + }); + }) + // Verify the allowed clusters are valid and they do not contain the peer clusters. + .thenCompose(allowedClusters -> clustersAsync() + .thenCompose(clusters -> { + List> futures = + allowedClusters.stream().map(clusterId -> { + if (!clusters.contains(clusterId)) { + throw new RestException(Status.FORBIDDEN, + "Invalid cluster id: " + clusterId); + } + return validatePeerClusterConflictAsync(clusterId, allowedClusters); + }).collect(Collectors.toList()); + return FutureUtil.waitForAll(futures).thenApply(__ -> allowedClusters); + })) + // Update allowed clusters into policies. + .thenCompose(allowedClusterSet -> updatePoliciesAsync(namespaceName, policies -> { + policies.allowed_clusters = allowedClusterSet; + return policies; + })); + } + + protected CompletableFuture> internalGetNamespaceAllowedClustersAsync() { + return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.ALLOW_CLUSTERS, PolicyOperation.READ) + .thenAccept(__ -> { + if (!namespaceName.isGlobal()) { + throw new RestException(Status.PRECONDITION_FAILED, + "Cannot get the allowed clusters for a non-global namespace"); + } + }).thenCompose(__ -> getNamespacePoliciesAsync(namespaceName)) + .thenApply(policies -> policies.allowed_clusters); + } + + } 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 3a7c614a7c6f8..54cceaf09e9fe 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 @@ -3041,5 +3041,52 @@ public void getDispatcherPauseOnAckStatePersistent(@Suspended final AsyncRespons }); } + + @POST + @Path("/{tenant}/{namespace}/allowedClusters") + @ApiOperation(value = "Set the allowed clusters for a namespace.") + @ApiResponses(value = { + @ApiResponse(code = 400, message = "The list of allowed clusters should include all replication clusters."), + @ApiResponse(code = 403, message = "The requester does not have admin permissions."), + @ApiResponse(code = 404, message = "The specified tenant, cluster, or namespace does not exist."), + @ApiResponse(code = 409, message = "A peer-cluster cannot be part of an allowed-cluster."), + @ApiResponse(code = 412, message = "The namespace is not global or the provided cluster IDs are invalid.")}) + public void setNamespaceAllowedClusters(@Suspended AsyncResponse asyncResponse, + @PathParam("tenant") String tenant, + @PathParam("namespace") String namespace, + @ApiParam(value = "List of allowed clusters", required = true) + List clusterIds) { + validateNamespaceName(tenant, namespace); + internalSetNamespaceAllowedClusters(clusterIds) + .thenAccept(asyncResponse::resume) + .exceptionally(e -> { + log.error("[{}] Failed to set namespace allowed clusters on namespace {}", + clientAppId(), namespace, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return null; + }); + } + + @GET + @Path("/{tenant}/{namespace}/allowedClusters") + @ApiOperation(value = "Get the allowed clusters for a namespace.", + response = String.class, responseContainer = "List") + @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), + @ApiResponse(code = 412, message = "Namespace is not global")}) + public void getNamespaceAllowedClusters(@Suspended AsyncResponse asyncResponse, + @PathParam("tenant") String tenant, + @PathParam("namespace") String namespace) { + validateNamespaceName(tenant, namespace); + internalGetNamespaceAllowedClustersAsync() + .thenAccept(asyncResponse::resume) + .exceptionally(e -> { + log.error("[{}] Failed to get namespace allowed clusters on namespace {}", clientAppId(), + namespace, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return null; + }); + } + private static final Logger log = LoggerFactory.getLogger(Namespaces.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index f3121ec152642..1983fa3c383e3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -1878,52 +1878,78 @@ public CompletableFuture checkReplication() { if (log.isDebugEnabled()) { log.debug("[{}] Checking replication status", name); } - List configuredClusters = topicPolicies.getReplicationClusters().get(); if (CollectionUtils.isEmpty(configuredClusters)) { log.warn("[{}] No replication clusters configured", name); return CompletableFuture.completedFuture(null); } - int newMessageTTLInSeconds = topicPolicies.getMessageTTLInSeconds().get(); - String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); - // if local cluster is removed from global namespace cluster-list : then delete topic forcefully - // because pulsar doesn't serve global topic without local repl-cluster configured. - if (TopicName.get(topic).isGlobal() && !configuredClusters.contains(localCluster)) { - log.info("Deleting topic [{}] because local cluster is not part of " - + " global namespace repl list {}", topic, configuredClusters); - return deleteForcefully(); - } - - removeTerminatedReplicators(replicators); - List> futures = new ArrayList<>(); - - // Check for missing replicators - for (String cluster : configuredClusters) { - if (cluster.equals(localCluster)) { - continue; + return checkAllowedCluster(localCluster).thenCompose(success -> { + if (!success) { + // if local cluster is removed from global namespace cluster-list : then delete topic forcefully + // because pulsar doesn't serve global topic without local repl-cluster configured. + return deleteForcefully(); } - if (!replicators.containsKey(cluster)) { - futures.add(startReplicator(cluster)); - } - } - // Check for replicators to be stopped - replicators.forEach((cluster, replicator) -> { - // Update message TTL - ((PersistentReplicator) replicator).updateMessageTTL(newMessageTTLInSeconds); - if (!cluster.equals(localCluster)) { - if (!configuredClusters.contains(cluster)) { - futures.add(removeReplicator(cluster)); + int newMessageTTLInSeconds = topicPolicies.getMessageTTLInSeconds().get(); + + removeTerminatedReplicators(replicators); + List> futures = new ArrayList<>(); + + // The replication clusters at namespace level will get local cluster when creating a namespace. + // If there are only one cluster in the replication clusters, it means the replication is not enabled. + // If the cluster 1 and cluster 2 use the same configuration store and the namespace is created in cluster1 + // without enabling geo-replication, then the replication clusters always has cluster1. + // + // When a topic under the namespace is load in the cluster2, the `cluster1` may be identified as + // remote cluster and start geo-replication. This check is to avoid the above case. + if (!(configuredClusters.size() == 1 && replicators.isEmpty())) { + // Check for missing replicators + for (String cluster : configuredClusters) { + if (cluster.equals(localCluster)) { + continue; + } + if (!replicators.containsKey(cluster)) { + futures.add(startReplicator(cluster)); + } } + // Check for replicators to be stopped + replicators.forEach((cluster, replicator) -> { + // Update message TTL + ((PersistentReplicator) replicator).updateMessageTTL(newMessageTTLInSeconds); + if (!cluster.equals(localCluster)) { + if (!configuredClusters.contains(cluster)) { + futures.add(removeReplicator(cluster)); + } + } + }); } - }); - futures.add(checkShadowReplication()); + futures.add(checkShadowReplication()); - return FutureUtil.waitForAll(futures); + return FutureUtil.waitForAll(futures); + }); + } + + private CompletableFuture checkAllowedCluster(String localCluster) { + List replicationClusters = topicPolicies.getReplicationClusters().get(); + return brokerService.pulsar().getPulsarResources().getNamespaceResources() + .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()).thenCompose(policiesOptional -> { + Set allowedClusters = Set.of(); + if (policiesOptional.isPresent()) { + allowedClusters = policiesOptional.get().allowed_clusters; + } + if (TopicName.get(topic).isGlobal() && !replicationClusters.contains(localCluster) + && !allowedClusters.contains(localCluster)) { + log.warn("Local cluster {} is not part of global namespace repl list {} and allowed list {}", + localCluster, replicationClusters, allowedClusters); + return CompletableFuture.completedFuture(false); + } else { + return CompletableFuture.completedFuture(true); + } + }); } private CompletableFuture checkShadowReplication() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java index 99f0a30d1a5f2..2e198eb99752e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java @@ -901,14 +901,16 @@ public static CompletableFuture checkLocalOrGetPeerReplicationC log.warn(msg); validationFuture.completeExceptionally(new RestException(Status.NOT_FOUND, "Namespace is deleted")); - } else if (policies.replication_clusters.isEmpty()) { + } else if (policies.replication_clusters.isEmpty() && policies.allowed_clusters.isEmpty()) { String msg = String.format( "Namespace does not have any clusters configured : local_cluster=%s ns=%s", localCluster, namespace.toString()); log.warn(msg); validationFuture.completeExceptionally(new RestException(Status.PRECONDITION_FAILED, msg)); - } else if (!policies.replication_clusters.contains(localCluster)) { - getOwnerFromPeerClusterListAsync(pulsarService, policies.replication_clusters) + } else if (!policies.replication_clusters.contains(localCluster) && !policies.allowed_clusters + .contains(localCluster)) { + getOwnerFromPeerClusterListAsync(pulsarService, policies.replication_clusters, + policies.allowed_clusters) .thenAccept(ownerPeerCluster -> { if (ownerPeerCluster != null) { // found a peer that own this namespace @@ -948,9 +950,9 @@ public static CompletableFuture checkLocalOrGetPeerReplicationC } private static CompletableFuture getOwnerFromPeerClusterListAsync(PulsarService pulsar, - Set replicationClusters) { + Set replicationClusters, Set allowedClusters) { String currentCluster = pulsar.getConfiguration().getClusterName(); - if (replicationClusters == null || replicationClusters.isEmpty() || isBlank(currentCluster)) { + if (replicationClusters.isEmpty() && allowedClusters.isEmpty() || isBlank(currentCluster)) { return CompletableFuture.completedFuture(null); } @@ -960,7 +962,8 @@ private static CompletableFuture getOwnerFromPeerClusterListAsy return CompletableFuture.completedFuture(null); } for (String peerCluster : cluster.get().getPeerClusterNames()) { - if (replicationClusters.contains(peerCluster)) { + if (replicationClusters.contains(peerCluster) + || allowedClusters.contains(peerCluster)) { return pulsar.getPulsarResources().getClusterResources().getClusterAsync(peerCluster) .thenApply(ret -> { if (!ret.isPresent()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java index 0b0d38a071e9b..422e9b80aeffa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java @@ -33,6 +33,7 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.github.benmanes.caffeine.cache.AsyncLoadingCache; +import com.google.common.collect.Sets; import com.google.common.hash.Hashing; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -41,6 +42,7 @@ import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -65,6 +67,7 @@ import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.admin.Namespaces; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.PulsarClient; @@ -77,8 +80,11 @@ import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BundlesData; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.LocalPolicies; import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.GetResult; @@ -827,6 +833,127 @@ public void testCheckTopicExists(String topicDomain) throws Exception { }); } + @Test + public void testAllowedClustersAtNamespaceLevelShouldBeIncludedInAllowedClustersAtTenantLevel() throws Exception { + // 1. Setup + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(true); + pulsar.getConfiguration().setForceDeleteTenantAllowed(true); + Set tenantAllowedClusters = Set.of("test", "r1", "r2"); + Set allowedClusters1 = Set.of("test", "r1", "r2", "r3"); + Set allowedClusters2 = Set.of("test", "r1", "r2"); + Set clusters = Set.of("r1", "r2", "r3", "r4"); + final String tenant = "my-tenant"; + final String namespace = tenant + "/testAllowedCluster"; + admin.tenants().createTenant(tenant, + new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test"))); + admin.namespaces().createNamespace(namespace); + pulsar.getPulsarResources().getTenantResources().updateTenantAsync(tenant, tenantInfo -> + TenantInfo.builder().allowedClusters(tenantAllowedClusters).build()); + for (String cluster : clusters) { + pulsar.getPulsarResources().getClusterResources().createCluster(cluster, ClusterData.builder().build()); + } + // 2. Verify + admin.namespaces().setNamespaceAllowedClusters(namespace, allowedClusters2); + + try { + admin.namespaces().setNamespaceAllowedClusters(namespace, allowedClusters1); + fail(); + } catch (PulsarAdminException e) { + assertEquals(e.getStatusCode(), 403); + assertEquals(e.getMessage(), + "Cluster [r3] is not in the list of allowed clusters list for tenant [my-tenant]"); + } + // 3. Clean up + admin.namespaces().deleteNamespace(namespace, true); + admin.tenants().deleteTenant(tenant, true); + for (String cluster : clusters) { + pulsar.getPulsarResources().getClusterResources().deleteCluster(cluster); + } + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(false); + pulsar.getConfiguration().setForceDeleteTenantAllowed(false); + } + + /** + * Test case: + * 1. Replication clusters should be included in the allowed clusters. For compatibility, the replication + * clusters could be set before the allowed clusters are set. + * 2. Peer cluster can not be a part of the allowed clusters. + */ + @Test + public void testNewAllowedClusterAdminAPIAndItsImpactOnReplicationClusterAPI() throws Exception { + // 1. Setup + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(true); + pulsar.getConfiguration().setForceDeleteTenantAllowed(true); + // Setup: Prepare cluster resource, tenant and namespace + Set replicationClusters = Set.of("test", "r1", "r2"); + Set tenantAllowedClusters = Set.of("test", "r1", "r2", "r3"); + Set allowedClusters = Set.of("test", "r1", "r2", "r3"); + Set clusters = Set.of("r1", "r2", "r3", "r4"); + final String tenant = "my-tenant"; + final String namespace = tenant + "/testAllowedCluster"; + admin.tenants().createTenant(tenant, + new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test"))); + admin.namespaces().createNamespace(namespace); + pulsar.getPulsarResources().getTenantResources().updateTenantAsync(tenant, tenantInfo -> + TenantInfo.builder().allowedClusters(tenantAllowedClusters).build()); + + Namespaces namespaces = admin.namespaces(); + for (String cluster : clusters) { + pulsar.getPulsarResources().getClusterResources().createCluster(cluster, ClusterData.builder().build()); + } + // 2. Verify + // 2.1 Replication clusters should be included in the allowed clusters. + + // SUCCESS + // 2.1.1. Set replication clusters without allowed clusters at namespace level. + namespaces.setNamespaceReplicationClusters(namespace, replicationClusters); + // 2..1.2 Set allowed clusters. + namespaces.setNamespaceAllowedClusters(namespace, allowedClusters); + // 2.1.3. Get allowed clusters and replication clusters. + List allowedClustersResponse = namespaces.getNamespaceAllowedClusters(namespace); + + List replicationClustersResponse = namespaces.getNamespaceReplicationClusters(namespace); + + assertEquals(replicationClustersResponse.size(), replicationClusters.size()); + assertEquals(allowedClustersResponse.size(), allowedClusters.size()); + + // FAIL + // 2.1.4. Fail: Set allowed clusters whose scope is smaller than replication clusters. + Set allowedClustersSmallScope = Set.of("r1", "r3"); + try { + namespaces.setNamespaceAllowedClusters(namespace, allowedClustersSmallScope); + fail(); + } catch (PulsarAdminException ignore) {} + // 2.1.5. Fail: Set replication clusters whose scope is excel the allowed clusters. + Set replicationClustersExcel = Set.of("r1", "r4"); + try { + namespaces.setNamespaceReplicationClusters(namespace, replicationClustersExcel); + fail(); + //Todo: The status code in the old implementation is confused. + } catch (PulsarAdminException.NotAuthorizedException ignore) {} + + // 2.2 Peer cluster can not be a part of the allowed clusters. + LinkedHashSet peerCluster = new LinkedHashSet<>(); + peerCluster.add("r2"); + pulsar.getPulsarResources().getClusterResources().deleteCluster("r1"); + pulsar.getPulsarResources().getClusterResources().createCluster("r1", + ClusterData.builder().peerClusterNames(peerCluster).build()); + try { + namespaces.setNamespaceAllowedClusters(namespace, Set.of("test", "r1", "r2", "r3")); + fail(); + } catch (PulsarAdminException.ConflictException ignore) {} + + // CleanUp: Namespace with replication clusters can not be deleted by force. + namespaces.setNamespaceReplicationClusters(namespace, Set.of(conf.getClusterName())); + admin.namespaces().deleteNamespace(namespace, true); + admin.tenants().deleteTenant(tenant, true); + for (String cluster : clusters) { + pulsar.getPulsarResources().getClusterResources().deleteCluster(cluster); + } + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(false); + pulsar.getConfiguration().setForceDeleteTenantAllowed(false); + } + /** * 1. Manually trigger "LoadReportUpdaterTask" * 2. Registry another new zk-node-listener "waitForBrokerChangeNotice". diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index fb92f3f47b22b..d83b2ed4ee6c9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -62,6 +62,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.commons.lang3.RandomUtils; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -1812,6 +1813,61 @@ public void testReplicatorWithTTL() throws Exception { assertEquals(result, Lists.newArrayList("V1", "V2", "V3", "V4")); } + @Test + public void testEnableReplicationWithNamespaceAllowedClustersPolices() throws Exception { + log.info("--- testEnableReplicationWithNamespaceAllowedClustersPolices ---"); + String namespace1 = "pulsar/ns" + RandomUtils.nextLong(); + admin1.namespaces().createNamespace(namespace1); + admin2.namespaces().createNamespace(namespace1 + "init_cluster_node"); + admin1.namespaces().setNamespaceAllowedClusters(namespace1, Sets.newHashSet("r1", "r2")); + final TopicName topicName = TopicName.get( + BrokerTestUtil.newUniqueName("persistent://" + namespace1 + "/testReplicatorProducerNotExceed1")); + + @Cleanup PulsarClient client1 = PulsarClient + .builder() + .serviceUrl(pulsar1.getBrokerServiceUrl()) + .build(); + @Cleanup Producer producer = client1 + .newProducer() + .topic(topicName.toString()) + .create(); + producer.newMessage().send(); + // Enable replication at the topic level in the cluster1. + admin1.topics().setReplicationClusters(topicName.toString(), List.of("r1", "r2")); + + PersistentTopic persistentTopic1 = (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName.toString(), + false) + .get() + .get(); + // Verify the replication from cluster1 to cluster2 is ready, but the replication form the cluster2 to cluster1 + // is not ready. + Awaitility.await().untilAsserted(() -> { + ConcurrentOpenHashMap replicatorMap = persistentTopic1.getReplicators(); + assertEquals(replicatorMap.size(), 1); + Replicator replicator = replicatorMap.get(replicatorMap.keys().get(0)); + assertTrue(replicator.isConnected()); + }); + + PersistentTopic persistentTopic2 = (PersistentTopic) pulsar2.getBrokerService().getTopic(topicName.toString(), + false) + .get() + .get(); + + Awaitility.await().untilAsserted(() -> { + ConcurrentOpenHashMap replicatorMap = persistentTopic2.getReplicators(); + assertEquals(replicatorMap.size(), 0); + }); + // Enable replication at the topic level in the cluster2. + admin2.topics().setReplicationClusters(topicName.toString(), List.of("r1", "r2")); + // Verify the replication between cluster1 and cluster2 is ready. + Awaitility.await().untilAsserted(() -> { + ConcurrentOpenHashMap replicatorMap = persistentTopic2.getReplicators(); + assertEquals(replicatorMap.size(), 1); + Replicator replicator = replicatorMap.get(replicatorMap.keys().get(0)); + assertTrue(replicator.isConnected()); + }); + } + private void pauseReplicator(PersistentReplicator replicator) { Awaitility.await().untilAsserted(() -> { assertTrue(replicator.isConnected()); diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java index fa9cf3ef21678..65124a6a76a8f 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java @@ -4680,4 +4680,87 @@ void setIsAllowAutoUpdateSchema(String namespace, boolean isAllowAutoUpdateSchem */ boolean getDispatcherPauseOnAckStatePersistent(String namespace) throws PulsarAdminException; + /** + * Get the allowed clusters for a namespace. + *

+ * Response example: + * + *

+     * ["use", "usw", "usc"]
+     * 
+ * + * @param namespace + * Namespace name + * @throws NotAuthorizedException + * Don't have admin permission + * @throws NotFoundException + * Namespace does not exist + * @throws PreconditionFailedException + * Namespace is not global + * @throws PulsarAdminException + * Unexpected error + */ + List getNamespaceAllowedClusters(String namespace) throws PulsarAdminException; + + /** + * Get the allowed clusters for a namespace asynchronously. + *

+ * Response example: + * + *

+     * ["use", "usw", "usc"]
+     * 
+ * + * @param namespace + * Namespace name + */ + CompletableFuture> getNamespaceAllowedClustersAsync(String namespace); + + /** + * Set the allowed clusters for a namespace. + *

+ * Request example: + * + *

+     * ["us-west", "us-east", "us-cent"]
+     * 
+ * + * @param namespace + * Namespace name + * @param clusterIds + * Pulsar Cluster Ids + * + * @throws ConflictException + * Peer-cluster cannot be part of an allowed-cluster + * @throws NotAuthorizedException + * Don't have admin permission + * @throws NotFoundException + * Namespace does not exist + * @throws PreconditionFailedException + * Namespace is not global + * @throws PreconditionFailedException + * Invalid cluster ids + * @throws PulsarAdminException + * The list of allowed clusters should include all replication clusters. + * @throws PulsarAdminException + * Unexpected error + */ + void setNamespaceAllowedClusters(String namespace, Set clusterIds) throws PulsarAdminException; + + /** + * Set the allowed clusters for a namespace asynchronously. + *

+ * Request example: + * + *

+     * ["us-west", "us-east", "us-cent"]
+     * 
+ * + * @param namespace + * Namespace name + * @param clusterIds + * Pulsar Cluster Ids + */ + CompletableFuture setNamespaceAllowedClustersAsync(String namespace, Set clusterIds); + } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java index 48e02b705ed71..d5e08a1f50cc0 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java @@ -36,6 +36,8 @@ public class Policies { public final AuthPolicies auth_policies = AuthPolicies.builder().build(); @SuppressWarnings("checkstyle:MemberName") public Set replication_clusters = new HashSet<>(); + @SuppressWarnings("checkstyle:MemberName") + public Set allowed_clusters = new HashSet<>(); public BundlesData bundles; @SuppressWarnings("checkstyle:MemberName") public Map backlog_quota_map = new HashMap<>(); @@ -139,7 +141,7 @@ public enum BundleType { @Override public int hashCode() { - return Objects.hash(auth_policies, replication_clusters, + return Objects.hash(auth_policies, replication_clusters, allowed_clusters, backlog_quota_map, publishMaxMessageRate, clusterDispatchRate, topicDispatchRate, subscriptionDispatchRate, replicatorDispatchRate, clusterSubscribeRate, deduplicationEnabled, autoTopicCreationOverride, @@ -170,6 +172,7 @@ public boolean equals(Object obj) { Policies other = (Policies) obj; return Objects.equals(auth_policies, other.auth_policies) && Objects.equals(replication_clusters, other.replication_clusters) + && Objects.equals(allowed_clusters, other.allowed_clusters) && Objects.equals(backlog_quota_map, other.backlog_quota_map) && Objects.equals(clusterDispatchRate, other.clusterDispatchRate) && Objects.equals(topicDispatchRate, other.topicDispatchRate) diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java index 5f70444fa0a76..c7492a26ab324 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java @@ -1993,4 +1993,28 @@ public CompletableFuture getDispatcherPauseOnAckStatePersistentAsync(St public boolean getDispatcherPauseOnAckStatePersistent(String namespace) throws PulsarAdminException { return sync(() -> getDispatcherPauseOnAckStatePersistentAsync(namespace)); } + + @Override + public List getNamespaceAllowedClusters(String namespace) throws PulsarAdminException { + return sync(() -> getNamespaceAllowedClustersAsync(namespace)); + } + + @Override + public CompletableFuture> getNamespaceAllowedClustersAsync(String namespace) { + return asyncGetNamespaceParts(new FutureCallback>(){}, namespace, "allowedClusters"); + } + + @Override + public void setNamespaceAllowedClusters(String namespace, Set clusterIds) throws PulsarAdminException { + sync(() -> setNamespaceAllowedClustersAsync(namespace, clusterIds)); + } + + @Override + public CompletableFuture setNamespaceAllowedClustersAsync(String namespace, Set clusterIds) { + NamespaceName ns = NamespaceName.get(namespace); + WebTarget path = namespacePath(ns, "allowedClusters"); + return asyncPostRequest(path, Entity.entity(clusterIds, MediaType.APPLICATION_JSON)); + } + + } diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java index a3b1fa075cffc..6e9782a0c2b91 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java @@ -437,7 +437,15 @@ public void namespaces() throws Exception { namespaces.run(split("get-clusters myprop/clust/ns1")); verify(mockNamespaces).getNamespaceReplicationClusters("myprop/clust/ns1"); - namespaces.run(split("set-subscription-types-enabled myprop/clust/ns1 -t Shared,Failover")); + namespaces.run(split("set-allowed-clusters myprop/clust/ns1 -c use,usw,usc")); + verify(mockNamespaces).setNamespaceAllowedClusters("myprop/clust/ns1", + Sets.newHashSet("use", "usw", "usc")); + + namespaces.run(split("get-allowed-clusters myprop/clust/ns1")); + verify(mockNamespaces).getNamespaceAllowedClusters("myprop/clust/ns1"); + + + namespaces.run(split("set-subscription-types-enabled myprop/clust/ns1 -t Shared,Failover")); verify(mockNamespaces).setSubscriptionTypesEnabled("myprop/clust/ns1", Sets.newHashSet(SubscriptionType.Shared, SubscriptionType.Failover)); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java index da8929da97cca..e8e644b688029 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java @@ -2630,6 +2630,35 @@ void run() throws PulsarAdminException { } } + @Command(description = "Set allowed clusters for a namespace") + private class SetAllowedClusters extends CliCommand { + @Parameters(description = "tenant/namespace", arity = "1") + private String namespaceName; + + @Option(names = { "--clusters", + "-c" }, description = "Replication Cluster Ids list (comma separated values)", required = true) + private String clusterIds; + + @Override + void run() throws PulsarAdminException { + String namespace = validateNamespace(namespaceName); + List clusters = Lists.newArrayList(clusterIds.split(",")); + getAdmin().namespaces().setNamespaceAllowedClusters(namespace, Sets.newHashSet(clusters)); + } + } + + @Command(description = "Get allowed clusters for a namespace") + private class GetAllowedClusters extends CliCommand { + @Parameters(description = "tenant/namespace", arity = "1") + private String namespaceName; + + @Override + void run() throws PulsarAdminException { + String namespace = validateNamespace(namespaceName); + print(getAdmin().namespaces().getNamespaceAllowedClusters(namespace)); + } + } + public CmdNamespaces(Supplier admin) { super("namespaces", admin); addCommand("list", new GetNamespacesPerProperty()); @@ -2657,6 +2686,9 @@ public CmdNamespaces(Supplier admin) { addCommand("get-subscription-types-enabled", new GetSubscriptionTypesEnabled()); addCommand("remove-subscription-types-enabled", new RemoveSubscriptionTypesEnabled()); + addCommand("set-allowed-clusters", new SetAllowedClusters()); + addCommand("get-allowed-clusters", new GetAllowedClusters()); + addCommand("get-backlog-quotas", new GetBacklogQuotaMap()); addCommand("set-backlog-quota", new SetBacklogQuota()); addCommand("remove-backlog-quota", new RemoveBacklogQuota()); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PolicyName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PolicyName.java index 69dc576fcf892..86ab545215e99 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PolicyName.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/PolicyName.java @@ -52,5 +52,6 @@ public enum PolicyName { RESOURCEGROUP, ENTRY_FILTERS, SHADOW_TOPIC, - DISPATCHER_PAUSE_ON_ACK_STATE_PERSISTENT + DISPATCHER_PAUSE_ON_ACK_STATE_PERSISTENT, + ALLOW_CLUSTERS } From ada47a327e08c1866c2b6f102c844e4c83fe93f3 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 20 Jun 2024 15:47:25 +0300 Subject: [PATCH 327/580] [fix][fn] Support compression type and crypto config for all producers in Functions and Connectors (#22950) --- .../functions/instance/ContextImpl.java | 65 +++---- .../instance/ProducerBuilderFactory.java | 159 ++++++++++++++++ .../pulsar/functions/sink/PulsarSink.java | 158 +++------------- .../src/main/resources/findbugsExclude.xml | 7 +- .../functions/instance/ContextImplTest.java | 4 +- .../instance/ProducerBuilderFactoryTest.java | 178 ++++++++++++++++++ .../functions/utils/FunctionConfigUtils.java | 87 +++++---- .../functions/utils/SourceConfigUtils.java | 47 +---- .../utils/FunctionConfigUtilsTest.java | 35 ++++ 9 files changed, 480 insertions(+), 260 deletions(-) create mode 100644 pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerBuilderFactory.java create mode 100644 pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerBuilderFactoryTest.java diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java index 6664a00510e56..eeeaa8b3627e9 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java @@ -27,6 +27,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; import java.util.List; @@ -40,14 +41,11 @@ import lombok.ToString; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.api.BatcherBuilder; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; -import org.apache.pulsar.client.api.HashingScheme; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -55,7 +53,7 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; -import org.apache.pulsar.client.impl.ProducerBuilderImpl; +import org.apache.pulsar.common.functions.ProducerConfig; import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.naming.TopicName; @@ -77,6 +75,7 @@ import org.apache.pulsar.functions.source.PulsarFunctionRecord; import org.apache.pulsar.functions.source.TopicSchema; import org.apache.pulsar.functions.utils.FunctionCommon; +import org.apache.pulsar.functions.utils.FunctionConfigUtils; import org.apache.pulsar.functions.utils.SinkConfigUtils; import org.apache.pulsar.functions.utils.SourceConfigUtils; import org.apache.pulsar.io.core.SinkContext; @@ -88,6 +87,8 @@ */ @ToString(exclude = {"pulsarAdmin"}) class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable { + private final ProducerBuilderFactory producerBuilderFactory; + private final Map producerProperties; private InstanceConfig config; private Logger logger; @@ -99,7 +100,6 @@ class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable private final PulsarAdmin pulsarAdmin; private Map> publishProducers; private ThreadLocal>> tlPublishProducers; - private ProducerBuilderImpl producerBuilder; private final TopicSchema topicSchema; @@ -154,27 +154,27 @@ public ContextImpl(InstanceConfig config, Logger logger, PulsarClient client, this.statsManager = statsManager; this.fatalHandler = fatalHandler; - this.producerBuilder = (ProducerBuilderImpl) client.newProducer().blockIfQueueFull(true).enableBatching(true) - .batchingMaxPublishDelay(1, TimeUnit.MILLISECONDS); boolean useThreadLocalProducers = false; + Function.ProducerSpec producerSpec = config.getFunctionDetails().getSink().getProducerSpec(); + ProducerConfig producerConfig = null; if (producerSpec != null) { - if (producerSpec.getMaxPendingMessages() != 0) { - this.producerBuilder.maxPendingMessages(producerSpec.getMaxPendingMessages()); - } - if (producerSpec.getMaxPendingMessagesAcrossPartitions() != 0) { - this.producerBuilder - .maxPendingMessagesAcrossPartitions(producerSpec.getMaxPendingMessagesAcrossPartitions()); - } - if (producerSpec.getBatchBuilder() != null) { - if (producerSpec.getBatchBuilder().equals("KEY_BASED")) { - this.producerBuilder.batcherBuilder(BatcherBuilder.KEY_BASED); - } else { - this.producerBuilder.batcherBuilder(BatcherBuilder.DEFAULT); - } - } + producerConfig = FunctionConfigUtils.convertProducerSpecToProducerConfig(producerSpec); useThreadLocalProducers = producerSpec.getUseThreadLocalProducers(); } + producerBuilderFactory = new ProducerBuilderFactory(client, producerConfig, + Thread.currentThread().getContextClassLoader(), + // This is for backwards compatibility. The PR https://github.com/apache/pulsar/pull/19470 removed + // the default and made it configurable for the producers created in PulsarSink, but not in ContextImpl. + // This is to keep the default unchanged for the producers created in ContextImpl. + producerBuilder -> producerBuilder.compressionType(CompressionType.LZ4)); + producerProperties = Collections.unmodifiableMap(InstanceUtils.getProperties(componentType, + FunctionCommon.getFullyQualifiedName( + this.config.getFunctionDetails().getTenant(), + this.config.getFunctionDetails().getNamespace(), + this.config.getFunctionDetails().getName()), + this.config.getInstanceId())); + if (useThreadLocalProducers) { tlPublishProducers = new ThreadLocal<>(); } else { @@ -556,26 +556,9 @@ private Producer getProducer(String topicName, Schema schema) throws P } if (producer == null) { - - Producer newProducer = ((ProducerBuilderImpl) producerBuilder.clone()) - .schema(schema) - .blockIfQueueFull(true) - .enableBatching(true) - .batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS) - .compressionType(CompressionType.LZ4) - .hashingScheme(HashingScheme.Murmur3_32Hash) // - .messageRoutingMode(MessageRoutingMode.CustomPartition) - .messageRouter(FunctionResultRouter.of()) - // set send timeout to be infinity to prevent potential deadlock with consumer - // that might happen when consumer is blocked due to unacked messages - .sendTimeout(0, TimeUnit.SECONDS) - .topic(topicName) - .properties(InstanceUtils.getProperties(componentType, - FunctionCommon.getFullyQualifiedName( - this.config.getFunctionDetails().getTenant(), - this.config.getFunctionDetails().getNamespace(), - this.config.getFunctionDetails().getName()), - this.config.getInstanceId())) + Producer newProducer = producerBuilderFactory + .createProducerBuilder(topicName, schema, null) + .properties(producerProperties) .create(); if (tlPublishProducers != null) { diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerBuilderFactory.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerBuilderFactory.java new file mode 100644 index 0000000000000..b08f7f3f2cb0f --- /dev/null +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerBuilderFactory.java @@ -0,0 +1,159 @@ +/* + * 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.functions.instance; + +import static org.apache.commons.lang.StringUtils.isEmpty; +import com.google.common.annotations.VisibleForTesting; +import java.security.Security; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import lombok.Builder; +import lombok.Data; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.BatcherBuilder; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.CryptoKeyReader; +import org.apache.pulsar.client.api.HashingScheme; +import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.ProducerCryptoFailureAction; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.functions.CryptoConfig; +import org.apache.pulsar.common.functions.ProducerConfig; +import org.apache.pulsar.functions.utils.CryptoUtils; +import org.bouncycastle.jce.provider.BouncyCastleProvider; + +/** + * This class is responsible for creating ProducerBuilders with the appropriate configurations to + * match the ProducerConfig provided. Producers are created in 2 locations in Pulsar Functions and Connectors + * and this class is used to unify the configuration of the producers without duplicating code. + */ +@Slf4j +public class ProducerBuilderFactory { + + private final PulsarClient client; + private final ProducerConfig producerConfig; + private final Consumer> defaultConfigurer; + private final Crypto crypto; + + public ProducerBuilderFactory(PulsarClient client, ProducerConfig producerConfig, ClassLoader functionClassLoader, + Consumer> defaultConfigurer) { + this.client = client; + this.producerConfig = producerConfig; + this.defaultConfigurer = defaultConfigurer; + try { + this.crypto = initializeCrypto(functionClassLoader); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Unable to initialize crypto config " + producerConfig.getCryptoConfig(), e); + } + if (crypto == null) { + log.info("crypto key reader is not provided, not enabling end to end encryption"); + } + } + + public ProducerBuilder createProducerBuilder(String topic, Schema schema, String producerName) { + ProducerBuilder builder = client.newProducer(schema); + if (defaultConfigurer != null) { + defaultConfigurer.accept(builder); + } + builder.blockIfQueueFull(true) + .enableBatching(true) + .batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS) + .hashingScheme(HashingScheme.Murmur3_32Hash) // + .messageRoutingMode(MessageRoutingMode.CustomPartition) + .messageRouter(FunctionResultRouter.of()) + // set send timeout to be infinity to prevent potential deadlock with consumer + // that might happen when consumer is blocked due to unacked messages + .sendTimeout(0, TimeUnit.SECONDS) + .topic(topic); + if (producerName != null) { + builder.producerName(producerName); + } + if (producerConfig != null) { + if (producerConfig.getCompressionType() != null) { + builder.compressionType(producerConfig.getCompressionType()); + } else { + // TODO: address this inconsistency. + // PR https://github.com/apache/pulsar/pull/19470 removed the default compression type of LZ4 + // from the top level. This default is only used if producer config is provided. + builder.compressionType(CompressionType.LZ4); + } + if (producerConfig.getMaxPendingMessages() != null && producerConfig.getMaxPendingMessages() != 0) { + builder.maxPendingMessages(producerConfig.getMaxPendingMessages()); + } + if (producerConfig.getMaxPendingMessagesAcrossPartitions() != null + && producerConfig.getMaxPendingMessagesAcrossPartitions() != 0) { + builder.maxPendingMessagesAcrossPartitions(producerConfig.getMaxPendingMessagesAcrossPartitions()); + } + if (producerConfig.getCryptoConfig() != null) { + builder.cryptoKeyReader(crypto.keyReader); + builder.cryptoFailureAction(crypto.failureAction); + for (String encryptionKeyName : crypto.getEncryptionKeys()) { + builder.addEncryptionKey(encryptionKeyName); + } + } + if (producerConfig.getBatchBuilder() != null) { + if (producerConfig.getBatchBuilder().equals("KEY_BASED")) { + builder.batcherBuilder(BatcherBuilder.KEY_BASED); + } else { + builder.batcherBuilder(BatcherBuilder.DEFAULT); + } + } + } + return builder; + } + + + @SuppressWarnings("unchecked") + @VisibleForTesting + Crypto initializeCrypto(ClassLoader functionClassLoader) throws ClassNotFoundException { + if (producerConfig == null + || producerConfig.getCryptoConfig() == null + || isEmpty(producerConfig.getCryptoConfig().getCryptoKeyReaderClassName())) { + return null; + } + + CryptoConfig cryptoConfig = producerConfig.getCryptoConfig(); + + // add provider only if it's not in the JVM + if (Security.getProvider(BouncyCastleProvider.PROVIDER_NAME) == null) { + Security.addProvider(new BouncyCastleProvider()); + } + + final String[] encryptionKeys = cryptoConfig.getEncryptionKeys(); + Crypto.CryptoBuilder bldr = Crypto.builder() + .failureAction(cryptoConfig.getProducerCryptoFailureAction()) + .encryptionKeys(encryptionKeys); + + bldr.keyReader(CryptoUtils.getCryptoKeyReaderInstance( + cryptoConfig.getCryptoKeyReaderClassName(), cryptoConfig.getCryptoKeyReaderConfig(), + functionClassLoader)); + + return bldr.build(); + } + + @Data + @Builder + private static class Crypto { + private CryptoKeyReader keyReader; + private ProducerCryptoFailureAction failureAction; + private String[] encryptionKeys; + } +} diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java index 97a0ad0a2ce17..18e55e8e84de1 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java @@ -18,10 +18,8 @@ */ package org.apache.pulsar.functions.sink; -import static org.apache.commons.lang.StringUtils.isEmpty; import com.google.common.annotations.VisibleForTesting; import java.nio.charset.StandardCharsets; -import java.security.Security; import java.util.ArrayList; import java.util.Base64; import java.util.List; @@ -29,21 +27,12 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; import java.util.function.Function; -import lombok.Builder; -import lombok.Data; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.client.api.BatcherBuilder; -import org.apache.pulsar.client.api.CompressionType; -import org.apache.pulsar.client.api.CryptoKeyReader; -import org.apache.pulsar.client.api.HashingScheme; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; -import org.apache.pulsar.client.api.ProducerCryptoFailureAction; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; @@ -52,22 +41,18 @@ import org.apache.pulsar.client.api.schema.KeyValueSchema; import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.common.functions.ConsumerConfig; -import org.apache.pulsar.common.functions.CryptoConfig; import org.apache.pulsar.common.functions.FunctionConfig; -import org.apache.pulsar.common.functions.ProducerConfig; import org.apache.pulsar.common.schema.KeyValueEncodingType; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.util.Reflections; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.instance.AbstractSinkRecord; -import org.apache.pulsar.functions.instance.FunctionResultRouter; +import org.apache.pulsar.functions.instance.ProducerBuilderFactory; import org.apache.pulsar.functions.instance.stats.ComponentStatsManager; import org.apache.pulsar.functions.source.PulsarRecord; import org.apache.pulsar.functions.source.TopicSchema; -import org.apache.pulsar.functions.utils.CryptoUtils; import org.apache.pulsar.io.core.Sink; import org.apache.pulsar.io.core.SinkContext; -import org.bouncycastle.jce.provider.BouncyCastleProvider; @Slf4j public class PulsarSink implements Sink { @@ -82,6 +67,8 @@ public class PulsarSink implements Sink { PulsarSinkProcessor pulsarSinkProcessor; private final TopicSchema topicSchema; + private Schema schema; + private ProducerBuilderFactory producerBuilderFactory; private interface PulsarSinkProcessor { @@ -94,60 +81,6 @@ private interface PulsarSinkProcessor { abstract class PulsarSinkProcessorBase implements PulsarSinkProcessor { protected Map> publishProducers = new ConcurrentHashMap<>(); - protected Schema schema; - protected Crypto crypto; - - protected PulsarSinkProcessorBase(Schema schema, Crypto crypto) { - this.schema = schema; - this.crypto = crypto; - } - - public Producer createProducer(PulsarClient client, String topic, String producerName, Schema schema) - throws PulsarClientException { - ProducerBuilder builder = client.newProducer(schema) - .blockIfQueueFull(true) - .enableBatching(true) - .batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS) - .hashingScheme(HashingScheme.Murmur3_32Hash) // - .messageRoutingMode(MessageRoutingMode.CustomPartition) - .messageRouter(FunctionResultRouter.of()) - // set send timeout to be infinity to prevent potential deadlock with consumer - // that might happen when consumer is blocked due to unacked messages - .sendTimeout(0, TimeUnit.SECONDS) - .topic(topic); - if (producerName != null) { - builder.producerName(producerName); - } - if (pulsarSinkConfig.getProducerConfig() != null) { - ProducerConfig producerConfig = pulsarSinkConfig.getProducerConfig(); - if (producerConfig.getCompressionType() != null) { - builder.compressionType(producerConfig.getCompressionType()); - } else { - builder.compressionType(CompressionType.LZ4); - } - if (producerConfig.getMaxPendingMessages() != 0) { - builder.maxPendingMessages(producerConfig.getMaxPendingMessages()); - } - if (producerConfig.getMaxPendingMessagesAcrossPartitions() != 0) { - builder.maxPendingMessagesAcrossPartitions(producerConfig.getMaxPendingMessagesAcrossPartitions()); - } - if (producerConfig.getCryptoConfig() != null) { - builder.cryptoKeyReader(crypto.keyReader); - builder.cryptoFailureAction(crypto.failureAction); - for (String encryptionKeyName : crypto.getEncryptionKeys()) { - builder.addEncryptionKey(encryptionKeyName); - } - } - if (producerConfig.getBatchBuilder() != null) { - if (producerConfig.getBatchBuilder().equals("KEY_BASED")) { - builder.batcherBuilder(BatcherBuilder.KEY_BASED); - } else { - builder.batcherBuilder(BatcherBuilder.DEFAULT); - } - } - } - return builder.properties(properties).create(); - } protected Producer getProducer(String destinationTopic, Schema schema) { return getProducer(destinationTopic, null, destinationTopic, schema); @@ -159,10 +92,9 @@ protected Producer getProducer(String producerId, String producerName, String log.info("Initializing producer {} on topic {} with schema {}", producerName, topicName, schema); Producer producer = createProducer( - client, topicName, - producerName, - schema != null ? schema : this.schema); + schema, producerName + ); log.info("Initialized producer {} on topic {} with schema {}: {} -> {}", producerName, topicName, schema, producerId, producer); return producer; @@ -218,13 +150,12 @@ public Function getPublishErrorHandler(AbstractSinkRecord re @VisibleForTesting class PulsarSinkAtMostOnceProcessor extends PulsarSinkProcessorBase { - public PulsarSinkAtMostOnceProcessor(Schema schema, Crypto crypto) { - super(schema, crypto); + public PulsarSinkAtMostOnceProcessor() { if (!(schema instanceof AutoConsumeSchema)) { // initialize default topic try { publishProducers.put(pulsarSinkConfig.getTopic(), - createProducer(client, pulsarSinkConfig.getTopic(), null, schema)); + createProducer(pulsarSinkConfig.getTopic(), schema, null)); } catch (PulsarClientException e) { log.error("Failed to create Producer while doing user publish", e); throw new RuntimeException(e); @@ -270,10 +201,6 @@ public void sendOutputMessage(TypedMessageBuilder msg, AbstractSinkRecord @VisibleForTesting class PulsarSinkAtLeastOnceProcessor extends PulsarSinkAtMostOnceProcessor { - public PulsarSinkAtLeastOnceProcessor(Schema schema, Crypto crypto) { - super(schema, crypto); - } - @Override public void sendOutputMessage(TypedMessageBuilder msg, AbstractSinkRecord record) { msg.sendAsync() @@ -284,11 +211,6 @@ public void sendOutputMessage(TypedMessageBuilder msg, AbstractSinkRecord @VisibleForTesting class PulsarSinkManualProcessor extends PulsarSinkAtMostOnceProcessor { - - public PulsarSinkManualProcessor(Schema schema, Crypto crypto) { - super(schema, crypto); - } - @Override public void sendOutputMessage(TypedMessageBuilder msg, AbstractSinkRecord record) { super.sendOutputMessage(msg, record); @@ -297,11 +219,6 @@ public void sendOutputMessage(TypedMessageBuilder msg, AbstractSinkRecord @VisibleForTesting class PulsarSinkEffectivelyOnceProcessor extends PulsarSinkProcessorBase { - - public PulsarSinkEffectivelyOnceProcessor(Schema schema, Crypto crypto) { - super(schema, crypto); - } - @Override public TypedMessageBuilder newMessage(AbstractSinkRecord record) { if (!record.getPartitionId().isPresent()) { @@ -359,30 +276,27 @@ public PulsarSink(PulsarClient client, PulsarSinkConfig pulsarSinkConfig, Map config, SinkContext sinkContext) throws Exception { log.info("Opening pulsar sink with config: {}", pulsarSinkConfig); - Schema schema = initializeSchema(); + schema = initializeSchema(); if (schema == null) { log.info("Since output type is null, not creating any real sink"); return; } - - Crypto crypto = initializeCrypto(); - if (crypto == null) { - log.info("crypto key reader is not provided, not enabling end to end encryption"); - } + producerBuilderFactory = + new ProducerBuilderFactory(client, pulsarSinkConfig.getProducerConfig(), functionClassLoader, null); FunctionConfig.ProcessingGuarantees processingGuarantees = this.pulsarSinkConfig.getProcessingGuarantees(); switch (processingGuarantees) { case ATMOST_ONCE: - this.pulsarSinkProcessor = new PulsarSinkAtMostOnceProcessor(schema, crypto); + this.pulsarSinkProcessor = new PulsarSinkAtMostOnceProcessor(); break; case ATLEAST_ONCE: - this.pulsarSinkProcessor = new PulsarSinkAtLeastOnceProcessor(schema, crypto); + this.pulsarSinkProcessor = new PulsarSinkAtLeastOnceProcessor(); break; case EFFECTIVELY_ONCE: - this.pulsarSinkProcessor = new PulsarSinkEffectivelyOnceProcessor(schema, crypto); + this.pulsarSinkProcessor = new PulsarSinkEffectivelyOnceProcessor(); break; case MANUAL: - this.pulsarSinkProcessor = new PulsarSinkManualProcessor(schema, crypto); + this.pulsarSinkProcessor = new PulsarSinkManualProcessor(); break; } } @@ -427,6 +341,16 @@ public void close() throws Exception { } } + Producer createProducer(String topic, Schema schema, String producerName) + throws PulsarClientException { + ProducerBuilder builder = + producerBuilderFactory.createProducerBuilder(topic, schema != null ? schema : this.schema, + producerName); + return builder + .properties(properties) + .create(); + } + @SuppressWarnings("unchecked") @VisibleForTesting Schema initializeSchema() throws ClassNotFoundException { @@ -461,39 +385,5 @@ Schema initializeSchema() throws ClassNotFoundException { } } - @SuppressWarnings("unchecked") - @VisibleForTesting - Crypto initializeCrypto() throws ClassNotFoundException { - if (pulsarSinkConfig.getProducerConfig() == null - || pulsarSinkConfig.getProducerConfig().getCryptoConfig() == null - || isEmpty(pulsarSinkConfig.getProducerConfig().getCryptoConfig().getCryptoKeyReaderClassName())) { - return null; - } - - CryptoConfig cryptoConfig = pulsarSinkConfig.getProducerConfig().getCryptoConfig(); - - // add provider only if it's not in the JVM - if (Security.getProvider(BouncyCastleProvider.PROVIDER_NAME) == null) { - Security.addProvider(new BouncyCastleProvider()); - } - - final String[] encryptionKeys = cryptoConfig.getEncryptionKeys(); - Crypto.CryptoBuilder bldr = Crypto.builder() - .failureAction(cryptoConfig.getProducerCryptoFailureAction()) - .encryptionKeys(encryptionKeys); - bldr.keyReader(CryptoUtils.getCryptoKeyReaderInstance( - cryptoConfig.getCryptoKeyReaderClassName(), cryptoConfig.getCryptoKeyReaderConfig(), - functionClassLoader)); - - return bldr.build(); - } - - @Data - @Builder - private static class Crypto { - private CryptoKeyReader keyReader; - private ProducerCryptoFailureAction failureAction; - private String[] encryptionKeys; - } } diff --git a/pulsar-functions/instance/src/main/resources/findbugsExclude.xml b/pulsar-functions/instance/src/main/resources/findbugsExclude.xml index 7fe247d2ab20a..40e3e91112328 100644 --- a/pulsar-functions/instance/src/main/resources/findbugsExclude.xml +++ b/pulsar-functions/instance/src/main/resources/findbugsExclude.xml @@ -49,7 +49,12 @@ - + + + + + + diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java index 6516b9284c9ca..115ef1e8a3f2b 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java @@ -103,7 +103,9 @@ public void setup() throws PulsarClientException { client = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); when(client.getCnxPool()).thenReturn(connectionPool); - when(client.newProducer()).thenReturn(new ProducerBuilderImpl(client, Schema.BYTES)); + when(client.newProducer()).thenAnswer(invocation -> new ProducerBuilderImpl(client, Schema.BYTES)); + when(client.newProducer(any())).thenAnswer( + invocation -> new ProducerBuilderImpl(client, invocation.getArgument(0))); when(client.createProducerAsync(any(ProducerConfigurationData.class), any(), any())) .thenReturn(CompletableFuture.completedFuture(producer)); when(client.getSchema(anyString())).thenReturn(CompletableFuture.completedFuture(Optional.empty())); diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerBuilderFactoryTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerBuilderFactoryTest.java new file mode 100644 index 0000000000000..42940f7e2dae3 --- /dev/null +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerBuilderFactoryTest.java @@ -0,0 +1,178 @@ +/* + * 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.functions.instance; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.client.api.BatcherBuilder; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.CryptoKeyReader; +import org.apache.pulsar.client.api.EncryptionKeyInfo; +import org.apache.pulsar.client.api.HashingScheme; +import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.ProducerCryptoFailureAction; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.functions.CryptoConfig; +import org.apache.pulsar.common.functions.ProducerConfig; +import org.mockito.internal.util.MockUtil; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class ProducerBuilderFactoryTest { + private PulsarClient pulsarClient; + private ProducerBuilder producerBuilder; + + @BeforeMethod + public void setup() { + pulsarClient = mock(PulsarClient.class); + + producerBuilder = mock(ProducerBuilder.class); + doReturn(producerBuilder).when(producerBuilder).blockIfQueueFull(anyBoolean()); + doReturn(producerBuilder).when(producerBuilder).enableBatching(anyBoolean()); + doReturn(producerBuilder).when(producerBuilder).batchingMaxPublishDelay(anyLong(), any()); + doReturn(producerBuilder).when(producerBuilder).compressionType(any()); + doReturn(producerBuilder).when(producerBuilder).hashingScheme(any()); + doReturn(producerBuilder).when(producerBuilder).messageRoutingMode(any()); + doReturn(producerBuilder).when(producerBuilder).messageRouter(any()); + doReturn(producerBuilder).when(producerBuilder).topic(anyString()); + doReturn(producerBuilder).when(producerBuilder).producerName(anyString()); + doReturn(producerBuilder).when(producerBuilder).property(anyString(), anyString()); + doReturn(producerBuilder).when(producerBuilder).properties(any()); + doReturn(producerBuilder).when(producerBuilder).sendTimeout(anyInt(), any()); + + doReturn(producerBuilder).when(pulsarClient).newProducer(); + doReturn(producerBuilder).when(pulsarClient).newProducer(any()); + } + + @AfterMethod + public void tearDown() { + MockUtil.resetMock(pulsarClient); + pulsarClient = null; + MockUtil.resetMock(producerBuilder); + producerBuilder = null; + TestCryptoKeyReader.LAST_INSTANCE = null; + } + + @Test + public void testCreateProducerBuilder() { + ProducerBuilderFactory builderFactory = new ProducerBuilderFactory(pulsarClient, null, null, null); + builderFactory.createProducerBuilder("topic", Schema.STRING, "producerName"); + verifyCommon(); + verifyNoMoreInteractions(producerBuilder); + } + + private void verifyCommon() { + verify(pulsarClient).newProducer(Schema.STRING); + verify(producerBuilder).blockIfQueueFull(true); + verify(producerBuilder).enableBatching(true); + verify(producerBuilder).batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS); + verify(producerBuilder).hashingScheme(HashingScheme.Murmur3_32Hash); + verify(producerBuilder).messageRoutingMode(MessageRoutingMode.CustomPartition); + verify(producerBuilder).messageRouter(FunctionResultRouter.of()); + verify(producerBuilder).sendTimeout(0, TimeUnit.SECONDS); + verify(producerBuilder).topic("topic"); + verify(producerBuilder).producerName("producerName"); + } + + @Test + public void testCreateProducerBuilderWithDefaultConfigurer() { + ProducerBuilderFactory builderFactory = new ProducerBuilderFactory(pulsarClient, null, null, + builder -> builder.property("key", "value")); + builderFactory.createProducerBuilder("topic", Schema.STRING, "producerName"); + verifyCommon(); + verify(producerBuilder).property("key", "value"); + verifyNoMoreInteractions(producerBuilder); + } + + @Test + public void testCreateProducerBuilderWithSimpleProducerConfig() { + ProducerConfig producerConfig = new ProducerConfig(); + producerConfig.setBatchBuilder("KEY_BASED"); + ProducerBuilderFactory builderFactory = new ProducerBuilderFactory(pulsarClient, producerConfig, null, null); + builderFactory.createProducerBuilder("topic", Schema.STRING, "producerName"); + verifyCommon(); + verify(producerBuilder).compressionType(CompressionType.LZ4); + verify(producerBuilder).batcherBuilder(BatcherBuilder.KEY_BASED); + verifyNoMoreInteractions(producerBuilder); + } + + @Test + public void testCreateProducerBuilderWithAdvancedProducerConfig() { + ProducerConfig producerConfig = new ProducerConfig(); + producerConfig.setBatchBuilder("KEY_BASED"); + producerConfig.setCompressionType(CompressionType.SNAPPY); + producerConfig.setMaxPendingMessages(5000); + producerConfig.setMaxPendingMessagesAcrossPartitions(50000); + CryptoConfig cryptoConfig = new CryptoConfig(); + cryptoConfig.setProducerCryptoFailureAction(ProducerCryptoFailureAction.FAIL); + cryptoConfig.setEncryptionKeys(new String[]{"key1", "key2"}); + cryptoConfig.setCryptoKeyReaderConfig(Map.of("key", "value")); + cryptoConfig.setCryptoKeyReaderClassName(TestCryptoKeyReader.class.getName()); + producerConfig.setCryptoConfig(cryptoConfig); + ProducerBuilderFactory builderFactory = new ProducerBuilderFactory(pulsarClient, producerConfig, null, null); + builderFactory.createProducerBuilder("topic", Schema.STRING, "producerName"); + verifyCommon(); + verify(producerBuilder).compressionType(CompressionType.SNAPPY); + verify(producerBuilder).batcherBuilder(BatcherBuilder.KEY_BASED); + verify(producerBuilder).maxPendingMessages(5000); + verify(producerBuilder).maxPendingMessagesAcrossPartitions(50000); + TestCryptoKeyReader lastInstance = TestCryptoKeyReader.LAST_INSTANCE; + assertNotNull(lastInstance); + assertEquals(lastInstance.configs, cryptoConfig.getCryptoKeyReaderConfig()); + verify(producerBuilder).cryptoKeyReader(lastInstance); + verify(producerBuilder).cryptoFailureAction(ProducerCryptoFailureAction.FAIL); + verify(producerBuilder).addEncryptionKey("key1"); + verify(producerBuilder).addEncryptionKey("key2"); + verifyNoMoreInteractions(producerBuilder); + } + + public static class TestCryptoKeyReader implements CryptoKeyReader { + static TestCryptoKeyReader LAST_INSTANCE; + Map configs; + public TestCryptoKeyReader(Map configs) { + this.configs = configs; + assert LAST_INSTANCE == null; + LAST_INSTANCE = this; + } + + @Override + public EncryptionKeyInfo getPublicKey(String keyName, Map metadata) { + throw new UnsupportedOperationException(); + } + + @Override + public EncryptionKeyInfo getPrivateKey(String keyName, Map metadata) { + throw new UnsupportedOperationException(); + } + } +} \ No newline at end of file diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java index 9dc9d5428eda3..45fb4c1cb1ee7 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java @@ -250,29 +250,7 @@ public static FunctionDetails convert(FunctionConfig functionConfig, ExtractedFu sinkSpecBuilder.setTypeClassName(functionConfig.getOutputTypeClassName()); } if (functionConfig.getProducerConfig() != null) { - ProducerConfig producerConf = functionConfig.getProducerConfig(); - Function.ProducerSpec.Builder pbldr = Function.ProducerSpec.newBuilder(); - if (producerConf.getMaxPendingMessages() != null) { - pbldr.setMaxPendingMessages(producerConf.getMaxPendingMessages()); - } - if (producerConf.getMaxPendingMessagesAcrossPartitions() != null) { - pbldr.setMaxPendingMessagesAcrossPartitions(producerConf.getMaxPendingMessagesAcrossPartitions()); - } - if (producerConf.getUseThreadLocalProducers() != null) { - pbldr.setUseThreadLocalProducers(producerConf.getUseThreadLocalProducers()); - } - if (producerConf.getCryptoConfig() != null) { - pbldr.setCryptoSpec(CryptoUtils.convert(producerConf.getCryptoConfig())); - } - if (producerConf.getBatchBuilder() != null) { - pbldr.setBatchBuilder(producerConf.getBatchBuilder()); - } - if (producerConf.getCompressionType() != null) { - pbldr.setCompressionType(convertFromCompressionType(producerConf.getCompressionType())); - } else { - pbldr.setCompressionType(Function.CompressionType.LZ4); - } - sinkSpecBuilder.setProducerSpec(pbldr.build()); + sinkSpecBuilder.setProducerSpec(convertProducerConfigToProducerSpec(functionConfig.getProducerConfig())); } if (functionConfig.getBatchBuilder() != null) { Function.ProducerSpec.Builder builder = sinkSpecBuilder.getProducerSpec() != null @@ -463,23 +441,8 @@ public static FunctionConfig convertFromDetails(FunctionDetails functionDetails) functionConfig.setOutputSchemaType(functionDetails.getSink().getSchemaType()); } if (functionDetails.getSink().getProducerSpec() != null) { - Function.ProducerSpec spec = functionDetails.getSink().getProducerSpec(); - ProducerConfig producerConfig = new ProducerConfig(); - if (spec.getMaxPendingMessages() != 0) { - producerConfig.setMaxPendingMessages(spec.getMaxPendingMessages()); - } - if (spec.getMaxPendingMessagesAcrossPartitions() != 0) { - producerConfig.setMaxPendingMessagesAcrossPartitions(spec.getMaxPendingMessagesAcrossPartitions()); - } - if (spec.hasCryptoSpec()) { - producerConfig.setCryptoConfig(CryptoUtils.convertFromSpec(spec.getCryptoSpec())); - } - if (spec.getBatchBuilder() != null) { - producerConfig.setBatchBuilder(spec.getBatchBuilder()); - } - producerConfig.setUseThreadLocalProducers(spec.getUseThreadLocalProducers()); - producerConfig.setCompressionType(convertFromFunctionDetailsCompressionType(spec.getCompressionType())); - functionConfig.setProducerConfig(producerConfig); + functionConfig.setProducerConfig( + convertProducerSpecToProducerConfig(functionDetails.getSink().getProducerSpec())); } if (!isEmpty(functionDetails.getLogTopic())) { functionConfig.setLogTopic(functionDetails.getLogTopic()); @@ -544,6 +507,50 @@ public static FunctionConfig convertFromDetails(FunctionDetails functionDetails) return functionConfig; } + public static Function.ProducerSpec convertProducerConfigToProducerSpec(ProducerConfig producerConf) { + Function.ProducerSpec.Builder builder = Function.ProducerSpec.newBuilder(); + if (producerConf.getMaxPendingMessages() != null) { + builder.setMaxPendingMessages(producerConf.getMaxPendingMessages()); + } + if (producerConf.getMaxPendingMessagesAcrossPartitions() != null) { + builder.setMaxPendingMessagesAcrossPartitions(producerConf.getMaxPendingMessagesAcrossPartitions()); + } + if (producerConf.getUseThreadLocalProducers() != null) { + builder.setUseThreadLocalProducers(producerConf.getUseThreadLocalProducers()); + } + if (producerConf.getCryptoConfig() != null) { + builder.setCryptoSpec(CryptoUtils.convert(producerConf.getCryptoConfig())); + } + if (producerConf.getBatchBuilder() != null) { + builder.setBatchBuilder(producerConf.getBatchBuilder()); + } + if (producerConf.getCompressionType() != null) { + builder.setCompressionType(convertFromCompressionType(producerConf.getCompressionType())); + } else { + builder.setCompressionType(Function.CompressionType.LZ4); + } + return builder.build(); + } + + public static ProducerConfig convertProducerSpecToProducerConfig(Function.ProducerSpec spec) { + ProducerConfig producerConfig = new ProducerConfig(); + if (spec.getMaxPendingMessages() != 0) { + producerConfig.setMaxPendingMessages(spec.getMaxPendingMessages()); + } + if (spec.getMaxPendingMessagesAcrossPartitions() != 0) { + producerConfig.setMaxPendingMessagesAcrossPartitions(spec.getMaxPendingMessagesAcrossPartitions()); + } + if (spec.hasCryptoSpec()) { + producerConfig.setCryptoConfig(CryptoUtils.convertFromSpec(spec.getCryptoSpec())); + } + if (spec.getBatchBuilder() != null) { + producerConfig.setBatchBuilder(spec.getBatchBuilder()); + } + producerConfig.setUseThreadLocalProducers(spec.getUseThreadLocalProducers()); + producerConfig.setCompressionType(convertFromFunctionDetailsCompressionType(spec.getCompressionType())); + return producerConfig; + } + public static void inferMissingArguments(FunctionConfig functionConfig, boolean forwardSourceMessagePropertyEnabled) { if (StringUtils.isEmpty(functionConfig.getName())) { diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java index 692d7459268dd..6229bffff5317 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java @@ -19,10 +19,10 @@ package org.apache.pulsar.functions.utils; import static org.apache.commons.lang3.StringUtils.isEmpty; -import static org.apache.pulsar.functions.utils.FunctionCommon.convertFromCompressionType; -import static org.apache.pulsar.functions.utils.FunctionCommon.convertFromFunctionDetailsCompressionType; import static org.apache.pulsar.functions.utils.FunctionCommon.convertProcessingGuarantee; import static org.apache.pulsar.functions.utils.FunctionCommon.getSourceType; +import static org.apache.pulsar.functions.utils.FunctionConfigUtils.convertProducerConfigToProducerSpec; +import static org.apache.pulsar.functions.utils.FunctionConfigUtils.convertProducerSpecToProducerConfig; import com.fasterxml.jackson.core.type.TypeReference; import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; @@ -39,7 +39,6 @@ import net.bytebuddy.description.type.TypeDescription; import net.bytebuddy.pool.TypePool; import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.common.functions.ProducerConfig; import org.apache.pulsar.common.functions.Resources; import org.apache.pulsar.common.io.BatchSourceConfig; import org.apache.pulsar.common.io.ConnectorDefinition; @@ -152,29 +151,7 @@ public static FunctionDetails convert(SourceConfig sourceConfig, ExtractedSource } if (sourceConfig.getProducerConfig() != null) { - ProducerConfig conf = sourceConfig.getProducerConfig(); - Function.ProducerSpec.Builder pbldr = Function.ProducerSpec.newBuilder(); - if (conf.getMaxPendingMessages() != null) { - pbldr.setMaxPendingMessages(conf.getMaxPendingMessages()); - } - if (conf.getMaxPendingMessagesAcrossPartitions() != null) { - pbldr.setMaxPendingMessagesAcrossPartitions(conf.getMaxPendingMessagesAcrossPartitions()); - } - if (conf.getUseThreadLocalProducers() != null) { - pbldr.setUseThreadLocalProducers(conf.getUseThreadLocalProducers()); - } - if (conf.getCryptoConfig() != null) { - pbldr.setCryptoSpec(CryptoUtils.convert(conf.getCryptoConfig())); - } - if (conf.getBatchBuilder() != null) { - pbldr.setBatchBuilder(conf.getBatchBuilder()); - } - if (conf.getCompressionType() != null) { - pbldr.setCompressionType(convertFromCompressionType(conf.getCompressionType())); - } else { - pbldr.setCompressionType(Function.CompressionType.LZ4); - } - sinkSpecBuilder.setProducerSpec(pbldr.build()); + sinkSpecBuilder.setProducerSpec(convertProducerConfigToProducerSpec(sourceConfig.getProducerConfig())); } if (sourceConfig.getBatchBuilder() != null) { @@ -259,23 +236,7 @@ public static SourceConfig convertFromDetails(FunctionDetails functionDetails) { sourceConfig.setSerdeClassName(sinkSpec.getSerDeClassName()); } if (sinkSpec.getProducerSpec() != null) { - Function.ProducerSpec spec = sinkSpec.getProducerSpec(); - ProducerConfig producerConfig = new ProducerConfig(); - if (spec.getMaxPendingMessages() != 0) { - producerConfig.setMaxPendingMessages(spec.getMaxPendingMessages()); - } - if (spec.getMaxPendingMessagesAcrossPartitions() != 0) { - producerConfig.setMaxPendingMessagesAcrossPartitions(spec.getMaxPendingMessagesAcrossPartitions()); - } - if (spec.hasCryptoSpec()) { - producerConfig.setCryptoConfig(CryptoUtils.convertFromSpec(spec.getCryptoSpec())); - } - if (spec.getBatchBuilder() != null) { - producerConfig.setBatchBuilder(spec.getBatchBuilder()); - } - producerConfig.setUseThreadLocalProducers(spec.getUseThreadLocalProducers()); - producerConfig.setCompressionType(convertFromFunctionDetailsCompressionType(spec.getCompressionType())); - sourceConfig.setProducerConfig(producerConfig); + sourceConfig.setProducerConfig(convertProducerSpecToProducerConfig(sinkSpec.getProducerSpec())); } if (!isEmpty(functionDetails.getLogTopic())) { sourceConfig.setLogTopic(functionDetails.getLogTopic()); diff --git a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java index 954eef44a7366..cf4e7dd92a8f7 100644 --- a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java +++ b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java @@ -35,9 +35,12 @@ import java.util.Map; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; +import org.apache.pulsar.client.api.ProducerCryptoFailureAction; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.impl.schema.JSONSchema; import org.apache.pulsar.common.functions.ConsumerConfig; +import org.apache.pulsar.common.functions.CryptoConfig; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.ProducerConfig; import org.apache.pulsar.common.functions.Resources; @@ -667,4 +670,36 @@ public void testPoolMessages() { convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); assertTrue(convertedConfig.getInputSpecs().get("test-input").isPoolMessages()); } + + @Test + public void testConvertProducerSpecToProducerConfigAndBackToProducerSpec() { + // given + Function.ProducerSpec producerSpec = Function.ProducerSpec.newBuilder() + .setBatchBuilder("KEY_BASED") + .setCompressionType(Function.CompressionType.ZSTD) + .setCryptoSpec(Function.CryptoSpec.newBuilder() + .addProducerEncryptionKeyName("key1") + .addProducerEncryptionKeyName("key2") + .setConsumerCryptoFailureAction(Function.CryptoSpec.FailureAction.DISCARD) + .setProducerCryptoFailureAction(Function.CryptoSpec.FailureAction.SEND) + .setCryptoKeyReaderClassName("ReaderClassName") + .setCryptoKeyReaderConfig("{\"key\":\"value\"}") + .build()) + .build(); + // when + ProducerConfig producerConfig = FunctionConfigUtils.convertProducerSpecToProducerConfig(producerSpec); + // then + assertEquals(producerConfig.getBatchBuilder(), "KEY_BASED"); + assertEquals(producerConfig.getCompressionType(), CompressionType.ZSTD); + CryptoConfig cryptoConfig = producerConfig.getCryptoConfig(); + assertEquals(cryptoConfig.getProducerCryptoFailureAction(), ProducerCryptoFailureAction.SEND); + assertEquals(cryptoConfig.getConsumerCryptoFailureAction(), ConsumerCryptoFailureAction.DISCARD); + assertEquals(cryptoConfig.getEncryptionKeys(), new String[]{"key1", "key2"}); + assertEquals(cryptoConfig.getCryptoKeyReaderClassName(), "ReaderClassName"); + // and when + // converted back to producer spec + Function.ProducerSpec producerSpec2 = FunctionConfigUtils.convertProducerConfigToProducerSpec(producerConfig); + // then + assertEquals(producerSpec2, producerSpec); + } } From 82b8d98a488191d279612d5cf2b4846627863543 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Thu, 20 Jun 2024 21:47:27 +0800 Subject: [PATCH 328/580] [fix][broker] Check the markDeletePosition and calculate the backlog (#22947) Signed-off-by: Zixuan Liu --- .../bookkeeper/mledger/impl/ManagedCursorImpl.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index c0992e48dba8a..bf46aa2fdffa9 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -1112,6 +1112,13 @@ public long getEstimatedSizeSinceMarkDeletePosition() { return ledger.estimateBacklogFromPosition(markDeletePosition); } + private long getNumberOfEntriesInBacklog() { + if (markDeletePosition.compareTo(ledger.getLastPosition()) >= 0) { + return 0; + } + return getNumberOfEntries(Range.openClosed(markDeletePosition, ledger.getLastPosition())); + } + @Override public long getNumberOfEntriesInBacklog(boolean isPrecise) { if (log.isDebugEnabled()) { @@ -1120,16 +1127,13 @@ public long getNumberOfEntriesInBacklog(boolean isPrecise) { messagesConsumedCounter, markDeletePosition, readPosition); } if (isPrecise) { - if (markDeletePosition.compareTo(ledger.getLastPosition()) >= 0) { - return 0; - } - return getNumberOfEntries(Range.openClosed(markDeletePosition, ledger.getLastPosition())); + return getNumberOfEntriesInBacklog(); } long backlog = ManagedLedgerImpl.ENTRIES_ADDED_COUNTER_UPDATER.get(ledger) - messagesConsumedCounter; if (backlog < 0) { // In some case the counters get incorrect values, fall back to the precise backlog count - backlog = getNumberOfEntries(Range.openClosed(markDeletePosition, ledger.getLastPosition())); + backlog = getNumberOfEntriesInBacklog(); } return backlog; From 6692bc8e327ea6958149ea0fb207691f4bce907d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 20 Jun 2024 23:39:58 +0300 Subject: [PATCH 329/580] [cleanup][misc] Remove classifier from netty-transport-native-unix-common dependency (#22951) --- distribution/server/src/assemble/LICENSE.bin.txt | 1 - distribution/shell/src/assemble/LICENSE.bin.txt | 1 - pulsar-common/pom.xml | 1 - 3 files changed, 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 3b30a40ff83e9..c2216378c278e 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -312,7 +312,6 @@ The Apache Software License, Version 2.0 - io.netty-netty-transport-native-epoll-4.1.111.Final-linux-aarch_64.jar - io.netty-netty-transport-native-epoll-4.1.111.Final-linux-x86_64.jar - io.netty-netty-transport-native-unix-common-4.1.111.Final.jar - - io.netty-netty-transport-native-unix-common-4.1.111.Final-linux-x86_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.65.Final.jar - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 6cefa42bed85a..86e7d2d560808 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -363,7 +363,6 @@ The Apache Software License, Version 2.0 - netty-transport-native-epoll-4.1.111.Final-linux-aarch_64.jar - netty-transport-native-epoll-4.1.111.Final-linux-x86_64.jar - netty-transport-native-unix-common-4.1.111.Final.jar - - netty-transport-native-unix-common-4.1.111.Final-linux-x86_64.jar - netty-tcnative-boringssl-static-2.0.65.Final.jar - netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar - netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index d29ce3126635f..3f73a43698ea4 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -107,7 +107,6 @@ io.netty netty-transport-native-unix-common - linux-x86_64 From 1517e63556a432fea088b81cc7cd5bcc89bcfad0 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 21 Jun 2024 10:06:30 +0300 Subject: [PATCH 330/580] [improve][misc] Set Alpine base image to 3.20 instead of 3.19.1 (#22941) --- docker/glibc-package/Dockerfile | 3 ++- docker/pulsar/Dockerfile | 10 ++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/docker/glibc-package/Dockerfile b/docker/glibc-package/Dockerfile index f9c238cbdfc55..016e5c622365f 100644 --- a/docker/glibc-package/Dockerfile +++ b/docker/glibc-package/Dockerfile @@ -19,6 +19,7 @@ ARG GLIBC_VERSION=2.38 +ARG ALPINE_VERSION=3.20 FROM ubuntu:22.04 as build ARG GLIBC_VERSION @@ -51,7 +52,7 @@ RUN tar --dereference --hard-dereference -zcf /glibc-bin.tar.gz /usr/glibc-compa ################################################ ## Build the APK package -FROM alpine:3.19 as apk +FROM alpine:$ALPINE_VERSION as apk ARG GLIBC_VERSION RUN apk add abuild sudo build-base diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index b75519fa91a07..b4294dd10da38 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -17,8 +17,10 @@ # under the License. # +ARG ALPINE_VERSION=3.20 + # First create a stage with just the Pulsar tarball and scripts -FROM alpine as pulsar +FROM alpine:$ALPINE_VERSION as pulsar RUN apk add zip @@ -52,7 +54,7 @@ RUN chmod -R o+rx /pulsar RUN echo 'OPTS="$OPTS -Dorg.xerial.snappy.use.systemlib=true"' >> /pulsar/conf/bkenv.sh ### Create one stage to include JVM distribution -FROM alpine AS jvm +FROM alpine:$ALPINE_VERSION AS jvm RUN wget -O /etc/apk/keys/amazoncorretto.rsa.pub https://apk.corretto.aws/amazoncorretto.rsa.pub RUN echo "https://apk.corretto.aws" >> /etc/apk/repositories @@ -68,7 +70,7 @@ RUN echo networkaddress.cache.negative.ttl=1 >> /opt/jvm/conf/security/java.secu # Fix the issue when using snappy-java in x86 arch alpine # See https://github.com/xerial/snappy-java/issues/181 https://github.com/xerial/snappy-java/issues/579 # We need to ensure that the version of the native library matches the version of snappy-java imported via Maven -FROM alpine AS snappy-java +FROM alpine:$ALPINE_VERSION AS snappy-java ARG SNAPPY_VERSION RUN apk add git alpine-sdk util-linux cmake autoconf automake libtool openjdk17 maven curl bash tar @@ -78,7 +80,7 @@ FROM apachepulsar/glibc-base:2.38 as glibc ## Create final stage from Alpine image ## and add OpenJDK and Python dependencies (for Pulsar functions) -FROM alpine:3.19.1 +FROM alpine:$ALPINE_VERSION ENV LANG C.UTF-8 # Install some utilities, some are required by Pulsar scripts From ddb03bb6a3b67ffcc71c7e95a87b35eb302a7393 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 21 Jun 2024 23:06:56 +0300 Subject: [PATCH 331/580] [fix][misc] Rename netty native libraries in pulsar-client-admin-shaded (#22954) --- pulsar-client-admin-shaded/pom.xml | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index 96ca2f8de9fd4..ab17e69ad4530 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -304,6 +304,31 @@ + + + + exec-maven-plugin + org.codehaus.mojo + + + rename-epoll-library + package + + exec + + + ${project.parent.basedir}/src/${rename.netty.native.libs} + + ${project.artifactId} + + + + +
From 263c6948fb3dd10480f39a9202c6fcc4a7d55d8e Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Mon, 24 Jun 2024 17:09:51 +0800 Subject: [PATCH 332/580] [improve] [broker] make system topic distribute evenly. (#22953) --- .../broker/loadbalance/impl/ModularLoadManagerImpl.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index 3af372607cb16..08c9483e87063 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -893,7 +893,9 @@ Optional selectBroker(final ServiceUnitId serviceUnit) { brokerToNamespaceToBundleRange, brokerToFailureDomainMap); // distribute bundles evenly to candidate-brokers if enable - if (conf.isLoadBalancerDistributeBundlesEvenlyEnabled()) { + // or system-namespace bundles + if (conf.isLoadBalancerDistributeBundlesEvenlyEnabled() + || serviceUnit.getNamespaceObject().equals(NamespaceName.SYSTEM_NAMESPACE)) { LoadManagerShared.removeMostServicingBrokersForNamespace(bundle, brokerCandidateCache, brokerToNamespaceToBundleRange); From 10eeaccbc5f01e53603c625555abffa50d0dcb17 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 24 Jun 2024 18:24:17 +0300 Subject: [PATCH 333/580] [fix][ci] Replace removed macos-11 with macos-latest in GitHub Actions (#22965) --- .github/workflows/ci-maven-cache-update.yaml | 2 +- .github/workflows/pulsar-ci.yaml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/ci-maven-cache-update.yaml b/.github/workflows/ci-maven-cache-update.yaml index 53dff03c248cc..5a0d4d840c655 100644 --- a/.github/workflows/ci-maven-cache-update.yaml +++ b/.github/workflows/ci-maven-cache-update.yaml @@ -64,7 +64,7 @@ jobs: mvn_arguments: '' - name: all modules - macos - runs-on: macos-11 + runs-on: macos-latest cache_name: 'm2-dependencies-all' - name: core-modules diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 1160a0d1ec363..8decde1c999ca 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -1319,7 +1319,7 @@ jobs: macos-build: name: Build Pulsar on MacOS - runs-on: macos-11 + runs-on: macos-latest timeout-minutes: 120 needs: ['preconditions', 'integration-tests'] if: ${{ needs.preconditions.outputs.docs_only != 'true' }} From f728b2ebb9bfe2dfe1f64643640700f762524c40 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 24 Jun 2024 19:54:27 +0300 Subject: [PATCH 334/580] [improve][misc] Replace rename-netty-native-libs.sh script with renaming with maven-shade-plugin (#22957) --- README.md | 2 - pom.xml | 7 --- pulsar-client-admin-shaded/pom.xml | 31 ++-------- pulsar-client-all/pom.xml | 31 ++-------- pulsar-client-shaded/pom.xml | 31 ++-------- src/rename-netty-native-libs.cmd | 98 ------------------------------ src/rename-netty-native-libs.sh | 70 --------------------- 7 files changed, 18 insertions(+), 252 deletions(-) delete mode 100644 src/rename-netty-native-libs.cmd delete mode 100755 src/rename-netty-native-libs.sh diff --git a/README.md b/README.md index 3eae0ae29c334..1d53af9f08149 100644 --- a/README.md +++ b/README.md @@ -141,8 +141,6 @@ components in the Pulsar ecosystem, including connectors, adapters, and other la > > This project includes a [Maven Wrapper](https://maven.apache.org/wrapper/) that can be used instead of a system-installed Maven. > Use it by replacing `mvn` by `./mvnw` on Linux and `mvnw.cmd` on Windows in the commands below. -> -> It's better to use CMD rather than Powershell on Windows. Because maven will activate the `windows` profile which runs `rename-netty-native-libs.cmd`. ### Build diff --git a/pom.xml b/pom.xml index 8325336aa9684..2373bb91c2f0d 100644 --- a/pom.xml +++ b/pom.xml @@ -321,9 +321,6 @@ flexible messaging model and an intuitive client API. 1.6.1 6.4.0 3.33.0 - - - rename-netty-native-libs.sh @@ -2372,10 +2369,6 @@ flexible messaging model and an intuitive client API. Windows - - rename-netty-native-libs.cmd - - diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index ab17e69ad4530..1376cefe80368 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -295,6 +295,12 @@ org.apache.bookkeeper org.apache.pulsar.shade.org.apache.bookkeeper + + + (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ + $1org_apache_pulsar_shade_$3 + true + @@ -304,31 +310,6 @@ - - - - exec-maven-plugin - org.codehaus.mojo - - - rename-epoll-library - package - - exec - - - ${project.parent.basedir}/src/${rename.netty.native.libs} - - ${project.artifactId} - - - - - diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 27abc1a24c38c..65d24e3394d10 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -387,6 +387,12 @@ org.tukaani org.apache.pulsar.shade.org.tukaani + + + (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ + $1org_apache_pulsar_shade_$3 + true + @@ -396,31 +402,6 @@ - - - - exec-maven-plugin - org.codehaus.mojo - - - rename-epoll-library - package - - exec - - - ${project.parent.basedir}/src/${rename.netty.native.libs} - - ${project.artifactId} - - - - - org.apache.maven.plugins maven-enforcer-plugin diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index ca018308731d6..c18d3123e66be 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -300,6 +300,12 @@ org.apache.bookkeeper org.apache.pulsar.shade.org.apache.bookkeeper + + + (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ + $1org_apache_pulsar_shade_$3 + true + @@ -323,31 +329,6 @@ - - - - exec-maven-plugin - org.codehaus.mojo - - - rename-epoll-library - package - - exec - - - ${project.parent.basedir}/src/${rename.netty.native.libs} - - ${project.artifactId} - - - - - diff --git a/src/rename-netty-native-libs.cmd b/src/rename-netty-native-libs.cmd deleted file mode 100644 index bfaa16de0812c..0000000000000 --- a/src/rename-netty-native-libs.cmd +++ /dev/null @@ -1,98 +0,0 @@ -@REM -@REM Licensed to the Apache Software Foundation (ASF) under one -@REM or more contributor license agreements. See the NOTICE file -@REM distributed with this work for additional information -@REM regarding copyright ownership. The ASF licenses this file -@REM to you under the Apache License, Version 2.0 (the -@REM "License"); you may not use this file except in compliance -@REM with the License. You may obtain a copy of the License at -@REM -@REM http://www.apache.org/licenses/LICENSE-2.0 -@REM -@REM Unless required by applicable law or agreed to in writing, -@REM software distributed under the License is distributed on an -@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -@REM KIND, either express or implied. See the License for the -@REM specific language governing permissions and limitations -@REM under the License. -@REM - -@echo off - -set ARTIFACT_ID=%1 -set JAR_PATH=%cd%/target/%ARTIFACT_ID%.jar -set FILE_PREFIX=META-INF/native - -:: echo %JAR_PATH% -:: echo %FILE_PREFIX% - -ECHO. -echo ----- Renaming epoll lib in %JAR_PATH% ------ -set TMP_DIR=%temp%\tmp_pulsar - -rd %TMP_DIR% /s /q -mkdir %TMP_DIR% - -set UNZIP_CMD=unzip -q %JAR_PATH% -d %TMP_DIR% -call %UNZIP_CMD% - -:: echo %UNZIP_CMD% -:: echo %TMP_DIR% - -cd /d %TMP_DIR%/%FILE_PREFIX% - -:: Loop through the number of groups -SET Obj_Length=10 -SET Obj[0].FROM=libnetty_transport_native_epoll_x86_64.so -SET Obj[0].TO=liborg_apache_pulsar_shade_netty_transport_native_epoll_x86_64.so -SET Obj[1].FROM=libnetty_transport_native_epoll_aarch_64.so -SET Obj[1].TO=liborg_apache_pulsar_shade_netty_transport_native_epoll_aarch_64.so -SET Obj[2].FROM=libnetty_tcnative_linux_x86_64.so -SET Obj[2].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so -SET Obj[3].FROM=libnetty_tcnative_linux_aarch_64.so -SET Obj[3].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_aarch_64.so -SET Obj[4].FROM=libnetty_tcnative_osx_x86_64.jnilib -SET Obj[4].TO=liborg_apache_pulsar_shade_netty_tcnative_osx_x86_64.jnilib -SET Obj[5].FROM=libnetty_tcnative_osx_aarch_64.jnilib -SET Obj[5].TO=liborg_apache_pulsar_shade_netty_tcnative_osx_aarch_64.jnilib -SET Obj[6].FROM=libnetty_transport_native_io_uring_x86_64.so -SET Obj[6].TO=liborg_apache_pulsar_shade_netty_transport_native_io_uring_x86_64.so -SET Obj[7].FROM=libnetty_transport_native_io_uring_aarch_64.so -SET Obj[7].TO=liborg_apache_pulsar_shade_netty_transport_native_io_uring_aarch_64.so -SET Obj[8].FROM=libnetty_resolver_dns_native_macos_aarch_64.jnilib -SET Obj[8].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_aarch_64.jnilib -SET Obj[9].FROM=libnetty_resolver_dns_native_macos_x86_64.jnilib -SET Obj[9].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_x86_64.jnilib -SET Obj_Index=0 - -:LoopStart -IF %Obj_Index% EQU %Obj_Length% GOTO END - -SET Obj_Current.FROM=0 -SET Obj_Current.TO=0 - -FOR /F "usebackq delims==. tokens=1-3" %%I IN (`SET Obj[%Obj_Index%]`) DO ( - SET Obj_Current.%%J=%%K.so -) - -echo "Renaming %Obj_Current.FROM% -> %Obj_Current.TO%" -call ren %Obj_Current.FROM% %Obj_Current.TO% - -SET /A Obj_Index=%Obj_Index% + 1 - -GOTO LoopStart -:: Loop end - -:END -cd /d %TMP_DIR% - -:: Overwrite the original ZIP archive -rd %JAR_PATH% /s /q -set ZIP_CMD=zip -q -r %JAR_PATH% . -:: echo %ZIP_CMD% -call %ZIP_CMD% -:: echo %TMP_DIR% -rd %TMP_DIR% /s /q - -exit /b 0 -:: echo.&pause&goto:eof \ No newline at end of file diff --git a/src/rename-netty-native-libs.sh b/src/rename-netty-native-libs.sh deleted file mode 100755 index ea2a4c0e2421e..0000000000000 --- a/src/rename-netty-native-libs.sh +++ /dev/null @@ -1,70 +0,0 @@ -#!/usr/bin/env bash -# -# 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. -# - -set -e - -ARTIFACT_ID=$1 -JAR_PATH="$PWD/target/$ARTIFACT_ID.jar" - -FILE_PREFIX='META-INF/native' - -FILES_TO_RENAME=( - 'libnetty_transport_native_epoll_x86_64.so liborg_apache_pulsar_shade_netty_transport_native_epoll_x86_64.so' - 'libnetty_transport_native_epoll_aarch_64.so liborg_apache_pulsar_shade_netty_transport_native_epoll_aarch_64.so' - 'libnetty_tcnative_linux_x86_64.so liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so' - 'libnetty_tcnative_linux_aarch_64.so liborg_apache_pulsar_shade_netty_tcnative_linux_aarch_64.so' - 'libnetty_tcnative_osx_x86_64.jnilib liborg_apache_pulsar_shade_netty_tcnative_osx_x86_64.jnilib' - 'libnetty_tcnative_osx_aarch_64.jnilib liborg_apache_pulsar_shade_netty_tcnative_osx_aarch_64.jnilib' - 'libnetty_transport_native_io_uring_x86_64.so liborg_apache_pulsar_shade_netty_transport_native_io_uring_x86_64.so' - 'libnetty_transport_native_io_uring_aarch_64.so liborg_apache_pulsar_shade_netty_transport_native_io_uring_aarch_64.so' - 'libnetty_resolver_dns_native_macos_aarch_64.jnilib liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_aarch_64.jnilib' - 'libnetty_resolver_dns_native_macos_x86_64.jnilib liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_x86_64.jnilib' -) - -echo "----- Renaming epoll lib in $JAR_PATH ------" -TMP_DIR=`mktemp -d` -CUR_DIR=$(pwd) -cd ${TMP_DIR} -# exclude `META-INF/LICENSE` -unzip -q $JAR_PATH -x "META-INF/LICENSE" -# include `META-INF/LICENSE` as LICENSE.netty. -# This approach is to get around the issue that MacOS is not able to recognize the difference between `META-INF/LICENSE` and `META-INF/license/`. -unzip -p $JAR_PATH META-INF/LICENSE > META-INF/LICENSE.netty -cd ${CUR_DIR} - -pushd $TMP_DIR - -for line in "${FILES_TO_RENAME[@]}"; do - read -r -a A <<< "$line" - FROM=${A[0]} - TO=${A[1]} - - if [ -f $FILE_PREFIX/$FROM ]; then - echo "Renaming $FROM -> $TO" - mv $FILE_PREFIX/$FROM $FILE_PREFIX/$TO - fi -done - -# Overwrite the original ZIP archive -rm $JAR_PATH -zip -q -r $JAR_PATH . -popd - -rm -rf $TMP_DIR From c78585ad286c846a26cf7c33b3902189e407d46a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 24 Jun 2024 21:21:41 +0300 Subject: [PATCH 335/580] [improve][misc] Upgrade Bookkeeper to 4.17.1 (#22962) --- .../server/src/assemble/LICENSE.bin.txt | 56 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 6 +- pom.xml | 2 +- 3 files changed, 32 insertions(+), 32 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index c2216378c278e..24c601b184afa 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -355,34 +355,34 @@ The Apache Software License, Version 2.0 - net.java.dev.jna-jna-jpms-5.12.1.jar - net.java.dev.jna-jna-platform-jpms-5.12.1.jar * BookKeeper - - org.apache.bookkeeper-bookkeeper-common-4.17.0.jar - - org.apache.bookkeeper-bookkeeper-common-allocator-4.17.0.jar - - org.apache.bookkeeper-bookkeeper-proto-4.17.0.jar - - org.apache.bookkeeper-bookkeeper-server-4.17.0.jar - - org.apache.bookkeeper-bookkeeper-tools-framework-4.17.0.jar - - org.apache.bookkeeper-circe-checksum-4.17.0.jar - - org.apache.bookkeeper-cpu-affinity-4.17.0.jar - - org.apache.bookkeeper-statelib-4.17.0.jar - - org.apache.bookkeeper-stream-storage-api-4.17.0.jar - - org.apache.bookkeeper-stream-storage-common-4.17.0.jar - - org.apache.bookkeeper-stream-storage-java-client-4.17.0.jar - - org.apache.bookkeeper-stream-storage-java-client-base-4.17.0.jar - - org.apache.bookkeeper-stream-storage-proto-4.17.0.jar - - org.apache.bookkeeper-stream-storage-server-4.17.0.jar - - org.apache.bookkeeper-stream-storage-service-api-4.17.0.jar - - org.apache.bookkeeper-stream-storage-service-impl-4.17.0.jar - - org.apache.bookkeeper.http-http-server-4.17.0.jar - - org.apache.bookkeeper.http-vertx-http-server-4.17.0.jar - - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.17.0.jar - - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.17.0.jar - - org.apache.distributedlog-distributedlog-common-4.17.0.jar - - org.apache.distributedlog-distributedlog-core-4.17.0-tests.jar - - org.apache.distributedlog-distributedlog-core-4.17.0.jar - - org.apache.distributedlog-distributedlog-protocol-4.17.0.jar - - org.apache.bookkeeper.stats-codahale-metrics-provider-4.17.0.jar - - org.apache.bookkeeper-bookkeeper-slogger-api-4.17.0.jar - - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.17.0.jar - - org.apache.bookkeeper-native-io-4.17.0.jar + - org.apache.bookkeeper-bookkeeper-common-4.17.1.jar + - org.apache.bookkeeper-bookkeeper-common-allocator-4.17.1.jar + - org.apache.bookkeeper-bookkeeper-proto-4.17.1.jar + - org.apache.bookkeeper-bookkeeper-server-4.17.1.jar + - org.apache.bookkeeper-bookkeeper-tools-framework-4.17.1.jar + - org.apache.bookkeeper-circe-checksum-4.17.1.jar + - org.apache.bookkeeper-cpu-affinity-4.17.1.jar + - org.apache.bookkeeper-statelib-4.17.1.jar + - org.apache.bookkeeper-stream-storage-api-4.17.1.jar + - org.apache.bookkeeper-stream-storage-common-4.17.1.jar + - org.apache.bookkeeper-stream-storage-java-client-4.17.1.jar + - org.apache.bookkeeper-stream-storage-java-client-base-4.17.1.jar + - org.apache.bookkeeper-stream-storage-proto-4.17.1.jar + - org.apache.bookkeeper-stream-storage-server-4.17.1.jar + - org.apache.bookkeeper-stream-storage-service-api-4.17.1.jar + - org.apache.bookkeeper-stream-storage-service-impl-4.17.1.jar + - org.apache.bookkeeper.http-http-server-4.17.1.jar + - org.apache.bookkeeper.http-vertx-http-server-4.17.1.jar + - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.17.1.jar + - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.17.1.jar + - org.apache.distributedlog-distributedlog-common-4.17.1.jar + - org.apache.distributedlog-distributedlog-core-4.17.1-tests.jar + - org.apache.distributedlog-distributedlog-core-4.17.1.jar + - org.apache.distributedlog-distributedlog-protocol-4.17.1.jar + - org.apache.bookkeeper.stats-codahale-metrics-provider-4.17.1.jar + - org.apache.bookkeeper-bookkeeper-slogger-api-4.17.1.jar + - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.17.1.jar + - org.apache.bookkeeper-native-io-4.17.1.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 86e7d2d560808..2971147c2c8df 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -395,9 +395,9 @@ The Apache Software License, Version 2.0 - opentelemetry-context-1.38.0.jar * BookKeeper - - bookkeeper-common-allocator-4.17.0.jar - - cpu-affinity-4.17.0.jar - - circe-checksum-4.17.0.jar + - bookkeeper-common-allocator-4.17.1.jar + - cpu-affinity-4.17.1.jar + - circe-checksum-4.17.1.jar * AirCompressor - aircompressor-0.27.jar * AsyncHttpClient diff --git a/pom.xml b/pom.xml index 2373bb91c2f0d..c7675ae88fcbf 100644 --- a/pom.xml +++ b/pom.xml @@ -141,7 +141,7 @@ flexible messaging model and an intuitive client API. 1.26.0 - 4.17.0 + 4.17.1 3.9.2 1.5.0 1.10.0 From aa03c0efebc9e0f46c8653629ae92206b47591c6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 24 Jun 2024 21:42:30 +0300 Subject: [PATCH 336/580] [fix][ci] Fix jacoco code coverage report aggregation (#22964) --- build/pulsar_ci_tool.sh | 9 +++++---- jetcd-core-shaded/pom.xml | 11 +++++++++++ pom.xml | 2 +- 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/build/pulsar_ci_tool.sh b/build/pulsar_ci_tool.sh index 034b2ce60cf37..3d63f104cd500 100755 --- a/build/pulsar_ci_tool.sh +++ b/build/pulsar_ci_tool.sh @@ -353,6 +353,7 @@ _ci_upload_coverage_files() { --transform="flags=r;s|\\(/jacoco.*\\).exec$|\\1_${testtype}_${testgroup}.exec|" \ --transform="flags=r;s|\\(/tmp/jacocoDir/.*\\).exec$|\\1_${testtype}_${testgroup}.exec|" \ --exclude="*/META-INF/bundled-dependencies/*" \ + --exclude="*/META-INF/versions/*" \ $GITHUB_WORKSPACE/target/classpath_* \ $(find "$GITHUB_WORKSPACE" -path "*/target/jacoco*.exec" -printf "%p\n%h/classes\n" | sort | uniq) \ $([ -d /tmp/jacocoDir ] && echo "/tmp/jacocoDir" ) \ @@ -494,11 +495,11 @@ ci_create_test_coverage_report() { local classfilesArgs="--classfiles $({ { for classpathEntry in $(cat $completeClasspathFile | { grep -v -f $filterArtifactsFile || true; } | sort | uniq | { grep -v -E "$excludeJarsPattern" || true; }); do - if [[ -f $classpathEntry && -n "$(unzip -Z1C $classpathEntry 'META-INF/bundled-dependencies/*' 2>/dev/null)" ]]; then - # file must be processed by removing META-INF/bundled-dependencies + if [[ -f $classpathEntry && -n "$(unzip -Z1C $classpathEntry 'META-INF/bundled-dependencies/*' 'META-INF/versions/*' 2>/dev/null)" ]]; then + # file must be processed by removing META-INF/bundled-dependencies and META-INF/versions local jartempfile=$(mktemp -t jarfile.XXXX --suffix=.jar) cp $classpathEntry $jartempfile - zip -q -d $jartempfile 'META-INF/bundled-dependencies/*' &> /dev/null + zip -q -d $jartempfile 'META-INF/bundled-dependencies/*' 'META-INF/versions/*' &> /dev/null echo $jartempfile else echo $classpathEntry @@ -560,7 +561,7 @@ ci_create_inttest_coverage_report() { # remove jar file that causes duplicate classes issue rm /tmp/jacocoDir/pulsar_lib/org.apache.pulsar-bouncy-castle* || true # remove any bundled dependencies as part of .jar/.nar files - find /tmp/jacocoDir/pulsar_lib '(' -name "*.jar" -or -name "*.nar" ')' -exec echo "Processing {}" \; -exec zip -q -d {} 'META-INF/bundled-dependencies/*' \; |grep -E -v "Nothing to do|^$" || true + find /tmp/jacocoDir/pulsar_lib '(' -name "*.jar" -or -name "*.nar" ')' -exec echo "Processing {}" \; -exec zip -q -d {} 'META-INF/bundled-dependencies/*' 'META-INF/versions/*' \; |grep -E -v "Nothing to do|^$" || true fi # projects that aren't considered as production code and their own src/main/java source code shouldn't be analysed local excludeProjectsPattern="testmocks|testclient|buildtools" diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml index d8819a1148a21..de5d654851a52 100644 --- a/jetcd-core-shaded/pom.xml +++ b/jetcd-core-shaded/pom.xml @@ -100,6 +100,12 @@ io.vertx org.apache.pulsar.jetcd.shaded.io.vertx + + + META-INF/versions/(\d+)/io/vertx/ + META-INF/versions/$1/org/apache/pulsar/jetcd/shaded/io/vertx/ + true + io.grpc.netty @@ -123,6 +129,11 @@ + + + true + + diff --git a/pom.xml b/pom.xml index c7675ae88fcbf..1e200d04d68fd 100644 --- a/pom.xml +++ b/pom.xml @@ -309,7 +309,7 @@ flexible messaging model and an intuitive client API. 4.9.10 3.5.3 1.7.0 - 0.8.11 + 0.8.12 4.7.3.6 4.7.3 2.24.0 From 69b2739eaa2974d93e32f6b84dd777b5112b07fa Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 25 Jun 2024 11:25:43 +0800 Subject: [PATCH 337/580] [fix][client] Fix orphan consumer when reconnection and closing are concurrency executing (#22958) --- .../main/java/org/apache/pulsar/client/impl/ConsumerImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index c8f4b0acec36d..6ddb0e1bc01db 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1088,7 +1088,7 @@ public void connectionFailed(PulsarClientException exception) { } @Override - public CompletableFuture closeAsync() { + public synchronized CompletableFuture closeAsync() { CompletableFuture closeFuture = new CompletableFuture<>(); if (getState() == State.Closing || getState() == State.Closed) { From 6fe8100b1fd5d37a6e1bf33803a8904fa3879321 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 25 Jun 2024 08:37:43 +0300 Subject: [PATCH 338/580] [improve][fn] Make producer cache bounded and expiring in Functions/Connectors (#22945) --- pulsar-functions/instance/pom.xml | 5 + .../functions/instance/ContextImpl.java | 84 +++-------- .../instance/JavaInstanceRunnable.java | 8 +- .../functions/instance/ProducerCache.java | 130 +++++++++++++++++ .../pulsar/functions/sink/PulsarSink.java | 89 +++++------- .../functions/instance/ContextImplTest.java | 24 +++- .../pulsar/functions/sink/PulsarSinkTest.java | 132 ++++++++---------- 7 files changed, 267 insertions(+), 205 deletions(-) create mode 100644 pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerCache.java diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index 160885a8ea4d7..d4eccab2303d6 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -157,6 +157,11 @@ guava + + com.github.ben-manes.caffeine + caffeine + + info.picocli picocli diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java index eeeaa8b3627e9..f613f749bd0fe 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java @@ -29,16 +29,15 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Stream; import lombok.ToString; +import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.ClientBuilder; @@ -85,6 +84,7 @@ /** * This class implements the Context interface exposed to the user. */ +@Slf4j @ToString(exclude = {"pulsarAdmin"}) class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable { private final ProducerBuilderFactory producerBuilderFactory; @@ -98,8 +98,6 @@ class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable private final ClientBuilder clientBuilder; private final PulsarClient client; private final PulsarAdmin pulsarAdmin; - private Map> publishProducers; - private ThreadLocal>> tlPublishProducers; private final TopicSchema topicSchema; @@ -139,12 +137,15 @@ class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable private final java.util.function.Consumer fatalHandler; + private final ProducerCache producerCache; + private final boolean useThreadLocalProducers; + public ContextImpl(InstanceConfig config, Logger logger, PulsarClient client, SecretsProvider secretsProvider, FunctionCollectorRegistry collectorRegistry, String[] metricsLabels, Function.FunctionDetails.ComponentType componentType, ComponentStatsManager statsManager, StateManager stateManager, PulsarAdmin pulsarAdmin, ClientBuilder clientBuilder, - java.util.function.Consumer fatalHandler) { + java.util.function.Consumer fatalHandler, ProducerCache producerCache) { this.config = config; this.logger = logger; this.clientBuilder = clientBuilder; @@ -154,14 +155,17 @@ public ContextImpl(InstanceConfig config, Logger logger, PulsarClient client, this.statsManager = statsManager; this.fatalHandler = fatalHandler; - boolean useThreadLocalProducers = false; + this.producerCache = producerCache; Function.ProducerSpec producerSpec = config.getFunctionDetails().getSink().getProducerSpec(); ProducerConfig producerConfig = null; if (producerSpec != null) { producerConfig = FunctionConfigUtils.convertProducerSpecToProducerConfig(producerSpec); useThreadLocalProducers = producerSpec.getUseThreadLocalProducers(); + } else { + useThreadLocalProducers = false; } + producerBuilderFactory = new ProducerBuilderFactory(client, producerConfig, Thread.currentThread().getContextClassLoader(), // This is for backwards compatibility. The PR https://github.com/apache/pulsar/pull/19470 removed @@ -175,12 +179,6 @@ public ContextImpl(InstanceConfig config, Logger logger, PulsarClient client, this.config.getFunctionDetails().getName()), this.config.getInstanceId())); - if (useThreadLocalProducers) { - tlPublishProducers = new ThreadLocal<>(); - } else { - publishProducers = new ConcurrentHashMap<>(); - } - if (config.getFunctionDetails().getUserConfig().isEmpty()) { userConfigs = new HashMap<>(); } else { @@ -543,39 +541,15 @@ public void fatal(Throwable t) { } private Producer getProducer(String topicName, Schema schema) throws PulsarClientException { - Producer producer; - if (tlPublishProducers != null) { - Map> producerMap = tlPublishProducers.get(); - if (producerMap == null) { - producerMap = new HashMap<>(); - tlPublishProducers.set(producerMap); - } - producer = (Producer) producerMap.get(topicName); - } else { - producer = (Producer) publishProducers.get(topicName); - } - - if (producer == null) { - Producer newProducer = producerBuilderFactory - .createProducerBuilder(topicName, schema, null) - .properties(producerProperties) - .create(); - - if (tlPublishProducers != null) { - tlPublishProducers.get().put(topicName, newProducer); - } else { - Producer existingProducer = (Producer) publishProducers.putIfAbsent(topicName, newProducer); - - if (existingProducer != null) { - // The value in the map was not updated after the concurrent put - newProducer.close(); - producer = existingProducer; - } else { - producer = newProducer; - } - } - } - return producer; + Long additionalCacheKey = useThreadLocalProducers ? Thread.currentThread().getId() : null; + return producerCache.getOrCreateProducer(ProducerCache.CacheArea.CONTEXT_CACHE, + topicName, additionalCacheKey, () -> { + log.info("Initializing producer on topic {} with schema {}", topicName, schema); + return producerBuilderFactory + .createProducerBuilder(topicName, schema, null) + .properties(producerProperties) + .create(); + }); } public Map getAndResetMetrics() { @@ -714,29 +688,9 @@ public void setUnderlyingBuilder(TypedMessageBuilder underlyingBuilder) { @Override public void close() { - List futures = new LinkedList<>(); - - if (publishProducers != null) { - for (Producer producer : publishProducers.values()) { - futures.add(producer.closeAsync()); - } - } - - if (tlPublishProducers != null) { - for (Producer producer : tlPublishProducers.get().values()) { - futures.add(producer.closeAsync()); - } - } - if (pulsarAdmin != null) { pulsarAdmin.close(); } - - try { - CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(); - } catch (InterruptedException | ExecutionException e) { - logger.warn("Failed to close producers", e); - } } @Override diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java index f1b9af00f9d0b..baf0c5f7400ec 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/JavaInstanceRunnable.java @@ -168,6 +168,8 @@ public class JavaInstanceRunnable implements AutoCloseable, Runnable { private final AtomicReference> sinkSchema = new AtomicReference<>(); private SinkSchemaInfoProvider sinkSchemaInfoProvider = null; + private final ProducerCache producerCache = new ProducerCache(); + public JavaInstanceRunnable(InstanceConfig instanceConfig, ClientBuilder clientBuilder, PulsarClient pulsarClient, @@ -292,7 +294,7 @@ ContextImpl setupContext() throws PulsarClientException { Thread.currentThread().setContextClassLoader(functionClassLoader); return new ContextImpl(instanceConfig, instanceLog, client, secretsProvider, collectorRegistry, metricsLabels, this.componentType, this.stats, stateManager, - pulsarAdmin, clientBuilder, fatalHandler); + pulsarAdmin, clientBuilder, fatalHandler, producerCache); } finally { Thread.currentThread().setContextClassLoader(clsLoader); } @@ -607,6 +609,8 @@ public synchronized void close() { instanceCache = null; + producerCache.close(); + if (logAppender != null) { removeLogTopicAppender(LoggerContext.getContext()); removeLogTopicAppender(LoggerContext.getContext(false)); @@ -1050,7 +1054,7 @@ private void setupOutput(ContextImpl contextImpl) throws Exception { } object = new PulsarSink(this.client, pulsarSinkConfig, this.properties, this.stats, - this.functionClassLoader); + this.functionClassLoader, this.producerCache); } } else { object = Reflections.createInstance( diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerCache.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerCache.java new file mode 100644 index 0000000000000..f68c4e9589558 --- /dev/null +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerCache.java @@ -0,0 +1,130 @@ +/* + * 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.functions.instance; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.Scheduler; +import com.google.common.annotations.VisibleForTesting; +import java.io.Closeable; +import java.time.Duration; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.common.util.FutureUtil; + +@Slf4j +public class ProducerCache implements Closeable { + // allow tuning the cache timeout with PRODUCER_CACHE_TIMEOUT_SECONDS env variable + private static final int PRODUCER_CACHE_TIMEOUT_SECONDS = + Integer.parseInt(System.getenv().getOrDefault("PRODUCER_CACHE_TIMEOUT_SECONDS", "300")); + // allow tuning the cache size with PRODUCER_CACHE_MAX_SIZE env variable + private static final int PRODUCER_CACHE_MAX_SIZE = + Integer.parseInt(System.getenv().getOrDefault("PRODUCER_CACHE_MAX_SIZE", "10000")); + private static final int FLUSH_OR_CLOSE_TIMEOUT_SECONDS = 60; + + // prevents the different producers created in different code locations from mixing up + public enum CacheArea { + // producers created by calling Context, SinkContext, SourceContext methods + CONTEXT_CACHE, + // producers created in Pulsar Sources, multiple topics are possible by returning destination topics + // by SinkRecord.getDestinationTopic call + SINK_RECORD_CACHE, + } + + record ProducerCacheKey(CacheArea cacheArea, String topic, Object additionalKey) { + } + + private final Cache> cache; + private final AtomicBoolean closed = new AtomicBoolean(false); + private final CopyOnWriteArrayList> closeFutures = new CopyOnWriteArrayList<>(); + + public ProducerCache() { + Caffeine builder = Caffeine.newBuilder() + .scheduler(Scheduler.systemScheduler()) + .removalListener((key, producer, cause) -> { + log.info("Closing producer for topic {}, cause {}", key.topic(), cause); + CompletableFuture closeFuture = + producer.flushAsync() + .orTimeout(FLUSH_OR_CLOSE_TIMEOUT_SECONDS, TimeUnit.SECONDS) + .exceptionally(ex -> { + log.error("Error flushing producer for topic {}", key.topic(), ex); + return null; + }).thenCompose(__ -> + producer.closeAsync().orTimeout(FLUSH_OR_CLOSE_TIMEOUT_SECONDS, + TimeUnit.SECONDS) + .exceptionally(ex -> { + log.error("Error closing producer for topic {}", key.topic(), + ex); + return null; + })); + if (closed.get()) { + closeFutures.add(closeFuture); + } + }) + .weigher((key, producer) -> Math.max(producer.getNumOfPartitions(), 1)) + .maximumWeight(PRODUCER_CACHE_MAX_SIZE); + if (PRODUCER_CACHE_TIMEOUT_SECONDS > 0) { + builder.expireAfterAccess(Duration.ofSeconds(PRODUCER_CACHE_TIMEOUT_SECONDS)); + } + cache = builder.build(); + } + + public Producer getOrCreateProducer(CacheArea cacheArea, String topicName, Object additionalCacheKey, + Callable> supplier) { + if (closed.get()) { + throw new IllegalStateException("ProducerCache is already closed"); + } + return (Producer) cache.get(new ProducerCacheKey(cacheArea, topicName, additionalCacheKey), key -> { + try { + return supplier.call(); + } catch (RuntimeException e) { + throw e; + } catch (Exception e) { + throw new RuntimeException("Unable to create producer for topic '" + topicName + "'", e); + } + }); + } + + public void close() { + if (closed.compareAndSet(false, true)) { + cache.invalidateAll(); + try { + FutureUtil.waitForAll(closeFutures).get(); + } catch (InterruptedException | ExecutionException e) { + log.warn("Failed to close producers", e); + } + } + } + + @VisibleForTesting + public boolean containsKey(CacheArea cacheArea, String topic) { + return containsKey(cacheArea, topic, null); + } + + @VisibleForTesting + public boolean containsKey(CacheArea cacheArea, String topic, Object additionalCacheKey) { + return cache.getIfPresent(new ProducerCacheKey(cacheArea, topic, additionalCacheKey)) != null; + } +} diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java index 18e55e8e84de1..da6b8006eb987 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java @@ -20,19 +20,15 @@ import com.google.common.annotations.VisibleForTesting; import java.nio.charset.StandardCharsets; -import java.util.ArrayList; import java.util.Base64; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.function.Function; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; @@ -48,6 +44,7 @@ import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.instance.AbstractSinkRecord; import org.apache.pulsar.functions.instance.ProducerBuilderFactory; +import org.apache.pulsar.functions.instance.ProducerCache; import org.apache.pulsar.functions.instance.stats.ComponentStatsManager; import org.apache.pulsar.functions.source.PulsarRecord; import org.apache.pulsar.functions.source.TopicSchema; @@ -62,6 +59,7 @@ public class PulsarSink implements Sink { private final Map properties; private final ClassLoader functionClassLoader; private ComponentStatsManager stats; + private final ProducerCache producerCache; @VisibleForTesting PulsarSinkProcessor pulsarSinkProcessor; @@ -80,43 +78,25 @@ private interface PulsarSinkProcessor { } abstract class PulsarSinkProcessorBase implements PulsarSinkProcessor { - protected Map> publishProducers = new ConcurrentHashMap<>(); - protected Producer getProducer(String destinationTopic, Schema schema) { - return getProducer(destinationTopic, null, destinationTopic, schema); + return getProducer(destinationTopic, schema, null, null); } - protected Producer getProducer(String producerId, String producerName, String topicName, Schema schema) { - return publishProducers.computeIfAbsent(producerId, s -> { - try { - log.info("Initializing producer {} on topic {} with schema {}", - producerName, topicName, schema); - Producer producer = createProducer( - topicName, - schema, producerName - ); - log.info("Initialized producer {} on topic {} with schema {}: {} -> {}", - producerName, topicName, schema, producerId, producer); - return producer; - } catch (PulsarClientException e) { - log.error("Failed to create Producer while doing user publish", e); - throw new RuntimeException(e); - } - }); + protected Producer getProducer(String topicName, Schema schema, String producerName, String partitionId) { + return producerCache.getOrCreateProducer(ProducerCache.CacheArea.SINK_RECORD_CACHE, topicName, partitionId, + () -> { + Producer producer = createProducer(topicName, schema, producerName); + log.info( + "Initialized producer with name '{}' on topic '{}' with schema {} partitionId {} " + + "-> {}", + producerName, topicName, schema, partitionId, producer); + return producer; + }); } @Override public void close() throws Exception { - List> closeFutures = new ArrayList<>(publishProducers.size()); - for (Map.Entry> entry : publishProducers.entrySet()) { - Producer producer = entry.getValue(); - closeFutures.add(producer.closeAsync()); - } - try { - org.apache.pulsar.common.util.FutureUtil.waitForAll(closeFutures); - } catch (Exception e) { - log.warn("Failed to close all the producers", e); - } + // no op } public Function getPublishErrorHandler(AbstractSinkRecord record, boolean failSource) { @@ -153,13 +133,7 @@ class PulsarSinkAtMostOnceProcessor extends PulsarSinkProcessorBase { public PulsarSinkAtMostOnceProcessor() { if (!(schema instanceof AutoConsumeSchema)) { // initialize default topic - try { - publishProducers.put(pulsarSinkConfig.getTopic(), - createProducer(pulsarSinkConfig.getTopic(), schema, null)); - } catch (PulsarClientException e) { - log.error("Failed to create Producer while doing user publish", e); - throw new RuntimeException(e); - } + getProducer(pulsarSinkConfig.getTopic(), schema); } else { if (log.isDebugEnabled()) { log.debug("The Pulsar producer is not initialized until the first record is" @@ -232,13 +206,10 @@ public TypedMessageBuilder newMessage(AbstractSinkRecord record) { // we must use the destination topic schema schemaToWrite = schema; } - Producer producer = getProducer( - String.format("%s-%s", record.getDestinationTopic().orElse(pulsarSinkConfig.getTopic()), - record.getPartitionId().get()), - record.getPartitionId().get(), - record.getDestinationTopic().orElse(pulsarSinkConfig.getTopic()), - schemaToWrite - ); + String topicName = record.getDestinationTopic().orElse(pulsarSinkConfig.getTopic()); + String partitionId = record.getPartitionId().get(); + String producerName = partitionId; + Producer producer = getProducer(topicName, schemaToWrite, producerName, partitionId); if (schemaToWrite != null) { return producer.newMessage(schemaToWrite); } else { @@ -263,13 +234,14 @@ public void sendOutputMessage(TypedMessageBuilder msg, AbstractSinkRecord } public PulsarSink(PulsarClient client, PulsarSinkConfig pulsarSinkConfig, Map properties, - ComponentStatsManager stats, ClassLoader functionClassLoader) { + ComponentStatsManager stats, ClassLoader functionClassLoader, ProducerCache producerCache) { this.client = client; this.pulsarSinkConfig = pulsarSinkConfig; this.topicSchema = new TopicSchema(client, functionClassLoader); this.properties = properties; this.stats = stats; this.functionClassLoader = functionClassLoader; + this.producerCache = producerCache; } @Override @@ -341,14 +313,17 @@ public void close() throws Exception { } } - Producer createProducer(String topic, Schema schema, String producerName) - throws PulsarClientException { - ProducerBuilder builder = - producerBuilderFactory.createProducerBuilder(topic, schema != null ? schema : this.schema, - producerName); - return builder - .properties(properties) - .create(); + Producer createProducer(String topicName, Schema schema, String producerName) { + Schema schemaToUse = schema != null ? schema : this.schema; + try { + log.info("Initializing producer {} on topic {} with schema {}", producerName, topicName, schemaToUse); + return producerBuilderFactory.createProducerBuilder(topicName, schemaToUse, producerName) + .properties(properties) + .create(); + } catch (PulsarClientException e) { + throw new RuntimeException("Failed to create Producer for topic " + topicName + + " producerName " + producerName + " schema " + schemaToUse, e); + } } @SuppressWarnings("unchecked") diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java index 115ef1e8a3f2b..cb4c93f153fd9 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java @@ -72,6 +72,7 @@ import org.mockito.Mockito; import org.slf4j.Logger; import org.testng.Assert; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -87,6 +88,7 @@ public class ContextImplTest { private PulsarAdmin pulsarAdmin; private ContextImpl context; private Producer producer; + private ProducerCache producerCache; @BeforeMethod(alwaysRun = true) public void setup() throws PulsarClientException { @@ -117,16 +119,24 @@ public void setup() throws PulsarClientException { TypedMessageBuilder messageBuilder = spy(new TypedMessageBuilderImpl(mock(ProducerBase.class), Schema.STRING)); doReturn(new CompletableFuture<>()).when(messageBuilder).sendAsync(); when(producer.newMessage()).thenReturn(messageBuilder); + doReturn(CompletableFuture.completedFuture(null)).when(producer).flushAsync(); + producerCache = new ProducerCache(); context = new ContextImpl( config, logger, client, new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0], FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), - pulsarAdmin, clientBuilder, t -> {}); + pulsarAdmin, clientBuilder, t -> {}, producerCache); context.setCurrentMessageContext((Record) () -> null); } + @AfterMethod(alwaysRun = true) + public void tearDown() { + producerCache.close(); + producerCache = null; + } + @Test(expectedExceptions = IllegalStateException.class) public void testIncrCounterStateDisabled() { context.incrCounter("test-key", 10); @@ -237,7 +247,7 @@ public void testGetPulsarAdminWithExposePulsarAdminDisabled() throws PulsarClien new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0], FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), - pulsarAdmin, clientBuilder, t -> {}); + pulsarAdmin, clientBuilder, t -> {}, producerCache); context.getPulsarAdmin(); } @@ -251,7 +261,7 @@ public void testUnsupportedExtendedSinkContext() throws PulsarClientException { new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0], FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), - pulsarAdmin, clientBuilder, t -> {}); + pulsarAdmin, clientBuilder, t -> {}, producerCache); try { context.seek("z", 0, Mockito.mock(MessageId.class)); Assert.fail("Expected exception"); @@ -282,7 +292,7 @@ public void testExtendedSinkContext() throws PulsarClientException { new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0], FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), - pulsarAdmin, clientBuilder, t -> {}); + pulsarAdmin, clientBuilder, t -> {}, producerCache); Consumer mockConsumer = Mockito.mock(Consumer.class); when(mockConsumer.getTopic()).thenReturn(TopicName.get("z").toString()); context.setInputConsumers(Lists.newArrayList(mockConsumer)); @@ -314,7 +324,7 @@ public void testGetConsumer() throws PulsarClientException { new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0], FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), - pulsarAdmin, clientBuilder, t -> {}); + pulsarAdmin, clientBuilder, t -> {}, producerCache); Consumer mockConsumer = Mockito.mock(Consumer.class); when(mockConsumer.getTopic()).thenReturn(TopicName.get("z").toString()); context.setInputConsumers(Lists.newArrayList(mockConsumer)); @@ -338,7 +348,7 @@ public void testGetConsumerMultiTopic() throws PulsarClientException { new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0], FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), - pulsarAdmin, clientBuilder, t -> {}); + pulsarAdmin, clientBuilder, t -> {}, producerCache); ConsumerImpl consumer1 = Mockito.mock(ConsumerImpl.class); when(consumer1.getTopic()).thenReturn(TopicName.get("first").toString()); ConsumerImpl consumer2 = Mockito.mock(ConsumerImpl.class); @@ -456,7 +466,7 @@ FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), pulsarAdmin, clientBuilder, t -> { assertEquals(t, fatalException); fatalInvoked.set(true); - }); + }, producerCache); context.fatal(fatalException); assertTrue(fatalInvoked.get()); } diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java index 799bad839a451..8a946a3f7571b 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java @@ -18,13 +18,13 @@ */ package org.apache.pulsar.functions.sink; +import static org.apache.pulsar.functions.instance.ProducerCache.CacheArea.SINK_RECORD_CACHE; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyList; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -37,7 +37,6 @@ import static org.testng.Assert.fail; import java.io.IOException; import java.util.HashMap; -import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; import lombok.Getter; @@ -65,12 +64,14 @@ import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.api.SerDe; +import org.apache.pulsar.functions.instance.ProducerCache; import org.apache.pulsar.functions.instance.SinkRecord; import org.apache.pulsar.functions.instance.stats.ComponentStatsManager; import org.apache.pulsar.functions.sink.PulsarSink.PulsarSinkProcessorBase; import org.apache.pulsar.functions.source.TopicSchema; import org.apache.pulsar.io.core.SinkContext; import org.testng.Assert; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -132,6 +133,7 @@ private static PulsarClientImpl getPulsarClient() throws PulsarClientException { doReturn(producer).when(producerBuilder).create(); doReturn(typedMessageBuilder).when(producer).newMessage(); doReturn(typedMessageBuilder).when(producer).newMessage(any(Schema.class)); + doReturn(CompletableFuture.completedFuture(null)).when(producer).flushAsync(); doReturn(producerBuilder).when(pulsarClient).newProducer(); doReturn(producerBuilder).when(pulsarClient).newProducer(any()); @@ -139,9 +141,17 @@ private static PulsarClientImpl getPulsarClient() throws PulsarClientException { return pulsarClient; } - @BeforeMethod + ProducerCache producerCache; + + @BeforeMethod(alwaysRun = true) public void setup() { + producerCache = new ProducerCache(); + } + @AfterMethod(alwaysRun = true) + public void tearDown() { + producerCache.close(); + producerCache = null; } private static PulsarSinkConfig getPulsarConfigs() { @@ -182,7 +192,7 @@ public void testVoidOutputClasses() throws Exception { pulsarConfig.setTypeClassName(Void.class.getName()); PulsarSink pulsarSink = new PulsarSink(getPulsarClient(), pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); try { Schema schema = pulsarSink.initializeSchema(); @@ -202,7 +212,7 @@ public void testInconsistentOutputType() throws IOException { pulsarConfig.setSerdeClassName(TestSerDe.class.getName()); PulsarSink pulsarSink = new PulsarSink(getPulsarClient(), pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); try { pulsarSink.initializeSchema(); fail("Should fail constructing java instance if function type is inconsistent with serde type"); @@ -227,7 +237,7 @@ public void testDefaultSerDe() throws PulsarClientException { pulsarConfig.setTypeClassName(String.class.getName()); PulsarSink pulsarSink = new PulsarSink(getPulsarClient(), pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); try { pulsarSink.initializeSchema(); @@ -248,7 +258,7 @@ public void testExplicitDefaultSerDe() throws PulsarClientException { pulsarConfig.setSerdeClassName(TopicSchema.DEFAULT_SERDE); PulsarSink pulsarSink = new PulsarSink(getPulsarClient(), pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); try { pulsarSink.initializeSchema(); @@ -266,7 +276,7 @@ public void testComplexOuputType() throws PulsarClientException { pulsarConfig.setSerdeClassName(ComplexSerDe.class.getName()); PulsarSink pulsarSink = new PulsarSink(getPulsarClient(), pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); try { pulsarSink.initializeSchema(); @@ -286,7 +296,7 @@ public void testInitializeSchema() throws Exception { pulsarSinkConfig.setTypeClassName(GenericRecord.class.getName()); PulsarSink sink = new PulsarSink( pulsarClient, pulsarSinkConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); Schema schema = sink.initializeSchema(); assertTrue(schema instanceof AutoConsumeSchema); @@ -295,7 +305,7 @@ public void testInitializeSchema() throws Exception { pulsarSinkConfig.setTypeClassName(GenericRecord.class.getName()); sink = new PulsarSink( pulsarClient, pulsarSinkConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); schema = sink.initializeSchema(); assertTrue(schema instanceof AutoConsumeSchema); @@ -306,7 +316,7 @@ public void testInitializeSchema() throws Exception { pulsarSinkConfig.setTypeClassName(GenericRecord.class.getName()); sink = new PulsarSink( pulsarClient, pulsarSinkConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); schema = sink.initializeSchema(); assertTrue(schema instanceof AutoConsumeSchema); @@ -317,7 +327,7 @@ public void testInitializeSchema() throws Exception { pulsarSinkConfig.setTypeClassName(GenericRecord.class.getName()); sink = new PulsarSink( pulsarClient, pulsarSinkConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); schema = sink.initializeSchema(); assertTrue(schema instanceof AutoConsumeSchema); @@ -327,7 +337,7 @@ public void testInitializeSchema() throws Exception { pulsarSinkConfig.setTypeClassName(GenericRecord.class.getName()); sink = new PulsarSink( pulsarClient, pulsarSinkConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); schema = sink.initializeSchema(); assertTrue(schema instanceof AutoConsumeSchema); } @@ -344,9 +354,12 @@ public void testSinkAndMessageRouting() throws Exception { /** test MANUAL **/ pulsarClient = getPulsarClient(); pulsarConfig.setProcessingGuarantees(ProcessingGuarantees.MANUAL); - PulsarSink pulsarSink = new PulsarSink(pulsarClient, pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), Thread.currentThread().getContextClassLoader()); + PulsarSink pulsarSink = + new PulsarSink(pulsarClient, pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), + Thread.currentThread().getContextClassLoader(), producerCache); pulsarSink.open(new HashMap<>(), mock(SinkContext.class)); + verify(pulsarClient.newProducer(), times(1)).topic(defaultTopic); for (String topic : topics) { @@ -370,23 +383,19 @@ public Optional getDestinationTopic() { PulsarSink.PulsarSinkManualProcessor pulsarSinkManualProcessor = (PulsarSink.PulsarSinkManualProcessor) pulsarSink.pulsarSinkProcessor; if (topic != null) { - Assert.assertTrue(pulsarSinkManualProcessor.publishProducers.containsKey(topic)); + Assert.assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, topic)); } else { - Assert.assertTrue(pulsarSinkManualProcessor.publishProducers.containsKey(defaultTopic)); + Assert.assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, defaultTopic)); } - verify(pulsarClient.newProducer(), times(1)).topic(argThat(otherTopic -> { - if (topic != null) { - return topic.equals(otherTopic); - } else { - return defaultTopic.equals(otherTopic); - } - })); + String actualTopic = topic != null ? topic : defaultTopic; + verify(pulsarClient.newProducer(), times(1)).topic(actualTopic); } /** test At-least-once **/ pulsarClient = getPulsarClient(); pulsarConfig.setProcessingGuarantees(ProcessingGuarantees.ATLEAST_ONCE); - pulsarSink = new PulsarSink(pulsarClient, pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), Thread.currentThread().getContextClassLoader()); + pulsarSink = new PulsarSink(pulsarClient, pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), + Thread.currentThread().getContextClassLoader(), producerCache); pulsarSink.open(new HashMap<>(), mock(SinkContext.class)); @@ -410,24 +419,17 @@ public Optional getDestinationTopic() { PulsarSink.PulsarSinkAtLeastOnceProcessor pulsarSinkAtLeastOnceProcessor = (PulsarSink.PulsarSinkAtLeastOnceProcessor) pulsarSink.pulsarSinkProcessor; if (topic != null) { - Assert.assertTrue(pulsarSinkAtLeastOnceProcessor.publishProducers.containsKey(topic)); + Assert.assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, topic)); } else { - Assert.assertTrue(pulsarSinkAtLeastOnceProcessor.publishProducers.containsKey(defaultTopic)); + Assert.assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, defaultTopic)); } - verify(pulsarClient.newProducer(), times(1)).topic(argThat(otherTopic -> { - if (topic != null) { - return topic.equals(otherTopic); - } else { - return defaultTopic.equals(otherTopic); - } - })); } /** test At-most-once **/ pulsarClient = getPulsarClient(); pulsarConfig.setProcessingGuarantees(ProcessingGuarantees.ATMOST_ONCE); pulsarSink = new PulsarSink(pulsarClient, pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); pulsarSink.open(new HashMap<>(), mock(SinkContext.class)); @@ -457,20 +459,17 @@ public Optional getDestinationTopic() { PulsarSink.PulsarSinkAtMostOnceProcessor pulsarSinkAtLeastOnceProcessor = (PulsarSink.PulsarSinkAtMostOnceProcessor) pulsarSink.pulsarSinkProcessor; if (topic != null) { - Assert.assertTrue(pulsarSinkAtLeastOnceProcessor.publishProducers.containsKey(topic)); + Assert.assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, topic)); } else { - Assert.assertTrue(pulsarSinkAtLeastOnceProcessor.publishProducers.containsKey(defaultTopic)); + Assert.assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, defaultTopic)); } - verify(pulsarClient.newProducer(), times(1)).topic(argThat(o -> { - return getTopicEquals(o, topic, defaultTopic); - })); } /** test Effectively-once **/ pulsarClient = getPulsarClient(); pulsarConfig.setProcessingGuarantees(FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE); pulsarSink = new PulsarSink(pulsarClient, pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); pulsarSink.open(new HashMap<>(), mock(SinkContext.class)); @@ -520,23 +519,19 @@ public Optional getRecordSequence() { PulsarSink.PulsarSinkEffectivelyOnceProcessor pulsarSinkEffectivelyOnceProcessor = (PulsarSink.PulsarSinkEffectivelyOnceProcessor) pulsarSink.pulsarSinkProcessor; if (topic != null) { - Assert.assertTrue(pulsarSinkEffectivelyOnceProcessor.publishProducers - .containsKey(String.format("%s-%s-id-1", topic, topic))); + Assert.assertTrue(producerCache + .containsKey(SINK_RECORD_CACHE, topic, String.format("%s-id-1", topic))); } else { - Assert.assertTrue(pulsarSinkEffectivelyOnceProcessor.publishProducers - .containsKey(String.format("%s-%s-id-1", defaultTopic, defaultTopic))); + Assert.assertTrue(producerCache + .containsKey(SINK_RECORD_CACHE, + defaultTopic, String.format("%s-id-1", defaultTopic) + )); } - verify(pulsarClient.newProducer(), times(1)).topic(argThat(o -> { - return getTopicEquals(o, topic, defaultTopic); - })); - verify(pulsarClient.newProducer(), times(1)).producerName(argThat(o -> { - if (topic != null) { - return String.format("%s-id-1", topic).equals(o); - } else { - return String.format("%s-id-1", defaultTopic).equals(o); - } - })); + String expectedTopicName = topic != null ? topic : defaultTopic; + verify(pulsarClient.newProducer(), times(1)).topic(expectedTopicName); + String expectedProducerName = String.format("%s-id-1", expectedTopicName); + verify(pulsarClient.newProducer(), times(1)).producerName(expectedProducerName); } } @@ -566,7 +561,7 @@ private void testWriteGenericRecords(ProcessingGuarantees guarantees) throws Exc PulsarClient client = getPulsarClient(); PulsarSink pulsarSink = new PulsarSink( client, sinkConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); pulsarSink.open(new HashMap<>(), mock(SinkContext.class)); @@ -578,7 +573,7 @@ private void testWriteGenericRecords(ProcessingGuarantees guarantees) throws Exc assertTrue(pulsarSink.pulsarSinkProcessor instanceof PulsarSink.PulsarSinkEffectivelyOnceProcessor); } PulsarSinkProcessorBase processor = (PulsarSinkProcessorBase) pulsarSink.pulsarSinkProcessor; - assertFalse(processor.publishProducers.containsKey(defaultTopic)); + assertFalse(producerCache.containsKey(SINK_RECORD_CACHE, defaultTopic)); String[] topics = {"topic-1", "topic-2", "topic-3"}; for (String topic : topics) { @@ -625,17 +620,15 @@ public Optional getRecordSequence() { pulsarSink.write(record); if (ProcessingGuarantees.EFFECTIVELY_ONCE == guarantees) { - assertTrue(processor.publishProducers.containsKey(String.format("%s-%s-id-1", topic, topic))); + assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, + topic, String.format("%s-id-1", topic) + )); } else { - assertTrue(processor.publishProducers.containsKey(topic)); + assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, topic)); } - verify(client.newProducer(), times(1)) - .topic(argThat( - otherTopic -> topic != null ? topic.equals(otherTopic) : defaultTopic.equals(otherTopic))); - - verify(client, times(1)) - .newProducer(argThat( - otherSchema -> Objects.equals(otherSchema, schema))); + String expectedTopicName = topic != null ? topic : defaultTopic; + verify(client.newProducer(), times(1)).topic(expectedTopicName); + verify(client, times(1)).newProducer(schema); } } @@ -646,13 +639,4 @@ private Optional getTopicOptional(String topic) { return Optional.empty(); } } - - private boolean getTopicEquals(Object o, String topic, String defaultTopic) { - if (topic != null) { - return topic.equals(o); - } else { - return defaultTopic.equals(o); - } - } - } From 1c44fbb8a03e583e94aa9dbef87dfa0a165e1cd8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 25 Jun 2024 09:38:56 +0300 Subject: [PATCH 339/580] [fix][broker] Fix updatePartitionedTopic when replication at ns level and topic policy is set (#22971) --- .../admin/impl/PersistentTopicsBase.java | 5 ++-- .../broker/service/OneWayReplicatorTest.java | 29 +++++++++++++++++-- .../service/OneWayReplicatorTestBase.java | 9 +++--- 3 files changed, 33 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index beb8ecc8d799b..93e4234559ecc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -35,7 +35,6 @@ import java.util.Base64; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -458,8 +457,8 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean Set replicationClusters = policies.get().replication_clusters; TopicPolicies topicPolicies = pulsarService.getTopicPoliciesService().getTopicPoliciesIfExists(topicName); - if (topicPolicies != null) { - replicationClusters = new HashSet<>(topicPolicies.getReplicationClusters()); + if (topicPolicies != null && topicPolicies.getReplicationClusters() != null) { + replicationClusters = topicPolicies.getReplicationClustersSet(); } // Do check replicated clusters. if (replicationClusters.size() == 0) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index e686cd2c94f62..80091c9e5eb2c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -73,12 +73,12 @@ import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; -import org.apache.pulsar.common.policies.data.RetentionPolicies; -import org.apache.pulsar.common.policies.data.TopicStats; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; @@ -494,6 +494,29 @@ public void testPartitionedTopicLevelReplication() throws Exception { admin2.topics().deletePartitionedTopic(topicName); } + // https://github.com/apache/pulsar/issues/22967 + @Test + public void testPartitionedTopicWithTopicPolicyAndNoReplicationClusters() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createPartitionedTopic(topicName, 2); + try { + admin1.topicPolicies().setMessageTTL(topicName, 5); + Awaitility.await().ignoreExceptions().untilAsserted(() -> { + assertEquals(admin2.topics().getPartitionedTopicMetadata(topicName).partitions, 2); + }); + admin1.topics().updatePartitionedTopic(topicName, 3, false); + Awaitility.await().ignoreExceptions().untilAsserted(() -> { + assertEquals(admin2.topics().getPartitionedTopicMetadata(topicName).partitions, 3); + }); + } finally { + // cleanup. + admin1.topics().deletePartitionedTopic(topicName, true); + if (!usingGlobalZK) { + admin2.topics().deletePartitionedTopic(topicName, true); + } + } + } + @Test public void testPartitionedTopicLevelReplicationRemoteTopicExist() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index ffe6147412e56..d66e666e3a055 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -266,6 +266,7 @@ protected void setConfigDefaults(ServiceConfiguration config, String clusterName config.setEnableReplicatedSubscriptions(true); config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); config.setLoadBalancerSheddingEnabled(false); + config.setForceDeleteNamespaceAllowed(true); } @Override @@ -276,11 +277,11 @@ protected void cleanup() throws Exception { if (!usingGlobalZK) { admin2.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster2)); } - admin1.namespaces().deleteNamespace(replicatedNamespace); - admin1.namespaces().deleteNamespace(nonReplicatedNamespace); + admin1.namespaces().deleteNamespace(replicatedNamespace, true); + admin1.namespaces().deleteNamespace(nonReplicatedNamespace, true); if (!usingGlobalZK) { - admin2.namespaces().deleteNamespace(replicatedNamespace); - admin2.namespaces().deleteNamespace(nonReplicatedNamespace); + admin2.namespaces().deleteNamespace(replicatedNamespace, true); + admin2.namespaces().deleteNamespace(nonReplicatedNamespace, true); } // shutdown. From 7dba98bed46231ce9bd2bd2d8f5369b50a2119be Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 25 Jun 2024 18:04:02 +0800 Subject: [PATCH 340/580] [improve][build] Use amazoncorretto:21-alpine image instead of apk installation (#22973) Signed-off-by: Zixuan Liu --- docker/pulsar/Dockerfile | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index b4294dd10da38..81446ae5ee5ce 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -54,11 +54,9 @@ RUN chmod -R o+rx /pulsar RUN echo 'OPTS="$OPTS -Dorg.xerial.snappy.use.systemlib=true"' >> /pulsar/conf/bkenv.sh ### Create one stage to include JVM distribution -FROM alpine:$ALPINE_VERSION AS jvm +FROM amazoncorretto:21-alpine AS jvm -RUN wget -O /etc/apk/keys/amazoncorretto.rsa.pub https://apk.corretto.aws/amazoncorretto.rsa.pub -RUN echo "https://apk.corretto.aws" >> /etc/apk/repositories -RUN apk add --no-cache amazon-corretto-21 binutils +RUN apk add --no-cache binutils # Use JLink to create a slimmer JDK distribution (see: https://adoptium.net/blog/2021/10/jlink-to-produce-own-runtime/) # This still includes all JDK modules, though in the future we could compile a list of required modules From f323342a4aa158ac72a9a3dc3cc67b8c2c5fd986 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 25 Jun 2024 07:34:59 -0700 Subject: [PATCH 341/580] [feat][broker] PIP-264: Add OpenTelemetry broker replicator metrics (#22972) --- .../apache/pulsar/broker/PulsarService.java | 7 + .../broker/service/AbstractReplicator.java | 40 +++++ .../pulsar/broker/service/AbstractTopic.java | 8 + .../pulsar/broker/service/Replicator.java | 6 +- .../NonPersistentReplicator.java | 25 ++- .../nonpersistent/NonPersistentTopic.java | 2 +- .../persistent/GeoPersistentReplicator.java | 2 + .../persistent/PersistentReplicator.java | 48 ++--- .../service/persistent/PersistentTopic.java | 4 +- .../service/persistent/ShadowReplicator.java | 2 + .../stats/OpenTelemetryReplicatorStats.java | 166 ++++++++++++++++++ .../prometheus/NamespaceStatsAggregator.java | 2 +- .../service/AbstractReplicatorTest.java | 5 + .../pulsar/broker/service/ReplicatorTest.java | 107 ++++++++++- .../broker/service/ReplicatorTestBase.java | 56 ++++-- .../stats/BrokerOpenTelemetryTestUtil.java | 18 ++ .../client/api/BrokerServiceLookupTest.java | 1 + .../data/NonPersistentReplicatorStats.java | 3 + .../common/policies/data/ReplicatorStats.java | 20 +++ .../NonPersistentReplicatorStatsImpl.java | 24 ++- .../data/stats/ReplicatorStatsImpl.java | 62 ++++++- .../OpenTelemetryAttributes.java | 6 + 22 files changed, 539 insertions(+), 75 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryReplicatorStats.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 65dd90f7a1235..8cf1376642b88 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -113,6 +113,7 @@ import org.apache.pulsar.broker.stats.MetricsGenerator; import org.apache.pulsar.broker.stats.OpenTelemetryConsumerStats; import org.apache.pulsar.broker.stats.OpenTelemetryProducerStats; +import org.apache.pulsar.broker.stats.OpenTelemetryReplicatorStats; import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; import org.apache.pulsar.broker.stats.PulsarBrokerOpenTelemetry; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; @@ -260,6 +261,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private OpenTelemetryTopicStats openTelemetryTopicStats; private OpenTelemetryConsumerStats openTelemetryConsumerStats; private OpenTelemetryProducerStats openTelemetryProducerStats; + private OpenTelemetryReplicatorStats openTelemetryReplicatorStats; private TransactionMetadataStoreService transactionMetadataStoreService; private TransactionBufferProvider transactionBufferProvider; @@ -678,6 +680,10 @@ public CompletableFuture closeAsync() { brokerClientSharedTimer.stop(); monotonicSnapshotClock.close(); + if (openTelemetryReplicatorStats != null) { + openTelemetryReplicatorStats.close(); + openTelemetryReplicatorStats = null; + } if (openTelemetryProducerStats != null) { openTelemetryProducerStats.close(); openTelemetryProducerStats = null; @@ -834,6 +840,7 @@ public void start() throws PulsarServerException { openTelemetryTopicStats = new OpenTelemetryTopicStats(this); openTelemetryConsumerStats = new OpenTelemetryConsumerStats(this); openTelemetryProducerStats = new OpenTelemetryProducerStats(this); + openTelemetryReplicatorStats = new OpenTelemetryReplicatorStats(this); localMetadataSynchronizer = StringUtils.isNotBlank(config.getMetadataSyncEventTopic()) ? new PulsarMetadataEventSynchronizer(this, config.getMetadataSyncEventTopic()) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index 869a4bc81d310..8552a9f09e93b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service; import com.google.common.annotations.VisibleForTesting; +import io.opentelemetry.api.common.Attributes; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -42,6 +43,7 @@ import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.StringInterner; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,6 +57,7 @@ public abstract class AbstractReplicator implements Replicator { protected final PulsarClientImpl replicationClient; protected final PulsarClientImpl client; protected String replicatorId; + @Getter protected final Topic localTopic; protected volatile ProducerImpl producer; @@ -74,6 +77,10 @@ public abstract class AbstractReplicator implements Replicator { @Getter protected volatile State state = State.Disconnected; + private volatile Attributes attributes = null; + private static final AtomicReferenceFieldUpdater ATTRIBUTES_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(AbstractReplicator.class, Attributes.class, "attributes"); + public enum State { /** * This enum has two mean meanings: @@ -136,6 +143,17 @@ public AbstractReplicator(String localCluster, Topic localTopic, String remoteCl protected abstract void disableReplicatorRead(); + @Override + public boolean isConnected() { + var producer = this.producer; + return producer != null && producer.isConnected(); + } + + public long getReplicationDelayMs() { + var producer = this.producer; + return producer == null ? 0 : producer.getDelayInMillis(); + } + public String getRemoteCluster() { return remoteCluster; } @@ -476,4 +494,26 @@ protected ImmutablePair compareSetAndGetState(State expect, Stat public boolean isTerminated() { return state == State.Terminating || state == State.Terminated; } + + public Attributes getAttributes() { + if (attributes != null) { + return attributes; + } + return ATTRIBUTES_UPDATER.updateAndGet(this, old -> { + if (old != null) { + return old; + } + var topicName = TopicName.get(getLocalTopic().getName()); + var builder = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, topicName.getDomain().toString()) + .put(OpenTelemetryAttributes.PULSAR_TENANT, topicName.getTenant()) + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, topicName.getNamespace()) + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName.getPartitionedTopicName()); + if (topicName.isPartitioned()) { + builder.put(OpenTelemetryAttributes.PULSAR_PARTITION_INDEX, topicName.getPartitionIndex()); + } + builder.put(OpenTelemetryAttributes.PULSAR_REPLICATION_REMOTE_CLUSTER_NAME, getRemoteCluster()); + return builder.build(); + }); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index 572b54e0d3e79..fbf11f1d0ad62 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -933,6 +933,14 @@ public void incrementPublishCount(Producer producer, int numOfMessages, long msg if (isSystemTopic()) { systemTopicBytesInCounter.add(msgSizeInBytes); } + + if (producer.isRemote()) { + var remoteClusterName = producer.getRemoteCluster(); + var replicator = getReplicators().get(remoteClusterName); + if (replicator != null) { + replicator.getStats().incrementPublishCount(numOfMessages, msgSizeInBytes); + } + } } private void handlePublishThrottling(Producer producer, int numOfMessages, long msgSizeInBytes) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java index 5c314397da80e..667063e491085 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Replicator.java @@ -27,7 +27,9 @@ public interface Replicator { void startProducer(); - ReplicatorStatsImpl getStats(); + Topic getLocalTopic(); + + ReplicatorStatsImpl computeStats(); CompletableFuture terminate(); @@ -53,4 +55,6 @@ default Optional getRateLimiter() { long getNumberOfEntriesInBacklog(); boolean isTerminated(); + + ReplicatorStatsImpl getStats(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java index 51509f3818a28..6441230fad87b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java @@ -116,6 +116,8 @@ public void sendMessage(Entry entry) { } msgOut.recordEvent(headersAndPayload.readableBytes()); + stats.incrementMsgOutCounter(); + stats.incrementBytesOutCounter(headersAndPayload.readableBytes()); msg.setReplicatedFrom(localCluster); @@ -129,6 +131,7 @@ public void sendMessage(Entry entry) { replicatorId); } msgDrop.recordEvent(); + stats.incrementMsgDropCount(); entry.release(); } } @@ -143,11 +146,11 @@ public void updateRates() { } @Override - public NonPersistentReplicatorStatsImpl getStats() { - stats.connected = producer != null && producer.isConnected(); - stats.replicationDelayInSeconds = getReplicationDelayInSeconds(); - + public NonPersistentReplicatorStatsImpl computeStats() { ProducerImpl producer = this.producer; + stats.connected = isConnected(); + stats.replicationDelayInSeconds = TimeUnit.MILLISECONDS.toSeconds(getReplicationDelayMs()); + if (producer != null) { stats.outboundConnection = producer.getConnectionId(); stats.outboundConnectedSince = producer.getConnectedSince(); @@ -159,11 +162,9 @@ public NonPersistentReplicatorStatsImpl getStats() { return stats; } - private long getReplicationDelayInSeconds() { - if (producer != null) { - return TimeUnit.MILLISECONDS.toSeconds(producer.getDelayInMillis()); - } - return 0L; + @Override + public NonPersistentReplicatorStatsImpl getStats() { + return stats; } private static final class ProducerSendCallback implements SendCallback { @@ -256,10 +257,4 @@ public long getNumberOfEntriesInBacklog() { protected void disableReplicatorRead() { // No-op } - - @Override - public boolean isConnected() { - ProducerImpl producer = this.producer; - return producer != null && producer.isConnected(); - } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index a6f65f6da3284..0c6ebdfefa01f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -961,7 +961,7 @@ public CompletableFuture asyncGetStats(GetStatsOptions }); replicators.forEach((cluster, replicator) -> { - NonPersistentReplicatorStatsImpl replicatorStats = replicator.getStats(); + NonPersistentReplicatorStatsImpl replicatorStats = replicator.computeStats(); // Add incoming msg rates PublisherStatsImpl pubStats = remotePublishersStats.get(replicator.getRemoteCluster()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java index 1314b2d2ed06b..1d9df2bcccda3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java @@ -166,6 +166,8 @@ protected boolean replicateEntries(List entries) { msg.getMessageBuilder().clearTxnidMostBits(); msg.getMessageBuilder().clearTxnidLeastBits(); msgOut.recordEvent(headersAndPayload.readableBytes()); + stats.incrementMsgOutCounter(); + stats.incrementBytesOutCounter(headersAndPayload.readableBytes()); // Increment pending messages for messages produced locally PENDING_MESSAGES_UPDATER.incrementAndGet(this); producer.sendAsync(msg, ProducerSendCallback.create(this, entry, msg)); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index 6263c512997fa..aa53a93da5c4f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -28,11 +28,13 @@ import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; import java.util.List; +import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import lombok.Getter; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.ClearBacklogCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback; @@ -107,7 +109,8 @@ public abstract class PersistentReplicator extends AbstractReplicator // for connected subscriptions, message expiry will be checked if the backlog is greater than this threshold private static final int MINIMUM_BACKLOG_FOR_EXPIRY_CHECK = 1000; - private final ReplicatorStatsImpl stats = new ReplicatorStatsImpl(); + @Getter + protected final ReplicatorStatsImpl stats = new ReplicatorStatsImpl(); protected volatile boolean fetchSchemaInProgress = false; @@ -118,7 +121,7 @@ public PersistentReplicator(String localCluster, PersistentTopic localTopic, Man super(localCluster, localTopic, remoteCluster, remoteTopic, localTopic.getReplicatorPrefix(), brokerService, replicationClient); this.topic = localTopic; - this.cursor = cursor; + this.cursor = Objects.requireNonNull(cursor); this.expiryMonitor = new PersistentMessageExpiryMonitor(localTopic, Codec.decode(cursor.getName()), cursor, null); HAVE_PENDING_READ_UPDATER.set(this, FALSE); @@ -186,12 +189,14 @@ public long getNumberOfEntriesInBacklog() { return cursor.getNumberOfEntriesInBacklog(true); } + public long getMessageExpiredCount() { + return expiryMonitor.getTotalMessageExpired(); + } + @Override protected void disableReplicatorRead() { - if (this.cursor != null) { - // deactivate cursor after successfully close the producer - this.cursor.setInactive(); - } + // deactivate cursor after successfully close the producer + this.cursor.setInactive(); } /** @@ -330,12 +335,10 @@ protected CompletableFuture getSchemaInfo(MessageImpl msg) throws Ex } public void updateCursorState() { - if (this.cursor != null) { - if (producer != null && producer.isConnected()) { - this.cursor.setActive(); - } else { - this.cursor.setInactive(); - } + if (isConnected()) { + cursor.setActive(); + } else { + cursor.setInactive(); } } @@ -595,10 +598,10 @@ public void updateRates() { stats.msgRateExpired = msgExpired.getRate() + expiryMonitor.getMessageExpiryRate(); } - public ReplicatorStatsImpl getStats() { - stats.replicationBacklog = cursor != null ? cursor.getNumberOfEntriesInBacklog(false) : 0; - stats.connected = producer != null && producer.isConnected(); - stats.replicationDelayInSeconds = getReplicationDelayInSeconds(); + public ReplicatorStatsImpl computeStats() { + stats.replicationBacklog = cursor.getNumberOfEntriesInBacklog(false); + stats.connected = isConnected(); + stats.replicationDelayInSeconds = TimeUnit.MILLISECONDS.toSeconds(getReplicationDelayMs()); ProducerImpl producer = this.producer; if (producer != null) { @@ -616,13 +619,6 @@ public void updateMessageTTL(int messageTTLInSeconds) { this.messageTTLInSeconds = messageTTLInSeconds; } - private long getReplicationDelayInSeconds() { - if (producer != null) { - return TimeUnit.MILLISECONDS.toSeconds(producer.getDelayInMillis()); - } - return 0L; - } - @Override public boolean expireMessages(int messageTTLInSeconds) { if ((cursor.getNumberOfEntriesInBacklog(false) == 0) @@ -691,12 +687,6 @@ protected void checkReplicatedSubscriptionMarker(Position position, MessageImpl< } } - @Override - public boolean isConnected() { - ProducerImpl producer = this.producer; - return producer != null && producer.isConnected(); - } - @Override protected void doReleaseResources() { dispatchRateLimiter.ifPresent(DispatchRateLimiter::close); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 1983fa3c383e3..6e3d49fbe9ff1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -2353,7 +2353,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats } // Update replicator stats - ReplicatorStatsImpl rStat = replicator.getStats(); + ReplicatorStatsImpl rStat = replicator.computeStats(); // Add incoming msg rates PublisherStatsImpl pubStats = topicStatsHelper.remotePublishersStats.get(replicator.getRemoteCluster()); @@ -2636,7 +2636,7 @@ public CompletableFuture asyncGetStats(GetStatsOptions }); replicators.forEach((cluster, replicator) -> { - ReplicatorStatsImpl replicatorStats = replicator.getStats(); + ReplicatorStatsImpl replicatorStats = replicator.computeStats(); // Add incoming msg rates PublisherStatsImpl pubStats = remotePublishersStats.get(replicator.getRemoteCluster()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ShadowReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ShadowReplicator.java index 85e837ff1879a..25591857aa1b5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ShadowReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ShadowReplicator.java @@ -92,6 +92,8 @@ protected boolean replicateEntries(List entries) { dispatchRateLimiter.ifPresent(rateLimiter -> rateLimiter.consumeDispatchQuota(1, entry.getLength())); msgOut.recordEvent(headersAndPayload.readableBytes()); + stats.incrementMsgOutCounter(); + stats.incrementBytesOutCounter(headersAndPayload.readableBytes()); msg.setReplicatedFrom(localCluster); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryReplicatorStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryReplicatorStats.java new file mode 100644 index 0000000000000..04bc805a64bbf --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryReplicatorStats.java @@ -0,0 +1,166 @@ +/* + * 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.stats; + +import io.opentelemetry.api.metrics.BatchCallback; +import io.opentelemetry.api.metrics.ObservableDoubleMeasurement; +import io.opentelemetry.api.metrics.ObservableLongMeasurement; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.AbstractReplicator; +import org.apache.pulsar.broker.service.nonpersistent.NonPersistentReplicator; +import org.apache.pulsar.broker.service.persistent.PersistentReplicator; +import org.apache.pulsar.common.stats.MetricsUtil; + +public class OpenTelemetryReplicatorStats implements AutoCloseable { + + // Replaces pulsar_replication_rate_in + public static final String MESSAGE_IN_COUNTER = "pulsar.broker.replication.message.incoming.count"; + private final ObservableLongMeasurement messageInCounter; + + // Replaces pulsar_replication_rate_out + public static final String MESSAGE_OUT_COUNTER = "pulsar.broker.replication.message.outgoing.count"; + private final ObservableLongMeasurement messageOutCounter; + + // Replaces pulsar_replication_throughput_in + public static final String BYTES_IN_COUNTER = "pulsar.broker.replication.message.incoming.size"; + private final ObservableLongMeasurement bytesInCounter; + + // Replaces pulsar_replication_throughput_out + public static final String BYTES_OUT_COUNTER = "pulsar.broker.replication.message.outgoing.size"; + private final ObservableLongMeasurement bytesOutCounter; + + // Replaces pulsar_replication_backlog + public static final String BACKLOG_COUNTER = "pulsar.broker.replication.message.backlog.count"; + private final ObservableLongMeasurement backlogCounter; + + // Replaces pulsar_replication_delay_in_seconds + public static final String DELAY_GAUGE = "pulsar.broker.replication.message.backlog.age"; + private final ObservableDoubleMeasurement delayGauge; + + // Replaces pulsar_replication_rate_expired + public static final String EXPIRED_COUNTER = "pulsar.broker.replication.message.expired.count"; + private final ObservableLongMeasurement expiredCounter; + + public static final String DROPPED_COUNTER = "pulsar.broker.replication.message.dropped.count"; + private final ObservableLongMeasurement droppedCounter; + + private final BatchCallback batchCallback; + + public OpenTelemetryReplicatorStats(PulsarService pulsar) { + var meter = pulsar.getOpenTelemetry().getMeter(); + + messageInCounter = meter + .upDownCounterBuilder(MESSAGE_IN_COUNTER) + .setUnit("{message}") + .setDescription( + "The total number of messages received from the remote cluster through this replicator.") + .buildObserver(); + + messageOutCounter = meter + .upDownCounterBuilder(MESSAGE_OUT_COUNTER) + .setUnit("{message}") + .setDescription("The total number of messages sent to the remote cluster through this replicator.") + .buildObserver(); + + bytesInCounter = meter + .upDownCounterBuilder(BYTES_IN_COUNTER) + .setUnit("{By}") + .setDescription( + "The total number of messages bytes received from the remote cluster through this replicator.") + .buildObserver(); + + bytesOutCounter = meter + .upDownCounterBuilder(BYTES_OUT_COUNTER) + .setUnit("{By}") + .setDescription( + "The total number of messages bytes sent to the remote cluster through this replicator.") + .buildObserver(); + + backlogCounter = meter + .upDownCounterBuilder(BACKLOG_COUNTER) + .setUnit("{message}") + .setDescription("The total number of messages in the backlog for this replicator.") + .buildObserver(); + + delayGauge = meter + .gaugeBuilder(DELAY_GAUGE) + .setUnit("s") + .setDescription("The age of the oldest message in the replicator backlog.") + .buildObserver(); + + expiredCounter = meter + .upDownCounterBuilder(EXPIRED_COUNTER) + .setUnit("{message}") + .setDescription("The total number of messages that expired for this replicator.") + .buildObserver(); + + droppedCounter = meter + .upDownCounterBuilder(DROPPED_COUNTER) + .setUnit("{message}") + .setDescription("The total number of messages dropped by this replicator.") + .buildObserver(); + + batchCallback = meter.batchCallback(() -> pulsar.getBrokerService() + .getTopics() + .values() + .stream() + .filter(topicFuture -> topicFuture.isDone() && !topicFuture.isCompletedExceptionally()) + .map(CompletableFuture::join) + .filter(Optional::isPresent) + .map(Optional::get) + .flatMap(topic -> topic.getReplicators().values().stream()) + .map(AbstractReplicator.class::cast) + .forEach(this::recordMetricsForReplicator), + messageInCounter, + messageOutCounter, + bytesInCounter, + bytesOutCounter, + backlogCounter, + delayGauge, + expiredCounter, + droppedCounter); + } + + @Override + public void close() { + batchCallback.close(); + } + + private void recordMetricsForReplicator(AbstractReplicator replicator) { + var attributes = replicator.getAttributes(); + var stats = replicator.getStats(); + + messageInCounter.record(stats.getMsgInCount(), attributes); + messageOutCounter.record(stats.getMsgOutCount(), attributes); + bytesInCounter.record(stats.getBytesInCount(), attributes); + bytesOutCounter.record(stats.getBytesOutCount(), attributes); + var delaySeconds = MetricsUtil.convertToSeconds(replicator.getReplicationDelayMs(), TimeUnit.MILLISECONDS); + delayGauge.record(delaySeconds, attributes); + + if (replicator instanceof PersistentReplicator persistentReplicator) { + expiredCounter.record(persistentReplicator.getMessageExpiredCount(), attributes); + backlogCounter.record(persistentReplicator.getNumberOfEntriesInBacklog(), attributes); + } else if (replicator instanceof NonPersistentReplicator nonPersistentReplicator) { + droppedCounter.record(nonPersistentReplicator.getStats().getMsgDropCount(), attributes); + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java index 3bbc9100b364f..a229ef54c795d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java @@ -290,7 +290,7 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include } topic.getReplicators().forEach((cluster, replicator) -> { - ReplicatorStatsImpl replStats = replicator.getStats(); + ReplicatorStatsImpl replStats = replicator.computeStats(); AggregatedReplicationStats aggReplStats = stats.replicationStats.get(replicator.getRemoteCluster()); if (aggReplStats == null) { aggReplStats = new AggregatedReplicationStats(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java index d20f5f0d520e9..64d3088b20622 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java @@ -139,6 +139,11 @@ protected Position getReplicatorReadPosition() { return PositionFactory.EARLIEST; } + @Override + public ReplicatorStatsImpl computeStats() { + return null; + } + @Override public ReplicatorStatsImpl getStats() { return null; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index d83b2ed4ee6c9..1c47abab775b3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -18,11 +18,15 @@ */ package org.apache.pulsar.broker.service; +import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricDoubleGaugeValue; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -31,6 +35,7 @@ import com.google.common.collect.Sets; import com.scurrilous.circe.checksum.Crc32cIntChecksum; import io.netty.buffer.ByteBuf; +import io.opentelemetry.api.common.Attributes; import java.lang.reflect.Field; import java.lang.reflect.Method; import java.nio.charset.StandardCharsets; @@ -58,10 +63,10 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerException.CursorAlreadyClosedException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.commons.lang3.RandomUtils; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; @@ -69,6 +74,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException.NamingException; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.OpenTelemetryReplicatorStats; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -104,6 +110,7 @@ import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.apache.pulsar.schema.Schemas; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; @@ -667,7 +674,7 @@ public void testReplicatorClearBacklog() throws Exception { Thread.sleep(100); replicator.updateRates(); // for code-coverage replicator.expireMessages(1); // for code-coverage - ReplicatorStats status = replicator.getStats(); + ReplicatorStats status = replicator.computeStats(); assertEquals(status.getReplicationBacklog(), 0); } @@ -697,7 +704,7 @@ public void testResetReplicatorSubscriptionPosition() throws Exception { replicator.updateRates(); - ReplicatorStats status = replicator.getStats(); + ReplicatorStats status = replicator.computeStats(); assertEquals(status.getReplicationBacklog(), 0); } @@ -997,14 +1004,28 @@ public void testResumptionAfterBacklogRelaxed() throws Exception { Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); - assertEquals(replicator.getStats().replicationBacklog, 0); + assertEquals(replicator.computeStats().replicationBacklog, 0); + var attributes = Attributes.of( + OpenTelemetryAttributes.PULSAR_DOMAIN, dest.getDomain().value(), + OpenTelemetryAttributes.PULSAR_TENANT, dest.getTenant(), + OpenTelemetryAttributes.PULSAR_NAMESPACE, dest.getNamespace(), + OpenTelemetryAttributes.PULSAR_TOPIC, dest.getPartitionedTopicName(), + OpenTelemetryAttributes.PULSAR_REPLICATION_REMOTE_CLUSTER_NAME, cluster2 + ); + var metrics = metricReader1.collectAllMetrics(); + assertMetricLongSumValue(metrics, OpenTelemetryReplicatorStats.BACKLOG_COUNTER, attributes, 0); + assertMetricDoubleGaugeValue(metrics, OpenTelemetryReplicatorStats.DELAY_GAUGE, attributes, 0.0); // Next message will not be replicated, because r2 has reached the quota producer1.produce(1); Thread.sleep(500); - assertEquals(replicator.getStats().replicationBacklog, 1); + assertEquals(replicator.computeStats().replicationBacklog, 1); + metrics = metricReader1.collectAllMetrics(); + assertMetricLongSumValue(metrics, OpenTelemetryReplicatorStats.BACKLOG_COUNTER, attributes, 1); + assertMetricDoubleGaugeValue(metrics, OpenTelemetryReplicatorStats.DELAY_GAUGE, attributes, + aDouble -> assertThat(aDouble).isPositive()); // Consumer will now drain 1 message and the replication backlog will be cleared consumer2.receive(1); @@ -1013,13 +1034,16 @@ public void testResumptionAfterBacklogRelaxed() throws Exception { consumer2.receive(1); int retry = 10; - for (int i = 0; i < retry && replicator.getStats().replicationBacklog > 0; i++) { + for (int i = 0; i < retry && replicator.computeStats().replicationBacklog > 0; i++) { if (i != retry - 1) { Thread.sleep(100); } } - assertEquals(replicator.getStats().replicationBacklog, 0); + assertEquals(replicator.computeStats().replicationBacklog, 0); + metrics = metricReader1.collectAllMetrics(); + assertMetricLongSumValue(metrics, OpenTelemetryReplicatorStats.BACKLOG_COUNTER, attributes, 0); + assertMetricDoubleGaugeValue(metrics, OpenTelemetryReplicatorStats.DELAY_GAUGE, attributes, 0.0); } } @@ -1813,6 +1837,72 @@ public void testReplicatorWithTTL() throws Exception { assertEquals(result, Lists.newArrayList("V1", "V2", "V3", "V4")); } + @Test + public void testReplicationMetrics() throws Exception { + var destTopicName = TopicName.get(BrokerTestUtil.newUniqueName("persistent://pulsar/ns/replicationMetrics")); + + @Cleanup + var producer1 = new MessageProducer(url1, destTopicName); + + @Cleanup + var consumer1 = new MessageConsumer(url1, destTopicName); + + @Cleanup + var consumer2 = new MessageConsumer(url2, destTopicName); + + // Produce from cluster 1 and consume from the 1 and 2. + producer1.produce(3); + consumer1.receive(2); + consumer2.receive(1); + + { + // Validate replicator metrics on cluster 1 from cluster 2 + var attributes = Attributes.of( + OpenTelemetryAttributes.PULSAR_DOMAIN, destTopicName.getDomain().value(), + OpenTelemetryAttributes.PULSAR_TENANT, destTopicName.getTenant(), + OpenTelemetryAttributes.PULSAR_NAMESPACE, destTopicName.getNamespace(), + OpenTelemetryAttributes.PULSAR_TOPIC, destTopicName.getPartitionedTopicName(), + OpenTelemetryAttributes.PULSAR_REPLICATION_REMOTE_CLUSTER_NAME, cluster2 + ); + var metrics = metricReader1.collectAllMetrics(); + assertMetricLongSumValue(metrics, OpenTelemetryReplicatorStats.MESSAGE_OUT_COUNTER, attributes, 3); + assertMetricLongSumValue(metrics, OpenTelemetryReplicatorStats.BYTES_OUT_COUNTER, attributes, + aLong -> assertThat(aLong).isPositive()); + + var topicOpt = pulsar1.getBrokerService().getTopicReference(destTopicName.toString()); + assertThat(topicOpt).isPresent(); + var topic = topicOpt.get(); + var persistentReplicators = topic.getReplicators() + .values() + .stream() + .map(PersistentReplicator.class::cast) + .toList(); + persistentReplicators.forEach(this::pauseReplicator); + producer1.produce(5); + Awaitility.await().untilAsserted(() -> { + persistentReplicators.forEach(repl -> repl.expireMessages(1)); + assertMetricLongSumValue(metricReader1.collectAllMetrics(), + OpenTelemetryReplicatorStats.EXPIRED_COUNTER, + attributes, 5); + }); + } + + { + // Validate replicator metrics on cluster 2 from cluster 1 + var attributes = Attributes.of( + OpenTelemetryAttributes.PULSAR_DOMAIN, destTopicName.getDomain().value(), + OpenTelemetryAttributes.PULSAR_TENANT, destTopicName.getTenant(), + OpenTelemetryAttributes.PULSAR_NAMESPACE, destTopicName.getNamespace(), + OpenTelemetryAttributes.PULSAR_TOPIC, destTopicName.getPartitionedTopicName(), + OpenTelemetryAttributes.PULSAR_REPLICATION_REMOTE_CLUSTER_NAME, cluster1 + ); + var metrics = metricReader2.collectAllMetrics(); + assertMetricLongSumValue(metrics, OpenTelemetryReplicatorStats.MESSAGE_IN_COUNTER, attributes, 3); + assertMetricLongSumValue(metrics, OpenTelemetryReplicatorStats.BYTES_IN_COUNTER, attributes, + aLong -> assertThat(aLong).isPositive()); + } + } + @Test public void testEnableReplicationWithNamespaceAllowedClustersPolices() throws Exception { log.info("--- testEnableReplicationWithNamespaceAllowedClustersPolices ---"); @@ -1873,5 +1963,8 @@ private void pauseReplicator(PersistentReplicator replicator) { assertTrue(replicator.isConnected()); }); replicator.closeProducerAsync(true); + Awaitility.await().untilAsserted(() -> { + assertFalse(replicator.isConnected()); + }); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java index 838632febd889..33877b681184f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java @@ -21,12 +21,10 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; - -import com.google.common.io.Resources; import com.google.common.collect.Sets; - +import com.google.common.io.Resources; import io.netty.util.concurrent.DefaultThreadFactory; - +import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; import java.net.URL; import java.util.Optional; import java.util.Set; @@ -35,12 +33,9 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; - import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.TopicType; -import org.apache.pulsar.tests.TestRetrySupport; +import org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -51,7 +46,11 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.functions.worker.WorkerConfig; +import org.apache.pulsar.tests.TestRetrySupport; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.apache.pulsar.zookeeper.ZookeeperServerTest; import org.slf4j.Logger; @@ -63,6 +62,7 @@ public abstract class ReplicatorTestBase extends TestRetrySupport { ServiceConfiguration config1 = new ServiceConfiguration(); PulsarService pulsar1; BrokerService ns1; + protected InMemoryMetricReader metricReader1; PulsarAdmin admin1; LocalBookkeeperEnsemble bkEnsemble1; @@ -74,6 +74,7 @@ public abstract class ReplicatorTestBase extends TestRetrySupport { BrokerService ns2; PulsarAdmin admin2; LocalBookkeeperEnsemble bkEnsemble2; + protected InMemoryMetricReader metricReader2; URL url3; URL urlTls3; @@ -82,6 +83,7 @@ public abstract class ReplicatorTestBase extends TestRetrySupport { BrokerService ns3; PulsarAdmin admin3; LocalBookkeeperEnsemble bkEnsemble3; + protected InMemoryMetricReader metricReader3; URL url4; URL urlTls4; @@ -89,6 +91,7 @@ public abstract class ReplicatorTestBase extends TestRetrySupport { PulsarService pulsar4; PulsarAdmin admin4; LocalBookkeeperEnsemble bkEnsemble4; + protected InMemoryMetricReader metricReader4; ZookeeperServerTest globalZkS; @@ -154,7 +157,8 @@ protected void setup() throws Exception { // completely // independent config objects instead of referring to the same properties object setConfig1DefaultValue(); - pulsar1 = new PulsarService(config1); + metricReader1 = InMemoryMetricReader.create(); + pulsar1 = buildPulsarService(config1, metricReader1); pulsar1.start(); ns1 = pulsar1.getBrokerService(); @@ -169,7 +173,8 @@ protected void setup() throws Exception { bkEnsemble2.start(); setConfig2DefaultValue(); - pulsar2 = new PulsarService(config2); + metricReader2 = InMemoryMetricReader.create(); + pulsar2 = buildPulsarService(config2, metricReader2); pulsar2.start(); ns2 = pulsar2.getBrokerService(); @@ -184,7 +189,8 @@ protected void setup() throws Exception { bkEnsemble3.start(); setConfig3DefaultValue(); - pulsar3 = new PulsarService(config3); + metricReader3 = InMemoryMetricReader.create(); + pulsar3 = buildPulsarService(config3, metricReader3); pulsar3.start(); ns3 = pulsar3.getBrokerService(); @@ -199,7 +205,8 @@ protected void setup() throws Exception { bkEnsemble4.start(); setConfig4DefaultValue(); - pulsar4 = new PulsarService(config4); + metricReader4 = InMemoryMetricReader.create(); + pulsar4 = buildPulsarService(config4, metricReader4); pulsar4.start(); url4 = new URL(pulsar4.getWebServiceAddress()); @@ -312,6 +319,14 @@ protected void setup() throws Exception { } + private PulsarService buildPulsarService(ServiceConfiguration config, InMemoryMetricReader metricReader) { + return new PulsarService(config, + new WorkerConfig(), + Optional.empty(), + exitCode -> log.info("Pulsar service finished with exit code {}", exitCode), + BrokerOpenTelemetryTestUtil.getOpenTelemetrySdkBuilderConsumer(metricReader)); + } + public void setConfig3DefaultValue() { setConfigDefaults(config3, cluster3, bkEnsemble3); config3.setTlsEnabled(true); @@ -409,6 +424,23 @@ protected void cleanup() throws Exception { admin4 = null; } + if (metricReader4 != null) { + metricReader4.close(); + metricReader4 = null; + } + if (metricReader3 != null) { + metricReader3.close(); + metricReader3 = null; + } + if (metricReader2 != null) { + metricReader2.close(); + metricReader2 = null; + } + if (metricReader1 != null) { + metricReader1.close(); + metricReader1 = null; + } + if (pulsar4 != null) { pulsar4.close(); pulsar4 = null; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java index cb61677ab953d..d7ad0588201d4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/BrokerOpenTelemetryTestUtil.java @@ -89,4 +89,22 @@ public static void assertMetricLongGaugeValue(Collection metrics, St valueConsumer.accept(point.getValue()); })))); } + + public static void assertMetricDoubleGaugeValue(Collection metrics, String metricName, + Attributes attributes, double expected) { + assertMetricDoubleGaugeValue(metrics, metricName, attributes, actual -> assertThat(actual).isEqualTo(expected)); + } + + public static void assertMetricDoubleGaugeValue(Collection metrics, String metricName, + Attributes attributes, Consumer valueConsumer) { + assertThat(metrics) + .anySatisfy(metric -> assertThat(metric) + .hasName(metricName) + .hasDoubleGaugeSatisfying(gauge -> gauge.satisfies( + pointData -> assertThat(pointData.getPoints()).anySatisfy( + point -> { + assertThat(point.getAttributes()).isEqualTo(attributes); + valueConsumer.accept(point.getValue()); + })))); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java index e99802a5bc5c4..157df1185307a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/BrokerServiceLookupTest.java @@ -193,6 +193,7 @@ public void testMultipleBrokerLookup() throws Exception { pulsar2.getOpenTelemetryTopicStats().close(); pulsar2.getOpenTelemetryConsumerStats().close(); pulsar2.getOpenTelemetryProducerStats().close(); + pulsar2.getOpenTelemetryReplicatorStats().close(); var metricReader = pulsarTestContext.getOpenTelemetryMetricReader(); var lookupRequestSemaphoreField = BrokerService.class.getDeclaredField("lookupRequestSemaphore"); diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentReplicatorStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentReplicatorStats.java index 6c77de9195786..bfeeb6d037a78 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentReplicatorStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NonPersistentReplicatorStats.java @@ -27,4 +27,7 @@ public interface NonPersistentReplicatorStats extends ReplicatorStats { * for non-persistent topic: broker drops msg for replicator if replicator connection is not writable. **/ double getMsgDropRate(); + + /** Total number of messages dropped by the broker for the replicator. */ + long getMsgDropCount(); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ReplicatorStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ReplicatorStats.java index 24be2f9380bb7..1790cc35f50c5 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ReplicatorStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ReplicatorStats.java @@ -24,20 +24,40 @@ public interface ReplicatorStats { /** Total rate of messages received from the remote cluster (msg/s). */ + @Deprecated double getMsgRateIn(); + /** Total number of messages received from the remote cluster. */ + long getMsgInCount(); + /** Total throughput received from the remote cluster (bytes/s). */ + @Deprecated double getMsgThroughputIn(); + /** Total number of bytes received from the remote cluster. */ + long getBytesInCount(); + /** Total rate of messages delivered to the replication-subscriber (msg/s). */ + @Deprecated double getMsgRateOut(); + /** Total number of messages sent to the remote cluster. */ + long getMsgOutCount(); + /** Total throughput delivered to the replication-subscriber (bytes/s). */ + @Deprecated double getMsgThroughputOut(); + /** Total number of bytes sent to the remote cluster. */ + long getBytesOutCount(); + /** Total rate of messages expired (msg/s). */ + @Deprecated double getMsgRateExpired(); + /** Total number of messages expired. */ + long getMsgExpiredCount(); + /** Number of messages pending to be replicated to remote cluster. */ long getReplicationBacklog(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentReplicatorStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentReplicatorStatsImpl.java index 98f838a94493c..a09d03b21a03a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentReplicatorStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/NonPersistentReplicatorStatsImpl.java @@ -18,27 +18,43 @@ */ package org.apache.pulsar.common.policies.data.stats; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Objects; -import lombok.Getter; +import java.util.concurrent.atomic.LongAdder; +import lombok.Data; +import lombok.EqualsAndHashCode; import org.apache.pulsar.common.policies.data.NonPersistentReplicatorStats; /** * Statistics for a non-persistent replicator. */ -@SuppressFBWarnings("EQ_DOESNT_OVERRIDE_EQUALS") +@Data +@EqualsAndHashCode(callSuper = true) public class NonPersistentReplicatorStatsImpl extends ReplicatorStatsImpl implements NonPersistentReplicatorStats { /** * for non-persistent topic: broker drops msg for replicator if replicator connection is not writable. **/ - @Getter public double msgDropRate; + @JsonIgnore + private final LongAdder msgDropCount = new LongAdder(); + public NonPersistentReplicatorStatsImpl add(NonPersistentReplicatorStatsImpl stats) { Objects.requireNonNull(stats); super.add(stats); this.msgDropRate += stats.msgDropRate; return this; } + + @Override + @JsonProperty + public long getMsgDropCount() { + return msgDropCount.sum(); + } + + public void incrementMsgDropCount() { + msgDropCount.increment(); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ReplicatorStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ReplicatorStatsImpl.java index 6933f5cc7ed76..c19169cbee57f 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ReplicatorStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ReplicatorStatsImpl.java @@ -18,7 +18,10 @@ */ package org.apache.pulsar.common.policies.data.stats; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Objects; +import java.util.concurrent.atomic.LongAdder; import lombok.Data; import org.apache.pulsar.common.policies.data.ReplicatorStats; @@ -31,15 +34,27 @@ public class ReplicatorStatsImpl implements ReplicatorStats { /** Total rate of messages received from the remote cluster (msg/s). */ public double msgRateIn; + @JsonIgnore + private final LongAdder msgInCount = new LongAdder(); + /** Total throughput received from the remote cluster (bytes/s). */ public double msgThroughputIn; + @JsonIgnore + private final LongAdder bytesInCount = new LongAdder(); + /** Total rate of messages delivered to the replication-subscriber (msg/s). */ public double msgRateOut; + @JsonIgnore + private final LongAdder msgOutCount = new LongAdder(); + /** Total throughput delivered to the replication-subscriber (bytes/s). */ public double msgThroughputOut; + @JsonIgnore + private final LongAdder bytesOutCount = new LongAdder(); + /** Total rate of messages expired (msg/s). */ public double msgRateExpired; @@ -72,10 +87,51 @@ public ReplicatorStatsImpl add(ReplicatorStatsImpl stats) { this.msgThroughputOut += stats.msgThroughputOut; this.msgRateExpired += stats.msgRateExpired; this.replicationBacklog += stats.replicationBacklog; - if (this.connected) { - this.connected &= stats.connected; - } + this.connected &= stats.connected; this.replicationDelayInSeconds = Math.max(this.replicationDelayInSeconds, stats.replicationDelayInSeconds); return this; } + + @Override + @JsonProperty + public long getMsgInCount() { + return msgInCount.sum(); + } + + @Override + @JsonProperty + public long getBytesInCount() { + return bytesInCount.sum(); + } + + public void incrementPublishCount(int numOfMessages, long msgSizeInBytes) { + msgInCount.add(numOfMessages); + bytesInCount.add(msgSizeInBytes); + } + + @Override + @JsonProperty + public long getMsgOutCount() { + return msgOutCount.sum(); + } + + public void incrementMsgOutCounter() { + msgOutCount.increment(); + } + + @Override + @JsonProperty + public long getBytesOutCount() { + return bytesOutCount.sum(); + } + + public void incrementBytesOutCounter(long bytes) { + bytesOutCount.add(bytes); + } + + @Override + @JsonProperty + public long getMsgExpiredCount() { + return 0; + } } diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index 6639cd68b398e..31e527f02869e 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -143,6 +143,12 @@ enum BacklogQuotaType { public final Attributes attributes = Attributes.of(PULSAR_BACKLOG_QUOTA_TYPE, name().toLowerCase()); } + /** + * The name of the remote cluster for a Pulsar replicator. + */ + AttributeKey PULSAR_REPLICATION_REMOTE_CLUSTER_NAME = + AttributeKey.stringKey("pulsar.replication.remote.cluster.name"); + AttributeKey PULSAR_CONNECTION_STATUS = AttributeKey.stringKey("pulsar.connection.status"); enum ConnectionStatus { ACTIVE, From 2da4ee8b54aa1e15d501b57cd4c476186aff92eb Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 25 Jun 2024 18:04:13 +0300 Subject: [PATCH 342/580] [revert] "[improve][broker] Optimize `ConcurrentOpenLongPairRangeSet` by RoaringBitmap (#22908)" (#22968) --- .../server/src/assemble/LICENSE.bin.txt | 2 +- .../shell/src/assemble/LICENSE.bin.txt | 2 - pom.xml | 2 +- pulsar-common/pom.xml | 5 - .../ConcurrentOpenLongPairRangeSet.java | 12 +- .../collections/ConcurrentRoaringBitSet.java | 439 ------------------ 6 files changed, 9 insertions(+), 453 deletions(-) delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 24c601b184afa..cfbe991a8edd8 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -513,7 +513,7 @@ The Apache Software License, Version 2.0 * RxJava - io.reactivex.rxjava3-rxjava-3.0.1.jar * RoaringBitmap - - org.roaringbitmap-RoaringBitmap-1.1.0.jar + - org.roaringbitmap-RoaringBitmap-1.0.6.jar * OpenTelemetry - io.opentelemetry-opentelemetry-api-1.38.0.jar - io.opentelemetry-opentelemetry-api-incubator-1.38.0-alpha.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 2971147c2c8df..0da56c6afa8fc 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -382,8 +382,6 @@ The Apache Software License, Version 2.0 - simpleclient_tracer_common-0.16.0.jar - simpleclient_tracer_otel-0.16.0.jar - simpleclient_tracer_otel_agent-0.16.0.jar - * RoaringBitmap - - RoaringBitmap-1.1.0.jar * Log4J - log4j-api-2.23.1.jar - log4j-core-2.23.1.jar diff --git a/pom.xml b/pom.xml index 1e200d04d68fd..7c556fa127786 100644 --- a/pom.xml +++ b/pom.xml @@ -317,7 +317,7 @@ flexible messaging model and an intuitive client API. 1.3 0.4 9.1.0 - 1.1.0 + 1.0.6 1.6.1 6.4.0 3.33.0 diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 3f73a43698ea4..aa7e4998e5c3e 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -252,11 +252,6 @@ awaitility test - - - org.roaringbitmap - RoaringBitmap - diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java index b5ad89d1695d4..72215d7296cc3 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java @@ -29,7 +29,6 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.lang.mutable.MutableInt; -import org.roaringbitmap.RoaringBitSet; /** * A Concurrent set comprising zero or more ranges of type {@link LongPair}. This can be alternative of @@ -45,7 +44,7 @@ public class ConcurrentOpenLongPairRangeSet> implements LongPairRangeSet { protected final NavigableMap rangeBitSetMap = new ConcurrentSkipListMap<>(); - private final boolean threadSafe; + private boolean threadSafe = true; private final int bitSetSize; private final LongPairConsumer consumer; @@ -96,7 +95,9 @@ public void addOpenClosed(long lowerKey, long lowerValueOpen, long upperKey, lon // (2) set 0th-index to upper-index in upperRange.getKey() if (isValid(upperKey, upperValue)) { BitSet rangeBitSet = rangeBitSetMap.computeIfAbsent(upperKey, (key) -> createNewBitSet()); - rangeBitSet.set(0, (int) upperValue + 1); + if (rangeBitSet != null) { + rangeBitSet.set(0, (int) upperValue + 1); + } } // No-op if values are not valid eg: if lower == LongPair.earliest or upper == LongPair.latest then nothing // to set @@ -413,6 +414,7 @@ private int getSafeEntry(long value) { } private BitSet createNewBitSet() { - return this.threadSafe ? new ConcurrentRoaringBitSet() : new RoaringBitSet(); + return this.threadSafe ? new ConcurrentBitSet(bitSetSize) : new BitSet(bitSetSize); } -} \ No newline at end of file + +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java deleted file mode 100644 index 814e58400993b..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java +++ /dev/null @@ -1,439 +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.collections; - -import java.util.BitSet; -import java.util.concurrent.locks.StampedLock; -import java.util.stream.IntStream; -import org.roaringbitmap.RoaringBitSet; - -public class ConcurrentRoaringBitSet extends RoaringBitSet { - private final StampedLock rwLock = new StampedLock(); - - public ConcurrentRoaringBitSet() { - super(); - } - - @Override - public boolean get(int bitIndex) { - long stamp = rwLock.tryOptimisticRead(); - boolean isSet = super.get(bitIndex); - if (!rwLock.validate(stamp)) { - stamp = rwLock.readLock(); - try { - isSet = super.get(bitIndex); - } finally { - rwLock.unlockRead(stamp); - } - } - return isSet; - } - - @Override - public void set(int bitIndex) { - long stamp = rwLock.writeLock(); - try { - super.set(bitIndex); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void clear(int bitIndex) { - long stamp = rwLock.writeLock(); - try { - super.clear(bitIndex); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void set(int fromIndex, int toIndex) { - long stamp = rwLock.writeLock(); - try { - super.set(fromIndex, toIndex); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void clear(int fromIndex, int toIndex) { - long stamp = rwLock.writeLock(); - try { - super.clear(fromIndex, toIndex); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void clear() { - long stamp = rwLock.writeLock(); - try { - super.clear(); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public int nextSetBit(int fromIndex) { - long stamp = rwLock.tryOptimisticRead(); - int nextSetBit = super.nextSetBit(fromIndex); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - nextSetBit = super.nextSetBit(fromIndex); - } finally { - rwLock.unlockRead(stamp); - } - } - return nextSetBit; - } - - @Override - public int nextClearBit(int fromIndex) { - long stamp = rwLock.tryOptimisticRead(); - int nextClearBit = super.nextClearBit(fromIndex); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - nextClearBit = super.nextClearBit(fromIndex); - } finally { - rwLock.unlockRead(stamp); - } - } - return nextClearBit; - } - - @Override - public int previousSetBit(int fromIndex) { - long stamp = rwLock.tryOptimisticRead(); - int previousSetBit = super.previousSetBit(fromIndex); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - previousSetBit = super.previousSetBit(fromIndex); - } finally { - rwLock.unlockRead(stamp); - } - } - return previousSetBit; - } - - @Override - public int previousClearBit(int fromIndex) { - long stamp = rwLock.tryOptimisticRead(); - int previousClearBit = super.previousClearBit(fromIndex); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - previousClearBit = super.previousClearBit(fromIndex); - } finally { - rwLock.unlockRead(stamp); - } - } - return previousClearBit; - } - - @Override - public int length() { - long stamp = rwLock.tryOptimisticRead(); - int length = super.length(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - length = super.length(); - } finally { - rwLock.unlockRead(stamp); - } - } - return length; - } - - @Override - public boolean isEmpty() { - long stamp = rwLock.tryOptimisticRead(); - boolean isEmpty = super.isEmpty(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - isEmpty = super.isEmpty(); - } finally { - rwLock.unlockRead(stamp); - } - } - return isEmpty; - } - - @Override - public int cardinality() { - long stamp = rwLock.tryOptimisticRead(); - int cardinality = super.cardinality(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - cardinality = super.cardinality(); - } finally { - rwLock.unlockRead(stamp); - } - } - return cardinality; - } - - @Override - public int size() { - long stamp = rwLock.tryOptimisticRead(); - int size = super.size(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - size = super.size(); - } finally { - rwLock.unlockRead(stamp); - } - } - return size; - } - - @Override - public byte[] toByteArray() { - long stamp = rwLock.tryOptimisticRead(); - byte[] byteArray = super.toByteArray(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - byteArray = super.toByteArray(); - } finally { - rwLock.unlockRead(stamp); - } - } - return byteArray; - } - - @Override - public long[] toLongArray() { - long stamp = rwLock.tryOptimisticRead(); - long[] longArray = super.toLongArray(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - longArray = super.toLongArray(); - } finally { - rwLock.unlockRead(stamp); - } - } - return longArray; - } - - @Override - public void flip(int bitIndex) { - long stamp = rwLock.writeLock(); - try { - super.flip(bitIndex); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void flip(int fromIndex, int toIndex) { - long stamp = rwLock.writeLock(); - try { - super.flip(fromIndex, toIndex); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void set(int bitIndex, boolean value) { - long stamp = rwLock.writeLock(); - try { - super.set(bitIndex, value); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void set(int fromIndex, int toIndex, boolean value) { - long stamp = rwLock.writeLock(); - try { - super.set(fromIndex, toIndex, value); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public BitSet get(int fromIndex, int toIndex) { - long stamp = rwLock.tryOptimisticRead(); - BitSet bitSet = super.get(fromIndex, toIndex); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - bitSet = super.get(fromIndex, toIndex); - } finally { - rwLock.unlockRead(stamp); - } - } - return bitSet; - } - - @Override - public boolean intersects(BitSet set) { - long stamp = rwLock.writeLock(); - try { - return super.intersects(set); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void and(BitSet set) { - long stamp = rwLock.writeLock(); - try { - super.and(set); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void or(BitSet set) { - long stamp = rwLock.writeLock(); - try { - super.or(set); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void xor(BitSet set) { - long stamp = rwLock.writeLock(); - try { - super.xor(set); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void andNot(BitSet set) { - long stamp = rwLock.writeLock(); - try { - super.andNot(set); - } finally { - rwLock.unlockWrite(stamp); - } - } - - /** - * Returns the clone of the internal wrapped {@code BitSet}. - * This won't be a clone of the {@code ConcurrentBitSet} object. - * - * @return a clone of the internal wrapped {@code BitSet} - */ - @Override - public Object clone() { - long stamp = rwLock.tryOptimisticRead(); - RoaringBitSet clone = (RoaringBitSet) super.clone(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - clone = (RoaringBitSet) super.clone(); - } finally { - rwLock.unlockRead(stamp); - } - } - return clone; - } - - @Override - public String toString() { - long stamp = rwLock.tryOptimisticRead(); - String str = super.toString(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - str = super.toString(); - } finally { - rwLock.unlockRead(stamp); - } - } - return str; - } - - /** - * This operation is not supported on {@code ConcurrentBitSet}. - */ - @Override - public IntStream stream() { - throw new UnsupportedOperationException("stream is not supported"); - } - - public boolean equals(final Object o) { - long stamp = rwLock.tryOptimisticRead(); - boolean isEqual = super.equals(o); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - isEqual = super.equals(o); - } finally { - rwLock.unlockRead(stamp); - } - } - return isEqual; - } - - public int hashCode() { - long stamp = rwLock.tryOptimisticRead(); - int hashCode = super.hashCode(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - hashCode = super.hashCode(); - } finally { - rwLock.unlockRead(stamp); - } - } - return hashCode; - } -} From 20eb3c61ec2fdfb38b6d2830f14aa04ccf5383de Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 26 Jun 2024 10:10:54 +0800 Subject: [PATCH 343/580] [fix][doc] Fix the doc for the message redelivery backoff (#22855) ## Motivation The document states that `ackTimeoutRedeliveryBackoff` cannot be used with `consumer.negativeAcknowledge(MessageId messageId)`. However, this is confusing. The `ackTimeoutRedeliveryBackoff` should not relate to the nack. ## Modification - Fix the doc for the `ackTimeoutRedeliveryBackoff` - Improve the doc for `negativeAckRedeliveryBackoff` and `ackTimeoutRedeliveryBackoff` --- .../pulsar/client/api/ConsumerBuilder.java | 73 ++++++++++++++----- 1 file changed, 54 insertions(+), 19 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java index 6f3c3be972735..c7919fa473fd5 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java @@ -791,31 +791,66 @@ public interface ConsumerBuilder extends Cloneable { ConsumerBuilder messagePayloadProcessor(MessagePayloadProcessor payloadProcessor); /** - * negativeAckRedeliveryBackoff doesn't work with `consumer.negativeAcknowledge(MessageId messageId)` - * because we are unable to get the redelivery count from the message ID. + * negativeAckRedeliveryBackoff sets the redelivery backoff policy for messages that are negatively acknowledged + * using + * `consumer.negativeAcknowledge(Message message)` but not with `consumer.negativeAcknowledge(MessageId + * messageId)`. + * This setting allows specifying a backoff policy for messages that are negatively acknowledged, + * enabling more flexible control over the delay before such messages are redelivered. + * + *

This configuration accepts a {@link RedeliveryBackoff} object that defines the backoff policy. + * The policy can be either a fixed delay or an exponential backoff. An exponential backoff policy + * is beneficial in scenarios where increasing the delay between consecutive redeliveries can help + * mitigate issues like temporary resource constraints or processing bottlenecks. + * + *

Note: This backoff policy does not apply when using `consumer.negativeAcknowledge(MessageId messageId)` + * because the redelivery count cannot be determined from just the message ID. It is recommended to use + * `consumer.negativeAcknowledge(Message message)` if you want to leverage the redelivery backoff policy. + * + *

Example usage: + *

{@code
+     * client.newConsumer()
+     *       .negativeAckRedeliveryBackoff(ExponentialRedeliveryBackoff.builder()
+     *           .minDelayMs(1000)   // Set minimum delay to 1 second
+     *           .maxDelayMs(60000)  // Set maximum delay to 60 seconds
+     *           .build())
+     *       .subscribe();
+     * }
* - *

Example: - *

-     * client.newConsumer().negativeAckRedeliveryBackoff(ExponentialRedeliveryBackoff.builder()
-     *              .minNackTimeMs(1000)
-     *              .maxNackTimeMs(60 * 1000)
-     *              .build()).subscribe();
-     * 
+ * @param negativeAckRedeliveryBackoff the backoff policy to use for negatively acknowledged messages + * @return the consumer builder instance */ ConsumerBuilder negativeAckRedeliveryBackoff(RedeliveryBackoff negativeAckRedeliveryBackoff); + /** - * redeliveryBackoff doesn't work with `consumer.negativeAcknowledge(MessageId messageId)` - * because we are unable to get the redelivery count from the message ID. + * Sets the redelivery backoff policy for messages that are redelivered due to acknowledgement timeout. + * This setting allows you to specify a backoff policy for messages that are not acknowledged within + * the specified ack timeout. By using a backoff policy, you can control the delay before a message + * is redelivered, potentially improving consumer performance by avoiding immediate redelivery of + * messages that might still be processing. * - *

Example: - *

-     * client.newConsumer().ackTimeout(10, TimeUnit.SECOND)
-     *              .ackTimeoutRedeliveryBackoff(ExponentialRedeliveryBackoff.builder()
-     *              .minNackTimeMs(1000)
-     *              .maxNackTimeMs(60 * 1000)
-     *              .build()).subscribe();
-     * 
+ *

This method accepts a {@link RedeliveryBackoff} object that defines the backoff policy to be used. + * You can use either a fixed backoff policy or an exponential backoff policy. The exponential backoff + * policy is particularly useful for scenarios where it may be beneficial to progressively increase the + * delay between redeliveries, reducing the load on the consumer and giving more time to process messages. + * + *

Example usage: + *

{@code
+     * client.newConsumer()
+     *       .ackTimeout(10, TimeUnit.SECONDS)
+     *       .ackTimeoutRedeliveryBackoff(ExponentialRedeliveryBackoff.builder()
+     *           .minDelayMs(1000)   // Set minimum delay to 1 second
+     *           .maxDelayMs(60000)  // Set maximum delay to 60 seconds
+     *           .build())
+     *       .subscribe();
+     * }
+ * + *

Note: This configuration is effective only if the ack timeout is triggered. It does not apply to + * messages negatively acknowledged using the negative acknowledgment API. + * + * @param ackTimeoutRedeliveryBackoff the backoff policy to use for messages that exceed their ack timeout + * @return the consumer builder instance */ ConsumerBuilder ackTimeoutRedeliveryBackoff(RedeliveryBackoff ackTimeoutRedeliveryBackoff); From 2c6fcc7eb8343583ffb48dec937334a5f05afbae Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 26 Jun 2024 11:35:26 +0800 Subject: [PATCH 344/580] [fix][broker] Partitioned shadow topic not work properly (#22797) --- .../service/persistent/PersistentTopic.java | 9 ++++-- .../service/persistent/ShadowTopicTest.java | 31 +++++++++++++++++++ 2 files changed, 38 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 6e3d49fbe9ff1..3d620d3189863 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -2198,8 +2198,13 @@ protected CompletableFuture addShadowReplicationCluster(String shadowTopic .thenAccept(replicationClient -> { Replicator replicator = shadowReplicators.computeIfAbsent(shadowTopic, r -> { try { - return new ShadowReplicator(shadowTopic, PersistentTopic.this, cursor, brokerService, - (PulsarClientImpl) replicationClient); + TopicName sourceTopicName = TopicName.get(getName()); + String shadowPartitionTopic = shadowTopic; + if (sourceTopicName.isPartitioned()) { + shadowPartitionTopic += "-partition-" + sourceTopicName.getPartitionIndex(); + } + return new ShadowReplicator(shadowPartitionTopic, PersistentTopic.this, cursor, + brokerService, (PulsarClientImpl) replicationClient); } catch (PulsarServerException e) { log.error("[{}] ShadowReplicator startup failed {}", topic, shadowTopic, e); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicTest.java index 1dbfe109a93d4..5334339ae5b62 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicTest.java @@ -21,6 +21,8 @@ import com.google.common.collect.Lists; import java.nio.charset.StandardCharsets; +import java.util.HashSet; +import java.util.Set; import java.util.concurrent.TimeUnit; import lombok.AllArgsConstructor; import lombok.Cleanup; @@ -113,6 +115,35 @@ public void testPartitionedShadowTopicSetup() throws Exception { Assert.assertEquals(brokerShadowTopic.getShadowSourceTopic().get().toString(), sourceTopicPartition); } + @Test + public void testPartitionedShadowTopicProduceAndConsume() throws Exception { + String sourceTopic = newShadowSourceTopicName(); + String shadowTopic = sourceTopic + "-shadow"; + admin.topics().createPartitionedTopic(sourceTopic, 3); + admin.topics().createShadowTopic(shadowTopic, sourceTopic); + + admin.topics().setShadowTopics(sourceTopic, Lists.newArrayList(shadowTopic)); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(sourceTopic).create(); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(shadowTopic).subscriptionName("test") + .subscribe(); + + for (int i = 0; i < 10; i++) { + producer.send("msg-" + i); + } + + Set set = new HashSet<>(); + for (int i = 0; i < 10; i++) { + Message msg = consumer.receive(); + set.add(msg.getValue()); + } + for (int i = 0; i < 10; i++) { + Assert.assertTrue(set.contains("msg-" + i)); + } + } + @Test public void testShadowTopicNotWritable() throws Exception { String sourceTopic = newShadowSourceTopicName(); From 243ad5a22f7cc533e5a7382c01cb875dcdaee6bb Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 25 Jun 2024 22:42:57 -0700 Subject: [PATCH 345/580] [feat][broker] PIP-264: Add OpenTelemetry metadata store stats (#22952) --- .../apache/pulsar/broker/PulsarService.java | 15 ++- .../stats/PulsarBrokerOpenTelemetry.java | 2 + .../AntiAffinityNamespaceGroupTest.java | 6 +- .../OpenTelemetryMetadataStoreStatsTest.java | 94 +++++++++++++++++++ .../AbstractTestPulsarService.java | 7 +- .../metadata/api/MetadataStoreConfig.java | 7 ++ .../metadata/impl/AbstractMetadataStore.java | 5 +- .../impl/LocalMemoryMetadataStore.java | 2 +- .../metadata/impl/RocksdbMetadataStore.java | 2 +- .../AbstractBatchedMetadataStore.java | 4 +- .../metadata/impl/oxia/OxiaMetadataStore.java | 7 +- .../impl/stats/BatchMetadataStoreStats.java | 23 ++++- .../impl/stats/MetadataStoreStats.java | 20 +++- .../impl/MetadataStoreFactoryImplTest.java | 3 +- 14 files changed, 175 insertions(+), 22 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryMetadataStoreStatsTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 8cf1376642b88..617afc6e5d154 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -31,6 +31,7 @@ import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; import io.netty.util.concurrent.DefaultThreadFactory; +import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import java.io.IOException; import java.lang.reflect.Constructor; @@ -382,7 +383,8 @@ public PulsarService(ServiceConfiguration config, DEFAULT_MONOTONIC_CLOCK_GRANULARITY_MILLIS), System::nanoTime); } - public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchronizer synchronizer) + public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchronizer synchronizer, + OpenTelemetry openTelemetry) throws MetadataStoreException { return MetadataStoreFactory.create(config.getConfigurationMetadataStoreUrl(), MetadataStoreConfig.builder() @@ -395,6 +397,7 @@ public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchro .batchingMaxSizeKb(config.getMetadataStoreBatchingMaxSizeKb()) .metadataStoreName(MetadataStoreConfig.CONFIGURATION_METADATA_STORE) .synchronizer(synchronizer) + .openTelemetry(openTelemetry) .build()); } @@ -845,7 +848,8 @@ public void start() throws PulsarServerException { localMetadataSynchronizer = StringUtils.isNotBlank(config.getMetadataSyncEventTopic()) ? new PulsarMetadataEventSynchronizer(this, config.getMetadataSyncEventTopic()) : null; - localMetadataStore = createLocalMetadataStore(localMetadataSynchronizer); + localMetadataStore = createLocalMetadataStore(localMetadataSynchronizer, + openTelemetry.getOpenTelemetryService().getOpenTelemetry()); localMetadataStore.registerSessionListener(this::handleMetadataSessionEvent); coordinationService = new CoordinationServiceImpl(localMetadataStore); @@ -854,7 +858,8 @@ public void start() throws PulsarServerException { configMetadataSynchronizer = StringUtils.isNotBlank(config.getConfigurationMetadataSyncEventTopic()) ? new PulsarMetadataEventSynchronizer(this, config.getConfigurationMetadataSyncEventTopic()) : null; - configurationMetadataStore = createConfigurationMetadataStore(configMetadataSynchronizer); + configurationMetadataStore = createConfigurationMetadataStore(configMetadataSynchronizer, + openTelemetry.getOpenTelemetryService().getOpenTelemetry()); shouldShutdownConfigurationMetadataStore = true; } else { configurationMetadataStore = localMetadataStore; @@ -1209,7 +1214,8 @@ private void handleDeleteCluster(Notification notification) { } } - public MetadataStoreExtended createLocalMetadataStore(PulsarMetadataEventSynchronizer synchronizer) + public MetadataStoreExtended createLocalMetadataStore(PulsarMetadataEventSynchronizer synchronizer, + OpenTelemetry openTelemetry) throws MetadataStoreException, PulsarServerException { return MetadataStoreExtended.create(config.getMetadataStoreUrl(), MetadataStoreConfig.builder() @@ -1222,6 +1228,7 @@ public MetadataStoreExtended createLocalMetadataStore(PulsarMetadataEventSynchro .batchingMaxSizeKb(config.getMetadataStoreBatchingMaxSizeKb()) .synchronizer(synchronizer) .metadataStoreName(MetadataStoreConfig.METADATA_STORE) + .openTelemetry(openTelemetry) .build()); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java index 01ca65d2cc537..c1bcfadaf97f1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -31,6 +31,8 @@ public class PulsarBrokerOpenTelemetry implements Closeable { public static final String SERVICE_NAME = "pulsar-broker"; + + @Getter private final OpenTelemetryService openTelemetryService; @Getter diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java index 5fbda961c0e3d..fc2fec96294ef 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java @@ -26,6 +26,7 @@ import com.google.common.collect.Range; import com.google.common.collect.Sets; import com.google.common.hash.Hashing; +import io.opentelemetry.api.OpenTelemetry; import java.lang.reflect.Field; import java.util.Collections; import java.util.HashMap; @@ -136,8 +137,9 @@ protected void cleanup() throws Exception { protected void beforePulsarStart(PulsarService pulsar) throws Exception { if (resources == null) { - MetadataStoreExtended localStore = pulsar.createLocalMetadataStore(null); - MetadataStoreExtended configStore = (MetadataStoreExtended) pulsar.createConfigurationMetadataStore(null); + MetadataStoreExtended localStore = pulsar.createLocalMetadataStore(null, OpenTelemetry.noop()); + MetadataStoreExtended configStore = + (MetadataStoreExtended) pulsar.createConfigurationMetadataStore(null, OpenTelemetry.noop()); resources = new PulsarResources(localStore, configStore); } this.createNamespaceIfNotExists(resources, NamespaceName.SYSTEM_NAMESPACE.getTenant(), diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryMetadataStoreStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryMetadataStoreStatsTest.java new file mode 100644 index 0000000000000..15689fca5d7c0 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryMetadataStoreStatsTest.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.broker.stats; + +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; +import static org.assertj.core.api.Assertions.assertThat; +import io.opentelemetry.api.common.Attributes; +import java.util.concurrent.ExecutorService; +import lombok.Cleanup; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.testcontext.NonClosingProxyHandler; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.impl.stats.BatchMetadataStoreStats; +import org.apache.pulsar.metadata.impl.stats.MetadataStoreStats; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class OpenTelemetryMetadataStoreStatsTest extends BrokerTestBase { + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.baseSetup(); + setupDefaultTenantAndNamespace(); + + // In testing conditions, the metadata store gets initialized before Pulsar does, so the OpenTelemetry SDK is + // not yet initialized. Work around this issue by recreating the stats object once we have access to the SDK. + var localMetadataStore = (MetadataStore) NonClosingProxyHandler.getDelegate(pulsar.getLocalMetadataStore()); + var currentStats = (MetadataStoreStats) FieldUtils.readField(localMetadataStore, "metadataStoreStats", true); + var localMetadataStoreName = (String) FieldUtils.readField(currentStats, "metadataStoreName", true); + + currentStats.close(); + var newStats = new MetadataStoreStats( + localMetadataStoreName, pulsar.getOpenTelemetry().getOpenTelemetryService().getOpenTelemetry()); + FieldUtils.writeField(localMetadataStore, "metadataStoreStats", newStats, true); + + var currentBatchedStats = (BatchMetadataStoreStats) FieldUtils.readField(localMetadataStore, "batchMetadataStoreStats", true); + currentBatchedStats.close(); + var currentExecutor = (ExecutorService) FieldUtils.readField(currentBatchedStats, "executor", true); + var newBatchedStats = new BatchMetadataStoreStats( + localMetadataStoreName, currentExecutor, pulsar.getOpenTelemetry().getOpenTelemetryService().getOpenTelemetry()); + FieldUtils.writeField(localMetadataStore, "batchMetadataStoreStats", newBatchedStats, true); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder pulsarTestContextBuilder) { + super.customizeMainPulsarTestContextBuilder(pulsarTestContextBuilder); + pulsarTestContextBuilder.enableOpenTelemetry(true); + } + + @Test + public void testMetadataStoreStats() throws Exception { + var topicName = BrokerTestUtil.newUniqueName("persistent://public/default/test-metadata-store-stats"); + + @Cleanup + var producer = pulsarClient.newProducer().topic(topicName).create(); + + producer.newMessage().value("test".getBytes()).send(); + + var attributes = Attributes.of(MetadataStoreStats.METADATA_STORE_NAME, "metadata-store"); + + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, MetadataStoreStats.METADATA_STORE_PUT_BYTES_COUNTER_METRIC_NAME, + attributes, value -> assertThat(value).isPositive()); + assertMetricLongSumValue(metrics, BatchMetadataStoreStats.EXECUTOR_QUEUE_SIZE_METRIC_NAME, attributes, + value -> assertThat(value).isPositive()); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/AbstractTestPulsarService.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/AbstractTestPulsarService.java index c459098f6850c..c67714484f442 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/AbstractTestPulsarService.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/AbstractTestPulsarService.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.testcontext; +import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import java.io.IOException; import java.util.Optional; @@ -68,7 +69,8 @@ public AbstractTestPulsarService(SpyConfig spyConfig, ServiceConfiguration confi } @Override - public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchronizer synchronizer) + public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchronizer synchronizer, + OpenTelemetry openTelemetry) throws MetadataStoreException { if (synchronizer != null) { synchronizer.registerSyncListener( @@ -78,7 +80,8 @@ public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchro } @Override - public MetadataStoreExtended createLocalMetadataStore(PulsarMetadataEventSynchronizer synchronizer) + public MetadataStoreExtended createLocalMetadataStore(PulsarMetadataEventSynchronizer synchronizer, + OpenTelemetry openTelemetry) throws MetadataStoreException, PulsarServerException { if (synchronizer != null) { synchronizer.registerSyncListener( diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java index 5ddfe33c3912a..be29f843eea18 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreConfig.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.metadata.api; +import io.opentelemetry.api.OpenTelemetry; import lombok.Builder; import lombok.Getter; import lombok.ToString; @@ -92,4 +93,10 @@ public class MetadataStoreConfig { * separate clusters. */ private MetadataEventSynchronizer synchronizer; + + /** + * OpenTelemetry instance to monitor metadata store operations. + */ + @Builder.Default + private OpenTelemetry openTelemetry = OpenTelemetry.noop(); } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java index 7315e6a04a230..f35f197463222 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java @@ -26,6 +26,7 @@ import com.github.benmanes.caffeine.cache.LoadingCache; import com.google.common.annotations.VisibleForTesting; import io.netty.util.concurrent.DefaultThreadFactory; +import io.opentelemetry.api.OpenTelemetry; import java.time.Instant; import java.util.Collections; import java.util.EnumSet; @@ -88,7 +89,7 @@ public abstract class AbstractMetadataStore implements MetadataStoreExtended, Co protected abstract CompletableFuture existsFromStore(String path); - protected AbstractMetadataStore(String metadataStoreName) { + protected AbstractMetadataStore(String metadataStoreName, OpenTelemetry openTelemetry) { this.executor = new ScheduledThreadPoolExecutor(1, new DefaultThreadFactory( StringUtils.isNotBlank(metadataStoreName) ? metadataStoreName : getClass().getSimpleName())); @@ -137,7 +138,7 @@ public CompletableFuture asyncReload(String key, Boolean oldValue, }); this.metadataStoreName = metadataStoreName; - this.metadataStoreStats = new MetadataStoreStats(metadataStoreName); + this.metadataStoreStats = new MetadataStoreStats(metadataStoreName, openTelemetry); } @Override diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java index 3909a89cf5eb2..e95f1947740c8 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/LocalMemoryMetadataStore.java @@ -78,7 +78,7 @@ private static class Value { public LocalMemoryMetadataStore(String metadataURL, MetadataStoreConfig metadataStoreConfig) throws MetadataStoreException { - super(metadataStoreConfig.getMetadataStoreName()); + super(metadataStoreConfig.getMetadataStoreName(), metadataStoreConfig.getOpenTelemetry()); String name = metadataURL.substring(MEMORY_SCHEME_IDENTIFIER.length()); // Local means a private data set // update synchronizer and register sync listener diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java index 06f7b26053693..20e3c4c2b27b2 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java @@ -209,7 +209,7 @@ static long toLong(byte[] bytes) { */ private RocksdbMetadataStore(String metadataURL, MetadataStoreConfig metadataStoreConfig) throws MetadataStoreException { - super(metadataStoreConfig.getMetadataStoreName()); + super(metadataStoreConfig.getMetadataStoreName(), metadataStoreConfig.getOpenTelemetry()); this.metadataUrl = metadataURL; try { RocksDB.loadLibrary(); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java index 4fa1c6aca0fee..4275920d7f954 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/batching/AbstractBatchedMetadataStore.java @@ -56,7 +56,7 @@ public abstract class AbstractBatchedMetadataStore extends AbstractMetadataStore private final BatchMetadataStoreStats batchMetadataStoreStats; protected AbstractBatchedMetadataStore(MetadataStoreConfig conf) { - super(conf.getMetadataStoreName()); + super(conf.getMetadataStoreName(), conf.getOpenTelemetry()); this.enabled = conf.isBatchingEnabled(); this.maxDelayMillis = conf.getBatchingMaxDelayMillis(); @@ -77,7 +77,7 @@ protected AbstractBatchedMetadataStore(MetadataStoreConfig conf) { // update synchronizer and register sync listener updateMetadataEventSynchronizer(conf.getSynchronizer()); this.batchMetadataStoreStats = - new BatchMetadataStoreStats(metadataStoreName, executor); + new BatchMetadataStoreStats(metadataStoreName, executor, conf.getOpenTelemetry()); } @Override diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java index 154a0ec0c4fd8..e9da7ec7c1ab5 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.metadata.impl.oxia; +import io.opentelemetry.api.OpenTelemetry; import io.streamnative.oxia.client.api.AsyncOxiaClient; import io.streamnative.oxia.client.api.DeleteOption; import io.streamnative.oxia.client.api.Notification; @@ -58,7 +59,7 @@ public class OxiaMetadataStore extends AbstractMetadataStore { private Optional synchronizer; public OxiaMetadataStore(AsyncOxiaClient oxia, String identity) { - super("oxia-metadata"); + super("oxia-metadata", OpenTelemetry.noop()); this.client = oxia; this.identity = identity; this.synchronizer = Optional.empty(); @@ -68,10 +69,10 @@ public OxiaMetadataStore(AsyncOxiaClient oxia, String identity) { public OxiaMetadataStore( @NonNull String serviceAddress, @NonNull String namespace, - @NonNull MetadataStoreConfig metadataStoreConfig, + MetadataStoreConfig metadataStoreConfig, boolean enableSessionWatcher) throws Exception { - super("oxia-metadata"); + super("oxia-metadata", Objects.requireNonNull(metadataStoreConfig).getOpenTelemetry()); var linger = metadataStoreConfig.getBatchingMaxDelayMillis(); if (!metadataStoreConfig.isBatchingEnabled()) { diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/BatchMetadataStoreStats.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/BatchMetadataStoreStats.java index f87155b9259be..9549a8df8f9f1 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/BatchMetadataStoreStats.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/BatchMetadataStoreStats.java @@ -18,6 +18,9 @@ */ package org.apache.pulsar.metadata.impl.stats; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.ObservableLongUpDownCounter; import io.prometheus.client.Gauge; import io.prometheus.client.Histogram; import java.util.concurrent.ExecutorService; @@ -58,7 +61,10 @@ public final class BatchMetadataStoreStats implements AutoCloseable { private final Histogram.Child batchExecuteTimeChild; private final Histogram.Child opsPerBatchChild; - public BatchMetadataStoreStats(String metadataStoreName, ExecutorService executor) { + public static final String EXECUTOR_QUEUE_SIZE_METRIC_NAME = "pulsar.broker.metadata.store.executor.queue.size"; + private final ObservableLongUpDownCounter batchMetadataStoreSizeCounter; + + public BatchMetadataStoreStats(String metadataStoreName, ExecutorService executor, OpenTelemetry openTelemetry) { if (executor instanceof ThreadPoolExecutor tx) { this.executor = tx; } else { @@ -69,8 +75,7 @@ public BatchMetadataStoreStats(String metadataStoreName, ExecutorService executo EXECUTOR_QUEUE_SIZE.setChild(new Gauge.Child() { @Override public double get() { - return BatchMetadataStoreStats.this.executor == null ? 0 : - BatchMetadataStoreStats.this.executor.getQueue().size(); + return getQueueSize(); } }, metadataStoreName); @@ -78,6 +83,17 @@ public double get() { this.batchExecuteTimeChild = BATCH_EXECUTE_TIME.labels(metadataStoreName); this.opsPerBatchChild = OPS_PER_BATCH.labels(metadataStoreName); + var meter = openTelemetry.getMeter("org.apache.pulsar"); + var attributes = Attributes.of(MetadataStoreStats.METADATA_STORE_NAME, metadataStoreName); + this.batchMetadataStoreSizeCounter = meter + .upDownCounterBuilder(EXECUTOR_QUEUE_SIZE_METRIC_NAME) + .setDescription("The number of batch operations in the metadata store executor queue") + .setUnit("{operation}") + .buildWithCallback(measurement -> measurement.record(getQueueSize(), attributes)); + } + + private int getQueueSize() { + return executor == null ? 0 : executor.getQueue().size(); } public void recordOpWaiting(long millis) { @@ -99,6 +115,7 @@ public void close() throws Exception { OPS_WAITING.remove(this.metadataStoreName); BATCH_EXECUTE_TIME.remove(this.metadataStoreName); OPS_PER_BATCH.remove(metadataStoreName); + batchMetadataStoreSizeCounter.close(); } } } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java index 45024a68383bd..5f0383f9520a7 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/stats/MetadataStoreStats.java @@ -18,6 +18,10 @@ */ package org.apache.pulsar.metadata.impl.stats; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.LongCounter; import io.prometheus.client.Counter; import io.prometheus.client.Histogram; import java.util.concurrent.atomic.AtomicBoolean; @@ -48,6 +52,12 @@ public final class MetadataStoreStats implements AutoCloseable { .labelNames(METADATA_STORE_LABEL_NAME) .register(); + public static final AttributeKey METADATA_STORE_NAME = AttributeKey.stringKey("pulsar.metadata.store.name"); + public static final String METADATA_STORE_PUT_BYTES_COUNTER_METRIC_NAME = + "pulsar.broker.metadata.store.outgoing.size"; + private final Attributes attributes; + private final LongCounter putBytesCounter; + private final Histogram.Child getOpsSucceedChild; private final Histogram.Child delOpsSucceedChild; private final Histogram.Child putOpsSucceedChild; @@ -58,7 +68,7 @@ public final class MetadataStoreStats implements AutoCloseable { private final String metadataStoreName; private final AtomicBoolean closed = new AtomicBoolean(false); - public MetadataStoreStats(String metadataStoreName) { + public MetadataStoreStats(String metadataStoreName, OpenTelemetry openTelemetry) { this.metadataStoreName = metadataStoreName; this.getOpsSucceedChild = OPS_LATENCY.labels(metadataStoreName, OPS_TYPE_GET, STATUS_SUCCESS); @@ -68,6 +78,13 @@ public MetadataStoreStats(String metadataStoreName) { this.delOpsFailedChild = OPS_LATENCY.labels(metadataStoreName, OPS_TYPE_DEL, STATUS_FAIL); this.putOpsFailedChild = OPS_LATENCY.labels(metadataStoreName, OPS_TYPE_PUT, STATUS_FAIL); this.putBytesChild = PUT_BYTES.labels(metadataStoreName); + + attributes = Attributes.of(METADATA_STORE_NAME, metadataStoreName); + putBytesCounter = openTelemetry.getMeter("org.apache.pulsar") + .counterBuilder(METADATA_STORE_PUT_BYTES_COUNTER_METRIC_NAME) + .setDescription("The total amount of data written to the metadata store") + .setUnit("{By}") + .build(); } public void recordGetOpsSucceeded(long millis) { @@ -81,6 +98,7 @@ public void recordDelOpsSucceeded(long millis) { public void recordPutOpsSucceeded(long millis, int bytes) { this.putOpsSucceedChild.observe(millis); this.putBytesChild.inc(bytes); + this.putBytesCounter.add(bytes, attributes); } public void recordGetOpsFailed(long millis) { diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java index c0159be4303bc..6ede02b67136e 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/impl/MetadataStoreFactoryImplTest.java @@ -20,6 +20,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; +import io.opentelemetry.api.OpenTelemetry; import lombok.Cleanup; import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataStore; @@ -91,7 +92,7 @@ public MetadataStore create(String metadataURL, MetadataStoreConfig metadataStor public static class MyMetadataStore extends AbstractMetadataStore { protected MyMetadataStore() { - super("custom"); + super("custom", OpenTelemetry.noop()); } @Override From fe726db49c32eb539b6eb0b83c8735e48f742a35 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 25 Jun 2024 22:49:07 -0700 Subject: [PATCH 346/580] [feat][broker] PIP-264: Add managed ledger cache metrics (#22898) --- managed-ledger/pom.xml | 12 ++ .../mledger/ManagedLedgerFactoryMXBean.java | 25 +++ .../OpenTelemetryManagedLedgerCacheStats.java | 172 ++++++++++++++++++ .../impl/ManagedLedgerFactoryImpl.java | 20 +- .../impl/ManagedLedgerFactoryMBeanImpl.java | 25 +++ .../cache/PooledByteBufAllocatorStats.java | 68 +++++++ .../broker/ManagedLedgerClientFactory.java | 8 +- .../apache/pulsar/broker/PulsarService.java | 2 +- .../stats/PulsarBrokerOpenTelemetry.java | 3 +- .../metrics/ManagedLedgerCacheMetrics.java | 43 +---- .../broker/storage/ManagedLedgerStorage.java | 9 +- ...nTelemetryManagedLedgerCacheStatsTest.java | 127 +++++++++++++ .../broker/testcontext/PulsarTestContext.java | 6 +- .../client/impl/SequenceIdWithErrorTest.java | 3 +- .../pulsar/opentelemetry/Constants.java | 28 +++ .../OpenTelemetryAttributes.java | 44 +++++ 16 files changed, 544 insertions(+), 51 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/OpenTelemetryManagedLedgerCacheStats.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PooledByteBufAllocatorStats.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryManagedLedgerCacheStatsTest.java create mode 100644 pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/Constants.java diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index d8b31220d51be..60a4edab95b77 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -71,6 +71,12 @@ ${project.version} + + ${project.groupId} + pulsar-opentelemetry + ${project.version} + + com.google.guava guava @@ -120,6 +126,12 @@ test + + io.opentelemetry + opentelemetry-sdk-testing + test + + org.slf4j slf4j-api diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryMXBean.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryMXBean.java index 35c26c5dfdb89..43e8196daa9ae 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryMXBean.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryMXBean.java @@ -47,26 +47,51 @@ public interface ManagedLedgerFactoryMXBean { */ double getCacheHitsRate(); + /** + * Cumulative number of cache hits. + */ + long getCacheHitsTotal(); + /** * Get the number of cache misses per second. */ double getCacheMissesRate(); + /** + * Cumulative number of cache misses. + */ + long getCacheMissesTotal(); + /** * Get the amount of data is retrieved from the cache in byte/s. */ double getCacheHitsThroughput(); + /** + * Cumulative amount of data retrieved from the cache in bytes. + */ + long getCacheHitsBytesTotal(); + /** * Get the amount of data is retrieved from the bookkeeper in byte/s. */ double getCacheMissesThroughput(); + /** + * Cumulative amount of data retrieved from the bookkeeper in bytes. + */ + long getCacheMissesBytesTotal(); + /** * Get the number of cache evictions during the last minute. */ long getNumberOfCacheEvictions(); + /** + * Cumulative number of cache evictions. + */ + long getNumberOfCacheEvictionsTotal(); + /** * Cumulative number of entries inserted into the cache. */ diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/OpenTelemetryManagedLedgerCacheStats.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/OpenTelemetryManagedLedgerCacheStats.java new file mode 100644 index 0000000000000..13e7ed6ac6799 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/OpenTelemetryManagedLedgerCacheStats.java @@ -0,0 +1,172 @@ +/* + * 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.bookkeeper.mledger; + +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.metrics.BatchCallback; +import io.opentelemetry.api.metrics.ObservableLongMeasurement; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; +import org.apache.bookkeeper.mledger.impl.cache.PooledByteBufAllocatorStats; +import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl; +import org.apache.pulsar.opentelemetry.Constants; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.CacheEntryStatus; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.CacheOperationStatus; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.PoolArenaType; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.PoolChunkAllocationType; + +public class OpenTelemetryManagedLedgerCacheStats implements AutoCloseable { + + // Replaces pulsar_ml_count + public static final String MANAGED_LEDGER_COUNTER = "pulsar.broker.managed_ledger.count"; + private final ObservableLongMeasurement managedLedgerCounter; + + // Replaces pulsar_ml_cache_evictions + public static final String CACHE_EVICTION_OPERATION_COUNTER = "pulsar.broker.managed_ledger.cache.eviction.count"; + private final ObservableLongMeasurement cacheEvictionOperationCounter; + + // Replaces 'pulsar_ml_cache_entries', + // 'pulsar_ml_cache_inserted_entries_total', + // 'pulsar_ml_cache_evicted_entries_total' + public static final String CACHE_ENTRY_COUNTER = "pulsar.broker.managed_ledger.cache.entry.count"; + private final ObservableLongMeasurement cacheEntryCounter; + + // Replaces pulsar_ml_cache_used_size + public static final String CACHE_SIZE_COUNTER = "pulsar.broker.managed_ledger.cache.entry.size"; + private final ObservableLongMeasurement cacheSizeCounter; + + // Replaces pulsar_ml_cache_hits_rate, pulsar_ml_cache_misses_rate + public static final String CACHE_OPERATION_COUNTER = "pulsar.broker.managed_ledger.cache.operation.count"; + private final ObservableLongMeasurement cacheOperationCounter; + + // Replaces pulsar_ml_cache_hits_throughput, pulsar_ml_cache_misses_throughput + public static final String CACHE_OPERATION_BYTES_COUNTER = "pulsar.broker.managed_ledger.cache.operation.size"; + private final ObservableLongMeasurement cacheOperationBytesCounter; + + // Replaces 'pulsar_ml_cache_pool_active_allocations', + // 'pulsar_ml_cache_pool_active_allocations_huge', + // 'pulsar_ml_cache_pool_active_allocations_normal', + // 'pulsar_ml_cache_pool_active_allocations_small' + public static final String CACHE_POOL_ACTIVE_ALLOCATION_COUNTER = + "pulsar.broker.managed_ledger.cache.pool.allocation.active.count"; + private final ObservableLongMeasurement cachePoolActiveAllocationCounter; + + // Replaces ['pulsar_ml_cache_pool_allocated', 'pulsar_ml_cache_pool_used'] + public static final String CACHE_POOL_ACTIVE_ALLOCATION_SIZE_COUNTER = + "pulsar.broker.managed_ledger.cache.pool.allocation.size"; + private final ObservableLongMeasurement cachePoolActiveAllocationSizeCounter; + + private final BatchCallback batchCallback; + + public OpenTelemetryManagedLedgerCacheStats(OpenTelemetry openTelemetry, ManagedLedgerFactoryImpl factory) { + var meter = openTelemetry.getMeter(Constants.BROKER_INSTRUMENTATION_SCOPE_NAME); + + managedLedgerCounter = meter + .upDownCounterBuilder(MANAGED_LEDGER_COUNTER) + .setUnit("{managed_ledger}") + .setDescription("The total number of managed ledgers.") + .buildObserver(); + + cacheEvictionOperationCounter = meter + .counterBuilder(CACHE_EVICTION_OPERATION_COUNTER) + .setUnit("{eviction}") + .setDescription("The total number of cache eviction operations.") + .buildObserver(); + + cacheEntryCounter = meter + .upDownCounterBuilder(CACHE_ENTRY_COUNTER) + .setUnit("{entry}") + .setDescription("The number of entries in the entry cache.") + .buildObserver(); + + cacheSizeCounter = meter + .upDownCounterBuilder(CACHE_SIZE_COUNTER) + .setUnit("{By}") + .setDescription("The byte amount of entries stored in the entry cache.") + .buildObserver(); + + cacheOperationCounter = meter + .counterBuilder(CACHE_OPERATION_COUNTER) + .setUnit("{entry}") + .setDescription("The number of cache operations.") + .buildObserver(); + + cacheOperationBytesCounter = meter + .counterBuilder(CACHE_OPERATION_BYTES_COUNTER) + .setUnit("{By}") + .setDescription("The byte amount of data retrieved from cache operations.") + .buildObserver(); + + cachePoolActiveAllocationCounter = meter + .upDownCounterBuilder(CACHE_POOL_ACTIVE_ALLOCATION_COUNTER) + .setUnit("{allocation}") + .setDescription("The number of currently active allocations in the direct arena.") + .buildObserver(); + + cachePoolActiveAllocationSizeCounter = meter + .upDownCounterBuilder(CACHE_POOL_ACTIVE_ALLOCATION_SIZE_COUNTER) + .setUnit("{By}") + .setDescription("The memory allocated in the direct arena.") + .buildObserver(); + + + batchCallback = meter.batchCallback(() -> recordMetrics(factory), + managedLedgerCounter, + cacheEvictionOperationCounter, + cacheEntryCounter, + cacheSizeCounter, + cacheOperationCounter, + cacheOperationBytesCounter, + cachePoolActiveAllocationCounter, + cachePoolActiveAllocationSizeCounter); + } + + @Override + public void close() { + batchCallback.close(); + } + + private void recordMetrics(ManagedLedgerFactoryImpl factory) { + var stats = factory.getCacheStats(); + + managedLedgerCounter.record(stats.getNumberOfManagedLedgers()); + cacheEvictionOperationCounter.record(stats.getNumberOfCacheEvictionsTotal()); + + var entriesOut = stats.getCacheEvictedEntriesCount(); + var entriesIn = stats.getCacheInsertedEntriesCount(); + var entriesActive = entriesIn - entriesOut; + cacheEntryCounter.record(entriesActive, CacheEntryStatus.ACTIVE.attributes); + cacheEntryCounter.record(entriesIn, CacheEntryStatus.INSERTED.attributes); + cacheEntryCounter.record(entriesOut, CacheEntryStatus.EVICTED.attributes); + cacheSizeCounter.record(stats.getCacheUsedSize()); + + cacheOperationCounter.record(stats.getCacheHitsTotal(), CacheOperationStatus.HIT.attributes); + cacheOperationBytesCounter.record(stats.getCacheHitsBytesTotal(), CacheOperationStatus.HIT.attributes); + cacheOperationCounter.record(stats.getCacheMissesTotal(), CacheOperationStatus.MISS.attributes); + cacheOperationBytesCounter.record(stats.getCacheMissesBytesTotal(), CacheOperationStatus.MISS.attributes); + + var allocatorStats = new PooledByteBufAllocatorStats(RangeEntryCacheImpl.ALLOCATOR); + cachePoolActiveAllocationCounter.record(allocatorStats.activeAllocationsSmall, PoolArenaType.SMALL.attributes); + cachePoolActiveAllocationCounter.record(allocatorStats.activeAllocationsNormal, + PoolArenaType.NORMAL.attributes); + cachePoolActiveAllocationCounter.record(allocatorStats.activeAllocationsHuge, PoolArenaType.HUGE.attributes); + cachePoolActiveAllocationSizeCounter.record(allocatorStats.totalAllocated, + PoolChunkAllocationType.ALLOCATED.attributes); + cachePoolActiveAllocationSizeCounter.record(allocatorStats.totalUsed, PoolChunkAllocationType.USED.attributes); + } +} \ No newline at end of file diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 0b0f66d14c98c..fc291b801c896 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -24,6 +24,7 @@ import com.google.common.base.Predicates; import com.google.common.collect.Maps; import io.netty.util.concurrent.DefaultThreadFactory; +import io.opentelemetry.api.OpenTelemetry; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -67,6 +68,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerInfo.MessageRangeInfo; import org.apache.bookkeeper.mledger.ManagedLedgerInfo.PositionInfo; import org.apache.bookkeeper.mledger.MetadataCompressionConfig; +import org.apache.bookkeeper.mledger.OpenTelemetryManagedLedgerCacheStats; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.ManagedLedgerInitializeLedgerCallback; @@ -118,6 +120,8 @@ public class ManagedLedgerFactoryImpl implements ManagedLedgerFactory { private volatile long cacheEvictionTimeThresholdNanos; private final MetadataStore metadataStore; + private final OpenTelemetryManagedLedgerCacheStats openTelemetryCacheStats; + //indicate whether shutdown() is called. private volatile boolean closed; @@ -149,7 +153,7 @@ public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, ClientConfi ManagedLedgerFactoryConfig config) throws Exception { this(metadataStore, new DefaultBkFactory(bkClientConfiguration), - true /* isBookkeeperManaged */, config, NullStatsLogger.INSTANCE); + true /* isBookkeeperManaged */, config, NullStatsLogger.INSTANCE, OpenTelemetry.noop()); } public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, BookKeeper bookKeeper) @@ -168,21 +172,24 @@ public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, ManagedLedgerFactoryConfig config) throws Exception { this(metadataStore, bookKeeperGroupFactory, false /* isBookkeeperManaged */, - config, NullStatsLogger.INSTANCE); + config, NullStatsLogger.INSTANCE, OpenTelemetry.noop()); } public ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, BookkeeperFactoryForCustomEnsemblePlacementPolicy bookKeeperGroupFactory, - ManagedLedgerFactoryConfig config, StatsLogger statsLogger) + ManagedLedgerFactoryConfig config, StatsLogger statsLogger, + OpenTelemetry openTelemetry) throws Exception { this(metadataStore, bookKeeperGroupFactory, false /* isBookkeeperManaged */, - config, statsLogger); + config, statsLogger, openTelemetry); } private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, BookkeeperFactoryForCustomEnsemblePlacementPolicy bookKeeperGroupFactory, boolean isBookkeeperManaged, - ManagedLedgerFactoryConfig config, StatsLogger statsLogger) throws Exception { + ManagedLedgerFactoryConfig config, + StatsLogger statsLogger, + OpenTelemetry openTelemetry) throws Exception { MetadataCompressionConfig compressionConfigForManagedLedgerInfo = config.getCompressionConfigForManagedLedgerInfo(); MetadataCompressionConfig compressionConfigForManagedCursorInfo = @@ -220,6 +227,8 @@ private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, closed = false; metadataStore.registerSessionListener(this::handleMetadataStoreNotification); + + openTelemetryCacheStats = new OpenTelemetryManagedLedgerCacheStats(openTelemetry, this); } static class DefaultBkFactory implements BookkeeperFactoryForCustomEnsemblePlacementPolicy { @@ -611,6 +620,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { })); }).thenAcceptAsync(__ -> { //wait for tasks in scheduledExecutor executed. + openTelemetryCacheStats.close(); scheduledExecutor.shutdownNow(); entryCacheManager.clear(); }); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryMBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryMBeanImpl.java index cf3d7142d617e..a3038a0e7ff76 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryMBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryMBeanImpl.java @@ -99,26 +99,51 @@ public double getCacheHitsRate() { return cacheHits.getRate(); } + @Override + public long getCacheHitsTotal() { + return cacheHits.getTotalCount(); + } + @Override public double getCacheMissesRate() { return cacheMisses.getRate(); } + @Override + public long getCacheMissesTotal() { + return cacheMisses.getTotalCount(); + } + @Override public double getCacheHitsThroughput() { return cacheHits.getValueRate(); } + @Override + public long getCacheHitsBytesTotal() { + return cacheHits.getTotalValue(); + } + @Override public double getCacheMissesThroughput() { return cacheMisses.getValueRate(); } + @Override + public long getCacheMissesBytesTotal() { + return cacheMisses.getTotalValue(); + } + @Override public long getNumberOfCacheEvictions() { return cacheEvictions.getCount(); } + @Override + public long getNumberOfCacheEvictionsTotal() { + return cacheEvictions.getTotalCount(); + } + public long getCacheInsertedEntriesCount() { return insertedEntryCount.sum(); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PooledByteBufAllocatorStats.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PooledByteBufAllocatorStats.java new file mode 100644 index 0000000000000..4f6a18cb5d934 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PooledByteBufAllocatorStats.java @@ -0,0 +1,68 @@ +/* + * 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.bookkeeper.mledger.impl.cache; + +import io.netty.buffer.PooledByteBufAllocator; +import lombok.Value; + +@Value +public class PooledByteBufAllocatorStats { + + public long activeAllocations; + public long activeAllocationsSmall; + public long activeAllocationsNormal; + public long activeAllocationsHuge; + + public long totalAllocated; + public long totalUsed; + + public PooledByteBufAllocatorStats(PooledByteBufAllocator allocator) { + long activeAllocations = 0; + long activeAllocationsSmall = 0; + long activeAllocationsNormal = 0; + long activeAllocationsHuge = 0; + long totalAllocated = 0; + long totalUsed = 0; + + for (var arena : allocator.metric().directArenas()) { + activeAllocations += arena.numActiveAllocations(); + activeAllocationsSmall += arena.numActiveSmallAllocations(); + activeAllocationsNormal += arena.numActiveNormalAllocations(); + activeAllocationsHuge += arena.numActiveHugeAllocations(); + + for (var list : arena.chunkLists()) { + for (var chunk : list) { + int size = chunk.chunkSize(); + int used = size - chunk.freeBytes(); + + totalAllocated += size; + totalUsed += used; + } + } + } + + this.activeAllocations = activeAllocations; + this.activeAllocationsSmall = activeAllocationsSmall; + this.activeAllocationsNormal = activeAllocationsNormal; + this.activeAllocationsHuge = activeAllocationsHuge; + + this.totalAllocated = totalAllocated; + this.totalUsed = totalUsed; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java index 6ed95f167a15a..9bbc2857863ff 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java @@ -22,6 +22,7 @@ import com.github.benmanes.caffeine.cache.Caffeine; import com.google.common.annotations.VisibleForTesting; import io.netty.channel.EventLoopGroup; +import io.opentelemetry.api.OpenTelemetry; import java.io.IOException; import java.util.Map; import java.util.Optional; @@ -54,9 +55,11 @@ public class ManagedLedgerClientFactory implements ManagedLedgerStorage { bkEnsemblePolicyToBkClientMap = Caffeine.newBuilder().buildAsync(); private StatsProvider statsProvider = new NullStatsProvider(); + @Override public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadataStore, BookKeeperClientFactory bookkeeperProvider, - EventLoopGroup eventLoopGroup) throws Exception { + EventLoopGroup eventLoopGroup, + OpenTelemetry openTelemetry) throws Exception { ManagedLedgerFactoryConfig managedLedgerFactoryConfig = new ManagedLedgerFactoryConfig(); managedLedgerFactoryConfig.setMaxCacheSize(conf.getManagedLedgerCacheSizeMB() * 1024L * 1024L); managedLedgerFactoryConfig.setCacheEvictionWatermark(conf.getManagedLedgerCacheEvictionWatermark()); @@ -109,7 +112,8 @@ public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadata try { this.managedLedgerFactory = - new ManagedLedgerFactoryImpl(metadataStore, bkFactory, managedLedgerFactoryConfig, statsLogger); + new ManagedLedgerFactoryImpl(metadataStore, bkFactory, managedLedgerFactoryConfig, statsLogger, + openTelemetry); } catch (Exception e) { statsProvider.stop(); defaultBkClient.close(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 617afc6e5d154..4fa773dace918 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -1069,7 +1069,7 @@ protected OrderedExecutor newOrderedExecutor() { protected ManagedLedgerStorage newManagedLedgerClientFactory() throws Exception { return ManagedLedgerStorage.create( config, localMetadataStore, - bkClientFactory, ioEventLoopGroup + bkClientFactory, ioEventLoopGroup, openTelemetry.getOpenTelemetryService().getOpenTelemetry() ); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java index c1bcfadaf97f1..178da8b84983f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -26,6 +26,7 @@ import lombok.Getter; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.opentelemetry.Constants; import org.apache.pulsar.opentelemetry.OpenTelemetryService; public class PulsarBrokerOpenTelemetry implements Closeable { @@ -46,7 +47,7 @@ public PulsarBrokerOpenTelemetry(ServiceConfiguration config, .serviceVersion(PulsarVersion.getVersion()) .builderCustomizer(builderCustomizer) .build(); - meter = openTelemetryService.getOpenTelemetry().getMeter("org.apache.pulsar.broker"); + meter = openTelemetryService.getOpenTelemetry().getMeter(Constants.BROKER_INSTRUMENTATION_SCOPE_NAME); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerCacheMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerCacheMetrics.java index 890a37aa2d877..9eb4beb72fbf2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerCacheMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerCacheMetrics.java @@ -18,13 +18,10 @@ */ package org.apache.pulsar.broker.stats.metrics; -import io.netty.buffer.PoolArenaMetric; -import io.netty.buffer.PoolChunkListMetric; -import io.netty.buffer.PoolChunkMetric; -import io.netty.buffer.PooledByteBufAllocator; import java.util.ArrayList; import java.util.List; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryMXBean; +import org.apache.bookkeeper.mledger.impl.cache.PooledByteBufAllocatorStats; import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.common.stats.Metrics; @@ -57,37 +54,13 @@ public synchronized List generate() { m.put("brk_ml_cache_hits_throughput", mlCacheStats.getCacheHitsThroughput()); m.put("brk_ml_cache_misses_throughput", mlCacheStats.getCacheMissesThroughput()); - PooledByteBufAllocator allocator = RangeEntryCacheImpl.ALLOCATOR; - long activeAllocations = 0; - long activeAllocationsSmall = 0; - long activeAllocationsNormal = 0; - long activeAllocationsHuge = 0; - long totalAllocated = 0; - long totalUsed = 0; - - for (PoolArenaMetric arena : allocator.metric().directArenas()) { - activeAllocations += arena.numActiveAllocations(); - activeAllocationsSmall += arena.numActiveSmallAllocations(); - activeAllocationsNormal += arena.numActiveNormalAllocations(); - activeAllocationsHuge += arena.numActiveHugeAllocations(); - - for (PoolChunkListMetric list : arena.chunkLists()) { - for (PoolChunkMetric chunk : list) { - int size = chunk.chunkSize(); - int used = size - chunk.freeBytes(); - - totalAllocated += size; - totalUsed += used; - } - } - } - - m.put("brk_ml_cache_pool_allocated", totalAllocated); - m.put("brk_ml_cache_pool_used", totalUsed); - m.put("brk_ml_cache_pool_active_allocations", activeAllocations); - m.put("brk_ml_cache_pool_active_allocations_small", activeAllocationsSmall); - m.put("brk_ml_cache_pool_active_allocations_normal", activeAllocationsNormal); - m.put("brk_ml_cache_pool_active_allocations_huge", activeAllocationsHuge); + var allocatorStats = new PooledByteBufAllocatorStats(RangeEntryCacheImpl.ALLOCATOR); + m.put("brk_ml_cache_pool_allocated", allocatorStats.totalAllocated); + m.put("brk_ml_cache_pool_used", allocatorStats.totalUsed); + m.put("brk_ml_cache_pool_active_allocations", allocatorStats.activeAllocations); + m.put("brk_ml_cache_pool_active_allocations_small", allocatorStats.activeAllocationsSmall); + m.put("brk_ml_cache_pool_active_allocations_normal", allocatorStats.activeAllocationsNormal); + m.put("brk_ml_cache_pool_active_allocations_huge", allocatorStats.activeAllocationsHuge); metrics.clear(); metrics.add(m); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java index 0b5a102eed1e0..944d2badf75f2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.storage; import io.netty.channel.EventLoopGroup; +import io.opentelemetry.api.OpenTelemetry; import java.io.IOException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.mledger.ManagedLedgerFactory; @@ -47,7 +48,8 @@ public interface ManagedLedgerStorage extends AutoCloseable { void initialize(ServiceConfiguration conf, MetadataStoreExtended metadataStore, BookKeeperClientFactory bookkeeperProvider, - EventLoopGroup eventLoopGroup) throws Exception; + EventLoopGroup eventLoopGroup, + OpenTelemetry openTelemetry) throws Exception; /** * Return the factory to create {@link ManagedLedgerFactory}. @@ -87,11 +89,12 @@ void initialize(ServiceConfiguration conf, static ManagedLedgerStorage create(ServiceConfiguration conf, MetadataStoreExtended metadataStore, BookKeeperClientFactory bkProvider, - EventLoopGroup eventLoopGroup) throws Exception { + EventLoopGroup eventLoopGroup, + OpenTelemetry openTelemetry) throws Exception { ManagedLedgerStorage storage = Reflections.createInstance(conf.getManagedLedgerStorageClassName(), ManagedLedgerStorage.class, Thread.currentThread().getContextClassLoader()); - storage.initialize(conf, metadataStore, bkProvider, eventLoopGroup); + storage.initialize(conf, metadataStore, bkProvider, eventLoopGroup, openTelemetry); return storage; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryManagedLedgerCacheStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryManagedLedgerCacheStatsTest.java new file mode 100644 index 0000000000000..c3a4a2e054ef3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryManagedLedgerCacheStatsTest.java @@ -0,0 +1,127 @@ +/* + * 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.stats; + +import static org.apache.bookkeeper.mledger.OpenTelemetryManagedLedgerCacheStats.CACHE_ENTRY_COUNTER; +import static org.apache.bookkeeper.mledger.OpenTelemetryManagedLedgerCacheStats.CACHE_EVICTION_OPERATION_COUNTER; +import static org.apache.bookkeeper.mledger.OpenTelemetryManagedLedgerCacheStats.CACHE_OPERATION_BYTES_COUNTER; +import static org.apache.bookkeeper.mledger.OpenTelemetryManagedLedgerCacheStats.CACHE_OPERATION_COUNTER; +import static org.apache.bookkeeper.mledger.OpenTelemetryManagedLedgerCacheStats.CACHE_POOL_ACTIVE_ALLOCATION_COUNTER; +import static org.apache.bookkeeper.mledger.OpenTelemetryManagedLedgerCacheStats.CACHE_POOL_ACTIVE_ALLOCATION_SIZE_COUNTER; +import static org.apache.bookkeeper.mledger.OpenTelemetryManagedLedgerCacheStats.CACHE_SIZE_COUNTER; +import static org.apache.bookkeeper.mledger.OpenTelemetryManagedLedgerCacheStats.MANAGED_LEDGER_COUNTER; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; +import static org.assertj.core.api.Assertions.assertThat; +import io.opentelemetry.api.common.Attributes; +import lombok.Cleanup; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.CacheEntryStatus; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.CacheOperationStatus; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.PoolArenaType; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.PoolChunkAllocationType; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class OpenTelemetryManagedLedgerCacheStatsTest extends BrokerTestBase { + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.baseSetup(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder builder) { + super.customizeMainPulsarTestContextBuilder(builder); + builder.enableOpenTelemetry(true); + } + + @Test + public void testManagedLedgerCacheStats() throws Exception { + var topicName = BrokerTestUtil.newUniqueName("persistent://prop/ns-abc/testManagedLedgerCacheStats"); + + @Cleanup + var producer = pulsarClient.newProducer().topic(topicName).create(); + + @Cleanup + var consumer1 = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName(BrokerTestUtil.newUniqueName("sub")) + .subscribe(); + + @Cleanup + var consumer2 = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName(BrokerTestUtil.newUniqueName("sub")) + .subscribe(); + + producer.send("test".getBytes()); + consumer1.receive(); + + Awaitility.await().untilAsserted(() -> { + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, CACHE_ENTRY_COUNTER, CacheEntryStatus.ACTIVE.attributes, + value -> assertThat(value).isNotNegative()); + assertMetricLongSumValue(metrics, CACHE_ENTRY_COUNTER, CacheEntryStatus.INSERTED.attributes, + value -> assertThat(value).isPositive()); + assertMetricLongSumValue(metrics, CACHE_ENTRY_COUNTER, CacheEntryStatus.EVICTED.attributes, + value -> assertThat(value).isPositive()); + assertMetricLongSumValue(metrics, CACHE_SIZE_COUNTER, Attributes.empty(), + value -> assertThat(value).isNotNegative()); + }); + + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + + assertMetricLongSumValue(metrics, MANAGED_LEDGER_COUNTER, Attributes.empty(), 2); + assertMetricLongSumValue(metrics, CACHE_EVICTION_OPERATION_COUNTER, Attributes.empty(), 0); + + assertMetricLongSumValue(metrics, CACHE_OPERATION_COUNTER, CacheOperationStatus.HIT.attributes, + value -> assertThat(value).isPositive()); + assertMetricLongSumValue(metrics, CACHE_OPERATION_BYTES_COUNTER, CacheOperationStatus.HIT.attributes, + value -> assertThat(value).isPositive()); + assertMetricLongSumValue(metrics, CACHE_OPERATION_COUNTER, CacheOperationStatus.MISS.attributes, + value -> assertThat(value).isNotNegative()); + assertMetricLongSumValue(metrics, CACHE_OPERATION_BYTES_COUNTER, CacheOperationStatus.MISS.attributes, + value -> assertThat(value).isNotNegative()); + + assertMetricLongSumValue(metrics, CACHE_POOL_ACTIVE_ALLOCATION_COUNTER, PoolArenaType.SMALL.attributes, + value -> assertThat(value).isNotNegative()); + assertMetricLongSumValue(metrics, CACHE_POOL_ACTIVE_ALLOCATION_COUNTER, PoolArenaType.NORMAL.attributes, + value -> assertThat(value).isNotNegative()); + assertMetricLongSumValue(metrics, CACHE_POOL_ACTIVE_ALLOCATION_COUNTER, PoolArenaType.HUGE.attributes, + value -> assertThat(value).isNotNegative()); + assertMetricLongSumValue(metrics, CACHE_POOL_ACTIVE_ALLOCATION_SIZE_COUNTER, + PoolChunkAllocationType.ALLOCATED.attributes, value -> assertThat(value).isNotNegative()); + assertMetricLongSumValue(metrics, CACHE_POOL_ACTIVE_ALLOCATION_SIZE_COUNTER, + PoolChunkAllocationType.USED.attributes, value -> assertThat(value).isNotNegative()); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java index 09cd4f7cb1a93..3d79a17a90f50 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java @@ -21,6 +21,7 @@ import com.google.common.util.concurrent.MoreExecutors; import io.netty.channel.EventLoopGroup; +import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; import java.io.IOException; @@ -843,9 +844,8 @@ private static ManagedLedgerStorage createManagedLedgerClientFactory(BookKeeper @Override public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadataStore, - BookKeeperClientFactory bookkeeperProvider, EventLoopGroup eventLoopGroup) - throws Exception { - + BookKeeperClientFactory bookkeeperProvider, EventLoopGroup eventLoopGroup, + OpenTelemetry openTelemetry) { } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SequenceIdWithErrorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SequenceIdWithErrorTest.java index 7d330bb82addd..1395424b14123 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SequenceIdWithErrorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SequenceIdWithErrorTest.java @@ -21,6 +21,7 @@ import static org.testng.Assert.assertEquals; import io.netty.channel.EventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; +import io.opentelemetry.api.OpenTelemetry; import java.util.Collections; import lombok.Cleanup; import org.apache.bookkeeper.mledger.ManagedLedger; @@ -60,7 +61,7 @@ public void testCheckSequenceId() throws Exception { EventLoopGroup eventLoopGroup = new NioEventLoopGroup(1); ManagedLedgerClientFactory clientFactory = new ManagedLedgerClientFactory(); clientFactory.initialize(pulsar.getConfiguration(), pulsar.getLocalMetadataStore(), - pulsar.getBookKeeperClientFactory(), eventLoopGroup); + pulsar.getBookKeeperClientFactory(), eventLoopGroup, OpenTelemetry.noop()); ManagedLedgerFactory mlFactory = clientFactory.getManagedLedgerFactory(); ManagedLedger ml = mlFactory.open(TopicName.get(topicName).getPersistenceNamingEncoding()); ml.close(); diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/Constants.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/Constants.java new file mode 100644 index 0000000000000..6d61cafb5a01a --- /dev/null +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/Constants.java @@ -0,0 +1,28 @@ +/* + * 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.opentelemetry; + +/** + * Common OpenTelemetry constants to be used by Pulsar components. + */ +public interface Constants { + + String BROKER_INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.broker"; + +} diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index 31e527f02869e..b530b50ee59dc 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -164,4 +164,48 @@ enum ConnectionCreateStatus { FAILURE; public final Attributes attributes = Attributes.of(PULSAR_CONNECTION_CREATE_STATUS, name().toLowerCase()); } + + /** + * The type of the pool arena. + */ + AttributeKey ML_POOL_ARENA_TYPE = AttributeKey.stringKey("pulsar.managed_ledger.pool.arena.type"); + enum PoolArenaType { + SMALL, + NORMAL, + HUGE; + public final Attributes attributes = Attributes.of(ML_POOL_ARENA_TYPE, name().toLowerCase()); + } + + /** + * The type of the pool chunk allocation. + */ + AttributeKey ML_POOL_CHUNK_ALLOCATION_TYPE = + AttributeKey.stringKey("pulsar.managed_ledger.pool.chunk.allocation.type"); + enum PoolChunkAllocationType { + ALLOCATED, + USED; + public final Attributes attributes = Attributes.of(ML_POOL_CHUNK_ALLOCATION_TYPE, name().toLowerCase()); + } + + /** + * The status of the cache entry. + */ + AttributeKey ML_CACHE_ENTRY_STATUS = AttributeKey.stringKey("pulsar.managed_ledger.cache.entry.status"); + enum CacheEntryStatus { + ACTIVE, + EVICTED, + INSERTED; + public final Attributes attributes = Attributes.of(ML_CACHE_ENTRY_STATUS, name().toLowerCase()); + } + + /** + * The result of the cache operation. + */ + AttributeKey ML_CACHE_OPERATION_STATUS = + AttributeKey.stringKey("pulsar.managed_ledger.cache.operation.status"); + enum CacheOperationStatus { + HIT, + MISS; + public final Attributes attributes = Attributes.of(ML_CACHE_OPERATION_STATUS, name().toLowerCase()); + } } From 53df683b0f78f5f7c12f87e6fbb4d73637ca5bd5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 26 Jun 2024 17:54:19 +0300 Subject: [PATCH 347/580] [fix][broker] Ensure that PulsarService is ready for serving incoming requests (#22977) --- .../apache/pulsar/broker/PulsarService.java | 16 ++- .../extensions/ExtensibleLoadManagerImpl.java | 131 +++++++++++------- .../broker/namespace/NamespaceService.java | 4 +- .../service/PulsarChannelInitializer.java | 7 +- .../pulsar/broker/service/ServerCnx.java | 4 + .../apache/pulsar/broker/web/WebService.java | 50 +++++++ 6 files changed, 156 insertions(+), 56 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 4fa773dace918..0d8bc571c5750 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -291,6 +291,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private final ExecutorProvider transactionExecutorProvider; private final DefaultMonotonicSnapshotClock monotonicSnapshotClock; private String brokerId; + private final CompletableFuture readyForIncomingRequestsFuture = new CompletableFuture<>(); public enum State { Init, Started, Closing, Closed @@ -999,6 +1000,9 @@ public void start() throws PulsarServerException { this.metricsGenerator = new MetricsGenerator(this); + // the broker is ready to accept incoming requests by Pulsar binary protocol and http/https + readyForIncomingRequestsFuture.complete(null); + // Initialize the message protocol handlers. // start the protocol handlers only after the broker is ready, // so that the protocol handlers can access broker service properly. @@ -1047,12 +1051,22 @@ public void start() throws PulsarServerException { state = State.Started; } catch (Exception e) { LOG.error("Failed to start Pulsar service: {}", e.getMessage(), e); - throw new PulsarServerException(e); + PulsarServerException startException = new PulsarServerException(e); + readyForIncomingRequestsFuture.completeExceptionally(startException); + throw startException; } finally { mutex.unlock(); } } + public void runWhenReadyForIncomingRequests(Runnable runnable) { + readyForIncomingRequestsFuture.thenRun(runnable); + } + + public void waitUntilReadyForIncomingRequests() throws ExecutionException, InterruptedException { + readyForIncomingRequestsFuture.get(); + } + protected BrokerInterceptor newBrokerInterceptor() throws IOException { return BrokerInterceptors.load(config); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 92dcf8001ada5..4a7ba90aad919 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -36,7 +36,6 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -167,10 +166,10 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS private TopBundleLoadDataReporter topBundleLoadDataReporter; - private ScheduledFuture brokerLoadDataReportTask; - private ScheduledFuture topBundlesLoadDataReportTask; + private volatile ScheduledFuture brokerLoadDataReportTask; + private volatile ScheduledFuture topBundlesLoadDataReportTask; - private ScheduledFuture monitorTask; + private volatile ScheduledFuture monitorTask; private SplitScheduler splitScheduler; private UnloadManager unloadManager; @@ -199,7 +198,7 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS private final ConcurrentHashMap>> lookupRequests = new ConcurrentHashMap<>(); - private final CountDownLatch initWaiter = new CountDownLatch(1); + private final CompletableFuture initWaiter = new CompletableFuture<>(); /** * Get all the bundles that are owned by this broker. @@ -376,12 +375,14 @@ public void start() throws PulsarServerException { pulsar.getCoordinationService(), pulsar.getBrokerId(), pulsar.getSafeWebServiceAddress(), ELECTION_ROOT, state -> { - pulsar.getLoadManagerExecutor().execute(() -> { - if (state == LeaderElectionState.Leading) { - playLeader(); - } else { - playFollower(); - } + pulsar.runWhenReadyForIncomingRequests(() -> { + pulsar.getLoadManagerExecutor().execute(() -> { + if (state == LeaderElectionState.Leading) { + playLeader(); + } else { + playFollower(); + } + }); }); }); this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar); @@ -391,7 +392,13 @@ public void start() throws PulsarServerException { this.serviceUnitStateChannel.listen(unloadManager); this.serviceUnitStateChannel.listen(splitManager); this.leaderElectionService.start(); - this.serviceUnitStateChannel.start(); + pulsar.runWhenReadyForIncomingRequests(() -> { + try { + this.serviceUnitStateChannel.start(); + } catch (Exception e) { + failStarting(e); + } + }); this.antiAffinityGroupPolicyHelper = new AntiAffinityGroupPolicyHelper(pulsar, serviceUnitStateChannel); antiAffinityGroupPolicyHelper.listenFailureDomainUpdate(); @@ -423,54 +430,72 @@ public void start() throws PulsarServerException { new TopBundleLoadDataReporter(pulsar, brokerRegistry.getBrokerId(), topBundlesLoadDataStore); this.serviceUnitStateChannel.listen(brokerLoadDataReporter); this.serviceUnitStateChannel.listen(topBundleLoadDataReporter); - var interval = conf.getLoadBalancerReportUpdateMinIntervalMillis(); - this.brokerLoadDataReportTask = this.pulsar.getLoadManagerExecutor() - .scheduleAtFixedRate(() -> { - try { - brokerLoadDataReporter.reportAsync(false); - // TODO: update broker load metrics using getLocalData - } catch (Throwable e) { - log.error("Failed to run the broker load manager executor job.", e); - } - }, - interval, - interval, TimeUnit.MILLISECONDS); - - this.topBundlesLoadDataReportTask = this.pulsar.getLoadManagerExecutor() - .scheduleAtFixedRate(() -> { - try { - // TODO: consider excluding the bundles that are in the process of split. - topBundleLoadDataReporter.reportAsync(false); - } catch (Throwable e) { - log.error("Failed to run the top bundles load manager executor job.", e); - } - }, - interval, - interval, TimeUnit.MILLISECONDS); - - this.monitorTask = this.pulsar.getLoadManagerExecutor() - .scheduleAtFixedRate(() -> { - monitor(); - }, - MONITOR_INTERVAL_IN_MILLIS, - MONITOR_INTERVAL_IN_MILLIS, TimeUnit.MILLISECONDS); this.unloadScheduler = new UnloadScheduler( pulsar, pulsar.getLoadManagerExecutor(), unloadManager, context, serviceUnitStateChannel, unloadCounter, unloadMetrics); this.splitScheduler = new SplitScheduler( pulsar, serviceUnitStateChannel, splitManager, splitCounter, splitMetrics, context); - this.splitScheduler.start(); - this.initWaiter.countDown(); - this.started = true; - log.info("Started load manager."); + + pulsar.runWhenReadyForIncomingRequests(() -> { + try { + var interval = conf.getLoadBalancerReportUpdateMinIntervalMillis(); + + this.brokerLoadDataReportTask = this.pulsar.getLoadManagerExecutor() + .scheduleAtFixedRate(() -> { + try { + brokerLoadDataReporter.reportAsync(false); + // TODO: update broker load metrics using getLocalData + } catch (Throwable e) { + log.error("Failed to run the broker load manager executor job.", e); + } + }, + interval, + interval, TimeUnit.MILLISECONDS); + + this.topBundlesLoadDataReportTask = this.pulsar.getLoadManagerExecutor() + .scheduleAtFixedRate(() -> { + try { + // TODO: consider excluding the bundles that are in the process of split. + topBundleLoadDataReporter.reportAsync(false); + } catch (Throwable e) { + log.error("Failed to run the top bundles load manager executor job.", e); + } + }, + interval, + interval, TimeUnit.MILLISECONDS); + + this.monitorTask = this.pulsar.getLoadManagerExecutor() + .scheduleAtFixedRate(() -> { + monitor(); + }, + MONITOR_INTERVAL_IN_MILLIS, + MONITOR_INTERVAL_IN_MILLIS, TimeUnit.MILLISECONDS); + + this.splitScheduler.start(); + this.initWaiter.complete(null); + this.started = true; + log.info("Started load manager."); + } catch (Exception ex) { + failStarting(ex); + } + }); } catch (Exception ex) { - log.error("Failed to start the extensible load balance and close broker registry {}.", - this.brokerRegistry, ex); - if (this.brokerRegistry != null) { + failStarting(ex); + } + } + + private void failStarting(Exception ex) { + log.error("Failed to start the extensible load balance and close broker registry {}.", + this.brokerRegistry, ex); + if (this.brokerRegistry != null) { + try { brokerRegistry.close(); + } catch (PulsarServerException e) { + // ignore } } + initWaiter.completeExceptionally(ex); } @Override @@ -816,7 +841,7 @@ synchronized void playLeader() { boolean becameFollower = false; while (!Thread.currentThread().isInterrupted()) { try { - initWaiter.await(); + initWaiter.get(); if (!serviceUnitStateChannel.isChannelOwner()) { becameFollower = true; break; @@ -866,7 +891,7 @@ synchronized void playFollower() { boolean becameLeader = false; while (!Thread.currentThread().isInterrupted()) { try { - initWaiter.await(); + initWaiter.get(); if (serviceUnitStateChannel.isChannelOwner()) { becameLeader = true; break; @@ -936,7 +961,7 @@ private List getIgnoredCommandMetrics(String advertisedBrokerAddress) { @VisibleForTesting protected void monitor() { try { - initWaiter.await(); + initWaiter.get(); // Monitor role // Periodically check the role in case ZK watcher fails. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index df6a141ddcf1a..dfd03dfbc6e43 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -1335,7 +1335,9 @@ public void addNamespaceBundleOwnershipListener(NamespaceBundleOwnershipListener bundleOwnershipListeners.add(listener); } } - getOwnedServiceUnits().forEach(bundle -> notifyNamespaceBundleOwnershipListener(bundle, listeners)); + pulsar.runWhenReadyForIncomingRequests(() -> { + getOwnedServiceUnits().forEach(bundle -> notifyNamespaceBundleOwnershipListener(bundle, listeners)); + }); } public void addNamespaceBundleSplitListener(NamespaceBundleSplitListener... listeners) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java index 5308b3c981eb4..e276ea24fed18 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service; import com.google.common.annotations.VisibleForTesting; +import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; import io.netty.handler.codec.LengthFieldBasedFrameDecoder; @@ -104,6 +105,9 @@ public PulsarChannelInitializer(PulsarService pulsar, PulsarChannelOptions opts) @Override protected void initChannel(SocketChannel ch) throws Exception { + // disable auto read explicitly so that requests aren't served until auto read is enabled + // ServerCnx must enable auto read in channelActive after PulsarService is ready to accept incoming requests + ch.config().setAutoRead(false); ch.pipeline().addLast("consolidation", new FlushConsolidationHandler(1024, true)); if (this.enableTls) { if (this.tlsEnabledWithKeyStore) { @@ -128,7 +132,8 @@ protected void initChannel(SocketChannel ch) throws Exception { // ServerCnx ends up reading higher number of messages and broker can not throttle the messages by disabling // auto-read. ch.pipeline().addLast("flowController", new FlowControlHandler()); - ServerCnx cnx = newServerCnx(pulsar, listenerName); + // using "ChannelHandler" type to workaround an IntelliJ bug that shows a false positive error + ChannelHandler cnx = newServerCnx(pulsar, listenerName); ch.pipeline().addLast("handler", cnx); } 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 b184f79494998..4933aee974d08 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 @@ -369,6 +369,10 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { this.commandSender = new PulsarCommandSenderImpl(brokerInterceptor, this); this.service.getPulsarStats().recordConnectionCreate(); cnxsPerThread.get().add(this); + service.getPulsar().runWhenReadyForIncomingRequests(() -> { + // enable auto read after PulsarService is ready to accept incoming requests + ctx.channel().config().setAutoRead(true); + }); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java index bf484d4f41f65..c969f40ad4382 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java @@ -20,12 +20,21 @@ import io.prometheus.client.CollectorRegistry; import io.prometheus.client.jetty.JettyStatisticsCollector; +import java.io.IOException; import java.util.ArrayList; import java.util.EnumSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.ExecutionException; import javax.servlet.DispatcherType; +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.http.HttpServletResponse; import lombok.Getter; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -232,6 +241,7 @@ private static class FilterInitializer { private final FilterHolder authenticationFilterHolder; FilterInitializer(PulsarService pulsarService) { ServiceConfiguration config = pulsarService.getConfiguration(); + if (config.getMaxConcurrentHttpRequests() > 0) { FilterHolder filterHolder = new FilterHolder(QoSFilter.class); filterHolder.setInitParameter("maxRequests", String.valueOf(config.getMaxConcurrentHttpRequests())); @@ -243,6 +253,10 @@ private static class FilterInitializer { new RateLimitingFilter(config.getHttpRequestsMaxPerSecond()))); } + // wait until the PulsarService is ready to serve incoming requests + filterHolders.add( + new FilterHolder(new WaitUntilPulsarServiceIsReadyForIncomingRequestsFilter(pulsarService))); + boolean brokerInterceptorEnabled = pulsarService.getBrokerInterceptor() != null; if (brokerInterceptorEnabled) { ExceptionHandler handler = new ExceptionHandler(); @@ -284,6 +298,42 @@ public void addFilters(ServletContextHandler context, boolean requiresAuthentica } } + // Filter that waits until the PulsarService is ready to serve incoming requests + private static class WaitUntilPulsarServiceIsReadyForIncomingRequestsFilter implements Filter { + private final PulsarService pulsarService; + + public WaitUntilPulsarServiceIsReadyForIncomingRequestsFilter(PulsarService pulsarService) { + this.pulsarService = pulsarService; + } + + @Override + public void init(FilterConfig filterConfig) throws ServletException { + + } + + @Override + public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) + throws IOException, ServletException { + try { + // Wait until the PulsarService is ready to serve incoming requests + pulsarService.waitUntilReadyForIncomingRequests(); + } catch (ExecutionException e) { + ((HttpServletResponse) response).sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE, + "PulsarService failed to start."); + return; + } catch (InterruptedException e) { + ((HttpServletResponse) response).sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE, + "PulsarService is not ready."); + return; + } + chain.doFilter(request, response); + } + + @Override + public void destroy() { + + } + } } public void addServlet(String path, ServletHolder servletHolder, boolean requiresAuthentication, From 4ac9bc42f22f8163f59273a0b4ffc46cf3cffdea Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Thu, 27 Jun 2024 16:57:04 +0800 Subject: [PATCH 348/580] [improve] [pip] PIP-364: Introduce a new load balance algorithm AvgShedder (#22946) --- pip/pip-364.md | 476 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 476 insertions(+) create mode 100644 pip/pip-364.md diff --git a/pip/pip-364.md b/pip/pip-364.md new file mode 100644 index 0000000000000..c589b3b47fc47 --- /dev/null +++ b/pip/pip-364.md @@ -0,0 +1,476 @@ + +# PIP-364: Introduce a new load balance algorithm AvgShedder + +# Background knowledge + +Pulsar has two load balance interfaces: +- `LoadSheddingStrategy` is an unloading strategy that identifies high load brokers and unloads some of the bundles they carry to reduce the load. +- `ModularLoadManagerStrategy` is a placement strategy responsible for assigning bundles to brokers. + +## LoadSheddingStrategy +There are three available algorithms: `ThresholdShedder`, `OverloadShedder`, `UniformLoadShedder`. + +### ThresholdShedder +`ThresholdShedder` uses the following method to calculate the maximum resource utilization rate for each broker, +which includes CPU, direct memory, bandwidth in, and bandwidth out. +``` + public double getMaxResourceUsageWithWeight(final double cpuWeight, + final double directMemoryWeight, final double bandwidthInWeight, + final double bandwidthOutWeight) { + return max(cpu.percentUsage() * cpuWeight, + directMemory.percentUsage() * directMemoryWeight, bandwidthIn.percentUsage() * bandwidthInWeight, + bandwidthOut.percentUsage() * bandwidthOutWeight) / 100; + } +``` + +After calculating the maximum resource utilization rate for each broker, a historical weight algorithm will +also be executed to obtain the final score. +``` +historyUsage = historyUsage == null ? resourceUsage : historyUsage * historyPercentage + (1 - historyPercentage) * resourceUsage; +``` +The historyPercentage is determined by configuring the `loadBalancerHistoryResourcePercentage`. +The default value is 0.9, which means that the last calculated score accounts for 90%, +while the current calculated score only accounts for 10%. + +The introduction of this historical weight algorithm is to avoid bundle switching caused by +short-term abnormal load increase or decrease, but in fact, this algorithm will introduce some +serious problems, which will be explained in detail later. + +Next, calculate the average score of all brokers in the entire cluster: `avgUsage=totalUsage/totalBrokers`. +When the score of any broker exceeds a certain threshold of avgUsage, it is determined that the broker is overloaded. +The threshold is determined by the configuration `loadBalancerBrokerThresholdShedderPercentage`, with a default value of 10. + + +### OverloadShedder +`OverloadShedder` use the same method `getMaxResourceUsageWithWeight` to calculate the maximum resource utilization rate for each broker. +The difference is that `OverloadShedder` will not use the historical weight algorithm to calculate the final score, +the final score is the current maximum resource utilization rate of the broker. + +After obtaining the load score for each broker, compare it with the `loadBalancerBrokerOverloadedThresholdPercentage`. +If the threshold is exceeded, it is considered overloaded, with a default value of 85%. + +This algorithm is relatively simple, but there are many serious corner cases, so it is not recommended to use `OverloadShedder`. +Here are two cases: +- When the load on each broker in the cluster reaches the threshold, the bundle unload will continue to be executed, + but it will only switch from one overloaded broker to another, which is meaningless. +- If there are no broker whose load reaches the threshold, adding new brokers will not balance the traffic to the new added brokers. +The impact of these two points is quite serious, so we won't talk about it next. + + +### UniformLoadShedder +`UniformLoadShedder` will first calculate the maximum and minimum message rates, as well as the maximum and minimum +traffic throughput and corresponding broker. Then calculate the maximum and minimum difference, with two thresholds +corresponding to message rate and throughput size, respectively. + +- loadBalancerMsgRateDifferenceShedderThreshold + +The message rate percentage threshold between the highest and lowest loaded brokers, with a default value of 50, +can trigger bundle unload when the maximum message rate is 1.5 times the minimum message rate. +For example, broker 1 with 50K msgRate and broker 2 with 30K msgRate will have a (50-30)/30=66%>50% difference in msgRate, +and the load balancer can unload the bundle from broker 1 to broker 2. + +- loadBalancerMsgThroughputMultiplierDifferenceShedderThreshold + +The threshold for the message throughput multiplier between the highest and lowest loaded brokers, +with a default value of 4, can trigger bundle unload when the maximum throughput is 4 times the minimum throughput. +For example, if the msgRate of broker 1 is 450MB, broker 2 is 100MB, and the difference in msgThrough +is 450/100=4.5>4 times, then the load balancer can unload the bundle from broker 1 to broker 2. + + +After introducing the algorithm of `UniformLoadShedder`, we can clearly obtain the following information: +#### load jitter +`UniformLoadShedder` does not have the logic to handle load jitter. For example, +when the traffic suddenly increases or decreases. This load data point is adopted, triggering a bundle unload. +However, the traffic of this topic will soon return to normal, so it is very likely to trigger a bundle unload again. +This type of bundle unload should be avoided. This kind of scenario is very common, actually. + +#### heterogeneous environment +`UniformLoadShedder` does not rely on indicators such as CPU usage and network card usage to determine high load +and low load brokers, but rather determines them based on message rate and traffic throughput size, +while `ThresholdShedder` and `OverloadShedder` rely on machine resource indicators such as CPU usage to determine. +If the cluster is heterogeneous, such as different machines with different hardware configurations, +or if there are other processes sharing resources on the machine where the broker is located, +`UniformLoadShedder` is likely to misjudge high and low load brokers, thereby migrating the load from high-performance +but low load brokers to low-performance but high load brokers. +Therefore, it is not recommended for users to use `UniformLoadShedder` in heterogeneous environments. + +#### slow load balancing +`UniformLoadShedder` will only unload the bundle from one of the highest loaded brokers at a time, +which may take a considerable amount of time for a large cluster to complete all load balancing tasks. +For example, if there are 100 high load brokers in the current cluster and 100 new machines to be added, +it is roughly estimated that it will take 100 shedding to complete the balancing. +However, since the execution time interval of the `LoadSheddingStrategy` policy is determined by the +configuration of `loadBalancerSheddingIntervalMinutes`, which defaults to once every 1 minute, +so it will take 100 minutes to complete all tasks. For users using large partition topics, their tasks +are likely to be disconnected multiple times within this 100 minutes, which greatly affects the user experience. + + +## ModularLoadManagerStrategy +The `LoadSheddingStrategy` strategy is used to unload bundles of high load brokers. However, in order to +achieve a good load balancing effect, it is necessary not only to "unload" correctly, but also to "load" correctly. +The `ModularLoadManagerStrategy` strategy is responsible for assigning bundles to brokers. +The coordination between `LoadSheddingStrategy` and `ModularLoadManagerStrategy` is also a key point worth paying attention to. + +### LeastLongTermMessageRate +The `LeastLongTermMessageRate` algorithm directly used the maximum resource usage of CPU and so on as the broker's score, +and reused the `OverloadShedder` configuration, `loadBalancerBrokerOverloadedThresholdPercentage`. +If the score is greater than it (default 85%), set `score=INF`; Otherwise, update the broker's score to the sum of the +message in and out rates obtained from the broker's long-term aggregation. +``` +score = longTerm MsgIn rate+longTerm MsgOut rate, +``` +Finally, randomly select a broker from the broker with the lowest score to return. If the score of each broker is INF, +randomly select broker from all brokers. + +The scoring algorithm in `LeastLongTermMessageRate` is essentially based on message rate. Although it initially examines +the maximum resource utilization, it is to exclude overloaded brokers only. +Therefore, in most cases, brokers are sorted based on the size of the message rate as a score, which results in the same +issues with heterogeneous environments, similar to `UniformLoadShedder`. + + +#### Effect of the combination of `LoadSheddingStrategy` and `LeastLongTermMessageRate` +Next, we will attempt to analyze the effect together with the `LoadSheddingStrategy`. +- **LeastLongTermMessageRate + OverloadShedder** +This is the initial combination, but due to some inherent flaws in `OverloadShedder`, **it is not recommended**. + +- **LeastLongTermMessageRate + ThresholdShedder** +This combination is even worse than `LeastLongTermMessageRate + OverloadShedder` and **is not recommended**. +Because `OverloadShedder` uses the maximum weighted resource usage and historical score to score brokers, +while LeastLongTermMessage Rate is scored based on message rate. Inconsistent unloading and placement criteria +can lead to incorrect load balancing execution. +This is also why a new placement strategy `LeastResourceUsageWithWeight` will be introduced later. + +- **LeastLongTermMessageRate + UniformLoadShedder** +This is **recommended**. Both uninstallation and placement policy are based on message rate, +but using message rate as a standard naturally leads to issues with heterogeneous environments. + + +### LeastResourceUsageWithWeight +`LeastResourceUsageWithWeight` uses the same scoring algorithm as `ThresholdShedder` to score brokers, which uses +weighted maximum resource usage and historical scores to calculate the current score. + +Next, select candidate brokers based on the configuration of `loadBalancerAverageResourceUsageDifferenceThresholdPercentage`. +If a broker's score plus this threshold is still not greater than the average score, the broker will be added to the +candidate broker list. After obtaining the candidate broker list, a broker will be randomly selected from it; +If there are no candidate brokers, randomly select from all brokers. + +For example, if the resource utilization rate of broker 1 is 10%, broker 2 is 30%, and broker 3 is 80%, +the average resource utilization rate is 40%. The placement strategy can choose Broker1 and Broker2 +as the best candidates, as the thresholds are 10, 10+10<=40, 30+10<=40. In this way, the bundles uninstalled +from broker 3 will be evenly distributed among broker 1 and broker 2, rather than being completely placed on broker 1. + +#### over placement problem +Over placement problem is that the bundle is placed on high load brokers and make them overloaded. + +In practice, it will be found that it is difficult to determine a suitable value for `loadBalancerAverageResourceUsageDifferenceThresholdPercentage`, +which often triggers a fallback global random selection logic. For example, if there are 6 brokers in the current +cluster, with scores of 40, 40, 40, 40, 69, and 70 respectively, the average score is 49.83. +Using the default configuration, there are no candidate brokers because 40+10>49.83. +Triggering a bottom-up global random selection logic and the bundle may be offloaded from the overloaded broker5 +to the overloaded broker6, or vice versa, **causing the over placement problem.** + +Attempting to reduce the configuration value to expand the random pool, such as setting it to 0, may also include some +overloaded brokers in the candidate broker list. For example, if there are 5 brokers in the current cluster with scores +of 10, 60, 70, 80, and 80 respectively, the average score is 60. As the configuration value is 0, then broker 1 and +broker 2 are both candidate brokers. If broker 2 shares half of the offloaded traffic, **it is highly likely to overload.** + +Therefore, it is difficult to configure the `LeastResourceUsageWithWeight` algorithm well to avoid incorrect load balancing. +Of course, if you want to use the `ThresholdShedder` algorithm, the combination of `ThresholdShedder+LeastResourceUsageWithWeight` +will still be superior to the combination of `ThresholdShedder+LeastLongTermMessageRate`, because at least the scoring algorithm +of `LeastResourceUsageWithWeight` is consistent with that of `ThresholdShedder`. + +#### why doesn't LeastLongTermMessage Rate have over placement problem? +The root of over placement problem is that the frequency of updating the load data is limited due to the performance +of zookeeper. If we assign a bundle to a broker, the broker's load will increase after a while, and it's load data +also need some time to be updated to leader broker. If there are many bundles unloaded in a shedding, +how can we assign these bundles to brokers? + +The most simple way is to assign them to the broker with the lowest load, but it may cause the over placement problem +as it is most likely that there is only one single broker with the lowest load. With all bundles assigned to this broker, +it will be overloaded. This is the reason why `LeastResourceUsageWithWeight` try to determine a candidate broker list +to avoid the over placement problem. But we also find that candidate broker list can be empty or include some overloaded +brokers, which will also cause the over placement problem. + +So why doesn't `LeastLongTermMessageRate` have over placement problem? The reason is that each time a bundle is assigned, +the bundle will be added into `PreallocatedBundleData`. When scoring a broker, not only will the long-term message rate +aggregated by the broker itself be used, but also the message rate of bundles in `PreallocatedBundleData` that have been +assigned to the broker but have not yet been reflected in the broker's load data will be calculated. + +For example, if there are two bundles with 20KB/s message rate to be assigned, and broker1 and broker2 at 100KB/s +and 110KB/s respectively. The first bundle is assigned to broker1, However, broker1's load data will not be updated +in the short term. Before the load data is updated, `LeastLongTermMessageRate` try to assign the second bundle. +At this time, the score of broker1 is 100+20=120KB/s, where 20KB/s is the message rate of the first bundle +from `PreallocatedBundleData`. As broker1's score is greater than broker2, the second bundle will be assigned to broker2. + +**`LeastLongTermMessageRate` predict the load of the broker after the bundle is assigned to avoid the over placement problem.** + +**Why doesn't `LeastResourceUsageWithWeight` have this feature? Because it is not possible to predict how much resource +utilization a broker will increase when loading a bundle. All algorithms scoring brokers based on resource utilization +can't fix the over placement problem with this feature.** +So `LeastResourceUsageWithWeight` try to determine a candidate broker list to avoid the over placement problem, which is +proved to be not a good solution. + + +#### over unloading problem +Over unloading problem is that the load offloaded from high load brokers is too much and make them underloaded. + +Finally, let's talk about the issue of historical weighted scoring algorithms. The historical weighted scoring algorithm +is used by the `ThresholdShedder` and `LeastResourceUsageWithWeight` algorithms, as follows: +``` +HistoryUsage=historyUsage=null? ResourceUsage: historyUsage * historyPercentage+(1- historyPercentage) * resourceUsage; +``` +The default value of historyPercentage is 0.9, indicating that the score calculated last time has a significant impact on the current score. +The current maximum resource utilization only accounts for 10%, which is to solves the problem of load jitter. +However, introducing this algorithm has its side effects, such as over unloading problem. + +For example, there is currently one broker1 in the cluster with a load of 90%, and broker2 is added with a current load of 10%. +- At the first execution of shedding: broker1 scores 90, broker2 scores 10. For simplicity, assuming that the algorithm will +move some bundles to make their load the same, thus the true load of broker 1 and broker 2 become 50 after load shedding is completed. +- At the second execution of shedding: broker1 scores 90*0.9+50*0.1=86, broker2 scores 10*0.9+50*0.1=14. +**Note that the actual load of broker1 here is 50, but it is overestimated as 86!** +**The true load of broker2 is also 50, but it is underestimated at 14!** +Due to the significant difference in ratings between the two, although their actual loads are already the same, +broker1 will continue to unload traffic corresponding to 36 points from broker1 to broker2, +resulting in broker1's actual load score becoming 14, broker2's actual load score becoming 86. + +- At the third execution of shedding: broker1 scored 86*0.9+14*0.1=78.8, broker2 scored 14*0.9+86*0.1=21.2. +It is ridiculous that broker1 is still considered overloaded, and broker2 is still considered underloaded. +All loads in broker1 are moved to broker2, which is the over unloading problem. + +Although this example is an idealized theoretical analysis, we can still see that using historical scoring algorithms +can seriously overestimate or underestimate the true load of the broker. Although it can avoid the problem of load jitter, +it will introduce a more serious and broader problem: **overestimating or underestimating the true load of the broker, +leading to incorrect load balancing execution**. + + +## Summary +Based on the previous analysis, although we have three shedding strategies and two placement strategies +that can generate 6 combinations of 3 * 2, we actually only have two recommended options: +- ThresholdShedder + LeastResourceUsageWithWeight +- UniformLoadShedder + LeastLongTermMessageRate + +These two options each have their own advantages and disadvantages, and users can choose one according to +their requirements. The following table summarizes the advantages and disadvantages of the two options: + +| Combination | heterogeneous environment | load jitter | over placement problem | over unloading problem | slow load balancing | +|---------------------------------------------|---------------------------|------------|-----------------------|-----------------------|---------------------| +| ThresholdShedder + LeastResourceUsageWithWeight | normal(1) | good | bad | bad | normal(1) | +| UniformLoadShedder + LeastLongTermMessageRate | bad(2) | bad | good | good | normal(1) | + +1. In terms of adapting to heterogeneous environments, `ThresholdShedder+LeastResourceUsageWithWeight` can +only be rated as `normal`. This is because `ThresholdShedder` is not fully adaptable to heterogeneous environments. +Although it does not misjudge overloaded brokers as underloaded, heterogeneous environments can still have a +significant impact on the load balancing effect of `ThresholdShedder`. +For example, there are three brokers in the current cluster with resource utilization rates of 10, 50, and 70, respectively. +Broker1 and Broker2 are isomorphic. Though Broker3 don't bear any load, its resource utilization rate has +reached to 70 due to the deployment of other processes at the same machine. +At this point, we would like broker 1 to share some of the pressure from broker2, but since the average load is +43.33, 43.33+10>50, broker2 will not be judged as overloaded, and overloaded broker 3 also has no traffic to +unload, causing the load balancing algorithm to be in an inoperable state. + +2. In the same scenario, if `UniformLoadShedder+LeastLongTermMessageRate` is used, the problem will be more +severe, as some of the load will be offloaded from broker2 to broker3. As a result, the performance of those +topics in broker3 services will experience significant performance degradation. +Therefore, it is not recommended to run Pulsar in heterogeneous environments as current load balancing algorithms +cannot adapt too well. If it is unavoidable, it is recommended to choose `ThresholdShedder+LeastResourceUsageWithWeight`. + +3. In terms of load balancing speed, although `ThresholdShedder+LeastResourceUsageWithWeight` can unload the load +of all overloaded brokers at once, historical scoring algorithms can seriously affect the accuracy of load +balancing decisions. Therefore, in reality, it also requires multiple load balancing executions to finally +stabilize. This is why the load balancing speed of `ThresholdShedder+LeastResourceUsageWithWeight` is rated as `normal`. + +4. In terms of load balancing speed, `UniformLoadShedder+LeastLongTermMessageRate` can only unload the load of one +overloaded broker at a time, so it takes a long time to complete load balancing when there are many brokers, +so it is also rated as `normal`. + + +# Motivation + +The current load balance algorithm has some serious problems, such as load jitter, heterogeneous environment, slow load balancing, etc. +This PIP aims to introduce a new load balance algorithm `AvgShedder` to solve these problems. + +# Goals + +Introduce a new load balance algorithm `AvgShedder` that can solve the problems of load jitter, heterogeneous environment, slow load balancing, etc. + + +# High Level Design + +## scoring criterion +First of all, to determine high load brokers, it is necessary to rate and sort them. +Currently, there are two scoring criteria: +- Resource utilization rate of broker +- The message rate and throughput of the broker +Based on the previous analysis, it can be seen that scoring based on message rate and throughput will face +the same problem as `UniformLoadShedder` in heterogeneous environments, while scoring based on resource utilization +rate will face the over placement problem like `LeastResourceUsageWithWeight`. + +**To solve the problem of heterogeneous environments, we use the resource utilization rate of the broker as the scoring criterion.** + + +## binding shedding and placement strategies +So how can we avoid the over placement problem? **The key is to bind the shedding and placement strategies together.** +If every bundle unloaded from the high load broker is assigned to the right low load broker in shedding strategy, +the over placement problem will be solved. + +For example, if the broker rating of the current cluster is 20,30,52,80,80, and the shedding and placement strategies are decoupled, +the bundles will be unloaded from the two brokers with score of 80, and then all these bundles will be placed on the broker with a +score of 20, causing the over placement problem. + +If the shedding and placement strategies are coupled, one broker with 80 score can unload some bundles to a broker with 20 score, +and another broker with 80 score can unload the bundle to the broker with 30 score. In this way, we can avoid the over placement problem. + + +## evenly distributed traffic between the highest and lowest loaded brokers +We will first pick out the highest and lowest loaded brokers, and then evenly distribute the traffic between them. + +For example, if the broker rating of the current cluster is 20,30,52,70,80, and the message rate of the highest loaded broker is 1000, +the message rate of the lowest loaded broker is 500. We introduce a threshold to whether trigger the bundle unload, for example, +the threshold is 40. As the difference between the score of the highest and lowest loaded brokers is 100-50=50>40, +the shedding strategy will be triggered. + +To achieve the goal of evenly distributing the traffic between the highest and lowest loaded brokers, the shedding strategy will +try to make the message rate of two brokers the same, which is (1000+500)/2=750. The shedding strategy will unload 250 message rate from the +highest loaded broker to the lowest loaded broker. After the shedding strategy is completed, the message rate of two brokers will be +same, which is 750. + + +## improve the load balancing speed +As we mentioned earlier in `UniformLoadShedder`, if strategy only handles one high load broker at a time, it will take a long time to +complete all load balancing tasks. Therefore, we further optimize it by matching multiple pairs of high and low load brokers in +a single shedding. After sorting the broker scores, the first and last place are paired, the second and and the second to last are paired, +and so on. When the score difference between the two paired brokers is greater than the threshold, the load will be evenly distributed +between the two, which can solve the problem of slow speed. + +For example, if the broker rating of the current cluster is 20,30,52,70,80, we will pair 20 and 80, 30 and 70. As the difference between +the two paired brokers is 80-20=60, 70-30=40, which are both greater than the threshold 40, the shedding strategy will be triggered. + + +## handle load jitter with multiple hits threshold +What about the historical weighting algorithm used in `ThresholdShedder`? It is used to solve the problem of load jitter, but previous +analysis and experiments have shown that it can bring serious negative effects, so we can no longer use this method to solve the +problem of load jitter. + +We mimic the way alarms are triggered: the threshold is triggered multiple times before the bundle unload is finally triggered. +For example, when the difference between a pair of brokers exceeds the threshold three times, load balancing is triggered. + +## high and low threshold +In situations of cluster rolling restart or expansion, there is often a significant load difference between +different brokers, and we hope to complete load balancing more quickly. + +Therefore, we introduce two thresholds: +- loadBalancerAvgShedderLowThreshold, default value is 15 +- loadBalancerAvgShedderHighThreshold, default value is 40 + +Two thresholds correspond to two continuous hit count requirements: +- loadBalancerAvgShedderHitCountLowThreshold, default value is 8 +- loadBalancerAvgShedderHitCountHighThreshold, default value of 2 + +When the difference in scores between two paired brokers exceeds the `loadBalancerAvgShedderLowThreshold` by +`loadBalancerAvgShedderHitCountLowThreshold` times, or exceeds the `loadBalancerAvgShedderHighThreshold` by +`loadBalancerAvgShedderHitCountHighThreshold` times, a bundle unload is triggered. +For example, with the default value, if the score difference exceeds 15, it needs to be triggered 8 times continuously, +and if the score difference exceeds 40, it needs to be triggered 2 times continuously. + +The larger the load difference between brokers, the smaller the number of times it takes to trigger bundle unloads, +which can adapt to scenarios such as cluster rolling restart or expansion. + +## placement strategy +As mentioned earlier, `AvgShedder` bundles the shedding and placement strategies, and a bundle has already determined +its next owner broker based on the shedding strategy during shedding. But we not only use placement strategies after +executing shedding, but also need to use placement strategies to assign bundles during cluster initialization, rolling +restart, and broker shutdown. So how should we assign these bundles without shedding strategies? + +We use a hash allocation method: hash mapping a random number to broker. Hash mapping roughly conforms to +a uniform distribution, so bundles will be roughly evenly distributed across all brokers. However, due to the different +throughput between different bundles, the cluster will exhibit a certain degree of imbalance. However, this problem is +not significant, and the subsequent balancing can be achieved through shedding strategies. Moreover, the frequency of +cluster initialization, rolling restart, and broker shutdown scenarios is not high, so the impact is slight. + +## summary +In summary, `AvgShedder` can solve the problems of load jitter, heterogeneous environment, slow load balancing, etc. +Following table summarizes the advantages and disadvantages of the three options: + +| Combination | heterogeneous environment | load jitter | over placement problem | over unloading problem | slow load balancing | +|---------------------------------------------|------------------------|------------|-----------------------|-----------------------|--------------| +| ThresholdShedder + LeastResourceUsageWithWeight | normal | good | bad | bad | normal | +| UniformLoadShedder + LeastLongTermMessageRate | bad | bad | good | good | normal | +| AvgShedder | normal | good | good | good | good | + + +# Detailed Design + +### Configuration + +To avoid introducing too many configurations when calculating how much traffic needs to be unloaded, `AvgShedder` reuses the +following three `UniformLoadShedder` configurations: +``` + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "In the UniformLoadShedder strategy, the minimum message that triggers unload." + ) + private int minUnloadMessage = 1000; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "In the UniformLoadShedder strategy, the minimum throughput that triggers unload." + ) + private int minUnloadMessageThroughput = 1 * 1024 * 1024; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "In the UniformLoadShedder strategy, the maximum unload ratio." + ) + private double maxUnloadPercentage = 0.2; +``` + +The `maxUnloadPercentage` controls the allocation ratio. Although the default value is 0.2, our goal is to evenly distribute the +pressure between two brokers. Therefore, we set the value to 0.5, so that after load balancing is completed, the message rate/throughput +of the two brokers will be almost equal. + +The following configurations are introduced to control the shedding strategy: +``` + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "The low threshold for the difference between the highest and lowest loaded brokers." + ) + private int loadBalancerAvgShedderLowThreshold = 15; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "The high threshold for the difference between the highest and lowest loaded brokers." + ) + private int loadBalancerAvgShedderHighThreshold = 40; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "The number of times the low threshold is triggered before the bundle is unloaded." + ) + private int loadBalancerAvgShedderHitCountLowThreshold = 8; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "The number of times the high threshold is triggered before the bundle is unloaded." + ) + private int loadBalancerAvgShedderHitCountHighThreshold = 2; +``` + + + +# Backward & Forward Compatibility + +Fully compatible. + +# General Notes + +# Links + +* Mailing List discussion thread: https://lists.apache.org/thread/cy39b6jp38n38zyzd3bbw8b9vm5fwf3f +* Mailing List voting thread: https://lists.apache.org/thread/2v9fw5t5m5hlmjkrvjz6ywxjcqpmd02q From 4e535cb3f4a3482b0d5dc5a3a0a63c87490704e3 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Thu, 27 Jun 2024 02:54:43 -0700 Subject: [PATCH 349/580] [feat][broker] PIP-264: Add transaction metrics (#22970) --- .../apache/pulsar/broker/PulsarService.java | 15 ++++ .../service/PersistentTopicAttributes.java | 30 +++++++ .../persistent/PersistentSubscription.java | 7 +- .../persistent/PersistentTopicMetrics.java | 14 ++- .../broker/stats/OpenTelemetryTopicStats.java | 27 +++++- ...nTelemetryTransactionCoordinatorStats.java | 87 +++++++++++++++++++ ...emetryTransactionPendingAckStoreStats.java | 72 +++++++++++++++ .../buffer/TransactionBufferClientStats.java | 7 +- .../impl/TransactionBufferClientImpl.java | 9 +- .../TransactionBufferClientStatsImpl.java | 61 +++++++++++-- .../pendingack/PendingAckHandle.java | 7 ++ .../PendingAckHandleAttributes.java | 63 ++++++++++++++ .../pendingack/PendingAckHandleStats.java | 7 ++ .../impl/PendingAckHandleDisabled.java | 6 ++ .../pendingack/impl/PendingAckHandleImpl.java | 28 +++--- .../impl/PendingAckHandleStatsImpl.java | 56 +++++++++++- .../broker/transaction/TransactionTest.java | 24 ++++- .../buffer/TopicTransactionBufferTest.java | 22 ++++- .../pendingack/PendingAckPersistentTest.java | 40 +++++++++ .../OpenTelemetryAttributes.java | 33 ++++++- pulsar-transaction/coordinator/pom.xml | 6 ++ .../coordinator/TransactionMetadataStore.java | 9 ++ .../TransactionMetadataStoreAttributes.java | 59 +++++++++++++ .../impl/InMemTransactionMetadataStore.java | 16 ++++ .../impl/MLTransactionMetadataStore.java | 16 ++++ 25 files changed, 678 insertions(+), 43 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTransactionCoordinatorStats.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTransactionPendingAckStoreStats.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckHandleAttributes.java create mode 100644 pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionMetadataStoreAttributes.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 0d8bc571c5750..848484fe3763d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -116,6 +116,8 @@ import org.apache.pulsar.broker.stats.OpenTelemetryProducerStats; import org.apache.pulsar.broker.stats.OpenTelemetryReplicatorStats; import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; +import org.apache.pulsar.broker.stats.OpenTelemetryTransactionCoordinatorStats; +import org.apache.pulsar.broker.stats.OpenTelemetryTransactionPendingAckStoreStats; import org.apache.pulsar.broker.stats.PulsarBrokerOpenTelemetry; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; @@ -263,6 +265,8 @@ public class PulsarService implements AutoCloseable, ShutdownService { private OpenTelemetryConsumerStats openTelemetryConsumerStats; private OpenTelemetryProducerStats openTelemetryProducerStats; private OpenTelemetryReplicatorStats openTelemetryReplicatorStats; + private OpenTelemetryTransactionCoordinatorStats openTelemetryTransactionCoordinatorStats; + private OpenTelemetryTransactionPendingAckStoreStats openTelemetryTransactionPendingAckStoreStats; private TransactionMetadataStoreService transactionMetadataStoreService; private TransactionBufferProvider transactionBufferProvider; @@ -684,6 +688,14 @@ public CompletableFuture closeAsync() { brokerClientSharedTimer.stop(); monotonicSnapshotClock.close(); + if (openTelemetryTransactionPendingAckStoreStats != null) { + openTelemetryTransactionPendingAckStoreStats.close(); + openTelemetryTransactionPendingAckStoreStats = null; + } + if (openTelemetryTransactionCoordinatorStats != null) { + openTelemetryTransactionCoordinatorStats.close(); + openTelemetryTransactionCoordinatorStats = null; + } if (openTelemetryReplicatorStats != null) { openTelemetryReplicatorStats.close(); openTelemetryReplicatorStats = null; @@ -996,6 +1008,9 @@ public void start() throws PulsarServerException { .newProvider(config.getTransactionBufferProviderClassName()); transactionPendingAckStoreProvider = TransactionPendingAckStoreProvider .newProvider(config.getTransactionPendingAckStoreProviderClassName()); + + openTelemetryTransactionCoordinatorStats = new OpenTelemetryTransactionCoordinatorStats(this); + openTelemetryTransactionPendingAckStoreStats = new OpenTelemetryTransactionPendingAckStoreStats(this); } this.metricsGenerator = new MetricsGenerator(this); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PersistentTopicAttributes.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PersistentTopicAttributes.java index 048edafe8848f..51f5bdb354dc9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PersistentTopicAttributes.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PersistentTopicAttributes.java @@ -36,6 +36,11 @@ public class PersistentTopicAttributes extends TopicAttributes { private final Attributes transactionCommittedAttributes; private final Attributes transactionAbortedAttributes; + private final Attributes transactionBufferClientCommitSucceededAttributes; + private final Attributes transactionBufferClientCommitFailedAttributes; + private final Attributes transactionBufferClientAbortSucceededAttributes; + private final Attributes transactionBufferClientAbortFailedAttributes; + public PersistentTopicAttributes(TopicName topicName) { super(topicName); @@ -61,6 +66,31 @@ public PersistentTopicAttributes(TopicName topicName) { .putAll(OpenTelemetryAttributes.TransactionStatus.ABORTED.attributes) .build(); + transactionBufferClientCommitSucceededAttributes = Attributes.builder() + .putAll(commonAttributes) + .remove(OpenTelemetryAttributes.PULSAR_DOMAIN) + .putAll(OpenTelemetryAttributes.TransactionStatus.COMMITTED.attributes) + .putAll(OpenTelemetryAttributes.TransactionBufferClientOperationStatus.SUCCESS.attributes) + .build(); + transactionBufferClientCommitFailedAttributes = Attributes.builder() + .putAll(commonAttributes) + .remove(OpenTelemetryAttributes.PULSAR_DOMAIN) + .putAll(OpenTelemetryAttributes.TransactionStatus.COMMITTED.attributes) + .putAll(OpenTelemetryAttributes.TransactionBufferClientOperationStatus.FAILURE.attributes) + .build(); + transactionBufferClientAbortSucceededAttributes = Attributes.builder() + .putAll(commonAttributes) + .remove(OpenTelemetryAttributes.PULSAR_DOMAIN) + .putAll(OpenTelemetryAttributes.TransactionStatus.ABORTED.attributes) + .putAll(OpenTelemetryAttributes.TransactionBufferClientOperationStatus.SUCCESS.attributes) + .build(); + transactionBufferClientAbortFailedAttributes = Attributes.builder() + .putAll(commonAttributes) + .remove(OpenTelemetryAttributes.PULSAR_DOMAIN) + .putAll(OpenTelemetryAttributes.TransactionStatus.ABORTED.attributes) + .putAll(OpenTelemetryAttributes.TransactionBufferClientOperationStatus.FAILURE.attributes) + .build(); + compactionSuccessAttributes = Attributes.builder() .putAll(commonAttributes) .putAll(OpenTelemetryAttributes.CompactionStatus.SUCCESS.attributes) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 7da339a420c89..a1d51668ca808 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -37,6 +37,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Predicate; import java.util.stream.Collectors; +import lombok.Getter; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.ClearBacklogCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback; @@ -128,6 +129,7 @@ public class PersistentSubscription extends AbstractSubscription { private static final Map NON_REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES = Map.of(); private volatile ReplicatedSubscriptionSnapshotCache replicatedSubscriptionSnapshotCache; + @Getter private final PendingAckHandle pendingAckHandle; private volatile Map subscriptionProperties; private volatile CompletableFuture fenceFuture; @@ -1439,11 +1441,6 @@ public ManagedCursor getCursor() { return cursor; } - @VisibleForTesting - public PendingAckHandle getPendingAckHandle() { - return pendingAckHandle; - } - public void syncBatchPositionBitSetForPendingAck(Position position) { this.pendingAckHandle.syncBatchPositionAckSetForTransaction(position); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java index f79d053a9790d..d8ebece7a51cb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java @@ -21,12 +21,13 @@ import java.util.concurrent.atomic.LongAdder; import lombok.Getter; -@SuppressWarnings("LombokGetterMayBeUsed") +@Getter public class PersistentTopicMetrics { - @Getter private final BacklogQuotaMetrics backlogQuotaMetrics = new BacklogQuotaMetrics(); + private final TransactionBufferClientMetrics transactionBufferClientMetrics = new TransactionBufferClientMetrics(); + public static class BacklogQuotaMetrics { private final LongAdder timeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); private final LongAdder sizeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); @@ -47,4 +48,13 @@ public long getTimeBasedBacklogQuotaExceededEvictionCount() { return timeBasedBacklogQuotaExceededEvictionCount.longValue(); } } + + @Getter + public static class TransactionBufferClientMetrics { + private final LongAdder commitSucceededCount = new LongAdder(); + private final LongAdder commitFailedCount = new LongAdder(); + + private final LongAdder abortSucceededCount = new LongAdder(); + private final LongAdder abortFailedCount = new LongAdder(); + } } \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStats.java index b6d3f08907792..0274cb7a7d4a6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTopicStats.java @@ -149,6 +149,12 @@ public class OpenTelemetryTopicStats implements AutoCloseable { public static final String TRANSACTION_COUNTER = "pulsar.broker.topic.transaction.count"; private final ObservableLongMeasurement transactionCounter; + // Replaces ['pulsar_txn_tb_client_abort_failed_total', 'pulsar_txn_tb_client_commit_failed_total', + // 'pulsar_txn_tb_client_abort_latency', 'pulsar_txn_tb_client_commit_latency'] + public static final String TRANSACTION_BUFFER_CLIENT_OPERATION_COUNTER = + "pulsar.broker.topic.transaction.buffer.client.operation.count"; + private final ObservableLongMeasurement transactionBufferClientOperationCounter; + // Replaces pulsar_subscription_delayed public static final String DELAYED_SUBSCRIPTION_COUNTER = "pulsar.broker.topic.subscription.delayed.entry.count"; private final ObservableLongMeasurement delayedSubscriptionCounter; @@ -333,6 +339,12 @@ public OpenTelemetryTopicStats(PulsarService pulsar) { .setDescription("The number of transactions on this topic.") .buildObserver(); + transactionBufferClientOperationCounter = meter + .counterBuilder(TRANSACTION_BUFFER_CLIENT_OPERATION_COUNTER) + .setUnit("{operation}") + .setDescription("The number of operations on the transaction buffer client.") + .buildObserver(); + delayedSubscriptionCounter = meter .upDownCounterBuilder(DELAYED_SUBSCRIPTION_COUNTER) .setUnit("{entry}") @@ -371,6 +383,7 @@ public OpenTelemetryTopicStats(PulsarService pulsar) { compactionEntriesCounter, compactionBytesCounter, transactionCounter, + transactionBufferClientOperationCounter, delayedSubscriptionCounter); } @@ -399,6 +412,8 @@ private void recordMetricsForTopic(Topic topic) { } if (topic instanceof PersistentTopic persistentTopic) { + var persistentTopicMetrics = persistentTopic.getPersistentTopicMetrics(); + var persistentTopicAttributes = persistentTopic.getTopicAttributes(); var managedLedger = persistentTopic.getManagedLedger(); var managedLedgerStats = persistentTopic.getManagedLedger().getStats(); @@ -416,7 +431,7 @@ private void recordMetricsForTopic(Topic topic) { topic.getBacklogQuota(BacklogQuota.BacklogQuotaType.message_age).getLimitTime(), attributes); backlogQuotaAge.record(topic.getBestEffortOldestUnacknowledgedMessageAgeSeconds(), attributes); - var backlogQuotaMetrics = persistentTopic.getPersistentTopicMetrics().getBacklogQuotaMetrics(); + var backlogQuotaMetrics = persistentTopicMetrics.getBacklogQuotaMetrics(); backlogEvictionCounter.record(backlogQuotaMetrics.getSizeBasedBacklogQuotaExceededEvictionCount(), persistentTopicAttributes.getSizeBasedQuotaAttributes()); backlogEvictionCounter.record(backlogQuotaMetrics.getTimeBasedBacklogQuotaExceededEvictionCount(), @@ -430,6 +445,16 @@ private void recordMetricsForTopic(Topic topic) { transactionCounter.record(txnBuffer.getAbortedTxnCount(), persistentTopicAttributes.getTransactionAbortedAttributes()); + var txnBufferClientMetrics = persistentTopicMetrics.getTransactionBufferClientMetrics(); + transactionBufferClientOperationCounter.record(txnBufferClientMetrics.getCommitSucceededCount().sum(), + persistentTopicAttributes.getTransactionBufferClientCommitSucceededAttributes()); + transactionBufferClientOperationCounter.record(txnBufferClientMetrics.getCommitFailedCount().sum(), + persistentTopicAttributes.getTransactionBufferClientCommitFailedAttributes()); + transactionBufferClientOperationCounter.record(txnBufferClientMetrics.getAbortSucceededCount().sum(), + persistentTopicAttributes.getTransactionBufferClientAbortSucceededAttributes()); + transactionBufferClientOperationCounter.record(txnBufferClientMetrics.getAbortFailedCount().sum(), + persistentTopicAttributes.getTransactionBufferClientAbortFailedAttributes()); + Optional.ofNullable(pulsar.getNullableCompactor()) .map(Compactor::getStats) .flatMap(compactorMXBean -> compactorMXBean.getCompactionRecordForTopic(topic.getName())) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTransactionCoordinatorStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTransactionCoordinatorStats.java new file mode 100644 index 0000000000000..ab73b2390b37d --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTransactionCoordinatorStats.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.broker.stats; + +import io.opentelemetry.api.metrics.BatchCallback; +import io.opentelemetry.api.metrics.ObservableLongMeasurement; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.transaction.coordinator.TransactionMetadataStore; + +public class OpenTelemetryTransactionCoordinatorStats implements AutoCloseable { + + // Replaces ['pulsar_txn_aborted_total', + // 'pulsar_txn_committed_total', + // 'pulsar_txn_created_total', + // 'pulsar_txn_timeout_total', + // 'pulsar_txn_active_count'] + public static final String TRANSACTION_COUNTER = "pulsar.broker.transaction.coordinator.transaction.count"; + private final ObservableLongMeasurement transactionCounter; + + // Replaces pulsar_txn_append_log_total + public static final String APPEND_LOG_COUNTER = "pulsar.broker.transaction.coordinator.append.log.count"; + private final ObservableLongMeasurement appendLogCounter; + + private final BatchCallback batchCallback; + + public OpenTelemetryTransactionCoordinatorStats(PulsarService pulsar) { + var meter = pulsar.getOpenTelemetry().getMeter(); + + transactionCounter = meter + .upDownCounterBuilder(TRANSACTION_COUNTER) + .setUnit("{transaction}") + .setDescription("The number of transactions handled by the coordinator.") + .buildObserver(); + + appendLogCounter = meter + .counterBuilder(APPEND_LOG_COUNTER) + .setUnit("{entry}") + .setDescription("The number of transaction metadata entries appended by the coordinator.") + .buildObserver(); + + batchCallback = meter.batchCallback(() -> { + var transactionMetadataStoreService = pulsar.getTransactionMetadataStoreService(); + // Avoid NPE during Pulsar shutdown. + if (transactionMetadataStoreService != null) { + transactionMetadataStoreService.getStores() + .values() + .forEach(this::recordMetricsForTransactionMetadataStore); + } + }, + transactionCounter, + appendLogCounter); + } + + @Override + public void close() { + batchCallback.close(); + } + + private void recordMetricsForTransactionMetadataStore(TransactionMetadataStore transactionMetadataStore) { + var attributes = transactionMetadataStore.getAttributes(); + var stats = transactionMetadataStore.getMetadataStoreStats(); + + transactionCounter.record(stats.getAbortedCount(), attributes.getTxnAbortedAttributes()); + transactionCounter.record(stats.getActives(), attributes.getTxnActiveAttributes()); + transactionCounter.record(stats.getCommittedCount(), attributes.getTxnCommittedAttributes()); + transactionCounter.record(stats.getCreatedCount(), attributes.getTxnCreatedAttributes()); + transactionCounter.record(stats.getTimeoutCount(), attributes.getTxnTimeoutAttributes()); + + appendLogCounter.record(stats.getAppendLogCount(), attributes.getCommonAttributes()); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTransactionPendingAckStoreStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTransactionPendingAckStoreStats.java new file mode 100644 index 0000000000000..562ad56e44db4 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryTransactionPendingAckStoreStats.java @@ -0,0 +1,72 @@ +/* + * 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.stats; + +import io.opentelemetry.api.metrics.ObservableLongCounter; +import io.opentelemetry.api.metrics.ObservableLongMeasurement; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; + +public class OpenTelemetryTransactionPendingAckStoreStats implements AutoCloseable { + + // Replaces ['pulsar_txn_tp_committed_count_total', 'pulsar_txn_tp_aborted_count_total'] + public static final String ACK_COUNTER = "pulsar.broker.transaction.pending.ack.store.transaction.count"; + private final ObservableLongCounter ackCounter; + + public OpenTelemetryTransactionPendingAckStoreStats(PulsarService pulsar) { + var meter = pulsar.getOpenTelemetry().getMeter(); + + ackCounter = meter + .counterBuilder(ACK_COUNTER) + .setUnit("{transaction}") + .setDescription("The number of transactions handled by the persistent ack store.") + .buildWithCallback(measurement -> pulsar.getBrokerService() + .getTopics() + .values() + .stream() + .filter(topicFuture -> topicFuture.isDone() && !topicFuture.isCompletedExceptionally()) + .map(CompletableFuture::join) + .filter(Optional::isPresent) + .map(Optional::get) + .filter(Topic::isPersistent) + .map(Topic::getSubscriptions) + .forEach(subs -> subs.forEach((__, sub) -> recordMetricsForSubscription(measurement, sub)))); + } + + @Override + public void close() { + ackCounter.close(); + } + + private void recordMetricsForSubscription(ObservableLongMeasurement measurement, Subscription subscription) { + assert subscription instanceof PersistentSubscription; // The topics have already been filtered for persistence. + var stats = ((PersistentSubscription) subscription).getPendingAckHandle().getPendingAckHandleStats(); + if (stats != null) { + var attributes = stats.getAttributes(); + measurement.record(stats.getCommitSuccessCount(), attributes.getCommitSuccessAttributes()); + measurement.record(stats.getCommitFailedCount(), attributes.getCommitFailureAttributes()); + measurement.record(stats.getAbortSuccessCount(), attributes.getAbortSuccessAttributes()); + measurement.record(stats.getAbortFailedCount(), attributes.getAbortFailureAttributes()); + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientStats.java index 8fda233ff1dfa..c21b212f981dd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientStats.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.transaction.buffer; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientStatsImpl; import org.apache.pulsar.client.impl.transaction.TransactionBufferHandler; @@ -34,10 +35,10 @@ public interface TransactionBufferClientStats { void close(); - static TransactionBufferClientStats create(boolean exposeTopicMetrics, TransactionBufferHandler handler, - boolean enableTxnCoordinator) { + static TransactionBufferClientStats create(PulsarService pulsarService, boolean exposeTopicMetrics, + TransactionBufferHandler handler, boolean enableTxnCoordinator) { return enableTxnCoordinator - ? TransactionBufferClientStatsImpl.getInstance(exposeTopicMetrics, handler) : NOOP; + ? TransactionBufferClientStatsImpl.getInstance(pulsarService, exposeTopicMetrics, handler) : NOOP; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferClientImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferClientImpl.java index 382d640ca8658..96ad020390055 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferClientImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferClientImpl.java @@ -39,10 +39,11 @@ public class TransactionBufferClientImpl implements TransactionBufferClient { private final TransactionBufferHandler tbHandler; private final TransactionBufferClientStats stats; - private TransactionBufferClientImpl(TransactionBufferHandler tbHandler, boolean exposeTopicLevelMetrics, - boolean enableTxnCoordinator) { + private TransactionBufferClientImpl(PulsarService pulsarService, TransactionBufferHandler tbHandler, + boolean exposeTopicLevelMetrics, boolean enableTxnCoordinator) { this.tbHandler = tbHandler; - this.stats = TransactionBufferClientStats.create(exposeTopicLevelMetrics, tbHandler, enableTxnCoordinator); + this.stats = TransactionBufferClientStats.create(pulsarService, exposeTopicLevelMetrics, tbHandler, + enableTxnCoordinator); } public static TransactionBufferClient create(PulsarService pulsarService, HashedWheelTimer timer, @@ -53,7 +54,7 @@ public static TransactionBufferClient create(PulsarService pulsarService, Hashed ServiceConfiguration config = pulsarService.getConfig(); boolean exposeTopicLevelMetrics = config.isExposeTopicLevelMetricsInPrometheus(); boolean enableTxnCoordinator = config.isTransactionCoordinatorEnabled(); - return new TransactionBufferClientImpl(handler, exposeTopicLevelMetrics, enableTxnCoordinator); + return new TransactionBufferClientImpl(pulsarService, handler, exposeTopicLevelMetrics, enableTxnCoordinator); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferClientStatsImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferClientStatsImpl.java index a447f70789311..4f1c2ca30cf54 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferClientStatsImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferClientStatsImpl.java @@ -18,31 +18,55 @@ */ package org.apache.pulsar.broker.transaction.buffer.impl; +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.Objects; +import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.LongAdder; +import lombok.NonNull; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics; +import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; import org.apache.pulsar.broker.transaction.buffer.TransactionBufferClientStats; import org.apache.pulsar.client.impl.transaction.TransactionBufferHandler; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric; public final class TransactionBufferClientStatsImpl implements TransactionBufferClientStats { private static final double[] QUANTILES = {0.50, 0.75, 0.95, 0.99, 0.999, 0.9999, 1}; private final AtomicBoolean closed = new AtomicBoolean(false); + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.TRANSACTION_BUFFER_CLIENT_OPERATION_COUNTER) private final Counter abortFailed; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.TRANSACTION_BUFFER_CLIENT_OPERATION_COUNTER) private final Counter commitFailed; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.TRANSACTION_BUFFER_CLIENT_OPERATION_COUNTER) private final Summary abortLatency; + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryTopicStats.TRANSACTION_BUFFER_CLIENT_OPERATION_COUNTER) private final Summary commitLatency; + + public static final String PENDING_TRANSACTION_COUNTER = "pulsar.broker.transaction.buffer.client.pending.count"; + private final ObservableLongUpDownCounter pendingTransactionCounter; + + @PulsarDeprecatedMetric(newMetricName = PENDING_TRANSACTION_COUNTER) private final Gauge pendingRequests; private final boolean exposeTopicLevelMetrics; + private final BrokerService brokerService; + private static TransactionBufferClientStats instance; - private TransactionBufferClientStatsImpl(boolean exposeTopicLevelMetrics, - TransactionBufferHandler handler) { + private TransactionBufferClientStatsImpl(@NonNull PulsarService pulsarService, + boolean exposeTopicLevelMetrics, + @NonNull TransactionBufferHandler handler) { + this.brokerService = Objects.requireNonNull(pulsarService.getBrokerService()); this.exposeTopicLevelMetrics = exposeTopicLevelMetrics; String[] labelNames = exposeTopicLevelMetrics ? new String[]{"namespace", "topic"} : new String[]{"namespace"}; @@ -63,9 +87,14 @@ private TransactionBufferClientStatsImpl(boolean exposeTopicLevelMetrics, .setChild(new Gauge.Child() { @Override public double get() { - return null == handler ? 0 : handler.getPendingRequestsCount(); + return handler.getPendingRequestsCount(); } }); + this.pendingTransactionCounter = pulsarService.getOpenTelemetry().getMeter() + .upDownCounterBuilder(PENDING_TRANSACTION_COUNTER) + .setDescription("The number of pending transactions in the transaction buffer client.") + .setUnit("{transaction}") + .buildWithCallback(measurement -> measurement.record(handler.getPendingRequestsCount())); } private Summary buildSummary(String name, String help, String[] labelNames) { @@ -77,33 +106,52 @@ private Summary buildSummary(String name, String help, String[] labelNames) { return builder.register(); } - public static synchronized TransactionBufferClientStats getInstance(boolean exposeTopicLevelMetrics, + public static synchronized TransactionBufferClientStats getInstance(PulsarService pulsarService, + boolean exposeTopicLevelMetrics, TransactionBufferHandler handler) { if (null == instance) { - instance = new TransactionBufferClientStatsImpl(exposeTopicLevelMetrics, handler); + instance = new TransactionBufferClientStatsImpl(pulsarService, exposeTopicLevelMetrics, handler); } - return instance; } @Override public void recordAbortFailed(String topic) { this.abortFailed.labels(labelValues(topic)).inc(); + getTransactionBufferClientMetrics(topic) + .map(PersistentTopicMetrics.TransactionBufferClientMetrics::getAbortFailedCount) + .ifPresent(LongAdder::increment); } @Override public void recordCommitFailed(String topic) { this.commitFailed.labels(labelValues(topic)).inc(); + getTransactionBufferClientMetrics(topic) + .map(PersistentTopicMetrics.TransactionBufferClientMetrics::getCommitFailedCount) + .ifPresent(LongAdder::increment); } @Override public void recordAbortLatency(String topic, long nanos) { this.abortLatency.labels(labelValues(topic)).observe(nanos); + getTransactionBufferClientMetrics(topic) + .map(PersistentTopicMetrics.TransactionBufferClientMetrics::getAbortSucceededCount) + .ifPresent(LongAdder::increment); } @Override public void recordCommitLatency(String topic, long nanos) { this.commitLatency.labels(labelValues(topic)).observe(nanos); + getTransactionBufferClientMetrics(topic) + .map(PersistentTopicMetrics.TransactionBufferClientMetrics::getCommitSucceededCount) + .ifPresent(LongAdder::increment); + } + + private Optional getTransactionBufferClientMetrics( + String topic) { + return brokerService.getTopicReference(topic) + .filter(t -> t instanceof PersistentTopic) + .map(t -> ((PersistentTopic) t).getPersistentTopicMetrics().getTransactionBufferClientMetrics()); } private String[] labelValues(String topic) { @@ -125,6 +173,7 @@ public void close() { CollectorRegistry.defaultRegistry.unregister(this.abortLatency); CollectorRegistry.defaultRegistry.unregister(this.commitLatency); CollectorRegistry.defaultRegistry.unregister(this.pendingRequests); + pendingTransactionCounter.close(); } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckHandle.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckHandle.java index 168a6b1483f86..dcebbb2829eec 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckHandle.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckHandle.java @@ -145,6 +145,13 @@ CompletableFuture individualAcknowledgeMessage(TxnID txnID, List closeAsync() { return CompletableFuture.completedFuture(null); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java index 98d0d3bf1b9e5..6a071c891ffa7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java @@ -50,7 +50,6 @@ import org.apache.commons.collections4.map.LinkedMap; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.commons.lang3.tuple.Pair; -import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException; @@ -155,20 +154,14 @@ public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { this.topicName = persistentSubscription.getTopicName(); this.subName = persistentSubscription.getName(); this.persistentSubscription = persistentSubscription; - internalPinnedExecutor = persistentSubscription - .getTopic() - .getBrokerService() - .getPulsar() - .getTransactionExecutorProvider() - .getExecutor(this); - - ServiceConfiguration config = persistentSubscription.getTopic().getBrokerService().pulsar().getConfig(); - boolean exposeTopicLevelMetrics = config.isExposeTopicLevelMetricsInPrometheus(); - this.handleStats = PendingAckHandleStats.create(topicName, subName, exposeTopicLevelMetrics); - - this.pendingAckStoreProvider = this.persistentSubscription.getTopic() - .getBrokerService().getPulsar().getTransactionPendingAckStoreProvider(); - transactionOpTimer = persistentSubscription.getTopic().getBrokerService().getPulsar().getTransactionTimer(); + var pulsar = persistentSubscription.getTopic().getBrokerService().getPulsar(); + internalPinnedExecutor = pulsar.getTransactionExecutorProvider().getExecutor(this); + + this.handleStats = PendingAckHandleStats.create( + topicName, subName, pulsar.getConfig().isExposeTopicLevelMetricsInPrometheus()); + + this.pendingAckStoreProvider = pulsar.getTransactionPendingAckStoreProvider(); + transactionOpTimer = pulsar.getTransactionTimer(); init(); } @@ -1021,6 +1014,11 @@ public TransactionInPendingAckStats getTransactionInPendingAckStats(TxnID txnID) return transactionInPendingAckStats; } + @Override + public PendingAckHandleStats getPendingAckHandleStats() { + return handleStats; + } + @Override public CompletableFuture closeAsync() { changeToCloseState(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleStatsImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleStatsImpl.java index f30c233af5993..a89b582b838dd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleStatsImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleStatsImpl.java @@ -22,7 +22,10 @@ import io.prometheus.client.Summary; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.concurrent.atomic.LongAdder; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.broker.transaction.pendingack.PendingAckHandleAttributes; import org.apache.pulsar.broker.transaction.pendingack.PendingAckHandleStats; import org.apache.pulsar.common.naming.TopicName; @@ -37,6 +40,19 @@ public class PendingAckHandleStatsImpl implements PendingAckHandleStats { private final String[] labelFailed; private final String[] commitLatencyLabel; + private final String topic; + private final String subscription; + + private final LongAdder commitTxnSucceedCounter = new LongAdder(); + private final LongAdder commitTxnFailedCounter = new LongAdder(); + private final LongAdder abortTxnSucceedCounter = new LongAdder(); + private final LongAdder abortTxnFailedCounter = new LongAdder(); + + private volatile PendingAckHandleAttributes attributes = null; + private static final AtomicReferenceFieldUpdater + ATTRIBUTES_UPDATER = AtomicReferenceFieldUpdater.newUpdater( + PendingAckHandleStatsImpl.class, PendingAckHandleAttributes.class, "attributes"); + public PendingAckHandleStatsImpl(String topic, String subscription, boolean exposeTopicLevelMetrics) { initialize(exposeTopicLevelMetrics); @@ -51,6 +67,9 @@ public PendingAckHandleStatsImpl(String topic, String subscription, boolean expo } } + this.topic = topic; + this.subscription = subscription; + labelSucceed = exposeTopicLevelMetrics0 ? new String[]{namespace, topic, subscription, "succeed"} : new String[]{namespace, "succeed"}; labelFailed = exposeTopicLevelMetrics0 @@ -62,18 +81,24 @@ public PendingAckHandleStatsImpl(String topic, String subscription, boolean expo @Override public void recordCommitTxn(boolean success, long nanos) { String[] labels; + LongAdder counter; if (success) { labels = labelSucceed; + counter = commitTxnSucceedCounter; commitTxnLatency.labels(commitLatencyLabel).observe(TimeUnit.NANOSECONDS.toMicros(nanos)); } else { labels = labelFailed; + counter = commitTxnFailedCounter; } commitTxnCounter.labels(labels).inc(); + counter.increment(); } @Override public void recordAbortTxn(boolean success) { abortTxnCounter.labels(success ? labelSucceed : labelFailed).inc(); + var counter = success ? abortTxnSucceedCounter : abortTxnFailedCounter; + counter.increment(); } @Override @@ -81,11 +106,40 @@ public void close() { if (exposeTopicLevelMetrics0) { commitTxnCounter.remove(this.labelSucceed); commitTxnCounter.remove(this.labelFailed); + abortTxnCounter.remove(this.labelSucceed); abortTxnCounter.remove(this.labelFailed); - abortTxnCounter.remove(this.labelFailed); } } + @Override + public long getCommitSuccessCount() { + return commitTxnSucceedCounter.sum(); + } + + @Override + public long getCommitFailedCount() { + return commitTxnFailedCounter.sum(); + } + + @Override + public long getAbortSuccessCount() { + return abortTxnSucceedCounter.sum(); + } + + @Override + public long getAbortFailedCount() { + return abortTxnFailedCounter.sum(); + } + + @Override + public PendingAckHandleAttributes getAttributes() { + if (attributes != null) { + return attributes; + } + return ATTRIBUTES_UPDATER.updateAndGet(PendingAckHandleStatsImpl.this, + old -> old != null ? old : new PendingAckHandleAttributes(topic, subscription)); + } + static void initialize(boolean exposeTopicLevelMetrics) { if (INITIALIZED.compareAndSet(false, true)) { exposeTopicLevelMetrics0 = exposeTopicLevelMetrics; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index 14d4375b7bf51..2a928084e648a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.transaction; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.apache.pulsar.common.naming.SystemTopicNames.PENDING_ACK_STORE_CURSOR_NAME; import static org.apache.pulsar.common.naming.SystemTopicNames.PENDING_ACK_STORE_SUFFIX; import static org.apache.pulsar.transaction.coordinator.impl.DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS; @@ -94,7 +95,6 @@ import org.apache.pulsar.broker.service.TransactionBufferSnapshotServiceFactory; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil; import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; import org.apache.pulsar.broker.systopic.SystemTopicClient; @@ -231,19 +231,35 @@ public void testTopicTransactionMetrics() throws Exception { .build(); var metrics = pulsarTestContexts.get(0).getOpenTelemetryMetricReader().collectAllMetrics(); - BrokerOpenTelemetryTestUtil.assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.TRANSACTION_COUNTER, + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.TRANSACTION_BUFFER_CLIENT_OPERATION_COUNTER, + Attributes.builder() + .putAll(attributes) + .remove(OpenTelemetryAttributes.PULSAR_DOMAIN) + .putAll(OpenTelemetryAttributes.TransactionStatus.COMMITTED.attributes) + .putAll(OpenTelemetryAttributes.TransactionBufferClientOperationStatus.SUCCESS.attributes) + .build(), + 1); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.TRANSACTION_BUFFER_CLIENT_OPERATION_COUNTER, + Attributes.builder() + .putAll(attributes) + .remove(OpenTelemetryAttributes.PULSAR_DOMAIN) + .putAll(OpenTelemetryAttributes.TransactionStatus.ABORTED.attributes) + .putAll(OpenTelemetryAttributes.TransactionBufferClientOperationStatus.SUCCESS.attributes) + .build(), + 1); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.TRANSACTION_COUNTER, Attributes.builder() .putAll(attributes) .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "committed") .build(), 1); - BrokerOpenTelemetryTestUtil.assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.TRANSACTION_COUNTER, + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.TRANSACTION_COUNTER, Attributes.builder() .putAll(attributes) .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "aborted") .build(), 1); - BrokerOpenTelemetryTestUtil.assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.TRANSACTION_COUNTER, + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.TRANSACTION_COUNTER, Attributes.builder() .putAll(attributes) .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "active") diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java index af12caf1efd61..dea79f391e39a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java @@ -18,12 +18,14 @@ */ package org.apache.pulsar.broker.transaction.buffer; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.Mockito.when; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertTrue; import static org.testng.AssertJUnit.fail; +import io.opentelemetry.api.common.Attributes; import java.time.Duration; import java.util.Collections; import java.util.List; @@ -44,6 +46,7 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBufferState; @@ -57,6 +60,7 @@ import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.TopicMessageIdImpl; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStore; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStoreState; @@ -70,7 +74,6 @@ public class TopicTransactionBufferTest extends TransactionTestBase { - @BeforeMethod(alwaysRun = true) protected void setup() throws Exception { setBrokerCount(1); @@ -101,10 +104,19 @@ protected void cleanup() throws Exception { @Test public void testTransactionBufferAppendMarkerWriteFailState() throws Exception { final String topic = "persistent://" + NAMESPACE1 + "/testPendingAckManageLedgerWriteFailState"; + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_TENANT, "tnx") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "tnx/ns1") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topic) + .putAll(OpenTelemetryAttributes.TransactionStatus.COMMITTED.attributes) + .putAll(OpenTelemetryAttributes.TransactionBufferClientOperationStatus.FAILURE.attributes) + .build(); + Transaction txn = pulsarClient.newTransaction() .withTransactionTimeout(5, TimeUnit.SECONDS) .build().get(); + @Cleanup Producer producer = pulsarClient .newProducer() .topic(topic) @@ -112,11 +124,19 @@ public void testTransactionBufferAppendMarkerWriteFailState() throws Exception { .enableBatching(false) .create(); + assertMetricLongSumValue( + pulsarTestContexts.get(0).getOpenTelemetryMetricReader().collectAllMetrics(), + OpenTelemetryTopicStats.TRANSACTION_BUFFER_CLIENT_OPERATION_COUNTER, attributes, 0); + producer.newMessage(txn).value("test".getBytes()).send(); PersistentTopic persistentTopic = (PersistentTopic) getPulsarServiceList().get(0) .getBrokerService().getTopic(TopicName.get(topic).toString(), false).get().get(); FieldUtils.writeField(persistentTopic.getManagedLedger(), "state", ManagedLedgerImpl.State.WriteFailed, true); txn.commit().get(); + + assertMetricLongSumValue( + pulsarTestContexts.get(0).getOpenTelemetryMetricReader().collectAllMetrics(), + OpenTelemetryTopicStats.TRANSACTION_BUFFER_CLIENT_OPERATION_COUNTER, attributes, 1); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java index 00cdb4162f0c4..9487e3d374642 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.transaction.pendingack; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.ArgumentMatchers.any; @@ -31,6 +32,7 @@ import static org.testng.AssertJUnit.assertTrue; import static org.testng.AssertJUnit.fail; import com.google.common.collect.Multimap; +import io.opentelemetry.api.common.Attributes; import java.io.ByteArrayOutputStream; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -56,6 +58,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.OpenTelemetryTransactionPendingAckStoreStats; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; @@ -78,6 +81,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -366,6 +370,42 @@ public void testPendingAckMetrics() throws Exception { assertTrue(metric.value > 0); } } + + var otelMetrics = pulsarTestContexts.get(0).getOpenTelemetryMetricReader().collectAllMetrics(); + var commonAttributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_TENANT, "tnx") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "tnx/ns1") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, TopicName.get(PENDING_ACK_REPLAY_TOPIC).toString()) + .put(OpenTelemetryAttributes.PULSAR_SUBSCRIPTION_NAME, subName) + .build(); + assertMetricLongSumValue(otelMetrics, OpenTelemetryTransactionPendingAckStoreStats.ACK_COUNTER, + Attributes.builder() + .putAll(commonAttributes) + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "committed") + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_ACK_STORE_OPERATION_STATUS, "success") + .build(), + 50); + assertMetricLongSumValue(otelMetrics, OpenTelemetryTransactionPendingAckStoreStats.ACK_COUNTER, + Attributes.builder() + .putAll(commonAttributes) + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "committed") + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_ACK_STORE_OPERATION_STATUS, "failure") + .build(), + 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryTransactionPendingAckStoreStats.ACK_COUNTER, + Attributes.builder() + .putAll(commonAttributes) + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "aborted") + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_ACK_STORE_OPERATION_STATUS, "success") + .build(), + 50); + assertMetricLongSumValue(otelMetrics, OpenTelemetryTransactionPendingAckStoreStats.ACK_COUNTER, + Attributes.builder() + .putAll(commonAttributes) + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_STATUS, "aborted") + .put(OpenTelemetryAttributes.PULSAR_TRANSACTION_ACK_STORE_OPERATION_STATUS, "failure") + .build(), + 0); } @Test diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index b530b50ee59dc..f485e30092604 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -117,12 +117,43 @@ public interface OpenTelemetryAttributes { */ AttributeKey PULSAR_TRANSACTION_STATUS = AttributeKey.stringKey("pulsar.transaction.status"); enum TransactionStatus { + ABORTED, ACTIVE, COMMITTED, - ABORTED; + CREATED, + TIMEOUT; public final Attributes attributes = Attributes.of(PULSAR_TRANSACTION_STATUS, name().toLowerCase()); } + /** + * The status of the Pulsar transaction ack store operation. + */ + AttributeKey PULSAR_TRANSACTION_ACK_STORE_OPERATION_STATUS = + AttributeKey.stringKey("pulsar.transaction.pending.ack.store.operation.status"); + enum TransactionPendingAckOperationStatus { + SUCCESS, + FAILURE; + public final Attributes attributes = + Attributes.of(PULSAR_TRANSACTION_ACK_STORE_OPERATION_STATUS, name().toLowerCase()); + } + + /** + * The ID of the Pulsar transaction coordinator. + */ + AttributeKey PULSAR_TRANSACTION_COORDINATOR_ID = AttributeKey.longKey("pulsar.transaction.coordinator.id"); + + /** + * The status of the Pulsar transaction buffer client operation. + */ + AttributeKey PULSAR_TRANSACTION_BUFFER_CLIENT_OPERATION_STATUS = + AttributeKey.stringKey("pulsar.transaction.buffer.client.operation.status"); + enum TransactionBufferClientOperationStatus { + SUCCESS, + FAILURE; + public final Attributes attributes = + Attributes.of(PULSAR_TRANSACTION_BUFFER_CLIENT_OPERATION_STATUS, name().toLowerCase()); + } + /** * The status of the Pulsar compaction operation. */ diff --git a/pulsar-transaction/coordinator/pom.xml b/pulsar-transaction/coordinator/pom.xml index 4728cd40634b5..fc326d9e9ba95 100644 --- a/pulsar-transaction/coordinator/pom.xml +++ b/pulsar-transaction/coordinator/pom.xml @@ -41,6 +41,12 @@ ${project.version} + + ${project.groupId} + pulsar-opentelemetry + ${project.version} + + ${project.groupId} managed-ledger diff --git a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionMetadataStore.java b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionMetadataStore.java index ff5adb4d409c7..850fcfb4d19ec 100644 --- a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionMetadataStore.java +++ b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionMetadataStore.java @@ -133,6 +133,15 @@ default long getLowWaterMark() { */ TransactionMetadataStoreStats getMetadataStoreStats(); + /** + * Get the transaction metadata store OpenTelemetry attributes. + * + * @return TransactionMetadataStoreAttributes {@link TransactionMetadataStoreAttributes} + */ + default TransactionMetadataStoreAttributes getAttributes() { + return new TransactionMetadataStoreAttributes(this); + } + /** * Get the transactions witch timeout is bigger than given timeout. * diff --git a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionMetadataStoreAttributes.java b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionMetadataStoreAttributes.java new file mode 100644 index 0000000000000..e8ae0f6d0391f --- /dev/null +++ b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/TransactionMetadataStoreAttributes.java @@ -0,0 +1,59 @@ +/* + * 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.transaction.coordinator; + +import io.opentelemetry.api.common.Attributes; +import lombok.Getter; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; + +@Getter +public class TransactionMetadataStoreAttributes { + + private final Attributes commonAttributes; + private final Attributes txnAbortedAttributes; + private final Attributes txnActiveAttributes; + private final Attributes txnCommittedAttributes; + private final Attributes txnCreatedAttributes; + private final Attributes txnTimeoutAttributes; + + public TransactionMetadataStoreAttributes(TransactionMetadataStore store) { + this.commonAttributes = Attributes.of( + OpenTelemetryAttributes.PULSAR_TRANSACTION_COORDINATOR_ID, store.getTransactionCoordinatorID().getId()); + this.txnAbortedAttributes = Attributes.builder() + .putAll(commonAttributes) + .putAll(OpenTelemetryAttributes.TransactionStatus.ABORTED.attributes) + .build(); + this.txnActiveAttributes = Attributes.builder() + .putAll(commonAttributes) + .putAll(OpenTelemetryAttributes.TransactionStatus.ACTIVE.attributes) + .build(); + this.txnCommittedAttributes = Attributes.builder() + .putAll(commonAttributes) + .putAll(OpenTelemetryAttributes.TransactionStatus.COMMITTED.attributes) + .build(); + this.txnCreatedAttributes = Attributes.builder() + .putAll(commonAttributes) + .putAll(OpenTelemetryAttributes.TransactionStatus.CREATED.attributes) + .build(); + this.txnTimeoutAttributes = Attributes.builder() + .putAll(commonAttributes) + .putAll(OpenTelemetryAttributes.TransactionStatus.TIMEOUT.attributes) + .build(); + } +} diff --git a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/InMemTransactionMetadataStore.java b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/InMemTransactionMetadataStore.java index 0f3c5e42d7a69..7817d48487568 100644 --- a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/InMemTransactionMetadataStore.java +++ b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/InMemTransactionMetadataStore.java @@ -23,12 +23,14 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.concurrent.atomic.LongAdder; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.common.policies.data.TransactionCoordinatorStats; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStore; +import org.apache.pulsar.transaction.coordinator.TransactionMetadataStoreAttributes; import org.apache.pulsar.transaction.coordinator.TransactionSubscription; import org.apache.pulsar.transaction.coordinator.TxnMeta; import org.apache.pulsar.transaction.coordinator.exceptions.CoordinatorException.InvalidTxnStatusException; @@ -49,6 +51,11 @@ class InMemTransactionMetadataStore implements TransactionMetadataStore { private final LongAdder abortTransactionCount; private final LongAdder transactionTimeoutCount; + private volatile TransactionMetadataStoreAttributes attributes = null; + private static final AtomicReferenceFieldUpdater + ATTRIBUTES_FIELD_UPDATER = AtomicReferenceFieldUpdater.newUpdater( + InMemTransactionMetadataStore.class, TransactionMetadataStoreAttributes.class, "attributes"); + InMemTransactionMetadataStore(TransactionCoordinatorID tcID) { this.tcID = tcID; this.localID = new AtomicLong(0L); @@ -165,4 +172,13 @@ public TransactionMetadataStoreStats getMetadataStoreStats() { public List getSlowTransactions(long timeout) { return null; } + + @Override + public TransactionMetadataStoreAttributes getAttributes() { + if (attributes != null) { + return attributes; + } + return ATTRIBUTES_FIELD_UPDATER.updateAndGet(this, + old -> old != null ? old : new TransactionMetadataStoreAttributes(this)); + } } diff --git a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionMetadataStore.java b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionMetadataStore.java index b6eaad2e3e38f..6bd7a947e3827 100644 --- a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionMetadataStore.java +++ b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionMetadataStore.java @@ -30,6 +30,7 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.concurrent.atomic.LongAdder; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.Position; @@ -45,6 +46,7 @@ import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.transaction.coordinator.TransactionLogReplayCallback; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStore; +import org.apache.pulsar.transaction.coordinator.TransactionMetadataStoreAttributes; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStoreState; import org.apache.pulsar.transaction.coordinator.TransactionRecoverTracker; import org.apache.pulsar.transaction.coordinator.TransactionSubscription; @@ -83,6 +85,11 @@ public class MLTransactionMetadataStore public final RecoverTimeRecord recoverTime = new RecoverTimeRecord(); private final long maxActiveTransactionsPerCoordinator; + private volatile TransactionMetadataStoreAttributes attributes = null; + private static final AtomicReferenceFieldUpdater + ATTRIBUTES_FIELD_UPDATER = AtomicReferenceFieldUpdater.newUpdater( + MLTransactionMetadataStore.class, TransactionMetadataStoreAttributes.class, "attributes"); + public MLTransactionMetadataStore(TransactionCoordinatorID tcID, MLTransactionLogImpl mlTransactionLog, TransactionTimeoutTracker timeoutTracker, @@ -549,4 +556,13 @@ public static List subscriptionToTxnSubscription( public ManagedLedger getManagedLedger() { return this.transactionLog.getManagedLedger(); } + + @Override + public TransactionMetadataStoreAttributes getAttributes() { + if (attributes != null) { + return attributes; + } + return ATTRIBUTES_FIELD_UPDATER.updateAndGet(this, + old -> old != null ? old : new TransactionMetadataStoreAttributes(this)); + } } From 7f4c0c535971d5b85c48a9cd658ae0e28dc46932 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 28 Jun 2024 23:52:08 +0800 Subject: [PATCH 350/580] [Fix][broker] Limit replication rate based on bytes (#22674) Signed-off-by: Zixuan Liu --- .../persistent/PersistentReplicator.java | 75 ++++++++++++------- .../service/ReplicatorRateLimiterTest.java | 60 +++++++++++++++ 2 files changed, 109 insertions(+), 26 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index aa53a93da5c4f..54b8993784e29 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -34,6 +34,8 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import lombok.AllArgsConstructor; +import lombok.Data; import lombok.Getter; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.ClearBacklogCallback; @@ -199,15 +201,31 @@ protected void disableReplicatorRead() { this.cursor.setInactive(); } + @Data + @AllArgsConstructor + private static class AvailablePermits { + private int messages; + private long bytes; + + /** + * messages, bytes + * 0, O: Producer queue is full, no permits. + * -1, -1: Rate Limiter reaches limit. + * >0, >0: available permits for read entries. + */ + public boolean isExceeded() { + return messages == -1 && bytes == -1; + } + + public boolean isReadable() { + return messages > 0 && bytes > 0; + } + } + /** * Calculate available permits for read entries. - * - * @return - * 0: Producer queue is full, no permits. - * -1: Rate Limiter reaches limit. - * >0: available permits for read entries. */ - private int getAvailablePermits() { + private AvailablePermits getAvailablePermits() { int availablePermits = producerQueueSize - PENDING_MESSAGES_UPDATER.get(this); // return 0, if Producer queue is full, it will pause read entries. @@ -216,15 +234,18 @@ private int getAvailablePermits() { log.debug("[{}] Producer queue is full, availablePermits: {}, pause reading", replicatorId, availablePermits); } - return 0; + return new AvailablePermits(0, 0); } + long availablePermitsOnMsg = -1; + long availablePermitsOnByte = -1; + // handle rate limit if (dispatchRateLimiter.isPresent() && dispatchRateLimiter.get().isDispatchRateLimitingEnabled()) { DispatchRateLimiter rateLimiter = dispatchRateLimiter.get(); // if dispatch-rate is in msg then read only msg according to available permit - long availablePermitsOnMsg = rateLimiter.getAvailableDispatchRateLimitOnMsg(); - long availablePermitsOnByte = rateLimiter.getAvailableDispatchRateLimitOnByte(); + availablePermitsOnMsg = rateLimiter.getAvailableDispatchRateLimitOnMsg(); + availablePermitsOnByte = rateLimiter.getAvailableDispatchRateLimitOnByte(); // no permits from rate limit if (availablePermitsOnByte == 0 || availablePermitsOnMsg == 0) { if (log.isDebugEnabled()) { @@ -235,14 +256,18 @@ private int getAvailablePermits() { rateLimiter.getDispatchRateOnByte(), MESSAGE_RATE_BACKOFF_MS); } - return -1; - } - if (availablePermitsOnMsg > 0) { - availablePermits = Math.min(availablePermits, (int) availablePermitsOnMsg); + return new AvailablePermits(-1, -1); } } - return availablePermits; + availablePermitsOnMsg = + availablePermitsOnMsg == -1 ? availablePermits : Math.min(availablePermits, availablePermitsOnMsg); + availablePermitsOnMsg = Math.min(availablePermitsOnMsg, readBatchSize); + + availablePermitsOnByte = + availablePermitsOnByte == -1 ? readMaxSizeBytes : Math.min(readMaxSizeBytes, availablePermitsOnByte); + + return new AvailablePermits((int) availablePermitsOnMsg, availablePermitsOnByte); } protected void readMoreEntries() { @@ -250,10 +275,10 @@ protected void readMoreEntries() { log.info("[{}] Skip the reading due to new detected schema", replicatorId); return; } - int availablePermits = getAvailablePermits(); - - if (availablePermits > 0) { - int messagesToRead = Math.min(availablePermits, readBatchSize); + AvailablePermits availablePermits = getAvailablePermits(); + if (availablePermits.isReadable()) { + int messagesToRead = availablePermits.getMessages(); + long bytesToRead = availablePermits.getBytes(); if (!isWritable()) { if (log.isDebugEnabled()) { log.debug("[{}] Throttling replication traffic because producer is not writable", replicatorId); @@ -262,23 +287,21 @@ protected void readMoreEntries() { messagesToRead = 1; } - // If messagesToRead is 0 or less, correct it to 1 to prevent IllegalArgumentException - messagesToRead = Math.max(messagesToRead, 1); - // Schedule read if (HAVE_PENDING_READ_UPDATER.compareAndSet(this, FALSE, TRUE)) { if (log.isDebugEnabled()) { - log.debug("[{}] Schedule read of {} messages", replicatorId, messagesToRead); + log.debug("[{}] Schedule read of {} messages or {} bytes", replicatorId, messagesToRead, + bytesToRead); } - cursor.asyncReadEntriesOrWait(messagesToRead, readMaxSizeBytes, this, + cursor.asyncReadEntriesOrWait(messagesToRead, bytesToRead, this, null, topic.getMaxReadPosition()); } else { if (log.isDebugEnabled()) { - log.debug("[{}] Not scheduling read due to pending read. Messages To Read {}", - replicatorId, messagesToRead); + log.debug("[{}] Not scheduling read due to pending read. Messages To Read {}, Bytes To Read {}", + replicatorId, messagesToRead, bytesToRead); } } - } else if (availablePermits == -1) { + } else if (availablePermits.isExceeded()) { // no permits from rate limit topic.getBrokerService().executor().schedule( () -> readMoreEntries(), MESSAGE_RATE_BACKOFF_MS, TimeUnit.MILLISECONDS); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java index 747ef3b7f5ce8..90df16360614d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -548,5 +549,64 @@ public void testReplicatorRateLimiterMessageReceivedAllMessages() throws Excepti producer.close(); } + @Test + public void testReplicatorRateLimiterByBytes() throws Exception { + final String namespace = "pulsar/replicatormsg-" + System.currentTimeMillis(); + final String topicName = "persistent://" + namespace + "/RateLimiterByBytes"; + + admin1.namespaces().createNamespace(namespace); + // 0. set 2 clusters, there will be 1 replicator in each topic + admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r2")); + + final int byteRate = 400; + final int payloadSize = 100; + DispatchRate dispatchRate = DispatchRate.builder() + .dispatchThrottlingRateInMsg(-1) + .dispatchThrottlingRateInByte(byteRate) + .ratePeriodInSecond(360) + .build(); + admin1.namespaces().setReplicatorDispatchRate(namespace, dispatchRate); + + @Cleanup + PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()).build(); + @Cleanup + Producer producer = client1.newProducer().topic(topicName) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + + PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getOrCreateTopic(topicName).get(); + + Awaitility.await() + .untilAsserted(() -> assertTrue(topic.getReplicators().values().get(0).getRateLimiter().isPresent())); + assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), byteRate); + + @Cleanup + PulsarClient client2 = PulsarClient.builder().serviceUrl(url2.toString()) + .build(); + final AtomicInteger totalReceived = new AtomicInteger(0); + + @Cleanup + Consumer ignored = client2.newConsumer().topic(topicName).subscriptionName("sub2-in-cluster2") + .messageListener((c1, msg) -> { + Assert.assertNotNull(msg, "Message cannot be null"); + String receivedMessage = new String(msg.getData()); + log.debug("Received message [{}] in the listener", receivedMessage); + totalReceived.incrementAndGet(); + }).subscribe(); + + // The total bytes is 5 times the rate limit value. + int numMessages = byteRate / payloadSize * 5; + for (int i = 0; i < numMessages * payloadSize; i++) { + producer.send(new byte[payloadSize]); + } + + Awaitility.await().pollDelay(5, TimeUnit.SECONDS) + .untilAsserted(() -> { + // The rate limit occurs in the next reading cycle, so a value fault tolerance needs to be added. + assertThat(totalReceived.get()).isLessThan((byteRate / payloadSize) + 2); + }); + } + private static final Logger log = LoggerFactory.getLogger(ReplicatorRateLimiterTest.class); } From 4c84788340b4a3df975bf4a919c7223b31835976 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 1 Jul 2024 21:41:43 +0800 Subject: [PATCH 351/580] [improve][broker] Improve exception for topic does not have schema to check (#22974) --- .../nonpersistent/NonPersistentTopic.java | 13 +++++- .../service/persistent/PersistentTopic.java | 13 +++++- .../schema/SchemaRegistryServiceImpl.java | 3 +- .../exceptions/NotExistSchemaException.java | 43 +++++++++++++++++++ .../org/apache/pulsar/schema/SchemaTest.java | 16 ++++--- 5 files changed, 80 insertions(+), 8 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/NotExistSchemaException.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 0c6ebdfefa01f..3801ac7f3ee82 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -66,6 +66,8 @@ import org.apache.pulsar.broker.service.TopicAttributes; import org.apache.pulsar.broker.service.TopicPolicyListener; import org.apache.pulsar.broker.service.TransportCnx; +import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; +import org.apache.pulsar.broker.service.schema.exceptions.NotExistSchemaException; import org.apache.pulsar.broker.stats.ClusterReplicationMetrics; import org.apache.pulsar.broker.stats.NamespaceStats; import org.apache.pulsar.client.api.MessageId; @@ -1239,7 +1241,16 @@ public CompletableFuture addSchemaIfIdleOrCheckCompatible(SchemaData schem || (!producers.isEmpty()) || (numActiveConsumersWithoutAutoSchema != 0) || ENTRIES_ADDED_COUNTER_UPDATER.get(this) != 0) { - return checkSchemaCompatibleForConsumer(schema); + return checkSchemaCompatibleForConsumer(schema) + .exceptionally(ex -> { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + if (realCause instanceof NotExistSchemaException) { + throw FutureUtil.wrapToCompletionException( + new IncompatibleSchemaException("Failed to add schema to an active topic" + + " with empty(BYTES) schema: new schema type " + schema.getType())); + } + throw FutureUtil.wrapToCompletionException(realCause); + }); } else { return addSchema(schema).thenCompose(schemaVersion -> CompletableFuture.completedFuture(null)); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 3d620d3189863..07deb1168072a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -132,6 +132,8 @@ import org.apache.pulsar.broker.service.TransportCnx; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; import org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage; +import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; +import org.apache.pulsar.broker.service.schema.exceptions.NotExistSchemaException; import org.apache.pulsar.broker.stats.ClusterReplicationMetrics; import org.apache.pulsar.broker.stats.NamespaceStats; import org.apache.pulsar.broker.stats.ReplicationMetrics; @@ -4048,7 +4050,16 @@ public CompletableFuture addSchemaIfIdleOrCheckCompatible(SchemaData schem || (userCreatedProducerCount > 0) || (numActiveConsumersWithoutAutoSchema != 0) || (ledger.getTotalSize() != 0)) { - return checkSchemaCompatibleForConsumer(schema); + return checkSchemaCompatibleForConsumer(schema) + .exceptionally(ex -> { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + if (realCause instanceof NotExistSchemaException) { + throw FutureUtil.wrapToCompletionException( + new IncompatibleSchemaException("Failed to add schema to an active topic" + + " with empty(BYTES) schema: new schema type " + schema.getType())); + } + throw FutureUtil.wrapToCompletionException(realCause); + }); } else { return addSchema(schema).thenCompose(schemaVersion -> CompletableFuture.completedFuture(null)); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java index 3e9e13b14fe46..c1a394dcfbbb7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java @@ -48,6 +48,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; +import org.apache.pulsar.broker.service.schema.exceptions.NotExistSchemaException; import org.apache.pulsar.broker.service.schema.exceptions.SchemaException; import org.apache.pulsar.broker.service.schema.proto.SchemaRegistryFormat; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; @@ -398,7 +399,7 @@ public CompletableFuture checkConsumerCompatibility(String schemaId, Schem return checkCompatibilityWithAll(schemaId, schemaData, strategy); } } else { - return FutureUtil.failedFuture(new IncompatibleSchemaException("Topic does not have schema to check")); + return FutureUtil.failedFuture(new NotExistSchemaException("Topic does not have schema to check")); } }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/NotExistSchemaException.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/NotExistSchemaException.java new file mode 100644 index 0000000000000..2fe0a09237545 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/NotExistSchemaException.java @@ -0,0 +1,43 @@ +/* + * 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.service.schema.exceptions; + +/** + * Exception is thrown when an schema not exist. + */ +public class NotExistSchemaException extends SchemaException { + + private static final long serialVersionUID = -8342983749283749283L; + + public NotExistSchemaException() { + super("The schema does not exist"); + } + + public NotExistSchemaException(String message) { + super(message); + } + + public NotExistSchemaException(String message, Throwable e) { + super(message, e); + } + + public NotExistSchemaException(Throwable e) { + super(e); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java index d21e853ba0982..ae9ea6d5ae6f4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java @@ -96,6 +96,7 @@ import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -125,6 +126,11 @@ public void cleanup() throws Exception { super.internalCleanup(); } + @DataProvider(name = "topicDomain") + public static Object[] topicDomain() { + return new Object[] { "persistent://", "non-persistent://" }; + } + @Test public void testGetSchemaWhenCreateAutoProduceBytesProducer() throws Exception{ final String tenant = PUBLIC_TENANT; @@ -1336,19 +1342,19 @@ private void testIncompatibleSchema() throws Exception { * the new consumer to register new schema. But before we can solve this problem, we need to modify * "CmdProducer" to let the Broker know that the Producer uses a schema of type "AUTO_PRODUCE_BYTES". */ - @Test - public void testAutoProduceAndSpecifiedConsumer() throws Exception { + @Test(dataProvider = "topicDomain") + public void testAutoProduceAndSpecifiedConsumer(String domain) throws Exception { final String namespace = PUBLIC_TENANT + "/ns_" + randomName(16); admin.namespaces().createNamespace(namespace, Sets.newHashSet(CLUSTER_NAME)); - final String topicName = "persistent://" + namespace + "/tp_" + randomName(16); + final String topicName = domain + namespace + "/tp_" + randomName(16); admin.topics().createNonPartitionedTopic(topicName); Producer producer = pulsarClient.newProducer(Schema.AUTO_PRODUCE_BYTES()).topic(topicName).create(); try { pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName("sub1").subscribe(); - fail("Should throw ex: Topic does not have schema to check"); + fail("Should throw ex: Failed to add schema to an active topic with empty(BYTES) schema"); } catch (Exception ex){ - assertTrue(ex.getMessage().contains("Topic does not have schema to check")); + assertTrue(ex.getMessage().contains("Failed to add schema to an active topic with empty(BYTES) schema")); } // Cleanup. From da2a1910a32e622ea609ff7b9e91711ecaf36de6 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Tue, 2 Jul 2024 08:46:56 +0800 Subject: [PATCH 352/580] [fix][broker] Fix broker OOM when upload a large package. (#22989) --- .../storage/bookkeeper/DLOutputStream.java | 53 +++++++++---------- .../bookkeeper/DLOutputStreamTest.java | 14 ++--- 2 files changed, 31 insertions(+), 36 deletions(-) diff --git a/pulsar-package-management/bookkeeper-storage/src/main/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStream.java b/pulsar-package-management/bookkeeper-storage/src/main/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStream.java index 222987aa49d43..67345ebd47e31 100644 --- a/pulsar-package-management/bookkeeper-storage/src/main/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStream.java +++ b/pulsar-package-management/bookkeeper-storage/src/main/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStream.java @@ -22,8 +22,6 @@ import io.netty.buffer.Unpooled; import java.io.IOException; import java.io.InputStream; -import java.util.ArrayList; -import java.util.List; import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; import org.apache.distributedlog.LogRecord; @@ -38,6 +36,7 @@ class DLOutputStream { private final DistributedLogManager distributedLogManager; private final AsyncLogWriter writer; + private final byte[] readBuffer = new byte[8192]; private long offset = 0L; private DLOutputStream(DistributedLogManager distributedLogManager, AsyncLogWriter writer) { @@ -50,42 +49,38 @@ static CompletableFuture openWriterAsync(DistributedLogManager d return distributedLogManager.openAsyncLogWriter().thenApply(w -> new DLOutputStream(distributedLogManager, w)); } - private CompletableFuture> getRecords(InputStream inputStream) { - CompletableFuture> future = new CompletableFuture<>(); - CompletableFuture.runAsync(() -> { - byte[] readBuffer = new byte[8192]; - List records = new ArrayList<>(); - try { - int read = 0; - while ((read = inputStream.read(readBuffer)) != -1) { - log.info("write something into the ledgers offset: {}, length: {}", offset, read); - ByteBuf writeBuf = Unpooled.copiedBuffer(readBuffer, 0, read); - offset += writeBuf.readableBytes(); - LogRecord record = new LogRecord(offset, writeBuf); - records.add(record); - } - future.complete(records); - } catch (IOException e) { - log.error("Failed to get all records from the input stream", e); - future.completeExceptionally(e); + private void writeAsyncHelper(InputStream is, CompletableFuture result) { + try { + int read = is.read(readBuffer); + if (read != -1) { + log.info("write something into the ledgers offset: {}, length: {}", offset, read); + final ByteBuf writeBuf = Unpooled.wrappedBuffer(readBuffer, 0, read); + offset += writeBuf.readableBytes(); + final LogRecord record = new LogRecord(offset, writeBuf); + writer.write(record).thenAccept(v -> writeAsyncHelper(is, result)) + .exceptionally(e -> { + result.completeExceptionally(e); + return null; + }); + } else { + result.complete(this); } - }); - return future; + } catch (IOException e) { + log.error("Failed to get all records from the input stream", e); + result.completeExceptionally(e); + } } /** * Write all input stream data to the distribute log. * * @param inputStream the data we need to write - * @return + * @return CompletableFuture */ CompletableFuture writeAsync(InputStream inputStream) { - return getRecords(inputStream) - .thenCompose(this::writeAsync); - } - - private CompletableFuture writeAsync(List records) { - return writer.writeBulk(records).thenApply(ignore -> this); + CompletableFuture result = new CompletableFuture<>(); + writeAsyncHelper(inputStream, result); + return result; } /** diff --git a/pulsar-package-management/bookkeeper-storage/src/test/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStreamTest.java b/pulsar-package-management/bookkeeper-storage/src/test/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStreamTest.java index 63fcf5e46ebe1..b55e0e0d34a4f 100644 --- a/pulsar-package-management/bookkeeper-storage/src/test/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStreamTest.java +++ b/pulsar-package-management/bookkeeper-storage/src/test/java/org/apache/pulsar/packages/management/storage/bookkeeper/DLOutputStreamTest.java @@ -21,17 +21,18 @@ import java.io.ByteArrayInputStream; import java.io.IOException; -import java.util.Collections; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import org.apache.distributedlog.DLSN; +import org.apache.distributedlog.LogRecord; import org.apache.distributedlog.api.AsyncLogWriter; import org.apache.distributedlog.api.DistributedLogManager; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.mockito.Mockito.anyList; @@ -53,9 +54,8 @@ public void setup() { when(dlm.asyncClose()).thenReturn(CompletableFuture.completedFuture(null)); when(writer.markEndOfStream()).thenReturn(CompletableFuture.completedFuture(null)); when(writer.asyncClose()).thenReturn(CompletableFuture.completedFuture(null)); - when(writer.writeBulk(anyList())) - .thenReturn(CompletableFuture.completedFuture( - Collections.singletonList(CompletableFuture.completedFuture(DLSN.InitialDLSN)))); + when(writer.write(any(LogRecord.class))) + .thenReturn(CompletableFuture.completedFuture(DLSN.InitialDLSN)); } @AfterMethod(alwaysRun = true) @@ -75,7 +75,7 @@ public void writeInputStreamData() throws ExecutionException, InterruptedExcepti .thenCompose(w -> w.writeAsync(new ByteArrayInputStream(data)) .thenCompose(DLOutputStream::closeAsync)).get(); - verify(writer, times(1)).writeBulk(anyList()); + verify(writer, times(1)).write(any(LogRecord.class)); verify(writer, times(1)).markEndOfStream(); verify(writer, times(1)).asyncClose(); verify(dlm, times(1)).asyncClose(); @@ -91,7 +91,7 @@ public void writeBytesArrayData() throws ExecutionException, InterruptedExceptio .thenCompose(w -> w.writeAsync(new ByteArrayInputStream(data)) .thenCompose(DLOutputStream::closeAsync)).get(); - verify(writer, times(1)).writeBulk(anyList()); + verify(writer, times(1)).write(any(LogRecord.class)); verify(writer, times(1)).markEndOfStream(); verify(writer, times(1)).asyncClose(); verify(dlm, times(1)).asyncClose(); @@ -104,7 +104,7 @@ public void writeLongBytesArrayData() throws ExecutionException, InterruptedExce .thenCompose(w -> w.writeAsync(new ByteArrayInputStream(data)) .thenCompose(DLOutputStream::closeAsync)).get(); - verify(writer, times(1)).writeBulk(anyList()); + verify(writer, times(4)).write(any(LogRecord.class)); verify(writer, times(1)).markEndOfStream(); verify(writer, times(1)).asyncClose(); verify(dlm, times(1)).asyncClose(); From dbbb6b66c99afd12762dec198482dbf766bff3bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 3 Jul 2024 21:09:31 +0800 Subject: [PATCH 353/580] [fix] Make operations on `individualDeletedMessages` in lock scope (#22966) --- .../mledger/ManagedLedgerConfig.java | 4 +- .../mledger/impl/ManagedCursorImpl.java | 91 +++++++++++++------ .../mledger/impl/RangeSetWrapper.java | 12 +-- ...angeSet.java => OpenLongPairRangeSet.java} | 10 +- .../util/collections/DefaultRangeSetTest.java | 4 +- ...est.java => OpenLongPairRangeSetTest.java} | 40 ++++---- 6 files changed, 100 insertions(+), 61 deletions(-) rename pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/{ConcurrentOpenLongPairRangeSet.java => OpenLongPairRangeSet.java} (97%) rename pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/{ConcurrentOpenLongPairRangeSetTest.java => OpenLongPairRangeSetTest.java} (92%) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index fb2c6de3c7423..03439f93ccad8 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -33,7 +33,7 @@ import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; import org.apache.commons.collections4.MapUtils; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; +import org.apache.pulsar.common.util.collections.OpenLongPairRangeSet; /** * Configuration class for a ManagedLedger. @@ -282,7 +282,7 @@ public ManagedLedgerConfig setPassword(String password) { } /** - * should use {@link ConcurrentOpenLongPairRangeSet} to store unacked ranges. + * should use {@link OpenLongPairRangeSet} to store unacked ranges. * @return */ public boolean isUnackedRangesOpenCacheSetEnabled() { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index bf46aa2fdffa9..98ba722ba1c9b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -336,7 +336,12 @@ public Map getProperties() { @Override public boolean isCursorDataFullyPersistable() { - return individualDeletedMessages.size() <= getConfig().getMaxUnackedRangesToPersist(); + lock.readLock().lock(); + try { + return individualDeletedMessages.size() <= getConfig().getMaxUnackedRangesToPersist(); + } finally { + lock.readLock().unlock(); + } } @Override @@ -1099,7 +1104,12 @@ public long getNumberOfEntriesSinceFirstNotAckedMessage() { @Override public int getTotalNonContiguousDeletedMessagesRange() { - return individualDeletedMessages.size(); + lock.readLock().lock(); + try { + return individualDeletedMessages.size(); + } finally { + lock.readLock().unlock(); + } } @Override @@ -2383,8 +2393,9 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb callback.deleteFailed(getManagedLedgerException(e), ctx); return; } finally { + boolean empty = individualDeletedMessages.isEmpty(); lock.writeLock().unlock(); - if (individualDeletedMessages.isEmpty()) { + if (empty) { callback.deleteComplete(ctx); } } @@ -2661,10 +2672,15 @@ public void operationFailed(MetaStoreException e) { } private boolean shouldPersistUnackRangesToLedger() { - return cursorLedger != null - && !isCursorLedgerReadOnly - && getConfig().getMaxUnackedRangesToPersist() > 0 - && individualDeletedMessages.size() > getConfig().getMaxUnackedRangesToPersistInMetadataStore(); + lock.readLock().lock(); + try { + return cursorLedger != null + && !isCursorLedgerReadOnly + && getConfig().getMaxUnackedRangesToPersist() > 0 + && individualDeletedMessages.size() > getConfig().getMaxUnackedRangesToPersistInMetadataStore(); + } finally { + lock.readLock().unlock(); + } } private void persistPositionMetaStore(long cursorsLedgerId, Position position, Map properties, @@ -3023,7 +3039,7 @@ private static List buildStringPropertiesMap(Map } private List buildIndividualDeletedMessageRanges() { - lock.readLock().lock(); + lock.writeLock().lock(); try { if (individualDeletedMessages.isEmpty()) { this.individualDeletedMessagesSerializedSize = 0; @@ -3065,7 +3081,7 @@ private List buildIndividualDeletedMessageRanges() { individualDeletedMessages.resetDirtyKeys(); return rangeList; } finally { - lock.readLock().unlock(); + lock.writeLock().unlock(); } } @@ -3451,8 +3467,13 @@ public LongPairRangeSet getIndividuallyDeletedMessagesSet() { } public boolean isMessageDeleted(Position position) { - return position.compareTo(markDeletePosition) <= 0 - || individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()); + lock.readLock().lock(); + try { + return position.compareTo(markDeletePosition) <= 0 + || individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()); + } finally { + lock.readLock().unlock(); + } } //this method will return a copy of the position's ack set @@ -3477,13 +3498,19 @@ public long[] getBatchPositionAckSet(Position position) { * @return next available position */ public Position getNextAvailablePosition(Position position) { - Range range = individualDeletedMessages.rangeContaining(position.getLedgerId(), - position.getEntryId()); - if (range != null) { - Position nextPosition = range.upperEndpoint().getNext(); - return (nextPosition != null && nextPosition.compareTo(position) > 0) ? nextPosition : position.getNext(); + lock.readLock().lock(); + try { + Range range = individualDeletedMessages.rangeContaining(position.getLedgerId(), + position.getEntryId()); + if (range != null) { + Position nextPosition = range.upperEndpoint().getNext(); + return (nextPosition != null && nextPosition.compareTo(position) > 0) + ? nextPosition : position.getNext(); + } + return position.getNext(); + } finally { + lock.readLock().unlock(); } - return position.getNext(); } public Position getNextLedgerPosition(long currentLedgerId) { @@ -3534,7 +3561,12 @@ public ManagedLedger getManagedLedger() { @Override public Range getLastIndividualDeletedRange() { - return individualDeletedMessages.lastRange(); + lock.readLock().lock(); + try { + return individualDeletedMessages.lastRange(); + } finally { + lock.readLock().unlock(); + } } @Override @@ -3664,15 +3696,20 @@ public ManagedLedgerConfig getConfig() { public ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) throws ManagedLedgerException { NonDurableCursorImpl newNonDurableCursor = (NonDurableCursorImpl) ledger.newNonDurableCursor(getMarkDeletedPosition(), nonDurableCursorName); - if (individualDeletedMessages != null) { - this.individualDeletedMessages.forEach(range -> { - newNonDurableCursor.individualDeletedMessages.addOpenClosed( - range.lowerEndpoint().getLedgerId(), - range.lowerEndpoint().getEntryId(), - range.upperEndpoint().getLedgerId(), - range.upperEndpoint().getEntryId()); - return true; - }); + lock.readLock().lock(); + try { + if (individualDeletedMessages != null) { + this.individualDeletedMessages.forEach(range -> { + newNonDurableCursor.individualDeletedMessages.addOpenClosed( + range.lowerEndpoint().getLedgerId(), + range.lowerEndpoint().getEntryId(), + range.upperEndpoint().getLedgerId(), + range.upperEndpoint().getEntryId()); + return true; + }); + } + } finally { + lock.readLock().unlock(); } if (batchDeletedIndexes != null) { for (Map.Entry entry : this.batchDeletedIndexes.entrySet()) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index f235ffc63ace5..299fd3dc74cb4 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -25,8 +25,8 @@ import java.util.Collection; import java.util.List; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet; +import org.apache.pulsar.common.util.collections.OpenLongPairRangeSet; /** * Wraps other Range classes, and adds LRU, marking dirty data and other features on this basis. @@ -55,7 +55,7 @@ public RangeSetWrapper(LongPairConsumer rangeConverter, this.config = managedCursor.getManagedLedger().getConfig(); this.rangeConverter = rangeConverter; this.rangeSet = config.isUnackedRangesOpenCacheSetEnabled() - ? new ConcurrentOpenLongPairRangeSet<>(4096, rangeConverter) + ? new OpenLongPairRangeSet<>(4096, rangeConverter) : new LongPairRangeSet.DefaultRangeSet<>(rangeConverter, rangeBoundConsumer); this.enableMultiEntry = config.isPersistentUnackedRangesWithMultipleEntriesEnabled(); } @@ -148,16 +148,16 @@ public int cardinality(long lowerKey, long lowerValue, long upperKey, long upper @VisibleForTesting void add(Range range) { - if (!(rangeSet instanceof ConcurrentOpenLongPairRangeSet)) { + if (!(rangeSet instanceof OpenLongPairRangeSet)) { throw new UnsupportedOperationException("Only ConcurrentOpenLongPairRangeSet support this method"); } - ((ConcurrentOpenLongPairRangeSet) rangeSet).add(range); + ((OpenLongPairRangeSet) rangeSet).add(range); } @VisibleForTesting void remove(Range range) { - if (rangeSet instanceof ConcurrentOpenLongPairRangeSet) { - ((ConcurrentOpenLongPairRangeSet) rangeSet).remove((Range) range); + if (rangeSet instanceof OpenLongPairRangeSet) { + ((OpenLongPairRangeSet) rangeSet).remove((Range) range); } else { ((DefaultRangeSet) rangeSet).remove(range); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java similarity index 97% rename from pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java rename to pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java index 72215d7296cc3..c053c106be206 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java @@ -28,6 +28,7 @@ import java.util.NavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; +import javax.annotation.concurrent.NotThreadSafe; import org.apache.commons.lang.mutable.MutableInt; /** @@ -41,7 +42,8 @@ * So, this rangeSet is not suitable for large number of unique keys. * */ -public class ConcurrentOpenLongPairRangeSet> implements LongPairRangeSet { +@NotThreadSafe +public class OpenLongPairRangeSet> implements LongPairRangeSet { protected final NavigableMap rangeBitSetMap = new ConcurrentSkipListMap<>(); private boolean threadSafe = true; @@ -54,15 +56,15 @@ public class ConcurrentOpenLongPairRangeSet> implements private volatile boolean updatedAfterCachedForSize = true; private volatile boolean updatedAfterCachedForToString = true; - public ConcurrentOpenLongPairRangeSet(LongPairConsumer consumer) { + public OpenLongPairRangeSet(LongPairConsumer consumer) { this(1024, true, consumer); } - public ConcurrentOpenLongPairRangeSet(int size, LongPairConsumer consumer) { + public OpenLongPairRangeSet(int size, LongPairConsumer consumer) { this(size, true, consumer); } - public ConcurrentOpenLongPairRangeSet(int size, boolean threadSafe, LongPairConsumer consumer) { + public OpenLongPairRangeSet(int size, boolean threadSafe, LongPairConsumer consumer) { this.threadSafe = threadSafe; this.bitSetSize = size; this.consumer = consumer; diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/DefaultRangeSetTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/DefaultRangeSetTest.java index f6103061a420c..730f4b4ceca22 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/DefaultRangeSetTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/DefaultRangeSetTest.java @@ -34,8 +34,8 @@ public class DefaultRangeSetTest { public void testBehavior() { LongPairRangeSet.DefaultRangeSet set = new LongPairRangeSet.DefaultRangeSet<>(consumer, reverseConsumer); - ConcurrentOpenLongPairRangeSet rangeSet = - new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet rangeSet = + new OpenLongPairRangeSet<>(consumer); assertNull(set.firstRange()); assertNull(set.lastRange()); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSetTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSetTest.java similarity index 92% rename from pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSetTest.java rename to pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSetTest.java index 40bb337935742..4dd0f5551f1f9 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSetTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSetTest.java @@ -37,14 +37,14 @@ import com.google.common.collect.Range; import com.google.common.collect.TreeRangeSet; -public class ConcurrentOpenLongPairRangeSetTest { +public class OpenLongPairRangeSetTest { static final LongPairConsumer consumer = LongPair::new; static final RangeBoundConsumer reverseConsumer = pair -> pair; @Test public void testIsEmpty() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); assertTrue(set.isEmpty()); // lowerValueOpen and upperValue are both -1 so that an empty set will be added set.addOpenClosed(0, -1, 0, -1); @@ -55,7 +55,7 @@ public void testIsEmpty() { @Test public void testAddForSameKey() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); // add 0 to 5 set.add(Range.closed(new LongPair(0, 0), new LongPair(0, 5))); // add 8,9,10 @@ -76,7 +76,7 @@ public void testAddForSameKey() { @Test public void testAddForDifferentKey() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); // [98,100],[(1,5),(1,5)],[(1,10,1,15)],[(1,20),(1,20)],[(2,0),(2,10)] set.addOpenClosed(0, 98, 0, 99); set.addOpenClosed(0, 100, 1, 5); @@ -93,7 +93,7 @@ public void testAddForDifferentKey() { @Test public void testAddCompareCompareWithGuava() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); // add 10K values for key 0 @@ -132,14 +132,14 @@ public void testAddCompareCompareWithGuava() { @Test public void testNPE() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); assertNull(set.span()); } @Test public void testDeleteCompareWithGuava() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); // add 10K values for key 0 @@ -193,7 +193,7 @@ public void testDeleteCompareWithGuava() { @Test public void testRemoveRangeInSameKey() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); set.addOpenClosed(0, 1, 0, 50); set.addOpenClosed(0, 97, 0, 99); set.addOpenClosed(0, 99, 1, 5); @@ -217,7 +217,7 @@ public void testRemoveRangeInSameKey() { @Test public void testSpanWithGuava() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); set.add(Range.openClosed(new LongPair(0, 97), new LongPair(0, 99))); gSet.add(Range.openClosed(new LongPair(0, 97), new LongPair(0, 99))); @@ -242,7 +242,7 @@ public void testSpanWithGuava() { @Test public void testFirstRange() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); assertNull(set.firstRange()); Range range = Range.openClosed(new LongPair(0, 97), new LongPair(0, 99)); set.add(range); @@ -260,7 +260,7 @@ public void testFirstRange() { @Test public void testLastRange() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); assertNull(set.lastRange()); Range range = Range.openClosed(new LongPair(0, 97), new LongPair(0, 99)); set.add(range); @@ -282,7 +282,7 @@ public void testLastRange() { @Test public void testToString() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); Range range = Range.openClosed(new LongPair(0, 97), new LongPair(0, 99)); set.add(range); assertEquals(set.toString(), "[(0:97..0:99]]"); @@ -296,7 +296,7 @@ public void testToString() { @Test public void testDeleteForDifferentKey() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); set.addOpenClosed(0, 97, 0, 99); set.addOpenClosed(0, 99, 1, 5); set.addOpenClosed(1, 9, 1, 15); @@ -327,7 +327,7 @@ public void testDeleteForDifferentKey() { @Test public void testDeleteWithAtMost() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); set.add(Range.closed(new LongPair(0, 98), new LongPair(0, 99))); set.add(Range.closed(new LongPair(0, 100), new LongPair(1, 5))); set.add(Range.closed(new LongPair(1, 10), new LongPair(1, 15))); @@ -353,7 +353,7 @@ public void testDeleteWithAtMost() { @Test public void testDeleteWithLeastMost() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); set.add(Range.closed(new LongPair(0, 98), new LongPair(0, 99))); set.add(Range.closed(new LongPair(0, 100), new LongPair(1, 5))); set.add(Range.closed(new LongPair(1, 10), new LongPair(1, 15))); @@ -382,7 +382,7 @@ public void testDeleteWithLeastMost() { @Test public void testRangeContaining() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); set.add(Range.closed(new LongPair(0, 98), new LongPair(0, 99))); set.add(Range.closed(new LongPair(0, 100), new LongPair(1, 5))); com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); @@ -423,7 +423,7 @@ public void testRangeContaining() { */ @Test public void testCacheFlagConflict() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); set.add(Range.openClosed(new LongPair(0, 1), new LongPair(0, 2))); set.add(Range.openClosed(new LongPair(0, 3), new LongPair(0, 4))); assertEquals(set.toString(), "[(0:1..0:2],(0:3..0:4]]"); @@ -466,7 +466,7 @@ private List> getConnectedRange(Set> gRanges) { @Test public void testCardinality() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); int v = set.cardinality(0, 0, Integer.MAX_VALUE, Integer.MAX_VALUE); assertEquals(v, 0 ); set.addOpenClosed(1, 0, 1, 20); @@ -486,8 +486,8 @@ public void testCardinality() { @Test public void testForEachResultTheSameAsForEachWithRangeBoundMapper() { - ConcurrentOpenLongPairRangeSet set = - new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = + new OpenLongPairRangeSet<>(consumer); LongPairRangeSet.DefaultRangeSet defaultRangeSet = new LongPairRangeSet.DefaultRangeSet<>(consumer, reverseConsumer); From e4390d357ea5c811fdb43da13e10471bf4830548 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 3 Jul 2024 11:00:47 -0700 Subject: [PATCH 354/580] [fix][test] Update OpenTelemetry receiver endpoint in integration test (#22998) --- .../src/test/resources/containers/otel-collector-config.yaml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/src/test/resources/containers/otel-collector-config.yaml b/tests/integration/src/test/resources/containers/otel-collector-config.yaml index bd332f0428307..2ba532f3c6cba 100644 --- a/tests/integration/src/test/resources/containers/otel-collector-config.yaml +++ b/tests/integration/src/test/resources/containers/otel-collector-config.yaml @@ -21,6 +21,7 @@ receivers: otlp: protocols: grpc: + endpoint: 0.0.0.0:4317 exporters: prometheus: From f4d1d05ee385bd730cbb4fa09a287614a00400a3 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 3 Jul 2024 12:25:39 -0700 Subject: [PATCH 355/580] [feat][broker] PIP-264: Add OpenTelemetry managed ledger metrics (#22987) --- .../bookkeeper/mledger/ManagedLedger.java | 7 + .../mledger/ManagedLedgerAttributes.java | 57 +++++++ .../mledger/ManagedLedgerMXBean.java | 35 ++++ .../impl/ManagedLedgerFactoryImpl.java | 3 + .../mledger/impl/ManagedLedgerImpl.java | 6 + .../mledger/impl/ManagedLedgerMBeanImpl.java | 35 ++++ .../impl/OpenTelemetryManagedLedgerStats.java | 153 ++++++++++++++++++ .../stats/ManagedLedgerMetricsTest.java | 100 +++++++++++- .../OpenTelemetryAttributes.java | 17 ++ 9 files changed, 406 insertions(+), 7 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerAttributes.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedLedgerStats.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java index 955a0d7850275..a9242d5cc65b4 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java @@ -696,4 +696,11 @@ default void skipNonRecoverableLedger(long ledgerId){} * Check if managed ledger should cache backlog reads. */ void checkCursorsToCacheEntries(); + + /** + * Get managed ledger attributes. + */ + default ManagedLedgerAttributes getManagedLedgerAttributes() { + return new ManagedLedgerAttributes(this); + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerAttributes.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerAttributes.java new file mode 100644 index 0000000000000..c3759a533a571 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerAttributes.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.bookkeeper.mledger; + +import io.opentelemetry.api.common.Attributes; +import lombok.Getter; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.ManagedLedgerOperationStatus; + +@Getter +public class ManagedLedgerAttributes { + + private final Attributes attributes; + private final Attributes attributesOperationSucceed; + private final Attributes attributesOperationFailure; + + public ManagedLedgerAttributes(ManagedLedger ml) { + var mlName = ml.getName(); + attributes = Attributes.of( + OpenTelemetryAttributes.ML_NAME, mlName, + OpenTelemetryAttributes.PULSAR_NAMESPACE, getNamespace(mlName) + ); + attributesOperationSucceed = Attributes.builder() + .putAll(attributes) + .putAll(ManagedLedgerOperationStatus.SUCCESS.attributes) + .build(); + attributesOperationFailure = Attributes.builder() + .putAll(attributes) + .putAll(ManagedLedgerOperationStatus.FAILURE.attributes) + .build(); + } + + private static String getNamespace(String mlName) { + try { + return TopicName.get(TopicName.fromPersistenceNamingEncoding(mlName)).getNamespace(); + } catch (RuntimeException e) { + return null; + } + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java index 44345c430b7fb..1d978e2378569 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerMXBean.java @@ -60,11 +60,21 @@ public interface ManagedLedgerMXBean { */ double getAddEntryBytesRate(); + /** + * @return the total number of bytes written + */ + long getAddEntryBytesTotal(); + /** * @return the bytes/s rate of messages added with replicas */ double getAddEntryWithReplicasBytesRate(); + /** + * @return the total number of bytes written, including replicas + */ + long getAddEntryWithReplicasBytesTotal(); + /** * @return the msg/s rate of messages read */ @@ -75,11 +85,21 @@ public interface ManagedLedgerMXBean { */ double getReadEntriesBytesRate(); + /** + * @return the total number of bytes read + */ + long getReadEntriesBytesTotal(); + /** * @return the rate of mark-delete ops/s */ double getMarkDeleteRate(); + /** + * @return the number of mark-delete ops + */ + long getMarkDeleteTotal(); + /** * @return the number of addEntry requests that succeeded */ @@ -95,6 +115,11 @@ public interface ManagedLedgerMXBean { */ long getAddEntryErrors(); + /** + * @return the total number of addEntry requests that failed + */ + long getAddEntryErrorsTotal(); + /** * @return the number of entries read from the managed ledger (from cache or BK) */ @@ -115,11 +140,21 @@ public interface ManagedLedgerMXBean { */ long getReadEntriesErrors(); + /** + * @return the total number of readEntries requests that failed + */ + long getReadEntriesErrorsTotal(); + /** * @return the number of readEntries requests that cache miss Rate */ double getReadEntriesOpsCacheMissesRate(); + /** + * @return the total number of readEntries requests that cache miss + */ + long getReadEntriesOpsCacheMissesTotal(); + // Entry size statistics double getEntrySizeAverage(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index fc291b801c896..b1939f40e9358 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -121,6 +121,7 @@ public class ManagedLedgerFactoryImpl implements ManagedLedgerFactory { private final MetadataStore metadataStore; private final OpenTelemetryManagedLedgerCacheStats openTelemetryCacheStats; + private final OpenTelemetryManagedLedgerStats openTelemetryManagedLedgerStats; //indicate whether shutdown() is called. private volatile boolean closed; @@ -229,6 +230,7 @@ private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, metadataStore.registerSessionListener(this::handleMetadataStoreNotification); openTelemetryCacheStats = new OpenTelemetryManagedLedgerCacheStats(openTelemetry, this); + openTelemetryManagedLedgerStats = new OpenTelemetryManagedLedgerStats(openTelemetry, this); } static class DefaultBkFactory implements BookkeeperFactoryForCustomEnsemblePlacementPolicy { @@ -620,6 +622,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { })); }).thenAcceptAsync(__ -> { //wait for tasks in scheduledExecutor executed. + openTelemetryManagedLedgerStats.close(); openTelemetryCacheStats.close(); scheduledExecutor.shutdownNow(); entryCacheManager.clear(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 8d1919dd0529c..b7734906f7553 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -96,6 +96,7 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerAttributes; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerException.BadVersionException; @@ -326,6 +327,9 @@ public enum PositionBound { */ final ConcurrentLinkedQueue pendingAddEntries = new ConcurrentLinkedQueue<>(); + @Getter + private final ManagedLedgerAttributes managedLedgerAttributes; + /** * This variable is used for testing the tests. * ManagedLedgerTest#testManagedLedgerWithPlacementPolicyInCustomMetadata() @@ -338,6 +342,7 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper final String name) { this(factory, bookKeeper, store, config, scheduledExecutor, name, null); } + public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, final String name, final Supplier> mlOwnershipChecker) { @@ -373,6 +378,7 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper this.minBacklogCursorsForCaching = config.getMinimumBacklogCursorsForCaching(); this.minBacklogEntriesForCaching = config.getMinimumBacklogEntriesForCaching(); this.maxBacklogBetweenCursorsForCaching = config.getMaxBacklogBetweenCursorsForCaching(); + this.managedLedgerAttributes = new ManagedLedgerAttributes(this); } synchronized void initialize(final ManagedLedgerInitializeLedgerCallback callback, final Object ctx) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java index 5e5161a29ca79..86320f9292468 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java @@ -210,11 +210,21 @@ public double getAddEntryBytesRate() { return addEntryOps.getValueRate(); } + @Override + public long getAddEntryBytesTotal() { + return addEntryOps.getTotalValue(); + } + @Override public double getAddEntryWithReplicasBytesRate() { return addEntryWithReplicasOps.getValueRate(); } + @Override + public long getAddEntryWithReplicasBytesTotal() { + return addEntryWithReplicasOps.getTotalValue(); + } + @Override public double getReadEntriesRate() { return readEntriesOps.getRate(); @@ -225,6 +235,11 @@ public double getReadEntriesBytesRate() { return readEntriesOps.getValueRate(); } + @Override + public long getReadEntriesBytesTotal() { + return readEntriesOps.getTotalValue(); + } + @Override public long getAddEntrySucceed() { return addEntryOps.getCount(); @@ -240,6 +255,11 @@ public long getAddEntryErrors() { return addEntryOpsFailed.getCount(); } + @Override + public long getAddEntryErrorsTotal() { + return addEntryOpsFailed.getTotalCount(); + } + @Override public long getReadEntriesSucceeded() { return readEntriesOps.getCount(); @@ -255,16 +275,31 @@ public long getReadEntriesErrors() { return readEntriesOpsFailed.getCount(); } + @Override + public long getReadEntriesErrorsTotal() { + return readEntriesOpsFailed.getTotalCount(); + } + @Override public double getReadEntriesOpsCacheMissesRate() { return readEntriesOpsCacheMisses.getRate(); } + @Override + public long getReadEntriesOpsCacheMissesTotal() { + return readEntriesOpsCacheMisses.getTotalCount(); + } + @Override public double getMarkDeleteRate() { return markDeleteOps.getRate(); } + @Override + public long getMarkDeleteTotal() { + return markDeleteOps.getTotalCount(); + } + @Override public double getEntrySizeAverage() { return entryStats.getAvg(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedLedgerStats.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedLedgerStats.java new file mode 100644 index 0000000000000..f7b9d91dff6ad --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedLedgerStats.java @@ -0,0 +1,153 @@ +/* + * 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.bookkeeper.mledger.impl; + +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.metrics.BatchCallback; +import io.opentelemetry.api.metrics.ObservableLongMeasurement; +import org.apache.pulsar.opentelemetry.Constants; + +public class OpenTelemetryManagedLedgerStats implements AutoCloseable { + + // Replaces pulsar_ml_AddEntryMessagesRate + public static final String ADD_ENTRY_COUNTER = "pulsar.broker.managed_ledger.message.outgoing.count"; + private final ObservableLongMeasurement addEntryCounter; + + // Replaces pulsar_ml_AddEntryBytesRate + public static final String BYTES_OUT_COUNTER = "pulsar.broker.managed_ledger.message.outgoing.logical.size"; + private final ObservableLongMeasurement bytesOutCounter; + + // Replaces pulsar_ml_AddEntryWithReplicasBytesRate + public static final String BYTES_OUT_WITH_REPLICAS_COUNTER = + "pulsar.broker.managed_ledger.message.outgoing.replicated.size"; + private final ObservableLongMeasurement bytesOutWithReplicasCounter; + + // Replaces pulsar_ml_NumberOfMessagesInBacklog + public static final String BACKLOG_COUNTER = "pulsar.broker.managed_ledger.backlog.count"; + private final ObservableLongMeasurement backlogCounter; + + // Replaces pulsar_ml_ReadEntriesRate + public static final String READ_ENTRY_COUNTER = "pulsar.broker.managed_ledger.message.incoming.count"; + private final ObservableLongMeasurement readEntryCounter; + + // Replaces pulsar_ml_ReadEntriesBytesRate + public static final String BYTES_IN_COUNTER = "pulsar.broker.managed_ledger.message.incoming.size"; + private final ObservableLongMeasurement bytesInCounter; + + // Replaces brk_ml_ReadEntriesOpsCacheMissesRate + public static final String READ_ENTRY_CACHE_MISS_COUNTER = + "pulsar.broker.managed_ledger.message.incoming.cache.miss.count"; + private final ObservableLongMeasurement readEntryCacheMissCounter; + + // Replaces pulsar_ml_MarkDeleteRate + public static final String MARK_DELETE_COUNTER = "pulsar.broker.managed_ledger.mark_delete.count"; + private final ObservableLongMeasurement markDeleteCounter; + + private final BatchCallback batchCallback; + + public OpenTelemetryManagedLedgerStats(OpenTelemetry openTelemetry, ManagedLedgerFactoryImpl factory) { + var meter = openTelemetry.getMeter(Constants.BROKER_INSTRUMENTATION_SCOPE_NAME); + + addEntryCounter = meter + .upDownCounterBuilder(ADD_ENTRY_COUNTER) + .setUnit("{operation}") + .setDescription("The number of write operations to this ledger.") + .buildObserver(); + + bytesOutCounter = meter + .counterBuilder(BYTES_OUT_COUNTER) + .setUnit("By") + .setDescription("The total number of messages bytes written to this ledger, excluding replicas.") + .buildObserver(); + + bytesOutWithReplicasCounter = meter + .counterBuilder(BYTES_OUT_WITH_REPLICAS_COUNTER) + .setUnit("By") + .setDescription("The total number of messages bytes written to this ledger, including replicas.") + .buildObserver(); + + backlogCounter = meter + .upDownCounterBuilder(BACKLOG_COUNTER) + .setUnit("{message}") + .setDescription("The number of messages in backlog for all consumers from this ledger.") + .buildObserver(); + + readEntryCounter = meter + .upDownCounterBuilder(READ_ENTRY_COUNTER) + .setUnit("{operation}") + .setDescription("The number of read operations from this ledger.") + .buildObserver(); + + bytesInCounter = meter + .counterBuilder(BYTES_IN_COUNTER) + .setUnit("By") + .setDescription("The total number of messages bytes read from this ledger.") + .buildObserver(); + + readEntryCacheMissCounter = meter + .upDownCounterBuilder(READ_ENTRY_CACHE_MISS_COUNTER) + .setUnit("{operation}") + .setDescription("The number of cache misses during read operations from this ledger.") + .buildObserver(); + + markDeleteCounter = meter + .counterBuilder(MARK_DELETE_COUNTER) + .setUnit("{operation}") + .setDescription("The total number of mark delete operations for this ledger.") + .buildObserver(); + + batchCallback = meter.batchCallback(() -> factory.getManagedLedgers() + .values() + .forEach(this::recordMetrics), + addEntryCounter, + bytesOutCounter, + bytesOutWithReplicasCounter, + backlogCounter, + readEntryCounter, + bytesInCounter, + readEntryCacheMissCounter, + markDeleteCounter); + } + + @Override + public void close() { + batchCallback.close(); + } + + private void recordMetrics(ManagedLedgerImpl ml) { + var stats = ml.getMbean(); + var ledgerAttributeSet = ml.getManagedLedgerAttributes(); + var attributes = ledgerAttributeSet.getAttributes(); + var attributesSucceed = ledgerAttributeSet.getAttributesOperationSucceed(); + var attributesFailure = ledgerAttributeSet.getAttributesOperationFailure(); + + addEntryCounter.record(stats.getAddEntrySucceedTotal(), attributesSucceed); + addEntryCounter.record(stats.getAddEntryErrorsTotal(), attributesFailure); + bytesOutCounter.record(stats.getAddEntryBytesTotal(), attributes); + bytesOutWithReplicasCounter.record(stats.getAddEntryWithReplicasBytesTotal(), attributes); + + readEntryCounter.record(stats.getReadEntriesSucceededTotal(), attributesSucceed); + readEntryCounter.record(stats.getReadEntriesErrorsTotal(), attributesFailure); + bytesInCounter.record(stats.getReadEntriesBytesTotal(), attributes); + + backlogCounter.record(stats.getNumberOfMessagesInBacklog(), attributes); + markDeleteCounter.record(stats.getMarkDeleteTotal(), attributes); + readEntryCacheMissCounter.record(stats.getReadEntriesOpsCacheMissesTotal(), attributes); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java index bec73121e487a..b9c0ab08e4ea1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java @@ -18,27 +18,38 @@ */ package org.apache.pulsar.broker.stats; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.apache.pulsar.transaction.coordinator.impl.DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS; +import static org.assertj.core.api.Assertions.assertThat; +import com.google.common.collect.Sets; import io.netty.util.HashedWheelTimer; import io.netty.util.concurrent.DefaultThreadFactory; +import io.opentelemetry.api.common.Attributes; import java.util.List; import java.util.Map.Entry; import java.util.concurrent.TimeUnit; -import com.google.common.collect.Sets; +import lombok.Cleanup; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerMBeanImpl; +import org.apache.bookkeeper.mledger.impl.OpenTelemetryManagedLedgerStats; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.stats.metrics.ManagedLedgerMetrics; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.stats.Metrics; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.transaction.coordinator.impl.MLTransactionLogImpl; import org.apache.pulsar.transaction.coordinator.impl.TxnLogBufferedWriterConfig; +import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -68,6 +79,12 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder pulsarTestContextBuilder) { + super.customizeMainPulsarTestContextBuilder(pulsarTestContextBuilder); + pulsarTestContextBuilder.enableOpenTelemetry(true); + } + @Test public void testManagedLedgerMetrics() throws Exception { ManagedLedgerMetrics metrics = new ManagedLedgerMetrics(pulsar); @@ -76,15 +93,20 @@ public void testManagedLedgerMetrics() throws Exception { List list1 = metrics.generate(); Assert.assertTrue(list1.isEmpty()); - Producer producer = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic1") - .create(); + var topicName = "persistent://my-property/use/my-ns/my-topic1"; + @Cleanup + Producer producer = pulsarClient.newProducer().topic(topicName).create(); + + @Cleanup + var consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("sub1").subscribe(); + for (int i = 0; i < 10; i++) { String message = "my-message-" + i; producer.send(message.getBytes()); } - for (Entry ledger : ((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()) - .getManagedLedgers().entrySet()) { + var managedLedgerFactory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory(); + for (Entry ledger : managedLedgerFactory.getManagedLedgers().entrySet()) { ManagedLedgerMBeanImpl stats = (ManagedLedgerMBeanImpl) ledger.getValue().getStats(); stats.refreshStats(1, TimeUnit.SECONDS); } @@ -96,14 +118,78 @@ public void testManagedLedgerMetrics() throws Exception { String message = "my-message-" + i; producer.send(message.getBytes()); } - for (Entry ledger : ((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()) - .getManagedLedgers().entrySet()) { + for (Entry ledger : managedLedgerFactory.getManagedLedgers().entrySet()) { ManagedLedgerMBeanImpl stats = (ManagedLedgerMBeanImpl) ledger.getValue().getStats(); stats.refreshStats(1, TimeUnit.SECONDS); } List list3 = metrics.generate(); Assert.assertEquals(list3.get(0).getMetrics().get(addEntryRateKey), 5.0D); + // Validate OpenTelemetry metrics. + var ledgers = managedLedgerFactory.getManagedLedgers(); + var topicNameObj = TopicName.get(topicName); + var mlName = topicNameObj.getPersistenceNamingEncoding(); + assertThat(ledgers).containsKey(mlName); + var ml = ledgers.get(mlName); + var attribCommon = Attributes.of( + OpenTelemetryAttributes.ML_NAME, mlName, + OpenTelemetryAttributes.PULSAR_NAMESPACE, topicNameObj.getNamespace() + ); + var metricReader = pulsarTestContext.getOpenTelemetryMetricReader(); + + Awaitility.await().untilAsserted(() -> { + var otelMetrics = metricReader.collectAllMetrics(); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedLedgerStats.BACKLOG_COUNTER, attribCommon, 15); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedLedgerStats.MARK_DELETE_COUNTER, attribCommon, 0); + }); + + for (int i = 0; i < 10; i++) { + var msg = consumer.receive(1, TimeUnit.SECONDS); + consumer.acknowledge(msg); + } + + Awaitility.await().untilAsserted(() -> { + var otelMetrics = metricReader.collectAllMetrics(); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedLedgerStats.BACKLOG_COUNTER, attribCommon, 5); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedLedgerStats.MARK_DELETE_COUNTER, attribCommon, + value -> assertThat(value).isPositive()); + }); + + Awaitility.await().untilAsserted(() -> { + @Cleanup + var cons = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionName(BrokerTestUtil.newUniqueName("sub")) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + cons.receive(1, TimeUnit.SECONDS); + + var attribSucceed = Attributes.of( + OpenTelemetryAttributes.ML_NAME, mlName, + OpenTelemetryAttributes.PULSAR_NAMESPACE, topicNameObj.getNamespace(), + OpenTelemetryAttributes.ML_OPERATION_STATUS, "success" + ); + var attribFailed = Attributes.of( + OpenTelemetryAttributes.ML_NAME, mlName, + OpenTelemetryAttributes.PULSAR_NAMESPACE, topicNameObj.getNamespace(), + OpenTelemetryAttributes.ML_OPERATION_STATUS, "failure" + ); + var otelMetrics = metricReader.collectAllMetrics(); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedLedgerStats.ADD_ENTRY_COUNTER, attribSucceed, 15); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedLedgerStats.ADD_ENTRY_COUNTER, attribFailed, 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedLedgerStats.BYTES_OUT_COUNTER, attribCommon, + value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedLedgerStats.BYTES_OUT_WITH_REPLICAS_COUNTER, + attribCommon, value -> assertThat(value).isPositive()); + + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedLedgerStats.READ_ENTRY_COUNTER, attribSucceed, + value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedLedgerStats.READ_ENTRY_COUNTER, attribFailed, 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedLedgerStats.BYTES_IN_COUNTER, attribCommon, + value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedLedgerStats.READ_ENTRY_CACHE_MISS_COUNTER, + attribCommon, value -> assertThat(value).isPositive()); + }); } @Test diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index f485e30092604..24dd1be8509bf 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -196,6 +196,23 @@ enum ConnectionCreateStatus { public final Attributes attributes = Attributes.of(PULSAR_CONNECTION_CREATE_STATUS, name().toLowerCase()); } + // Managed Ledger Attributes + + /** + * The name of the managed ledger. + */ + AttributeKey ML_NAME = AttributeKey.stringKey("pulsar.managed_ledger.name"); + + /** + * The status of the managed ledger operation. + */ + AttributeKey ML_OPERATION_STATUS = AttributeKey.stringKey("pulsar.managed_ledger.operation.status"); + enum ManagedLedgerOperationStatus { + SUCCESS, + FAILURE; + public final Attributes attributes = Attributes.of(ML_OPERATION_STATUS, name().toLowerCase()); + }; + /** * The type of the pool arena. */ From deb26f7662268def7f838f722de4a677b3d546ed Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 4 Jul 2024 07:02:26 +0800 Subject: [PATCH 356/580] [fix][broker] Can't connecte to non-persist topic when enable broker client tls (#22991) --- .../broker/namespace/NamespaceService.java | 10 ++++++- .../TokenExpirationProduceConsumerTest.java | 27 +++++++++++++++++++ 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index dfd03dfbc6e43..2a1584df961f7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -1471,7 +1471,15 @@ public CompletableFuture checkNonPersistentNonPartitionedTopicExists(St return FutureUtil.failedFuture(new ServiceUnitNotReadyException( "No broker was available to own " + topicName)); } - return pulsarClient.getLookup(lookupResult.get().getLookupData().getBrokerUrl()) + LookupData lookupData = lookupResult.get().getLookupData(); + String brokerUrl; + if (pulsar.getConfiguration().isBrokerClientTlsEnabled() + && StringUtils.isNotEmpty(lookupData.getBrokerUrlTls())) { + brokerUrl = lookupData.getBrokerUrlTls(); + } else { + brokerUrl = lookupData.getBrokerUrl(); + } + return pulsarClient.getLookup(brokerUrl) .getPartitionedTopicMetadata(topicName, false) .thenApply(metadata -> true) .exceptionallyCompose(ex -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenExpirationProduceConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenExpirationProduceConsumerTest.java index fa9099f3d2f50..d8ed105572033 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenExpirationProduceConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenExpirationProduceConsumerTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.client.api; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; import com.google.common.collect.Sets; @@ -32,6 +34,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import javax.crypto.SecretKey; +import java.nio.charset.StandardCharsets; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; @@ -107,6 +110,7 @@ protected void internalSetUpForBroker() { conf.setAuthenticationProviders(Sets.newHashSet(AuthenticationProviderToken.class.getName())); conf.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); conf.setBrokerClientAuthenticationParameters("token:" + ADMIN_TOKEN); + conf.setBrokerClientTlsEnabled(true); conf.getProperties().setProperty("tokenSecretKey", "data:;base64," + Base64.getEncoder().encodeToString(SECRET_KEY.getEncoded())); } @@ -132,6 +136,29 @@ private PulsarAdmin getAdmin(String token) throws Exception { return clientBuilder.build(); } + @Test + public void testNonPersistentTopic() throws Exception { + + @Cleanup + PulsarClient pulsarClient = getClient(ADMIN_TOKEN); + + String topic = "non-persistent://" + namespaceName + "/test-token-non-persistent"; + + @Cleanup + Consumer consumer = pulsarClient.newConsumer().topic(topic) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName("test").subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer().topic(topic).create(); + byte[] msg = "Hello".getBytes(StandardCharsets.UTF_8); + producer.send(msg); + + Message receive = consumer.receive(3, TimeUnit.SECONDS); + assertNotNull(receive); + assertEquals(receive.getData(), msg); + } + @Test public void testTokenExpirationProduceConsumer() throws Exception { Calendar calendar = Calendar.getInstance(); From 2086cc46c882df7fb2855a3cdb2580e1bc3adc5b Mon Sep 17 00:00:00 2001 From: Apurva007 Date: Thu, 4 Jul 2024 00:22:19 -0700 Subject: [PATCH 357/580] [improve][pip] PIP-337: SSL Factory Plugin to customize SSL Context and SSL Engine generation (#22016) Co-authored-by: Apurva Telang --- pip/pip-337.md | 382 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 382 insertions(+) create mode 100644 pip/pip-337.md diff --git a/pip/pip-337.md b/pip/pip-337.md new file mode 100644 index 0000000000000..283bb9710de84 --- /dev/null +++ b/pip/pip-337.md @@ -0,0 +1,382 @@ +# PIP-337: SSL Factory Plugin to customize SSLContext/SSLEngine generation + +# Background knowledge +Apache Pulsar supports TLS encrypted communication between the clients and servers. The TLS encryption setup requires +loading the TLS certificates and its respective passwords to generate the SSL Context. Pulsar supports loading these +certificates and passwords via the filesystem. It supports both Java based Keystores/Truststores and TLS information in +".crt", ".pem" & ".key" formats. This information is refreshed based on a configurable interval. + +Apache Pulsar internally uses 3 different frameworks for connection management: + +- Netty: Connection management for Pulsar server and client that understands Pulsar binary protocol. +- Jetty: HTTP Server creation for Pulsar Admin and websocket. Jetty Client is used by proxy for admin client calls. +- AsyncHttpClient: HTTP Client creation for Admin client and HTTP Lookup + +Each of the above frameworks supports customizing the generation of the SSL Context and SSL Engine. Currently, Pulsar +uses these features to feed the SSL Context via its internal security tools after loading the file based certificates. +One of the issues of using these features is that pulsar tries to bootstrap the SSL Context in multiple ways to suit +each framework and file type. + +```mermaid +flowchart TB + Proxy.DirectProxyHandler --> NettyClientSslContextRefresher + Proxy.DirectProxyHandler --> NettySSLContextAutoRefreshBuilder + Proxy.AdminProxyHandler --> KeyStoreSSLContext + Proxy.AdminProxyHandler --> SecurityUtility + Proxy.ServiceChannelInitializer --> NettySSLContextAutoRefreshBuilder + Proxy.ServiceChannelInitializer --> NettyServerSslContextBuilder + Broker.PulsarChannelInitializer --> NettyServerSslContextBuilder + Broker.PulsarChannelInitializer --> NettySSLContextAutoRefreshBuilder + Client.PulsarChannelInitializer --> NettySSLContextAutoRefreshBuilder + Client.PulsarChannelInitializer --> SecurityUtility + Broker.WebService --> JettySSlContextFactory + Proxy.WebServer --> JettySSlContextFactory + PulsarAdmin --> AsyncHttpConnector + AsyncHttpConnector --> KeyStoreSSLContext + AsyncHttpConnector --> SecurityUtility + JettySSlContextFactory --> NetSslContextBuilder + JettySSlContextFactory --> DefaultSslContextBuilder + NettyClientSslContextRefresher -.-> SslContextAutoRefreshBuilder + NettySSLContextAutoRefreshBuilder -.-> SslContextAutoRefreshBuilder + NettyServerSslContextBuilder -.-> SslContextAutoRefreshBuilder + NetSslContextBuilder -.-> SslContextAutoRefreshBuilder + DefaultSslContextBuilder -.-> SslContextAutoRefreshBuilder + Client.HttpLookup.HttpClient --> KeyStoreSSLContext + Client.HttpLookup.HttpClient --> SecurityUtility + SecurityUtility -.-> KeyManagerProxy + SecurityUtility -.-> TrustManagerProxy +``` +The above diagram is an example of the complexity of the TLS encryption setup within Pulsar. The above diagram only +contains the basic components of Pulsar excluding Websockets, Functions, etc. + +Pulsar uses 2 base classes to load the TLS information. + +- `SecurityUtility`: It loads files of type ".crt", ".pem" and ".key" and converts it into SSL Context. This SSL Context +can be of type `io.netty.handler.ssl.SslContext` or `javax.net.ssl.SSLContext` based on the caller. Security Utility +can be used to create SSL Context that internally has KeyManager and Trustmanager proxies that load cert changes +dynamically. +- `KeyStoreSSLContext`: It loads files of type Java Keystore/Truststore and converts it into SSL Context. This SSL +Context will be of type `javax.net.ssl.SSLContext`. This is always used to create the SSL Engine. + +Each of the above classes are either directly used by Pulsar Clients or used via implementations of the abstract class +`SslContextAutoRefreshBuilder`. + +- `SslContextAutoRefreshBuilder` - This abstract class is used to refresh certificates at a configurable interval. It +internally provides a public API to return the SSL Context. + +There are several implementations of the above abstract class to suit the needs of each of the framework and the +respective TLS certificate files: + +- `NettyClientSslContextRefresher` - It internally creates the `io.netty.handler.ssl.SslContext` using the ".crt", +".pem" and ".key" files for the proxy client. +- `NettySSLContextAutoRefreshBuilder` - It internally creates the `KeyStoreSSLContext` using the Java Keystores. +- `NettyServerSslContextBuilder` - It internally creates the `io.netty.handler.ssl.SslContext` using the ".crt", + ".pem" and ".key" files for the server. +- `NetSslContextBuilder` - It internally creates the `javax.net.ssl.SSLContext` using the Java Keystores for the web +server. +- `DefaultSslContextBuilder` - It internally creates the `javax.net.ssl.SSLContext` using the ".crt", ".pem" and ".key" +files for the web server. + +# Motivation +Apache Pulsar's TLS encryption configuration is not pluggable. It only supports file-based certificates. This makes +Pulsar difficult to adopt for organizations that require loading TLS certificates by other mechanisms. + +# Goals +The purpose of this PIP is to introduce the following: + +- Provide a mechanism to plugin a custom SSL Factory that can generate SSL Context and SSL Engine. +- Simplify the Pulsar code base to universally use `javax.net.ssl.SSLContext` and reduce the amount of code required to +build and configure the SSL context taking into consideration backwards compatibility. + +## In Scope + +- Creation of a new interface `PulsarSslFactory` that can generate a SSL Context, Client SSL Engine and Server SSL +Engine. +- Creation of a default implementation of `PulsarSslFactory` that supports loading the SSL Context and SSL Engine via +file-based certificates. Internally it will use the SecurityUtility and KeyStoreSSLContext. +- Creation of a new class called "PulsarSslConfiguration" to store the ssl configuration parameters which will be passed +to the SSL Factory. +- Modify the Pulsar Components to support the `PulsarSslFactory` instead of the SslContextAutoRefreshBuilder, SecurityUtility +and KeyStoreSSLContext. +- Remove the SslContextAutoRefreshBuilder and all its implementations. +- SSL Context refresh will be moved out of the factory. The responsibility of refreshing the ssl context will lie with +the components using the factory. +- The factory will not be thread safe. We are isolating responsibilities by having a single thread perform all writes, +while all channel initializer threads will perform only reads. SSL Context reads can be eventually consistent. +- Each component calling the factory will internally initialize it as part of the constructor as well as create the +ssl context at startup as a blocking call. If this creation/initialization fails then it will cause the Pulsar +Component to shutdown. This is true for all components except the Pulsar client due to past contracts where +authentication provider may not have started before the client. +- Each component will re-use its scheduled executor provider to schedule the refresh of the ssl context based on its +component's certificate refresh configurations. + +# High Level Design +```mermaid +flowchart TB + Proxy.DirectProxyHandler --> PulsarSslFactory + Proxy.AdminProxyHandler --> PulsarSslFactory + Proxy.ServiceChannelInitializer --> PulsarSslFactory + Broker.PulsarChannelInitializer --> PulsarSslFactory + Client.PulsarChannelInitializer --> PulsarSslFactory + Broker.WebService --> JettySSlContextFactory + Proxy.WebServer --> JettySSlContextFactory + PulsarAdmin --> AsyncHttpConnector + AsyncHttpConnector --> PulsarSslFactory + JettySSlContextFactory --> PulsarSslFactory + Client.HttpLookup.HttpClient --> PulsarSslFactory + PulsarSslFactory -.-> DefaultPulsarSslFactory + PulsarSslFactory -.-> CustomPulsarSslFactory +``` + +# Detailed Design + +## Design and Implementation Details + +### Pulsar Common Changes + +A new interface called `PulsarSslFactory` that provides public methods to create a SSL Context, Client SSL Engine and +Server SSL Engine. The SSL Context class returned will be of type `javax.net.ssl.SSLContext`. + +```java +public interface PulsarSslFactory extends AutoCloseable { + /* + * Utilizes the configuration to perform initialization operations and may store information in instance variables. + * @param config PulsarSslConfiguration required by the factory for SSL parameters + */ + void initialize(PulsarSslConfiguration config); + + /* + * Creates a client ssl engine based on the ssl context stored in the instance variable and the respective parameters. + * @param peerHost Name of the peer host + * @param peerPort Port number of the peer + * @return A SSlEngine created using the instance variable stored Ssl Context + */ + SSLEngine createClientSslEngine(String peerHost, int peerPort); + + /* + * Creates a server ssl engine based on the ssl context stored in the instance variable and the respective parameters. + * @return A SSLEngine created using the instance variable stored ssl context + */ + SSLEngine createServerSslEngine(); + + /* + * Returns A boolean stating if the ssl context needs to be updated + * @return Boolean value representing if ssl context needs to be updated + */ + boolean needsUpdate(); + + /* + * Checks if the SSL Context needs to be updated. If true, then a new SSL Context should be internally create and + * should atomically replace the old ssl context stored in the instance variable. + * @throws Exception It can throw an exception if the createInternalSslContext method fails + */ + default void update() throws Exception { + if (this.needsUpdate()) { + this.createInternalSslContext(); + } + } + + /* + * Creates a new SSL Context and internally stores it atomically into an instance variable + * @throws It can throw an exception if the internal ssl context creation fails. + */ + void createInternalSslContext() throws Exception; + + /* + * Returns the internally stored ssl context + * @throws IllegalStateException If the SSL Context has not be created before this call, then it wil throw this + * exception. + */ + SSLContext getInternalSslContext(); + + /* + * Shutdown the factory and close any internal dependencies + * @throws Exception It can throw an exception if there are any issues shutting down the factory. + */ + void close() throws Exception; + +} +``` + +A default implementation of the above SSLFactory class called `DefaultPulsarSslFactory` that will generate the SSL +Context and SSL Engines using File-based Certificates. It will be able to support both Java keystores and "pem/crt/key" +files. + +```java +public class DefaultPulsarSslFactory implements PulsarSslFactory { + public void initialize(PulsarSslConfiguration config); + public SSLEngine createClientSslEngine(String peerHost, int peerPort); + public SSLEngine createServerSslEngine(); + public boolean needsUpdate(); + public void createInternalSslContext() throws Exception; + public SSLContext getInternalSslContext(); + public void close() throws Exception; +} +``` + +### Pulsar Commmon Changes + +4 new configurations will need to be added into the Configurations like `ServiceConfiguration`, +`ClientConfigurationData`, `ProxyConfiguration`, etc. All of the below will be optional. It will use the default values +to match the current behavior of Pulsar. + +- `sslFactoryPlugin`: SSL Factory Plugin class to provide SSLEngine and SSLContext objects. +The default class used is `DefaultPulsarSslFactory`. +- `sslFactoryPluginParams`: SSL Factory plugin configuration parameters. It will be of type string. It can be parsed by +the plugin at its discretion. + +The below configs will be applicable only to the Pulsar Server components like Broker and Proxy: +- `brokerClientSslFactoryPlugin`: SSL Factory Plugin class used by internal client to provide SSLEngine and SSLContext +objects. The default class used is `DefaultPulsarSslFactory`. +- `brokerClientSslFactoryPluginParams`: SSL Factory plugin configuration parameters used by internal client. It can be +parsed by the plugin at its discretion. + +`JettySslContextFactory` class will need to be changed to internally use the `PulsarSslFactory` class to generate the +SslContext. + +### SslFactory Usage across Pulsar Netty based server components + +Example Changes in broker's `PulsarChannelInitializer` to initialize the PulsarSslFactory: +```java +PulsarSslConfiguration pulsarSslConfig = buildSslConfiguration(serviceConfig); +this.sslFactory = (PulsarSslFactory) Class.forName(serviceConfig.getSslFactoryPlugin()) + .getConstructor().newInstance(); +this.sslFactory.initialize(pulsarSslConfig); +this.sslFactory.createInternalSslContext(); +this.pulsar.getExecutor().scheduleWithFixedDelay(this::refreshSslContext, + serviceConfig.getTlsCertRefreshCheckDurationSec(), + serviceConfig.getTlsCertRefreshCheckDurationSec(), + TimeUnit.SECONDS); +``` + +Example changes in `PulsarChannelInitializer` to `initChannel(SocketChannel ch)`: +```java +ch.pipeline().addLast(TLS_HANDLER, new SslHandler(this.sslFactory.createServerSslEngine())); +``` + +The above changes is similar in all the Pulsar Server components that internally utilize Netty. + +### SslFactory Usage across Pulsar Netty based Client components + +Example Changes in Client's `PulsarChannelInitializer` to initialize the SslFactory: +```java +this.pulsarSslFactory = (PulsarSslFactory) Class.forName(conf.getSslFactoryPlugin()) + .getConstructor().newInstance(); +PulsarSslConfiguration sslConfiguration = buildSslConfiguration(conf); +this.pulsarSslFactory.initialize(sslConfiguration); +this.pulsarSslFactory.createInternalSslContext(); +scheduledExecutorProvider.getExecutor()) + .scheduleWithFixedDelay(() -> { + this.refreshSslContext(conf); + }, conf.getAutoCertRefreshSeconds(), + conf.getAutoCertRefreshSeconds(), TimeUnit.SECONDS); +``` + +Example changes in `PulsarChannelInitializer` to `initChannel(SocketChannel ch)`: +```java +SslHandler handler = new SslHandler(sslFactory + .createClientSslEngine(sniHost.getHostName(), sniHost.getPort())); +ch.pipeline().addFirst(TLS_HANDLER, handler); +``` + +The above changes is similar in all the Pulsar client components that internally utilize Netty. + +### SslFactory Usage across Pulsar Jetty Based Server Components + +The initialization of the PulsarSslFactory is similar to the [Netty Server initialization.](#sslfactory-usage-across-pulsar-jetty-based-server-components) + +The usage of the PulsarSslFactory requires changes in the `JettySslContextFactory`. It will internally accept +`PulsarSslFactory` as an input and utilize it to create the SSL Context. +```java +public class JettySslContextFactory { + private static class Server extends SslContextFactory.Server { + private final PulsarSslFactory sslFactory; + + // New + public Server(String sslProviderString, PulsarSslFactory sslFactory, + boolean requireTrustedClientCertOnConnect, Set ciphers, Set protocols) { + this.sslFactory = sslFactory; + // Current implementation + } + + @Override + public SSLContext getSslContext() { + return this.sslFactory.getInternalSslContext(); + } + } +} +``` + +The above `JettySslContextFactory` will be used to create the SSL Context within the Jetty Server. This pattern will be +common across all Web Server created using Jetty within Pulsar. + +### SslFactory Usage across Pulsar AsyncHttpClient based Client Components + +The initialization of the PulsarSslFactory is similar to the [Netty Server initialization.](#sslfactory-usage-across-pulsar-jetty-based-server-components) + +The usage of the PulsarSslFactory requires changes in the `AsyncHttpConnector`. It will internally initialize the +`PulsarSslFactory` and pass it to a new custom `PulsarHttpAsyncSslEngineFactory` that implements `org.asynchttpclient.SSLEngineFactory`. +This new custom class will incorporate the features of the existing `WithSNISslEngineFactory` and `JsseSslEngineFactory` +and replace it. + +```java +public class PulsarHttpAsyncSslEngineFactory extends DefaultSslEngineFactory { + + private final PulsarSslFactory sslFactory; + private final String host; + + public PulsarHttpAsyncSslEngineFactory(PulsarSslFactory sslFactory, String host) { + this.sslFactory = sslFactory; + this.host = host; + } + + @Override + protected void configureSslEngine(SSLEngine sslEngine, AsyncHttpClientConfig config) { + super.configureSslEngine(sslEngine, config); + if (StringUtils.isNotBlank(host)) { + SSLParameters parameters = sslEngine.getSSLParameters(); + parameters.setServerNames(Collections.singletonList(new SNIHostName(host))); + sslEngine.setSSLParameters(parameters); + } + } + + @Override + public SSLEngine newSslEngine(AsyncHttpClientConfig config, String peerHost, int peerPort) { + SSLContext sslContext = this.sslFactory.getInternalSslContext(); + SSLEngine sslEngine = config.isDisableHttpsEndpointIdentificationAlgorithm() + ? sslContext.createSSLEngine() : + sslContext.createSSLEngine(domain(peerHost), peerPort); + configureSslEngine(sslEngine, config); + return sslEngine; + } + +} +``` + +The above `PulsarHttpAsyncSslEngineFactory` will be passed to the DefaultAsyncHttpClientConfig.Builder while creating +the DefaultAsyncHttpClient. This pattern will be common across all HTTP Clients using AsyncHttpClient within Pulsar. + +## Public-facing Changes + +### Configuration + +Same as [Broker Common Changes](#pulsar-commmon-changes) + +### CLI +CLI tools like `PulsarClientTool` and `PulsarAdminTool` will need to be modified to support the new configurations. + +# Backward & Forward Compatibility + +## Revert +Rolling back to the previous version of Pulsar will revert to the previous behavior. + +## Upgrade +Upgrading to the version containing the `PulsarSslFactory` will not cause any behavior change. The `PulsarSslFactory` +for the server, client and brokerclient will default to using the `DefaultPulsarSslFactory` which will +read the TLS certificates via the file system. + +The Pulsar system will use the custom plugin behavior only if the `sslFactoryPlugin` configuration is set. + +# Links + +POC Changes: https://github.com/Apurva007/pulsar/pull/4 \ No newline at end of file From 8b7754f11f113af9d341a460795d0c7b8095f594 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 4 Jul 2024 12:41:21 +0300 Subject: [PATCH 358/580] [fix][ci] Fix OWASP Dependency Check download by using NVD API key (#22999) --- .../workflows/ci-owasp-dependency-check.yaml | 20 ++++++++----------- .github/workflows/pulsar-ci.yaml | 9 ++++----- distribution/io/pom.xml | 1 - pom.xml | 14 ++++++++++--- pulsar-io/docs/pom.xml | 1 - pulsar-io/flume/pom.xml | 1 - pulsar-io/hbase/pom.xml | 1 - pulsar-io/hdfs2/pom.xml | 7 +++---- pulsar-io/hdfs3/pom.xml | 9 ++++----- tiered-storage/file-system/pom.xml | 1 - 10 files changed, 30 insertions(+), 34 deletions(-) diff --git a/.github/workflows/ci-owasp-dependency-check.yaml b/.github/workflows/ci-owasp-dependency-check.yaml index a273e902c88d2..a70f4a82ff1af 100644 --- a/.github/workflows/ci-owasp-dependency-check.yaml +++ b/.github/workflows/ci-owasp-dependency-check.yaml @@ -24,8 +24,9 @@ on: workflow_dispatch: env: - MAVEN_OPTS: -Xss1500k -Xmx1024m -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000 + MAVEN_OPTS: -Xss1500k -Xmx1500m -Daether.connector.http.reuseConnections=false -Daether.connector.requestTimeout=60000 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true -Dmaven.wagon.http.serviceUnavailableRetryStrategy.class=standard -Dmaven.wagon.rto=60000 JDK_DISTRIBUTION: corretto + NIST_NVD_API_KEY: ${{ secrets.NIST_NVD_API_KEY }} jobs: run-owasp-dependency-check: @@ -42,12 +43,9 @@ jobs: matrix: include: - branch: master + - branch: branch-3.3 - branch: branch-3.2 - - branch: branch-3.1 - branch: branch-3.0 - - branch: branch-2.11 - - branch: branch-2.10 - jdk: 11 steps: - name: checkout @@ -58,16 +56,14 @@ jobs: - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm - - name: Cache local Maven repository - uses: actions/cache@v4 + - name: Restore Maven repository cache + uses: actions/cache/restore@v4 timeout-minutes: 5 with: path: | ~/.m2/repository/*/*/* !~/.m2/repository/org/apache/pulsar - !~/.m2/repository/org/owasp/dependency-check-data key: ${{ runner.os }}-m2-dependencies-all-${{ hashFiles('**/pom.xml') }} - lookup-only: true restore-keys: | ${{ runner.os }}-m2-dependencies-core-modules-${{ hashFiles('**/pom.xml') }} ${{ runner.os }}-m2-dependencies-core-modules- @@ -79,7 +75,7 @@ jobs: java-version: ${{ matrix.jdk || '17' }} - name: run install by skip tests - run: mvn -B -ntp clean install -DskipTests -Dspotbugs.skip=true -Dlicense.skip=true -Dcheckstyle.skip=true -Drat.skip=true -DskipDocker=true + run: mvn -B -ntp clean install -DskipTests -Dspotbugs.skip=true -Dlicense.skip=true -Dcheckstyle.skip=true -Drat.skip=true -DskipDocker=true -DnarPluginPhase=none -pl '!distribution/io,!distribution/offloaders' - name: OWASP cache key weeknum id: get-weeknum @@ -89,7 +85,7 @@ jobs: - name: Restore OWASP Dependency Check data id: restore-owasp-dependency-check-data - uses: actions/cache/restore@v3 + uses: actions/cache/restore@v4 timeout-minutes: 5 with: path: ~/.m2/repository/org/owasp/dependency-check-data @@ -105,7 +101,7 @@ jobs: - name: Save OWASP Dependency Check data if: ${{ steps.update-owasp-dependency-check-data.outcome == 'success' }} - uses: actions/cache/save@v3 + uses: actions/cache/save@v4 timeout-minutes: 5 with: path: ~/.m2/repository/org/owasp/dependency-check-data diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 8decde1c999ca..828f876f13194 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -1427,6 +1427,7 @@ jobs: env: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} + NIST_NVD_API_KEY: ${{ secrets.NIST_NVD_API_KEY }} steps: - name: checkout uses: actions/checkout@v4 @@ -1442,16 +1443,14 @@ jobs: with: limit-access-to-actor: true - - name: Cache Maven dependencies - uses: actions/cache@v4 + - name: Restore Maven repository cache + uses: actions/cache/restore@v4 timeout-minutes: 5 with: path: | ~/.m2/repository/*/*/* !~/.m2/repository/org/apache/pulsar - !~/.m2/repository/org/owasp/dependency-check-data key: ${{ runner.os }}-m2-dependencies-core-modules-${{ hashFiles('**/pom.xml') }} - lookup-only: true restore-keys: | ${{ runner.os }}-m2-dependencies-core-modules- @@ -1480,7 +1479,7 @@ jobs: - name: Restore OWASP Dependency Check data id: restore-owasp-dependency-check-data - uses: actions/cache/restore@v3 + uses: actions/cache/restore@v4 timeout-minutes: 5 with: path: ~/.m2/repository/org/owasp/dependency-check-data diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index bd65d5a81232b..96dd8b071106b 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -136,7 +136,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/pom.xml b/pom.xml index 7c556fa127786..d42eac2d5af59 100644 --- a/pom.xml +++ b/pom.xml @@ -316,7 +316,7 @@ flexible messaging model and an intuitive client API. 0.1.21 1.3 0.4 - 9.1.0 + 10.0.1 1.0.6 1.6.1 6.4.0 @@ -2192,6 +2192,16 @@ flexible messaging model and an intuitive client API. build-helper-maven-plugin ${build-helper-maven-plugin.version} + + org.owasp + dependency-check-maven + ${dependency-check-maven.version} + + NIST_NVD_API_KEY + + + + @@ -2639,7 +2649,6 @@ flexible messaging model and an intuitive client API. org.owasp dependency-check-maven - ${dependency-check-maven.version} ${pulsar.basedir}/src/owasp-dependency-check-false-positives.xml @@ -2674,7 +2683,6 @@ flexible messaging model and an intuitive client API. org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/pulsar-io/docs/pom.xml b/pulsar-io/docs/pom.xml index 82c8f0bb6f96a..1e21656305b6c 100644 --- a/pulsar-io/docs/pom.xml +++ b/pulsar-io/docs/pom.xml @@ -258,7 +258,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/pulsar-io/flume/pom.xml b/pulsar-io/flume/pom.xml index 9b2839970ab79..86cec763cbe4a 100644 --- a/pulsar-io/flume/pom.xml +++ b/pulsar-io/flume/pom.xml @@ -141,7 +141,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/pulsar-io/hbase/pom.xml b/pulsar-io/hbase/pom.xml index 0c38d4f06d029..9fb98069a8ceb 100644 --- a/pulsar-io/hbase/pom.xml +++ b/pulsar-io/hbase/pom.xml @@ -108,7 +108,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/pulsar-io/hdfs2/pom.xml b/pulsar-io/hdfs2/pom.xml index 81b67f8e095fa..3b73adae46caa 100644 --- a/pulsar-io/hdfs2/pom.xml +++ b/pulsar-io/hdfs2/pom.xml @@ -27,14 +27,14 @@ pulsar-io-hdfs2 Pulsar IO :: Hdfs2 - + ${project.groupId} pulsar-io-core ${project.version} - + com.fasterxml.jackson.core jackson-databind @@ -74,7 +74,7 @@ commons-lang3 - + @@ -113,7 +113,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index 3d9f185e37582..29a1c248c756f 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -27,14 +27,14 @@ pulsar-io-hdfs3 Pulsar IO :: Hdfs3 - + ${project.groupId} pulsar-io-core ${project.version} - + com.fasterxml.jackson.core jackson-databind @@ -49,7 +49,7 @@ org.apache.commons commons-collections4 - + org.apache.hadoop hadoop-client @@ -80,7 +80,7 @@ - + @@ -119,7 +119,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/tiered-storage/file-system/pom.xml b/tiered-storage/file-system/pom.xml index d20b92692fc58..03dc5371ef7f6 100644 --- a/tiered-storage/file-system/pom.xml +++ b/tiered-storage/file-system/pom.xml @@ -208,7 +208,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} From dd1b57944b117d16ebd371996b44c02af2ce325c Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 5 Jul 2024 00:55:06 -0700 Subject: [PATCH 359/580] [feat][misc] PIP-264: Copy OpenTelemetry resource attributes to Prometheus labels (#23005) --- .../opentelemetry/OpenTelemetryService.java | 15 +++++++ .../metrics/OpenTelemetrySanityTest.java | 39 +++++++++++-------- 2 files changed, 37 insertions(+), 17 deletions(-) diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java index b32d353eb5ae7..e6c6d95273e0e 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryService.java @@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.common.annotations.VisibleForTesting; import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.exporter.prometheus.PrometheusHttpServer; import io.opentelemetry.instrumentation.runtimemetrics.java17.RuntimeMetrics; import io.opentelemetry.sdk.OpenTelemetrySdk; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; @@ -97,6 +98,20 @@ public OpenTelemetryService(String clusterName, return resource.merge(resourceBuilder.build()); }); + sdkBuilder.addMetricReaderCustomizer((metricReader, configProperties) -> { + if (metricReader instanceof PrometheusHttpServer prometheusHttpServer) { + // At this point, the server is already started. We need to close it and create a new one with the + // correct resource attributes filter. + prometheusHttpServer.close(); + + // Allow all resource attributes to be exposed. + return prometheusHttpServer.toBuilder() + .setAllowedResourceAttributesFilter(s -> true) + .build(); + } + return metricReader; + }); + if (builderCustomizer != null) { builderCustomizer.accept(sdkBuilder); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java index 38afc1f127d18..31e600f3aa812 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/metrics/OpenTelemetrySanityTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.tests.integration.metrics; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; +import static org.awaitility.Awaitility.waitAtMost; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -37,7 +39,6 @@ import org.apache.pulsar.tests.integration.topologies.PulsarCluster; import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; import org.apache.pulsar.tests.integration.topologies.PulsarTestBase; -import org.awaitility.Awaitility; import org.testng.annotations.Test; public class OpenTelemetrySanityTest { @@ -71,17 +72,17 @@ public void testOpenTelemetryMetricsOtlpExport() throws Exception { // TODO: Validate cluster name and service version are present once // https://github.com/open-telemetry/opentelemetry-java/issues/6108 is solved. var metricName = "queueSize_ratio"; // Sent automatically by the OpenTelemetry SDK. - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { + waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var metrics = getMetricsFromPrometheus( openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT); return !metrics.findByNameAndLabels(metricName, "job", PulsarBrokerOpenTelemetry.SERVICE_NAME).isEmpty(); }); - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { + waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var metrics = getMetricsFromPrometheus( openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT); return !metrics.findByNameAndLabels(metricName, "job", PulsarProxyOpenTelemetry.SERVICE_NAME).isEmpty(); }); - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { + waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { var metrics = getMetricsFromPrometheus( openTelemetryCollectorContainer, OpenTelemetryCollectorContainer.PROMETHEUS_EXPORTER_PORT); return !metrics.findByNameAndLabels(metricName, "job", PulsarWorkerOpenTelemetry.SERVICE_NAME).isEmpty(); @@ -120,30 +121,34 @@ public void testOpenTelemetryMetricsPrometheusExport() throws Exception { pulsarCluster.start(); pulsarCluster.setupFunctionWorkers(PulsarTestBase.randomName(), FunctionRuntimeType.PROCESS, 1); - var metricName = "target_info"; // Sent automatically by the OpenTelemetry SDK. - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { - var metrics = getMetricsFromPrometheus(pulsarCluster.getBroker(0), prometheusExporterPort); - return !metrics.findByNameAndLabels(metricName, + var targetInfoMetricName = "target_info"; // Sent automatically by the OpenTelemetry SDK. + var cpuCountMetricName = "jvm_cpu_count"; // Configured by the OpenTelemetryService. + waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).untilAsserted(() -> { + var expectedMetrics = new String[] {targetInfoMetricName, cpuCountMetricName, "pulsar_broker_topic_producer_count"}; + var actualMetrics = getMetricsFromPrometheus(pulsarCluster.getBroker(0), prometheusExporterPort); + assertThat(expectedMetrics).allMatch(expectedMetric -> !actualMetrics.findByNameAndLabels(expectedMetric, Pair.of("pulsar_cluster", clusterName), Pair.of("service_name", PulsarBrokerOpenTelemetry.SERVICE_NAME), Pair.of("service_version", PulsarVersion.getVersion()), - Pair.of("host_name", pulsarCluster.getBroker(0).getHostname())).isEmpty(); + Pair.of("host_name", pulsarCluster.getBroker(0).getHostname())).isEmpty()); }); - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { - var metrics = getMetricsFromPrometheus(pulsarCluster.getProxy(), prometheusExporterPort); - return !metrics.findByNameAndLabels(metricName, + waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).untilAsserted(() -> { + var expectedMetrics = new String[] {targetInfoMetricName, cpuCountMetricName}; + var actualMetrics = getMetricsFromPrometheus(pulsarCluster.getProxy(), prometheusExporterPort); + assertThat(expectedMetrics).allMatch(expectedMetric -> !actualMetrics.findByNameAndLabels(expectedMetric, Pair.of("pulsar_cluster", clusterName), Pair.of("service_name", PulsarProxyOpenTelemetry.SERVICE_NAME), Pair.of("service_version", PulsarVersion.getVersion()), - Pair.of("host_name", pulsarCluster.getProxy().getHostname())).isEmpty(); + Pair.of("host_name", pulsarCluster.getProxy().getHostname())).isEmpty()); }); - Awaitility.waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).until(() -> { - var metrics = getMetricsFromPrometheus(pulsarCluster.getAnyWorker(), prometheusExporterPort); - return !metrics.findByNameAndLabels(metricName, + waitAtMost(90, TimeUnit.SECONDS).ignoreExceptions().pollInterval(1, TimeUnit.SECONDS).untilAsserted(() -> { + var expectedMetrics = new String[] {targetInfoMetricName, cpuCountMetricName}; + var actualMetrics = getMetricsFromPrometheus(pulsarCluster.getAnyWorker(), prometheusExporterPort); + assertThat(expectedMetrics).allMatch(expectedMetric -> !actualMetrics.findByNameAndLabels(expectedMetric, Pair.of("pulsar_cluster", clusterName), Pair.of("service_name", PulsarWorkerOpenTelemetry.SERVICE_NAME), Pair.of("service_version", PulsarVersion.getVersion()), - Pair.of("host_name", pulsarCluster.getAnyWorker().getHostname())).isEmpty(); + Pair.of("host_name", pulsarCluster.getAnyWorker().getHostname())).isEmpty()); }); } From 8351c079d8e8b162f964ed6a735edf76459070ec Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 5 Jul 2024 02:45:55 -0700 Subject: [PATCH 360/580] [feat][broker] PIP-264: Add OpenTelemetry managed cursor metrics (#23000) --- .../bookkeeper/mledger/ManagedCursor.java | 8 ++ .../mledger/ManagedCursorAttributes.java | 51 +++++++ .../mledger/impl/ManagedCursorImpl.java | 14 ++ .../impl/ManagedLedgerFactoryImpl.java | 3 + .../impl/OpenTelemetryManagedCursorStats.java | 136 ++++++++++++++++++ .../stats/ManagedCursorMetricsTest.java | 98 +++++++++++-- .../OpenTelemetryAttributes.java | 23 +++ 7 files changed, 321 insertions(+), 12 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursorAttributes.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedCursorStats.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java index 4aa3226a4dc2b..f6345e7b9ec5b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java @@ -877,4 +877,12 @@ default boolean periodicRollover() { return false; } + /** + * Get the attributes associated with the cursor. + * + * @return the attributes associated with the cursor + */ + default ManagedCursorAttributes getManagedCursorAttributes() { + return new ManagedCursorAttributes(this); + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursorAttributes.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursorAttributes.java new file mode 100644 index 0000000000000..6c06e68d75e24 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursorAttributes.java @@ -0,0 +1,51 @@ +/* + * 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.bookkeeper.mledger; + +import io.opentelemetry.api.common.Attributes; +import lombok.Getter; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.ManagedCursorOperationStatus; + +@Getter +public class ManagedCursorAttributes { + + private final Attributes attributes; + private final Attributes attributesOperationSucceed; + private final Attributes attributesOperationFailure; + + public ManagedCursorAttributes(ManagedCursor cursor) { + var mlName = cursor.getManagedLedger().getName(); + var topicName = TopicName.get(TopicName.fromPersistenceNamingEncoding(mlName)); + attributes = Attributes.of( + OpenTelemetryAttributes.ML_CURSOR_NAME, cursor.getName(), + OpenTelemetryAttributes.ML_LEDGER_NAME, mlName, + OpenTelemetryAttributes.PULSAR_NAMESPACE, topicName.getNamespace() + ); + attributesOperationSucceed = Attributes.builder() + .putAll(attributes) + .putAll(ManagedCursorOperationStatus.SUCCESS.attributes) + .build(); + attributesOperationFailure = Attributes.builder() + .putAll(attributes) + .putAll(ManagedCursorOperationStatus.FAILURE.attributes) + .build(); + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 98ba722ba1c9b..4ef9678f3e180 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -77,6 +77,7 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.SkipEntriesCallback; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedCursorAttributes; import org.apache.bookkeeper.mledger.ManagedCursorMXBean; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; @@ -286,6 +287,11 @@ public enum State { protected final ManagedCursorMXBean mbean; + private volatile ManagedCursorAttributes managedCursorAttributes; + private static final AtomicReferenceFieldUpdater ATTRIBUTES_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, ManagedCursorAttributes.class, + "managedCursorAttributes"); + @SuppressWarnings("checkstyle:javadoctype") public interface VoidCallback { void operationComplete(); @@ -3719,4 +3725,12 @@ public ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) thro } return newNonDurableCursor; } + + @Override + public ManagedCursorAttributes getManagedCursorAttributes() { + if (managedCursorAttributes != null) { + return managedCursorAttributes; + } + return ATTRIBUTES_UPDATER.updateAndGet(this, old -> old != null ? old : new ManagedCursorAttributes(this)); + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index b1939f40e9358..00afb85a9d486 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -122,6 +122,7 @@ public class ManagedLedgerFactoryImpl implements ManagedLedgerFactory { private final OpenTelemetryManagedLedgerCacheStats openTelemetryCacheStats; private final OpenTelemetryManagedLedgerStats openTelemetryManagedLedgerStats; + private final OpenTelemetryManagedCursorStats openTelemetryManagedCursorStats; //indicate whether shutdown() is called. private volatile boolean closed; @@ -231,6 +232,7 @@ private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, openTelemetryCacheStats = new OpenTelemetryManagedLedgerCacheStats(openTelemetry, this); openTelemetryManagedLedgerStats = new OpenTelemetryManagedLedgerStats(openTelemetry, this); + openTelemetryManagedCursorStats = new OpenTelemetryManagedCursorStats(openTelemetry, this); } static class DefaultBkFactory implements BookkeeperFactoryForCustomEnsemblePlacementPolicy { @@ -622,6 +624,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { })); }).thenAcceptAsync(__ -> { //wait for tasks in scheduledExecutor executed. + openTelemetryManagedCursorStats.close(); openTelemetryManagedLedgerStats.close(); openTelemetryCacheStats.close(); scheduledExecutor.shutdownNow(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedCursorStats.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedCursorStats.java new file mode 100644 index 0000000000000..93a749d4aef51 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedCursorStats.java @@ -0,0 +1,136 @@ +/* + * 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.bookkeeper.mledger.impl; + +import com.google.common.collect.Streams; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.metrics.BatchCallback; +import io.opentelemetry.api.metrics.ObservableLongMeasurement; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.pulsar.opentelemetry.Constants; + +public class OpenTelemetryManagedCursorStats implements AutoCloseable { + + // Replaces ['pulsar_ml_cursor_persistLedgerSucceed', 'pulsar_ml_cursor_persistLedgerErrors'] + public static final String PERSIST_OPERATION_COUNTER = "pulsar.broker.managed_ledger.persist.operation.count"; + private final ObservableLongMeasurement persistOperationCounter; + + // Replaces ['pulsar_ml_cursor_persistZookeeperSucceed', 'pulsar_ml_cursor_persistZookeeperErrors'] + public static final String PERSIST_OPERATION_METADATA_STORE_COUNTER = + "pulsar.broker.managed_ledger.persist.mds.operation.count"; + private final ObservableLongMeasurement persistOperationMetadataStoreCounter; + + // Replaces pulsar_ml_cursor_nonContiguousDeletedMessagesRange + public static final String NON_CONTIGUOUS_MESSAGE_RANGE_COUNTER = + "pulsar.broker.managed_ledger.message_range.count"; + private final ObservableLongMeasurement nonContiguousMessageRangeCounter; + + // Replaces pulsar_ml_cursor_writeLedgerSize + public static final String OUTGOING_BYTE_COUNTER = "pulsar.broker.managed_ledger.cursor.outgoing.size"; + private final ObservableLongMeasurement outgoingByteCounter; + + // Replaces pulsar_ml_cursor_writeLedgerLogicalSize + public static final String OUTGOING_BYTE_LOGICAL_COUNTER = + "pulsar.broker.managed_ledger.cursor.outgoing.logical.size"; + private final ObservableLongMeasurement outgoingByteLogicalCounter; + + // Replaces pulsar_ml_cursor_readLedgerSize + public static final String INCOMING_BYTE_COUNTER = "pulsar.broker.managed_ledger.cursor.incoming.size"; + private final ObservableLongMeasurement incomingByteCounter; + + private final BatchCallback batchCallback; + + public OpenTelemetryManagedCursorStats(OpenTelemetry openTelemetry, ManagedLedgerFactoryImpl factory) { + var meter = openTelemetry.getMeter(Constants.BROKER_INSTRUMENTATION_SCOPE_NAME); + + persistOperationCounter = meter + .counterBuilder(PERSIST_OPERATION_COUNTER) + .setUnit("{operation}") + .setDescription("The number of acknowledgment operations on the ledger.") + .buildObserver(); + + persistOperationMetadataStoreCounter = meter + .counterBuilder(PERSIST_OPERATION_METADATA_STORE_COUNTER) + .setUnit("{operation}") + .setDescription("The number of acknowledgment operations in the metadata store.") + .buildObserver(); + + nonContiguousMessageRangeCounter = meter + .upDownCounterBuilder(NON_CONTIGUOUS_MESSAGE_RANGE_COUNTER) + .setUnit("{range}") + .setDescription("The number of non-contiguous deleted messages ranges.") + .buildObserver(); + + outgoingByteCounter = meter + .counterBuilder(OUTGOING_BYTE_COUNTER) + .setUnit("{By}") + .setDescription("The total amount of data written to the ledger.") + .buildObserver(); + + outgoingByteLogicalCounter = meter + .counterBuilder(OUTGOING_BYTE_LOGICAL_COUNTER) + .setUnit("{By}") + .setDescription("The total amount of data written to the ledger, not including replicas.") + .buildObserver(); + + incomingByteCounter = meter + .counterBuilder(INCOMING_BYTE_COUNTER) + .setUnit("{By}") + .setDescription("The total amount of data read from the ledger.") + .buildObserver(); + + batchCallback = meter.batchCallback(() -> factory.getManagedLedgers() + .values() + .stream() + .map(ManagedLedgerImpl::getCursors) + .flatMap(Streams::stream) + .forEach(this::recordMetrics), + persistOperationCounter, + persistOperationMetadataStoreCounter, + nonContiguousMessageRangeCounter, + outgoingByteCounter, + outgoingByteLogicalCounter, + incomingByteCounter); + } + + @Override + public void close() { + batchCallback.close(); + } + + private void recordMetrics(ManagedCursor cursor) { + var stats = cursor.getStats(); + var cursorAttributesSet = cursor.getManagedCursorAttributes(); + var attributes = cursorAttributesSet.getAttributes(); + var attributesSucceed = cursorAttributesSet.getAttributesOperationSucceed(); + var attributesFailed = cursorAttributesSet.getAttributesOperationFailure(); + + persistOperationCounter.record(stats.getPersistLedgerSucceed(), attributesSucceed); + persistOperationCounter.record(stats.getPersistLedgerErrors(), attributesFailed); + + persistOperationMetadataStoreCounter.record(stats.getPersistZookeeperSucceed(), attributesSucceed); + persistOperationMetadataStoreCounter.record(stats.getPersistZookeeperErrors(), attributesFailed); + + nonContiguousMessageRangeCounter.record(cursor.getTotalNonContiguousDeletedMessagesRange(), attributes); + + outgoingByteCounter.record(stats.getWriteCursorLedgerSize(), attributes); + outgoingByteLogicalCounter.record(stats.getWriteCursorLedgerLogicalSize(), attributes); + incomingByteCounter.record(stats.getReadCursorLedgerSize(), attributes); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedCursorMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedCursorMetricsTest.java index baa4bea570155..8ddb5320588da 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedCursorMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedCursorMetricsTest.java @@ -18,20 +18,24 @@ */ package org.apache.pulsar.broker.stats; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; +import static org.assertj.core.api.Assertions.assertThat; import java.util.ArrayList; import java.util.List; import java.util.UUID; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursorMXBean; -import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.ManagedCursorAttributes; +import org.apache.bookkeeper.mledger.impl.OpenTelemetryManagedCursorStats; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.stats.metrics.ManagedCursorMetrics; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; @@ -80,6 +84,12 @@ protected PulsarClient createNewPulsarClient(ClientBuilder clientBuilder) throws return PulsarTestClient.create(clientBuilder); } + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder pulsarTestContextBuilder) { + super.customizeMainPulsarTestContextBuilder(pulsarTestContextBuilder); + pulsarTestContextBuilder.enableOpenTelemetry(true); + } + /*** * This method has overridden these case: * brk_ml_cursor_persistLedgerSucceed @@ -115,10 +125,7 @@ public void testManagedCursorMetrics() throws Exception { .topic(topicName) .enableBatching(false) .create(); - final PersistentSubscription persistentSubscription = - (PersistentSubscription) pulsar.getBrokerService() - .getTopic(topicName, false).get().get().getSubscription(subName); - final ManagedCursorImpl managedCursor = (ManagedCursorImpl) persistentSubscription.getCursor(); + var managedCursor = getManagedCursor(topicName, subName); ManagedCursorMXBean managedCursorMXBean = managedCursor.getStats(); // Assert. metricsList = metrics.generate(); @@ -128,6 +135,19 @@ public void testManagedCursorMetrics() throws Exception { Assert.assertEquals(metricsList.get(0).getMetrics().get("brk_ml_cursor_persistZookeeperSucceed"), 0L); Assert.assertEquals(metricsList.get(0).getMetrics().get("brk_ml_cursor_persistZookeeperErrors"), 0L); Assert.assertEquals(metricsList.get(0).getMetrics().get("brk_ml_cursor_nonContiguousDeletedMessagesRange"), 0L); + // Validate OpenTelemetry metrics as well + var attributesSet = new ManagedCursorAttributes(managedCursor); + var otelMetrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_COUNTER, + attributesSet.getAttributesOperationSucceed(), 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_COUNTER, + attributesSet.getAttributesOperationFailure(), 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_METADATA_STORE_COUNTER, + attributesSet.getAttributesOperationSucceed(), 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_METADATA_STORE_COUNTER, + attributesSet.getAttributesOperationFailure(), 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.NON_CONTIGUOUS_MESSAGE_RANGE_COUNTER, + attributesSet.getAttributes(), 0); /** * 1. Send many messages, and only ack half. After the cursor data is written to BK, * verify "brk_ml_cursor_persistLedgerSucceed" and "brk_ml_cursor_nonContiguousDeletedMessagesRange". @@ -156,6 +176,17 @@ public void testManagedCursorMetrics() throws Exception { Assert.assertEquals(metricsList.get(0).getMetrics().get("brk_ml_cursor_persistZookeeperErrors"), 0L); Assert.assertNotEquals(metricsList.get(0).getMetrics().get("brk_ml_cursor_nonContiguousDeletedMessagesRange"), 0L); + otelMetrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_COUNTER, + attributesSet.getAttributesOperationSucceed(), value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_COUNTER, + attributesSet.getAttributesOperationFailure(), 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_METADATA_STORE_COUNTER, + attributesSet.getAttributesOperationSucceed(), 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_METADATA_STORE_COUNTER, + attributesSet.getAttributesOperationFailure(), 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.NON_CONTIGUOUS_MESSAGE_RANGE_COUNTER, + attributesSet.getAttributes(), value -> assertThat(value).isPositive()); // Ack another half. for (MessageId messageId : keepsMessageIdList){ consumer.acknowledge(messageId); @@ -171,6 +202,17 @@ public void testManagedCursorMetrics() throws Exception { Assert.assertEquals(metricsList.get(0).getMetrics().get("brk_ml_cursor_persistZookeeperSucceed"), 0L); Assert.assertEquals(metricsList.get(0).getMetrics().get("brk_ml_cursor_persistZookeeperErrors"), 0L); Assert.assertEquals(metricsList.get(0).getMetrics().get("brk_ml_cursor_nonContiguousDeletedMessagesRange"), 0L); + otelMetrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_COUNTER, + attributesSet.getAttributesOperationSucceed(), value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_COUNTER, + attributesSet.getAttributesOperationFailure(), 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_METADATA_STORE_COUNTER, + attributesSet.getAttributesOperationSucceed(), 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_METADATA_STORE_COUNTER, + attributesSet.getAttributesOperationFailure(), 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.NON_CONTIGUOUS_MESSAGE_RANGE_COUNTER, + attributesSet.getAttributes(), 0); /** * Make BK error, and send many message, then wait cursor persistent finish. * After the cursor data is written to ZK, verify "brk_ml_cursor_persistLedgerErrors" and @@ -196,6 +238,17 @@ public void testManagedCursorMetrics() throws Exception { Assert.assertNotEquals(metricsList.get(0).getMetrics().get("brk_ml_cursor_persistZookeeperSucceed"), 0L); Assert.assertEquals(metricsList.get(0).getMetrics().get("brk_ml_cursor_persistZookeeperErrors"), 0L); Assert.assertEquals(metricsList.get(0).getMetrics().get("brk_ml_cursor_nonContiguousDeletedMessagesRange"), 0L); + otelMetrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_COUNTER, + attributesSet.getAttributesOperationSucceed(), value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_COUNTER, + attributesSet.getAttributesOperationFailure(), value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_METADATA_STORE_COUNTER, + attributesSet.getAttributesOperationSucceed(), value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.PERSIST_OPERATION_METADATA_STORE_COUNTER, + attributesSet.getAttributesOperationFailure(), 0); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.NON_CONTIGUOUS_MESSAGE_RANGE_COUNTER, + attributesSet.getAttributes(), 0); /** * TODO verify "brk_ml_cursor_persistZookeeperErrors". * This is not easy to implement, we can use {@link #mockZooKeeper} to fail ZK, but we cannot identify whether @@ -210,13 +263,16 @@ public void testManagedCursorMetrics() throws Exception { admin.topics().delete(topicName, true); } - private ManagedCursorMXBean getManagedCursorMXBean(String topicName, String subscriptionName) - throws ExecutionException, InterruptedException { + private ManagedCursorMXBean getManagedCursorMXBean(String topicName, String subscriptionName) throws Exception { + var managedCursor = getManagedCursor(topicName, subscriptionName); + return managedCursor.getStats(); + } + + private ManagedCursor getManagedCursor(String topicName, String subscriptionName) throws Exception { final PersistentSubscription persistentSubscription = (PersistentSubscription) pulsar.getBrokerService() .getTopic(topicName, false).get().get().getSubscription(subscriptionName); - final ManagedCursorImpl managedCursor = (ManagedCursorImpl) persistentSubscription.getCursor(); - return managedCursor.getStats(); + return persistentSubscription.getCursor(); } @Test @@ -265,9 +321,11 @@ public void testCursorReadWriteMetrics() throws Exception { } } + var managedCursor1 = getManagedCursor(topicName, subName1); + var cursorMXBean1 = managedCursor1.getStats(); + var managedCursor2 = getManagedCursor(topicName, subName2); + var cursorMXBean2 = managedCursor2.getStats(); // Wait for persistent cursor meta. - ManagedCursorMXBean cursorMXBean1 = getManagedCursorMXBean(topicName, subName1); - ManagedCursorMXBean cursorMXBean2 = getManagedCursorMXBean(topicName, subName2); Awaitility.await().until(() -> cursorMXBean1.getWriteCursorLedgerLogicalSize() > 0); Awaitility.await().until(() -> cursorMXBean2.getWriteCursorLedgerLogicalSize() > 0); @@ -281,6 +339,22 @@ public void testCursorReadWriteMetrics() throws Exception { Assert.assertNotEquals(metricsList.get(1).getMetrics().get("brk_ml_cursor_writeLedgerLogicalSize"), 0L); Assert.assertEquals(metricsList.get(1).getMetrics().get("brk_ml_cursor_readLedgerSize"), 0L); + var otelMetrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + var attributes1 = new ManagedCursorAttributes(managedCursor1).getAttributes(); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.OUTGOING_BYTE_COUNTER, + attributes1, value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.OUTGOING_BYTE_LOGICAL_COUNTER, + attributes1, value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.INCOMING_BYTE_COUNTER, + attributes1, 0); + + var attributes2 = new ManagedCursorAttributes(managedCursor2).getAttributes(); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.OUTGOING_BYTE_COUNTER, + attributes2, value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.OUTGOING_BYTE_LOGICAL_COUNTER, + attributes2, value -> assertThat(value).isPositive()); + assertMetricLongSumValue(otelMetrics, OpenTelemetryManagedCursorStats.INCOMING_BYTE_COUNTER, + attributes2, 0); // cleanup. consumer.close(); consumer2.close(); diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index 24dd1be8509bf..41358a72c0d90 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -116,6 +116,7 @@ public interface OpenTelemetryAttributes { * The status of the Pulsar transaction. */ AttributeKey PULSAR_TRANSACTION_STATUS = AttributeKey.stringKey("pulsar.transaction.status"); + enum TransactionStatus { ABORTED, ACTIVE, @@ -174,6 +175,28 @@ enum BacklogQuotaType { public final Attributes attributes = Attributes.of(PULSAR_BACKLOG_QUOTA_TYPE, name().toLowerCase()); } + // Managed Ledger Attributes + /** + * The name of the managed ledger. + */ + AttributeKey ML_LEDGER_NAME = AttributeKey.stringKey("pulsar.managed_ledger.name"); + + /** + * The name of the managed cursor. + */ + AttributeKey ML_CURSOR_NAME = AttributeKey.stringKey("pulsar.managed_ledger.cursor.name"); + + /** + * The status of the managed cursor operation. + */ + AttributeKey ML_CURSOR_OPERATION_STATUS = + AttributeKey.stringKey("pulsar.managed_ledger.cursor.operation.status"); + enum ManagedCursorOperationStatus { + SUCCESS, + FAILURE; + public final Attributes attributes = Attributes.of(ML_CURSOR_OPERATION_STATUS, name().toLowerCase()); + } + /** * The name of the remote cluster for a Pulsar replicator. */ From 41ef3f6fb1c0b209307d7b4e14300a377c52c5ab Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Sat, 6 Jul 2024 06:26:28 +0800 Subject: [PATCH 361/580] [fix][broker] Fix MessageDeduplication replay timeout cause topic loading stuck (#23004) Co-authored-by: fanjianye --- .../persistent/MessageDeduplication.java | 14 ++- .../persistent/TopicDuplicationTest.java | 104 ++++++++++++++++++ 2 files changed, 114 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java index 9d970479400ba..a4879f2e9520a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java @@ -159,11 +159,12 @@ private CompletableFuture recoverSequenceIdsMap() { log.info("[{}] Replaying {} entries for deduplication", topic.getName(), managedCursor.getNumberOfEntries()); CompletableFuture future = new CompletableFuture<>(); replayCursor(future); - return future.thenAccept(lastPosition -> { + return future.thenCompose(lastPosition -> { if (lastPosition != null && snapshotCounter >= snapshotInterval) { snapshotCounter = 0; - takeSnapshot(lastPosition); + return takeSnapshot(lastPosition); } + return CompletableFuture.completedFuture(null); }); } @@ -438,13 +439,15 @@ public void resetHighestSequenceIdPushed() { } } - private void takeSnapshot(Position position) { + private CompletableFuture takeSnapshot(Position position) { + CompletableFuture future = new CompletableFuture<>(); if (log.isDebugEnabled()) { log.debug("[{}] Taking snapshot of sequence ids map", topic.getName()); } if (!snapshotTaking.compareAndSet(false, true)) { - return; + future.complete(null); + return future; } Map snapshot = new TreeMap<>(); @@ -462,14 +465,17 @@ public void markDeleteComplete(Object ctx) { } lastSnapshotTimestamp = System.currentTimeMillis(); snapshotTaking.set(false); + future.complete(null); } @Override public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { log.warn("[{}] Failed to store new deduplication snapshot at {}", topic.getName(), position); snapshotTaking.set(false); + future.completeExceptionally(exception); } }, null); + return future; } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java index 2feaacd5b8209..ddc5eeab1d20e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service.persistent; +import static org.apache.pulsar.broker.service.persistent.PersistentTopic.DEDUPLICATION_CURSOR_NAME; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotEquals; @@ -25,6 +26,8 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; + +import java.lang.reflect.Field; import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -33,12 +36,18 @@ import lombok.Cleanup; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -529,6 +538,101 @@ public void testDisableNamespacePolicyTakeSnapshotShouldNotThrowException() thro persistentTopic.checkDeduplicationSnapshot(); } + @Test + public void testFinishTakeSnapshotWhenTopicLoading() throws Exception { + cleanup(); + setup(); + + // Create a topic and wait deduplication is started. + int brokerDeduplicationEntriesInterval = 1000; + pulsar.getConfiguration().setBrokerDeduplicationEnabled(true); + pulsar.getConfiguration().setBrokerDeduplicationEntriesInterval(brokerDeduplicationEntriesInterval); + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic); + final PersistentTopic persistentTopic1 = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + final ManagedLedgerImpl ml1 = (ManagedLedgerImpl) persistentTopic1.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor1 = + (ManagedCursorImpl) ml1.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor1); + }); + final MessageDeduplication deduplication1 = persistentTopic1.getMessageDeduplication(); + + + // Send 999 messages, it is less than "brokerDeduplicationEntriesInterval". + // So it would not trigger takeSnapshot + final Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic).enableBatching(false).create(); + for (int i = 0; i < brokerDeduplicationEntriesInterval - 1; i++) { + producer.send(i + ""); + } + producer.close(); + int snapshotCounter1 = WhiteboxImpl.getInternalState(deduplication1, "snapshotCounter"); + assertEquals(snapshotCounter1, brokerDeduplicationEntriesInterval - 1); + + + // Unload and load topic, simulate topic load is timeout. + // SetBrokerDeduplicationEntriesInterval to 10, therefore recoverSequenceIdsMap#takeSnapshot + // would trigger and should update the snapshot position. + // However, if topic close and takeSnapshot are concurrent, + // it would result in takeSnapshot throw exception + admin.topics().unload(topic); + pulsar.getConfiguration().setBrokerDeduplicationEntriesInterval(10); + + // Mock message deduplication recovery speed topicLoadTimeoutSeconds + pulsar.getConfiguration().setTopicLoadTimeoutSeconds(1); + String mlPath = BrokerService.MANAGED_LEDGER_PATH_ZNODE + "/" + + TopicName.get(topic).getPersistenceNamingEncoding() + "/" + DEDUPLICATION_CURSOR_NAME; + mockZooKeeper.delay(2 * 1000, (op, path) -> { + if (mlPath.equals(path)) { + return true; + } + return false; + }); + + Field field2 = BrokerService.class.getDeclaredField("topics"); + field2.setAccessible(true); + ConcurrentOpenHashMap>> topics = + (ConcurrentOpenHashMap>>) + field2.get(pulsar.getBrokerService()); + + try { + pulsar.getBrokerService().getTopic(topic, false).join().get(); + Assert.fail(); + } catch (Exception e) { + // topic loading should timeout. + } + Awaitility.await().untilAsserted(() -> { + // topic loading timeout then close topic and remove from topicsMap + Assert.assertFalse(topics.containsKey(topic)); + }); + + + // Load topic again, setBrokerDeduplicationEntriesInterval to 10000, + // make recoverSequenceIdsMap#takeSnapshot not trigger takeSnapshot. + // But actually it should not replay again in recoverSequenceIdsMap, + // since previous topic loading should finish the replay process. + pulsar.getConfiguration().setBrokerDeduplicationEntriesInterval(10000); + pulsar.getConfiguration().setTopicLoadTimeoutSeconds(60); + PersistentTopic persistentTopic2 = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) persistentTopic2.getManagedLedger(); + MessageDeduplication deduplication2 = persistentTopic2.getMessageDeduplication(); + + Awaitility.await().untilAsserted(() -> { + int snapshotCounter3 = WhiteboxImpl.getInternalState(deduplication2, "snapshotCounter"); + Assert.assertEquals(snapshotCounter3, 0); + Assert.assertEquals(ml2.getLedgersInfo().size(), 1); + }); + + + // cleanup. + admin.topics().delete(topic); + cleanup(); + setup(); + } + private void waitCacheInit(String topicName) throws Exception { pulsarClient.newConsumer().topic(topicName).subscriptionName("my-sub").subscribe().close(); TopicName topic = TopicName.get(topicName); From ed39c4db671c29057e51b9142a0d4cdb71e3eb88 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 6 Jul 2024 13:29:00 +0300 Subject: [PATCH 362/580] [improve][broker] Use RoaringBitmap in tracking individual acks to reduce memory usage (#23006) --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- distribution/shell/src/assemble/LICENSE.bin.txt | 2 ++ .../bookkeeper/mledger/impl/RangeSetWrapper.java | 2 +- pom.xml | 2 +- pulsar-common/pom.xml | 5 +++++ .../util/collections/OpenLongPairRangeSet.java | 15 ++------------- 6 files changed, 12 insertions(+), 16 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index cfbe991a8edd8..f46b18347c1eb 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -513,7 +513,7 @@ The Apache Software License, Version 2.0 * RxJava - io.reactivex.rxjava3-rxjava-3.0.1.jar * RoaringBitmap - - org.roaringbitmap-RoaringBitmap-1.0.6.jar + - org.roaringbitmap-RoaringBitmap-1.2.0.jar * OpenTelemetry - io.opentelemetry-opentelemetry-api-1.38.0.jar - io.opentelemetry-opentelemetry-api-incubator-1.38.0-alpha.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 0da56c6afa8fc..261fef74a102c 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -382,6 +382,8 @@ The Apache Software License, Version 2.0 - simpleclient_tracer_common-0.16.0.jar - simpleclient_tracer_otel-0.16.0.jar - simpleclient_tracer_otel_agent-0.16.0.jar + * RoaringBitmap + - RoaringBitmap-1.2.0.jar * Log4J - log4j-api-2.23.1.jar - log4j-core-2.23.1.jar diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index 299fd3dc74cb4..c193d71c64f7d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -55,7 +55,7 @@ public RangeSetWrapper(LongPairConsumer rangeConverter, this.config = managedCursor.getManagedLedger().getConfig(); this.rangeConverter = rangeConverter; this.rangeSet = config.isUnackedRangesOpenCacheSetEnabled() - ? new OpenLongPairRangeSet<>(4096, rangeConverter) + ? new OpenLongPairRangeSet<>(rangeConverter) : new LongPairRangeSet.DefaultRangeSet<>(rangeConverter, rangeBoundConsumer); this.enableMultiEntry = config.isPersistentUnackedRangesWithMultipleEntriesEnabled(); } diff --git a/pom.xml b/pom.xml index d42eac2d5af59..7767a1f626a95 100644 --- a/pom.xml +++ b/pom.xml @@ -317,7 +317,7 @@ flexible messaging model and an intuitive client API. 1.3 0.4 10.0.1 - 1.0.6 + 1.2.0 1.6.1 6.4.0 3.33.0 diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index aa7e4998e5c3e..3f73a43698ea4 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -252,6 +252,11 @@ awaitility test + + + org.roaringbitmap + RoaringBitmap + diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java index c053c106be206..5114675324ad7 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java @@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.concurrent.NotThreadSafe; import org.apache.commons.lang.mutable.MutableInt; +import org.roaringbitmap.RoaringBitSet; /** * A Concurrent set comprising zero or more ranges of type {@link LongPair}. This can be alternative of @@ -46,8 +47,6 @@ public class OpenLongPairRangeSet> implements LongPairRangeSet { protected final NavigableMap rangeBitSetMap = new ConcurrentSkipListMap<>(); - private boolean threadSafe = true; - private final int bitSetSize; private final LongPairConsumer consumer; // caching place-holder for cpu-optimization to avoid calculating ranges again @@ -57,16 +56,6 @@ public class OpenLongPairRangeSet> implements LongPairRa private volatile boolean updatedAfterCachedForToString = true; public OpenLongPairRangeSet(LongPairConsumer consumer) { - this(1024, true, consumer); - } - - public OpenLongPairRangeSet(int size, LongPairConsumer consumer) { - this(size, true, consumer); - } - - public OpenLongPairRangeSet(int size, boolean threadSafe, LongPairConsumer consumer) { - this.threadSafe = threadSafe; - this.bitSetSize = size; this.consumer = consumer; } @@ -416,7 +405,7 @@ private int getSafeEntry(long value) { } private BitSet createNewBitSet() { - return this.threadSafe ? new ConcurrentBitSet(bitSetSize) : new BitSet(bitSetSize); + return new RoaringBitSet(); } } From 17e3f860b050443de74413c1d0b4a3d47173f68a Mon Sep 17 00:00:00 2001 From: zhouyifan279 <88070094+zhouyifan279@users.noreply.github.com> Date: Mon, 8 Jul 2024 14:47:24 +0800 Subject: [PATCH 363/580] [fix][broker] PulsarStandalone started with error if --stream-storage-port is not 4181 (#22993) --- .../zookeeper/LocalBookkeeperEnsemble.java | 2 +- .../zookeeper/LocalBookkeeperEnsembleTest.java | 16 ++++++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java b/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java index cf1a30951ebdf..de3077959a444 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java @@ -360,7 +360,7 @@ public void runStreamStorage(CompositeConfiguration conf) throws Exception { // create a default namespace try (StorageAdminClient admin = StorageClientBuilder.newBuilder() .withSettings(StorageClientSettings.newBuilder() - .serviceUri("bk://localhost:4181") + .serviceUri("bk://localhost:" + streamStoragePort) .backoffPolicy(Backoff.Jitter.of( Type.EXPONENTIAL, 1000, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsembleTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsembleTest.java index a4bc69a7266cc..bfbdf675bd81d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsembleTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsembleTest.java @@ -21,6 +21,8 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; + +import org.apache.bookkeeper.conf.ServerConfiguration; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -54,4 +56,18 @@ public void testStartStop() throws Exception { assertFalse(ensemble.getZkClient().getState().isConnected()); assertFalse(ensemble.getBookies()[0].isRunning()); } + + @Test(timeOut = 10_000) + public void testStartWithSpecifiedStreamStoragePort() throws Exception { + LocalBookkeeperEnsemble ensemble = null; + try { + ensemble = + new LocalBookkeeperEnsemble(1, 0, 0, 4182, null, null, true, null); + ensemble.startStandalone(new ServerConfiguration(), true); + } finally { + if (ensemble != null) { + ensemble.stop(); + } + } + } } From 32e29a3d45c2de5560e22201b0b4bfd5409f12f2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 8 Jul 2024 13:55:56 +0300 Subject: [PATCH 364/580] [fix][misc] Remove RoaringBitmap dependency from pulsar-common (#23008) --- distribution/shell/src/assemble/LICENSE.bin.txt | 2 -- managed-ledger/pom.xml | 4 ++++ .../bookkeeper/mledger/impl/RangeSetWrapper.java | 3 ++- pulsar-common/pom.xml | 5 ----- .../common/util/collections/OpenLongPairRangeSet.java | 10 ++++++++-- 5 files changed, 14 insertions(+), 10 deletions(-) diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 261fef74a102c..0da56c6afa8fc 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -382,8 +382,6 @@ The Apache Software License, Version 2.0 - simpleclient_tracer_common-0.16.0.jar - simpleclient_tracer_otel-0.16.0.jar - simpleclient_tracer_otel_agent-0.16.0.jar - * RoaringBitmap - - RoaringBitmap-1.2.0.jar * Log4J - log4j-api-2.23.1.jar - log4j-core-2.23.1.jar diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index 60a4edab95b77..fac39103c49fb 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -109,6 +109,10 @@ + + org.roaringbitmap + RoaringBitmap + io.dropwizard.metrics metrics-core diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index c193d71c64f7d..a55e6444b2fd9 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -27,6 +27,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.common.util.collections.OpenLongPairRangeSet; +import org.roaringbitmap.RoaringBitSet; /** * Wraps other Range classes, and adds LRU, marking dirty data and other features on this basis. @@ -55,7 +56,7 @@ public RangeSetWrapper(LongPairConsumer rangeConverter, this.config = managedCursor.getManagedLedger().getConfig(); this.rangeConverter = rangeConverter; this.rangeSet = config.isUnackedRangesOpenCacheSetEnabled() - ? new OpenLongPairRangeSet<>(rangeConverter) + ? new OpenLongPairRangeSet<>(rangeConverter, RoaringBitSet::new) : new LongPairRangeSet.DefaultRangeSet<>(rangeConverter, rangeBoundConsumer); this.enableMultiEntry = config.isPersistentUnackedRangesWithMultipleEntriesEnabled(); } diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 3f73a43698ea4..aa7e4998e5c3e 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -252,11 +252,6 @@ awaitility test - - - org.roaringbitmap - RoaringBitmap - diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java index 5114675324ad7..6df6d414871ec 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java @@ -28,9 +28,9 @@ import java.util.NavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; import javax.annotation.concurrent.NotThreadSafe; import org.apache.commons.lang.mutable.MutableInt; -import org.roaringbitmap.RoaringBitSet; /** * A Concurrent set comprising zero or more ranges of type {@link LongPair}. This can be alternative of @@ -48,6 +48,7 @@ public class OpenLongPairRangeSet> implements LongPairRa protected final NavigableMap rangeBitSetMap = new ConcurrentSkipListMap<>(); private final LongPairConsumer consumer; + private final Supplier bitSetSupplier; // caching place-holder for cpu-optimization to avoid calculating ranges again private volatile int cachedSize = 0; @@ -56,7 +57,12 @@ public class OpenLongPairRangeSet> implements LongPairRa private volatile boolean updatedAfterCachedForToString = true; public OpenLongPairRangeSet(LongPairConsumer consumer) { + this(consumer, BitSet::new); + } + + public OpenLongPairRangeSet(LongPairConsumer consumer, Supplier bitSetSupplier) { this.consumer = consumer; + this.bitSetSupplier = bitSetSupplier; } /** @@ -405,7 +411,7 @@ private int getSafeEntry(long value) { } private BitSet createNewBitSet() { - return new RoaringBitSet(); + return bitSetSupplier.get(); } } From 7924f9c3e3ddb5c65338816c90d5d14d5db00198 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 8 Jul 2024 16:33:33 +0300 Subject: [PATCH 365/580] [improve][build] Upgrade dependency-check-maven-plugin to 10.0.2 (#23012) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7767a1f626a95..93e2e24c0558b 100644 --- a/pom.xml +++ b/pom.xml @@ -316,7 +316,7 @@ flexible messaging model and an intuitive client API. 0.1.21 1.3 0.4 - 10.0.1 + 10.0.2 1.2.0 1.6.1 6.4.0 From 9626e7e090e9481e12441a47cf7e89f209aadd03 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 9 Jul 2024 08:53:54 +0800 Subject: [PATCH 366/580] [fix][client] Fix pattern consumer create crash if a part of partitions of a topic have been deleted (#22854) --- .../pulsar/broker/service/TopicGCTest.java | 291 +++++++++++++++++- .../api/PatternMultiTopicsConsumerTest.java | 37 +++ .../impl/PatternTopicsConsumerImplTest.java | 13 +- .../client/impl/TopicsConsumerImplTest.java | 5 +- .../pulsar/client/impl/ConsumerBase.java | 6 + .../pulsar/client/impl/LookupService.java | 9 +- .../client/impl/MultiTopicsConsumerImpl.java | 188 ++++++----- .../impl/PatternConsumerUpdateQueue.java | 254 +++++++++++++++ .../impl/PatternMultiTopicsConsumerImpl.java | 249 ++++++++++----- .../pulsar/client/impl/PulsarClientImpl.java | 15 +- .../pulsar/client/impl/TopicListWatcher.java | 16 +- .../impl/PatternConsumerUpdateQueueTest.java | 247 +++++++++++++++ .../PatternMultiTopicsConsumerImplTest.java | 6 +- .../client/impl/TopicListWatcherTest.java | 12 +- .../pulsar/common/lookup/GetTopicsResult.java | 24 ++ .../pulsar/common/naming/TopicName.java | 9 + .../apache/pulsar/common/util/FutureUtil.java | 3 + 17 files changed, 1187 insertions(+), 197 deletions(-) create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java index 8fdf0723ea8d1..172bd3702e129 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java @@ -18,24 +18,34 @@ */ package org.apache.pulsar.broker.service; +import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; import lombok.EqualsAndHashCode; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TopicMessageId; +import org.apache.pulsar.client.impl.ConsumerImpl; +import org.apache.pulsar.client.impl.PatternMultiTopicsConsumerImpl; import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -58,14 +68,38 @@ protected void cleanup() throws Exception { @EqualsAndHashCode.Include protected void doInitConf() throws Exception { super.doInitConf(); - this.conf.setBrokerDeleteInactiveTopicsEnabled(true); - this.conf.setBrokerDeleteInactiveTopicsMode(InactiveTopicDeleteMode.delete_when_subscriptions_caught_up); - this.conf.setBrokerDeleteInactiveTopicsFrequencySeconds(10); + conf.setBrokerDeleteInactiveTopicsEnabled(true); + conf.setBrokerDeleteInactiveTopicsMode( + InactiveTopicDeleteMode.delete_when_subscriptions_caught_up); + conf.setBrokerDeleteInactiveTopicsFrequencySeconds(10); } - @Test - public void testCreateConsumerAfterOnePartDeleted() throws Exception { + private enum SubscribeTopicType { + MULTI_PARTITIONED_TOPIC, + REGEX_TOPIC; + } + + @DataProvider(name = "subscribeTopicTypes") + public Object[][] subTopicTypes() { + return new Object[][]{ + {SubscribeTopicType.MULTI_PARTITIONED_TOPIC}, + {SubscribeTopicType.REGEX_TOPIC} + }; + } + + private void setSubscribeTopic(ConsumerBuilder consumerBuilder, SubscribeTopicType subscribeTopicType, + String topicName, String topicPattern) { + if (subscribeTopicType.equals(SubscribeTopicType.MULTI_PARTITIONED_TOPIC)) { + consumerBuilder.topic(topicName); + } else { + consumerBuilder.topicsPattern(Pattern.compile(topicPattern)); + } + } + + @Test(dataProvider = "subscribeTopicTypes", timeOut = 300 * 1000) + public void testRecreateConsumerAfterOnePartGc(SubscribeTopicType subscribeTopicType) throws Exception { final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topicPattern = "persistent://public/default/tp.*"; final String partition0 = topic + "-partition-0"; final String partition1 = topic + "-partition-1"; final String subscription = "s1"; @@ -77,8 +111,12 @@ public void testCreateConsumerAfterOnePartDeleted() throws Exception { .enableBatching(false).create(); Producer producer1 = pulsarClient.newProducer(Schema.STRING).topic(partition1) .enableBatching(false).create(); - org.apache.pulsar.client.api.Consumer consumer1 = pulsarClient.newConsumer(Schema.STRING).topic(topic) - .subscriptionName(subscription).isAckReceiptEnabled(true).subscribe(); + ConsumerBuilder consumerBuilder1 = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared); + setSubscribeTopic(consumerBuilder1, subscribeTopicType, topic, topicPattern); + Consumer consumer1 = consumerBuilder1.subscribe(); // Make consume all messages for one topic, do not consume any messages for another one. producer0.send("1"); @@ -97,18 +135,247 @@ public void testCreateConsumerAfterOnePartDeleted() throws Exception { }); // Verify that the consumer subscribed with partitioned topic can be created successful. - Consumer consumerAllPartition = pulsarClient.newConsumer(Schema.STRING).topic(topic) - .subscriptionName(subscription).isAckReceiptEnabled(true).subscribe(); - Message msg = consumerAllPartition.receive(2, TimeUnit.SECONDS); + ConsumerBuilder consumerBuilder2 = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared); + setSubscribeTopic(consumerBuilder2, subscribeTopicType, topic, topicPattern); + Consumer consumer2 = consumerBuilder2.subscribe(); + Message msg = consumer2.receive(2, TimeUnit.SECONDS); + String receivedMsgValue = msg.getValue(); + log.info("received msg: {}", receivedMsgValue); + consumer2.acknowledge(msg); + + // cleanup. + consumer2.close(); + producer0.close(); + producer1.close(); + admin.topics().deletePartitionedTopic(topic); + } + + @Test(dataProvider = "subscribeTopicTypes", timeOut = 300 * 1000) + public void testAppendCreateConsumerAfterOnePartGc(SubscribeTopicType subscribeTopicType) throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topicPattern = "persistent://public/default/tp.*"; + final String partition0 = topic + "-partition-0"; + final String partition1 = topic + "-partition-1"; + final String subscription = "s1"; + admin.topics().createPartitionedTopic(topic, 2); + admin.topics().createSubscription(topic, subscription, MessageId.earliest); + + // create consumers and producers. + Producer producer0 = pulsarClient.newProducer(Schema.STRING).topic(partition0) + .enableBatching(false).create(); + Producer producer1 = pulsarClient.newProducer(Schema.STRING).topic(partition1) + .enableBatching(false).create(); + ConsumerBuilder consumerBuilder1 = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared); + setSubscribeTopic(consumerBuilder1, subscribeTopicType, topic, topicPattern); + Consumer consumer1 = consumerBuilder1.subscribe(); + + // Make consume all messages for one topic, do not consume any messages for another one. + producer0.send("partition-0-1"); + producer1.send("partition-1-1"); + producer1.send("partition-1-2"); + producer1.send("partition-1-4"); + admin.topics().skipAllMessages(partition0, subscription); + + // Wait for topic GC. + // Partition 0 will be deleted about 20s later, left 2min to avoid flaky. + producer0.close(); + Awaitility.await().atMost(2, TimeUnit.MINUTES).untilAsserted(() -> { + CompletableFuture> tp1 = pulsar.getBrokerService().getTopic(partition0, false); + CompletableFuture> tp2 = pulsar.getBrokerService().getTopic(partition1, false); + assertTrue(tp1 == null || !tp1.get().isPresent()); + assertTrue(tp2 != null && tp2.get().isPresent()); + }); + + // Verify that the messages under "partition-1" still can be ack. + for (int i = 0; i < 2; i++) { + Message msg = consumer1.receive(2, TimeUnit.SECONDS); + assertNotNull(msg, "Expected at least received 2 messages."); + log.info("received msg[{}]: {}", i, msg.getValue()); + TopicMessageId messageId = (TopicMessageId) msg.getMessageId(); + if (messageId.getOwnerTopic().equals(partition1)) { + consumer1.acknowledgeAsync(msg); + } + } + consumer1.close(); + + // Verify that the consumer subscribed with partitioned topic can be created successful. + ConsumerBuilder consumerBuilder2 = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared); + setSubscribeTopic(consumerBuilder2, subscribeTopicType, topic, topicPattern); + Consumer consumer2 = consumerBuilder2.subscribe(); + producer1.send("partition-1-5"); + Message msg = consumer2.receive(2, TimeUnit.SECONDS); assertNotNull(msg); String receivedMsgValue = msg.getValue(); log.info("received msg: {}", receivedMsgValue); - consumerAllPartition.acknowledge(msg); + consumer2.acknowledge(msg); // cleanup. - consumerAllPartition.close(); + consumer2.close(); producer0.close(); producer1.close(); admin.topics().deletePartitionedTopic(topic); } + + @Test(timeOut = 180 * 1000) + public void testPhasePartDeletion() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topicPattern = "persistent://public/default/tp.*"; + final String partition0 = topic + "-partition-0"; + final String partition1 = topic + "-partition-1"; + final String partition2 = topic + "-partition-2"; + final String subscription = "s1"; + admin.topics().createPartitionedTopic(topic, 3); + // Create consumer. + PatternMultiTopicsConsumerImpl c1 = (PatternMultiTopicsConsumerImpl) pulsarClient + .newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared) + .topicsPattern(Pattern.compile(topicPattern)).subscribe(); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertEquals(partitionedTopics.get(topic), 3); + assertEquals(consumers.size(), 3); + assertTrue(consumers.containsKey(partition0)); + assertTrue(consumers.containsKey(partition1)); + assertTrue(consumers.containsKey(partition2)); + }); + // Delete partitions the first time. + admin.topics().delete(partition0, true); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertEquals(partitionedTopics.get(topic), 3); + assertEquals(consumers.size(), 2); + assertTrue(consumers.containsKey(partition1)); + assertTrue(consumers.containsKey(partition2)); + }); + // Delete partitions the second time. + admin.topics().delete(partition1, true); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertEquals(partitionedTopics.get(topic), 3); + assertEquals(consumers.size(), 1); + assertTrue(consumers.containsKey(partition2)); + }); + // Delete partitions the third time. + admin.topics().delete(partition2, true); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 0); + assertEquals(consumers.size(), 0); + }); + + // cleanup. + c1.close(); + admin.topics().deletePartitionedTopic(topic); + } + + @Test(timeOut = 180 * 1000) + public void testExpandPartitions() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topicPattern = "persistent://public/default/tp.*"; + final String partition0 = topic + "-partition-0"; + final String partition1 = topic + "-partition-1"; + final String subscription = "s1"; + admin.topics().createPartitionedTopic(topic, 2); + // Delete partitions. + admin.topics().delete(partition0, true); + admin.topics().delete(partition1, true); + // Create consumer. + PatternMultiTopicsConsumerImpl c1 = (PatternMultiTopicsConsumerImpl) pulsarClient + .newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared) + .topicsPattern(Pattern.compile(topicPattern)).subscribe(); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 0); + assertEquals(consumers.size(), 0); + }); + // Trigger partitions creation. + pulsarClient.newConsumer(Schema.STRING).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).topic(topic).subscribe().close(); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertEquals(partitionedTopics.get(topic), 2); + assertEquals(consumers.size(), 2); + assertTrue(consumers.containsKey(partition0)); + assertTrue(consumers.containsKey(partition1)); + }); + // Expand partitions the first time. + admin.topics().updatePartitionedTopic(topic, 3); + final String partition2 = topic + "-partition-2"; + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertEquals(partitionedTopics.get(topic), 3); + assertEquals(consumers.size(), 3); + assertTrue(consumers.containsKey(partition0)); + assertTrue(consumers.containsKey(partition1)); + assertTrue(consumers.containsKey(partition2)); + }); + // Expand partitions the second time. + admin.topics().updatePartitionedTopic(topic, 4); + final String partition3 = topic + "-partition-3"; + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertEquals(partitionedTopics.get(topic), 4); + assertEquals(consumers.size(), 4); + assertTrue(consumers.containsKey(partition0)); + assertTrue(consumers.containsKey(partition1)); + assertTrue(consumers.containsKey(partition2)); + assertTrue(consumers.containsKey(partition3)); + }); + + // cleanup. + c1.close(); + admin.topics().deletePartitionedTopic(topic); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternMultiTopicsConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternMultiTopicsConsumerTest.java index 00a47c3957150..475477ac52149 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternMultiTopicsConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternMultiTopicsConsumerTest.java @@ -18,11 +18,14 @@ */ package org.apache.pulsar.client.api; +import static org.testng.Assert.fail; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.impl.PatternMultiTopicsConsumerImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -95,4 +98,38 @@ private void testWithConsumer(Consumer consumer) throws Exception { consumer.close(); } + @Test(timeOut = 30000) + public void testFailedSubscribe() throws Exception { + final String topicName1 = BrokerTestUtil.newUniqueName("persistent://public/default/tp_test"); + final String topicName2 = BrokerTestUtil.newUniqueName("persistent://public/default/tp_test"); + final String topicName3 = BrokerTestUtil.newUniqueName("persistent://public/default/tp_test"); + final String subName = "s1"; + admin.topics().createPartitionedTopic(topicName1, 2); + admin.topics().createPartitionedTopic(topicName2, 3); + admin.topics().createNonPartitionedTopic(topicName3); + + // Register a exclusive consumer to makes the pattern consumer failed to subscribe. + Consumer c1 = pulsarClient.newConsumer(Schema.STRING).topic(topicName3).subscriptionType(SubscriptionType.Exclusive) + .subscriptionName(subName).subscribe(); + + try { + PatternMultiTopicsConsumerImpl consumer = + (PatternMultiTopicsConsumerImpl) pulsarClient.newConsumer(Schema.STRING) + .topicsPattern("persistent://public/default/tp_test.*") + .subscriptionType(SubscriptionType.Failover) + .subscriptionName(subName) + .subscribe(); + fail("Expected a consumer busy error."); + } catch (Exception ex) { + log.info("consumer busy", ex); + } + + c1.close(); + // Verify all internal consumer will be closed. + // If delete topic without "-f" work, it means the internal consumers were closed. + admin.topics().delete(topicName3); + admin.topics().deletePartitionedTopic(topicName2); + admin.topics().deletePartitionedTopic(topicName1); + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java index c5504a0c02a0c..9c19fadffb137 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java @@ -19,7 +19,6 @@ package org.apache.pulsar.client.impl; import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertSame; import static org.testng.Assert.assertTrue; @@ -35,7 +34,6 @@ import java.util.regex.Pattern; import java.util.stream.IntStream; -import io.netty.util.Timeout; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.client.api.Consumer; @@ -53,6 +51,7 @@ import org.apache.pulsar.common.api.proto.BaseCommand; import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; @@ -1024,17 +1023,17 @@ public void testAutoUnsubscribePatternConsumer() throws Exception { // 6. remove producer 1,3; verify only consumer 2 left // seems no direct way to verify auto-unsubscribe, because this patternConsumer also referenced the topic. - List topicNames = Lists.newArrayList(topicName2); + String tp2p0 = TopicName.get(topicName2).getPartition(0).toString(); + String tp2p1 = TopicName.get(topicName2).getPartition(1).toString(); + List topicNames = Lists.newArrayList(tp2p0, tp2p1); NamespaceService nss = pulsar.getNamespaceService(); doReturn(CompletableFuture.completedFuture(topicNames)).when(nss) .getListOfPersistentTopics(NamespaceName.get("my-property/my-ns")); // 7. call recheckTopics to unsubscribe topic 1,3, verify topics number: 2=6-1-3 log.debug("recheck topics change"); - PatternMultiTopicsConsumerImpl consumer1 = ((PatternMultiTopicsConsumerImpl) consumer); - Timeout recheckPatternTimeout = spy(consumer1.getRecheckPatternTimeout()); - doReturn(false).when(recheckPatternTimeout).isCancelled(); - consumer1.run(recheckPatternTimeout); + PatternConsumerUpdateQueue taskQueue = WhiteboxImpl.getInternalState(consumer, "updateTaskQueue"); + taskQueue.appendRecheckOp(); Thread.sleep(100); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 2); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 2); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java index c343ab0d6e294..83cb5f2a4400b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import io.netty.util.Timeout; +import java.time.Duration; import lombok.Cleanup; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; @@ -1321,7 +1322,6 @@ public void testPartitionsUpdatesForMultipleTopics() throws Exception { Assert.assertEquals(consumer.allTopicPartitionsNumber.intValue(), 2); admin.topics().updatePartitionedTopic(topicName0, 5); - consumer.getPartitionsAutoUpdateTimeout().task().run(consumer.getPartitionsAutoUpdateTimeout()); Awaitility.await().untilAsserted(() -> { Assert.assertEquals(consumer.getPartitionsOfTheTopicMap(), 5); @@ -1341,9 +1341,8 @@ public void testPartitionsUpdatesForMultipleTopics() throws Exception { }); admin.topics().updatePartitionedTopic(topicName1, 5); - consumer.getPartitionsAutoUpdateTimeout().task().run(consumer.getPartitionsAutoUpdateTimeout()); - Awaitility.await().untilAsserted(() -> { + Awaitility.await().atMost(Duration.ofSeconds(30)).untilAsserted(() -> { Assert.assertEquals(consumer.getPartitionsOfTheTopicMap(), 10); Assert.assertEquals(consumer.allTopicPartitionsNumber.intValue(), 10); }); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 05081dcaa07ea..74abb82bfe809 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Queues; import io.netty.util.Timeout; import java.nio.charset.StandardCharsets; @@ -1285,5 +1286,10 @@ public boolean hasBatchReceiveTimeout() { return batchReceiveTimeout != null; } + @VisibleForTesting + CompletableFuture> getSubscribeFuture() { + return subscribeFuture; + } + private static final Logger log = LoggerFactory.getLogger(ConsumerBase.class); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java index ccd1f6b23f2f3..2fe457059c1e9 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java @@ -117,7 +117,14 @@ CompletableFuture getPartitionedTopicMetadata(TopicNam InetSocketAddress resolveHost(); /** - * Returns all the topics name for a given namespace. + * Returns all the topics that matches {@param topicPattern} for a given namespace. + * + * Note: {@param topicPattern} it relate to the topic name(without the partition suffix). For example: + * - There is a partitioned topic "tp-a" with two partitions. + * - tp-a-partition-0 + * - tp-a-partition-1 + * - If {@param topicPattern} is "tp-a", the consumer will subscribe to the two partitions. + * - if {@param topicPattern} is "tp-a-partition-0", the consumer will subscribe nothing. * * @param namespace : namespace-name * @return diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 8047e05351ac1..e8cbf71e500c9 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -47,6 +47,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; +import java.util.function.Predicate; +import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.IntStream; import javax.annotation.Nullable; @@ -68,6 +70,7 @@ import org.apache.pulsar.client.util.ConsumerName; import org.apache.pulsar.client.util.ExecutorProvider; import org.apache.pulsar.common.api.proto.CommandAck.AckType; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.util.CompletableFutureCancellationHandler; @@ -81,14 +84,14 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { public static final String DUMMY_TOPIC_NAME_PREFIX = "MultiTopicsConsumer-"; // Map , when get do ACK, consumer will by find by topic name - private final ConcurrentHashMap> consumers; + protected final ConcurrentHashMap> consumers; // Map , store partition number for each topic protected final ConcurrentHashMap partitionedTopics; // Queue of partition consumers on which we have stopped calling receiveAsync() because the // shared incoming queue was full - private final ConcurrentLinkedQueue> pausedConsumers; + protected final ConcurrentLinkedQueue> pausedConsumers; // sum of topicPartitions, simple topic has 1, partitioned topic equals to partition number. AtomicInteger allTopicPartitionsNumber; @@ -1009,8 +1012,12 @@ CompletableFuture subscribeAsync(String topicName, int numberPartitions) { new PulsarClientException.AlreadyClosedException("Topic name not valid")); } String fullTopicName = topicNameInstance.toString(); - if (consumers.containsKey(fullTopicName) - || partitionedTopics.containsKey(topicNameInstance.getPartitionedTopicName())) { + if (consumers.containsKey(fullTopicName)) { + return FutureUtil.failedFuture( + new PulsarClientException.AlreadyClosedException("Already subscribed to " + topicName)); + } + if (!topicNameInstance.isPartitioned() + && partitionedTopics.containsKey(topicNameInstance.getPartitionedTopicName())) { return FutureUtil.failedFuture( new PulsarClientException.AlreadyClosedException("Already subscribed to " + topicName)); } @@ -1047,7 +1054,7 @@ private void doSubscribeTopicPartitions(Schema schema, log.debug("Subscribe to topic {} metadata.partitions: {}", topicName, numPartitions); } - List>> futureList; + CompletableFuture subscribeAllPartitionsFuture; if (numPartitions != PartitionedTopicMetadata.NON_PARTITIONED) { // Below condition is true if subscribeAsync() has been invoked second time with same // topicName before the first invocation had reached this point. @@ -1067,30 +1074,50 @@ private void doSubscribeTopicPartitions(Schema schema, ConsumerConfigurationData configurationData = getInternalConsumerConfig(); configurationData.setReceiverQueueSize(receiverQueueSize); - futureList = IntStream - .range(0, numPartitions) - .mapToObj( - partitionIndex -> { - String partitionName = TopicName.get(topicName).getPartition(partitionIndex).toString(); - CompletableFuture> subFuture = new CompletableFuture<>(); - configurationData.setStartPaused(paused); - ConsumerImpl newConsumer = createInternalConsumer(configurationData, partitionName, - partitionIndex, subFuture, createIfDoesNotExist, schema); - synchronized (pauseMutex) { - if (paused) { - newConsumer.pause(); - } else { - newConsumer.resume(); - } - consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); + CompletableFuture> partitionsFuture; + if (createIfDoesNotExist || !TopicName.get(topicName).isPersistent()) { + partitionsFuture = CompletableFuture.completedFuture(IntStream.range(0, numPartitions) + .mapToObj(i -> Integer.valueOf(i)) + .collect(Collectors.toList())); + } else { + partitionsFuture = getExistsPartitions(topicName.toString()); + } + subscribeAllPartitionsFuture = partitionsFuture.thenCompose(partitions -> { + if (partitions.isEmpty()) { + partitionedTopics.remove(topicName, numPartitions); + return CompletableFuture.completedFuture(null); + } + List>> subscribeList = new ArrayList<>(); + for (int partitionIndex : partitions) { + String partitionName = TopicName.get(topicName).getPartition(partitionIndex).toString(); + CompletableFuture> subFuture = new CompletableFuture<>(); + configurationData.setStartPaused(paused); + ConsumerImpl newConsumer = createInternalConsumer(configurationData, partitionName, + partitionIndex, subFuture, createIfDoesNotExist, schema); + synchronized (pauseMutex) { + if (paused) { + newConsumer.pause(); + } else { + newConsumer.resume(); } - return subFuture; - }) - .collect(Collectors.toList()); + Consumer originalValue = consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); + if (originalValue != null) { + newConsumer.closeAsync().exceptionally(ex -> { + log.error("[{}] [{}] Failed to close the orphan consumer", + partitionName, subscription, ex); + return null; + }); + } + } + subscribeList.add(subFuture); + } + return FutureUtil.waitForAll(subscribeList); + }); } else { allTopicPartitionsNumber.incrementAndGet(); - CompletableFuture> subFuture = new CompletableFuture<>(); + CompletableFuture> subscribeFuture = new CompletableFuture<>(); + subscribeAllPartitionsFuture = subscribeFuture.thenAccept(__ -> {}); synchronized (pauseMutex) { consumers.compute(topicName, (key, existingValue) -> { @@ -1104,7 +1131,7 @@ private void doSubscribeTopicPartitions(Schema schema, } else { internalConfig.setStartPaused(paused); ConsumerImpl newConsumer = createInternalConsumer(internalConfig, topicName, - -1, subFuture, createIfDoesNotExist, schema); + -1, subscribeFuture, createIfDoesNotExist, schema); if (paused) { newConsumer.pause(); } else { @@ -1114,11 +1141,10 @@ private void doSubscribeTopicPartitions(Schema schema, } }); } - futureList = Collections.singletonList(subFuture); + } - FutureUtil.waitForAll(futureList) - .thenAccept(finalFuture -> { + subscribeAllPartitionsFuture.thenAccept(finalFuture -> { if (allTopicPartitionsNumber.get() > getCurrentReceiverQueueSize()) { setCurrentReceiverQueueSize(allTopicPartitionsNumber.get()); } @@ -1139,6 +1165,8 @@ private void doSubscribeTopicPartitions(Schema schema, return; }) .exceptionally(ex -> { + log.warn("[{}] Failed to subscribe for topic [{}] in topics consumer {}", topic, topicName, + ex.getMessage()); handleSubscribeOneTopicError(topicName, ex, subscribeResult); return null; }); @@ -1162,7 +1190,7 @@ private ConsumerImpl createInternalConsumer(ConsumerConfigurationData conf } // handling failure during subscribe new topic, unsubscribe success created partitions - private void handleSubscribeOneTopicError(String topicName, + protected void handleSubscribeOneTopicError(String topicName, Throwable error, CompletableFuture subscribeFuture) { log.warn("[{}] Failed to subscribe for topic [{}] in topics consumer {}", topic, topicName, error.getMessage()); @@ -1255,59 +1283,6 @@ public CompletableFuture unsubscribeAsync(String topicName) { return unsubscribeFuture; } - /*** - * Remove a consumer for a topic. - * @param topicName topic name contains the partition suffix. - */ - public CompletableFuture removeConsumerAsync(String topicName) { - checkArgument(TopicName.isValid(topicName), "Invalid topic name:" + topicName); - - if (getState() == State.Closing || getState() == State.Closed) { - return FutureUtil.failedFuture( - new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); - } - - CompletableFuture unsubscribeFuture = new CompletableFuture<>(); - String topicPartName = TopicName.get(topicName).getPartitionedTopicName(); - - - List> consumersToClose = consumers.values().stream() - .filter(consumer -> { - String consumerTopicName = consumer.getTopic(); - return TopicName.get(consumerTopicName).getPartitionedTopicName().equals(topicPartName); - }).collect(Collectors.toList()); - - List> futureList = consumersToClose.stream() - .map(ConsumerImpl::closeAsync).collect(Collectors.toList()); - - FutureUtil.waitForAll(futureList) - .whenComplete((r, ex) -> { - if (ex == null) { - consumersToClose.forEach(consumer1 -> { - consumers.remove(consumer1.getTopic()); - pausedConsumers.remove(consumer1); - allTopicPartitionsNumber.decrementAndGet(); - }); - - removeTopic(topicName); - if (unAckedMessageTracker instanceof UnAckedTopicMessageTracker) { - ((UnAckedTopicMessageTracker) unAckedMessageTracker).removeTopicMessages(topicName); - } - - unsubscribeFuture.complete(null); - log.info("[{}] [{}] [{}] Removed Topics Consumer, allTopicPartitionsNumber: {}", - topicName, subscription, consumerName, allTopicPartitionsNumber); - } else { - unsubscribeFuture.completeExceptionally(ex); - setState(State.Failed); - log.error("[{}] [{}] [{}] Could not remove Topics Consumer", - topicName, subscription, consumerName, ex.getCause()); - } - }); - - return unsubscribeFuture; - } - // get topics name public List getPartitionedTopics() { @@ -1573,4 +1548,51 @@ protected void setCurrentReceiverQueueSize(int newSize) { CURRENT_RECEIVER_QUEUE_SIZE_UPDATER.set(this, newSize); resumeReceivingFromPausedConsumersIfNeeded(); } + + /** + * Get the exists partitions of a partitioned topic, the result does not contain the partitions which has not been + * created yet(in other words, the partitions that do not exist in the response of "pulsar-admin topics list"). + * @return sorted partitions list if it is a partitioned topic; @return an empty list if it is a non-partitioned + * topic. + */ + private CompletableFuture> getExistsPartitions(String topic) { + TopicName topicName = TopicName.get(topic); + if (!topicName.isPersistent()) { + return FutureUtil.failedFuture(new IllegalArgumentException("The method getExistsPartitions" + + " does not support non-persistent topic yet.")); + } + return client.getLookup().getTopicsUnderNamespace(topicName.getNamespaceObject(), + CommandGetTopicsOfNamespace.Mode.PERSISTENT, + TopicName.getPattern(topicName.getPartitionedTopicName()), + null).thenApply(getTopicsResult -> { + if (getTopicsResult.getNonPartitionedOrPartitionTopics() == null + || getTopicsResult.getNonPartitionedOrPartitionTopics().isEmpty()) { + return Collections.emptyList(); + } + // If broker version is less than "2.11.x", it does not support broker-side pattern check, so append + // a client-side pattern check. + // If lookup service is typed HttpLookupService, the HTTP API does not support broker-side pattern + // check yet, so append a client-side pattern check. + Predicate clientSideFilter; + if (getTopicsResult.isFiltered()) { + clientSideFilter = __ -> true; + } else { + clientSideFilter = + tp -> Pattern.compile(TopicName.getPartitionPattern(topic)).matcher(tp).matches(); + } + ArrayList list = new ArrayList<>(getTopicsResult.getNonPartitionedOrPartitionTopics().size()); + for (String partition : getTopicsResult.getNonPartitionedOrPartitionTopics()) { + int partitionIndex = TopicName.get(partition).getPartitionIndex(); + if (partitionIndex < 0) { + // It is not a partition. + continue; + } + if (clientSideFilter.test(partition)) { + list.add(partitionIndex); + } + } + Collections.sort(list); + return list; + }); + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java new file mode 100644 index 0000000000000..d6eba6463a07d --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -0,0 +1,254 @@ +/* + * 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.impl; + +import com.google.common.annotations.VisibleForTesting; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.tuple.Pair; + +/** + * Used to make all tasks that will modify subscriptions will be executed one by one, and skip the unnecessary updating. + * + * So far, four three scenarios that will modify subscriptions: + * 1. When start pattern consumer. + * 2. After topic list watcher reconnected, it will call {@link PatternMultiTopicsConsumerImpl#recheckTopicsChange()}. + * this scenario only exists in the version >= 2.11 (both client-version and broker version are >= 2.11). + * 3. A scheduled task will call {@link PatternMultiTopicsConsumerImpl#recheckTopicsChange()}, this scenario only + * exists in the version < 2.11. + * 4. The topics change events will trigger a + * {@link PatternMultiTopicsConsumerImpl#topicsChangeListener#onTopicsRemoved(Collection)} or + * {@link PatternMultiTopicsConsumerImpl#topicsChangeListener#onTopicsAdded(Collection)}. + * + * When you are using this client connect to the broker whose version >= 2.11, there are three scenarios: [1, 2, 4]. + * When you are using this client connect to the broker whose version < 2.11, there is only one scenario: [3] and all + * the event will run in the same thread. + */ +@Slf4j +@SuppressFBWarnings("EI_EXPOSE_REP2") +public class PatternConsumerUpdateQueue { + + private static final Pair> RECHECK_OP = + Pair.of(UpdateSubscriptionType.RECHECK, null); + + private final LinkedBlockingQueue>> pendingTasks; + + private final PatternMultiTopicsConsumerImpl patternConsumer; + + private final PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener; + + /** + * Whether there is a task is in progress, this variable is used to confirm whether a next-task triggering is + * needed. + */ + private Pair> taskInProgress = null; + + /** + * Whether there is a recheck task in queue. + * - Since recheck task will do all changes, it can be used to compress multiple tasks to one. + * - To avoid skipping the newest changes, once the recheck task is starting to work, this variable will be set + * to "false". + */ + private boolean recheckTaskInQueue = false; + + private volatile long lastRecheckTaskStartingTimestamp = 0; + + private boolean closed; + + public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer) { + this(patternConsumer, patternConsumer.topicsChangeListener); + } + + /** This constructor is only for test. **/ + @VisibleForTesting + public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer, + PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener) { + this.patternConsumer = patternConsumer; + this.topicsChangeListener = topicsChangeListener; + this.pendingTasks = new LinkedBlockingQueue<>(); + // To avoid subscribing and topics changed events execute concurrently, let the change events starts after the + // subscribing task. + doAppend(Pair.of(UpdateSubscriptionType.CONSUMER_INIT, null)); + } + + synchronized void appendTopicsAddedOp(Collection topics) { + if (topics == null || topics.isEmpty()) { + return; + } + doAppend(Pair.of(UpdateSubscriptionType.TOPICS_ADDED, topics)); + } + + synchronized void appendTopicsRemovedOp(Collection topics) { + if (topics == null || topics.isEmpty()) { + return; + } + doAppend(Pair.of(UpdateSubscriptionType.TOPICS_REMOVED, topics)); + } + + synchronized void appendRecheckOp() { + doAppend(RECHECK_OP); + } + + synchronized void doAppend(Pair> task) { + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] try to append task. {} {}", patternConsumer.getSubscription(), + task.getLeft(), task.getRight() == null ? "" : task.getRight()); + } + // Once there is a recheck task in queue, it means other tasks can be skipped. + if (recheckTaskInQueue) { + return; + } + + // Once there are too many tasks in queue, compress them as a recheck task. + if (pendingTasks.size() >= 30 && !task.getLeft().equals(UpdateSubscriptionType.RECHECK)) { + appendRecheckOp(); + return; + } + + pendingTasks.add(task); + if (task.getLeft().equals(UpdateSubscriptionType.RECHECK)) { + recheckTaskInQueue = true; + } + + // If no task is in-progress, trigger a task execution. + if (taskInProgress == null) { + triggerNextTask(); + } + } + + synchronized void triggerNextTask() { + if (closed) { + return; + } + + final Pair> task = pendingTasks.poll(); + + // No pending task. + if (task == null) { + taskInProgress = null; + return; + } + + // If there is a recheck task in queue, skip others and only call the recheck task. + if (recheckTaskInQueue && !task.getLeft().equals(UpdateSubscriptionType.RECHECK)) { + triggerNextTask(); + return; + } + + // Execute pending task. + CompletableFuture newTaskFuture = null; + switch (task.getLeft()) { + case CONSUMER_INIT: { + newTaskFuture = patternConsumer.getSubscribeFuture().thenAccept(__ -> {}).exceptionally(ex -> { + // If the subscribe future was failed, the consumer will be closed. + synchronized (PatternConsumerUpdateQueue.this) { + this.closed = true; + patternConsumer.closeAsync().exceptionally(ex2 -> { + log.error("Pattern consumer failed to close, this error may left orphan consumers." + + " Subscription: {}", patternConsumer.getSubscription()); + return null; + }); + } + return null; + }); + break; + } + case TOPICS_ADDED: { + newTaskFuture = topicsChangeListener.onTopicsAdded(task.getRight()); + break; + } + case TOPICS_REMOVED: { + newTaskFuture = topicsChangeListener.onTopicsRemoved(task.getRight()); + break; + } + case RECHECK: { + recheckTaskInQueue = false; + lastRecheckTaskStartingTimestamp = System.currentTimeMillis(); + newTaskFuture = patternConsumer.recheckTopicsChange(); + break; + } + default: { + throw new RuntimeException("Un-support UpdateSubscriptionType"); + } + } + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] starting task. {} {} ", patternConsumer.getSubscription(), + task.getLeft(), task.getRight() == null ? "" : task.getRight()); + } + // Trigger next pending task. + taskInProgress = Pair.of(task.getLeft(), newTaskFuture); + newTaskFuture.thenAccept(ignore -> { + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] task finished. {} {} ", patternConsumer.getSubscription(), + task.getLeft(), task.getRight() == null ? "" : task.getRight()); + } + triggerNextTask(); + }).exceptionally(ex -> { + /** + * Once a updating fails, trigger a delayed new recheck task to guarantee all things is correct. + * - Skip if there is already a recheck task in queue. + * - Skip if the last recheck task has been executed after the current time. + */ + log.error("Pattern consumer [{}] task finished. {} {}. But it failed", patternConsumer.getSubscription(), + task.getLeft(), task.getRight() == null ? "" : task.getRight(), ex); + // Skip if there is already a recheck task in queue. + synchronized (PatternConsumerUpdateQueue.this) { + if (recheckTaskInQueue || PatternConsumerUpdateQueue.this.closed) { + return null; + } + } + // Skip if the last recheck task has been executed after the current time. + long failedTime = System.currentTimeMillis(); + patternConsumer.getClient().timer().newTimeout(timeout -> { + if (lastRecheckTaskStartingTimestamp <= failedTime) { + appendRecheckOp(); + } + }, 10, TimeUnit.SECONDS); + triggerNextTask(); + return null; + }); + } + + public synchronized CompletableFuture cancelAllAndWaitForTheRunningTask() { + this.closed = true; + if (taskInProgress == null) { + return CompletableFuture.completedFuture(null); + } + // If the in-progress task is consumer init task, it means nothing is in-progress. + if (taskInProgress.getLeft().equals(UpdateSubscriptionType.CONSUMER_INIT)) { + return CompletableFuture.completedFuture(null); + } + return taskInProgress.getRight().thenAccept(__ -> {}).exceptionally(ex -> null); + } + + private enum UpdateSubscriptionType { + /** A marker that indicates the consumer's subscribe task.**/ + CONSUMER_INIT, + /** Triggered by {@link PatternMultiTopicsConsumerImpl#topicsChangeListener}.**/ + TOPICS_ADDED, + /** Triggered by {@link PatternMultiTopicsConsumerImpl#topicsChangeListener}.**/ + TOPICS_REMOVED, + /** A fully check for pattern consumer. **/ + RECHECK; + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index ffca79dfa4342..70ba3e33963f4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -28,12 +28,12 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; @@ -42,6 +42,7 @@ import org.apache.pulsar.common.lookup.GetTopicsResult; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.common.util.BackoffBuilder; @@ -51,7 +52,7 @@ public class PatternMultiTopicsConsumerImpl extends MultiTopicsConsumerImpl implements TimerTask { private final Pattern topicsPattern; - private final TopicsChangedListener topicsChangeListener; + final TopicsChangedListener topicsChangeListener; private final Mode subscriptionMode; private final CompletableFuture watcherFuture = new CompletableFuture<>(); protected NamespaceName namespaceName; @@ -69,6 +70,8 @@ public class PatternMultiTopicsConsumerImpl extends MultiTopicsConsumerImpl recheckTopicsChangeAfterReconnect()); watcherFuture .thenAccept(__ -> recheckPatternTimeout.cancel()) .exceptionally(ex -> { - log.warn("Unable to create topic list watcher. Falling back to only polling for new topics", ex); + log.warn("Pattern consumer [{}] unable to create topic list watcher. Falling back to only polling" + + " for new topics", conf.getSubscriptionName(), ex); return null; }); } else { - log.debug("Not creating topic list watcher for subscription mode {}", subscriptionMode); + log.debug("Pattern consumer [{}] not creating topic list watcher for subscription mode {}", + conf.getSubscriptionName(), subscriptionMode); watcherFuture.complete(null); } } @@ -129,17 +135,7 @@ private void recheckTopicsChangeAfterReconnect() { return; } // Do check. - recheckTopicsChange().whenComplete((ignore, ex) -> { - if (ex != null) { - log.warn("[{}] Failed to recheck topics change: {}", topic, ex.getMessage()); - long delayMs = recheckPatternTaskBackoff.next(); - client.timer().newTimeout(timeout -> { - recheckTopicsChangeAfterReconnect(); - }, delayMs, TimeUnit.MILLISECONDS); - } else { - recheckPatternTaskBackoff.reset(); - } - }); + updateTaskQueue.appendRecheckOp(); } // TimerTask to recheck topics change, and trigger subscribe/unsubscribe based on the change. @@ -148,18 +144,10 @@ public void run(Timeout timeout) throws Exception { if (timeout.isCancelled()) { return; } - recheckTopicsChange().exceptionally(ex -> { - log.warn("[{}] Failed to recheck topics change: {}", topic, ex.getMessage()); - return null; - }).thenAccept(__ -> { - // schedule the next re-check task - this.recheckPatternTimeout = client.timer() - .newTimeout(PatternMultiTopicsConsumerImpl.this, - Math.max(1, conf.getPatternAutoDiscoveryPeriod()), TimeUnit.SECONDS); - }); + updateTaskQueue.appendRecheckOp(); } - private CompletableFuture recheckTopicsChange() { + CompletableFuture recheckTopicsChange() { String pattern = topicsPattern.pattern(); final int epoch = recheckPatternEpoch.incrementAndGet(); return client.getLookup().getTopicsUnderNamespace(namespaceName, subscriptionMode, pattern, topicsHash) @@ -172,22 +160,18 @@ private CompletableFuture recheckTopicsChange() { return CompletableFuture.completedFuture(null); } if (log.isDebugEnabled()) { - log.debug("Get topics under namespace {}, topics.size: {}, topicsHash: {}, filtered: {}", + log.debug("Pattern consumer [{}] get topics under namespace {}, topics.size: {}," + + " topicsHash: {}, filtered: {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), namespaceName, getTopicsResult.getTopics().size(), getTopicsResult.getTopicsHash(), getTopicsResult.isFiltered()); getTopicsResult.getTopics().forEach(topicName -> log.debug("Get topics under namespace {}, topic: {}", namespaceName, topicName)); } - final List oldTopics = new ArrayList<>(getPartitionedTopics()); - for (String partition : getPartitions()) { - TopicName topicName = TopicName.get(partition); - if (!topicName.isPartitioned() || !oldTopics.contains(topicName.getPartitionedTopicName())) { - oldTopics.add(partition); - } - } + final List oldTopics = new ArrayList<>(getPartitions()); return updateSubscriptions(topicsPattern, this::setTopicsHash, getTopicsResult, - topicsChangeListener, oldTopics); + topicsChangeListener, oldTopics, subscription); } }); } @@ -196,7 +180,8 @@ static CompletableFuture updateSubscriptions(Pattern topicsPattern, java.util.function.Consumer topicsHashSetter, GetTopicsResult getTopicsResult, TopicsChangedListener topicsChangedListener, - List oldTopics) { + List oldTopics, + String subscriptionForLog) { topicsHashSetter.accept(getTopicsResult.getTopicsHash()); if (!getTopicsResult.isChanged()) { return CompletableFuture.completedFuture(null); @@ -204,14 +189,20 @@ static CompletableFuture updateSubscriptions(Pattern topicsPattern, List newTopics; if (getTopicsResult.isFiltered()) { - newTopics = getTopicsResult.getTopics(); + newTopics = getTopicsResult.getNonPartitionedOrPartitionTopics(); } else { - newTopics = TopicList.filterTopics(getTopicsResult.getTopics(), topicsPattern); + newTopics = getTopicsResult.filterTopics(topicsPattern).getNonPartitionedOrPartitionTopics(); } final List> listenersCallback = new ArrayList<>(2); - listenersCallback.add(topicsChangedListener.onTopicsAdded(TopicList.minus(newTopics, oldTopics))); - listenersCallback.add(topicsChangedListener.onTopicsRemoved(TopicList.minus(oldTopics, newTopics))); + Set topicsAdded = TopicList.minus(newTopics, oldTopics); + Set topicsRemoved = TopicList.minus(oldTopics, newTopics); + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] Recheck pattern consumer's topics. topicsAdded: {}, topicsRemoved: {}", + subscriptionForLog, topicsAdded, topicsRemoved); + } + listenersCallback.add(topicsChangedListener.onTopicsAdded(topicsAdded)); + listenersCallback.add(topicsChangedListener.onTopicsRemoved(topicsRemoved)); return FutureUtil.waitForAll(Collections.unmodifiableList(listenersCallback)); } @@ -247,23 +238,68 @@ private class PatternTopicsChangedListener implements TopicsChangedListener { */ @Override public CompletableFuture onTopicsRemoved(Collection removedTopics) { - CompletableFuture removeFuture = new CompletableFuture<>(); - if (removedTopics.isEmpty()) { - removeFuture.complete(null); - return removeFuture; + return CompletableFuture.completedFuture(null); } - List> futures = Lists.newArrayListWithExpectedSize(partitionedTopics.size()); - removedTopics.stream().forEach(topic -> futures.add(removeConsumerAsync(topic))); - FutureUtil.waitForAll(futures) - .thenAccept(finalFuture -> removeFuture.complete(null)) - .exceptionally(ex -> { - log.warn("[{}] Failed to unsubscribe from topics: {}", topic, ex.getMessage()); - removeFuture.completeExceptionally(ex); + // Unsubscribe and remove consumers in memory. + List> unsubscribeList = new ArrayList<>(removedTopics.size()); + Set partialRemoved = new HashSet<>(removedTopics.size()); + Set partialRemovedForLog = new HashSet<>(removedTopics.size()); + for (String tp : removedTopics) { + TopicName topicName = TopicName.get(tp); + ConsumerImpl consumer = consumers.get(topicName.toString()); + if (consumer != null) { + CompletableFuture unsubscribeFuture = new CompletableFuture<>(); + consumer.closeAsync().whenComplete((__, ex) -> { + if (ex != null) { + log.error("Pattern consumer [{}] failed to unsubscribe from topics: {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), topicName.toString(), ex); + unsubscribeFuture.completeExceptionally(ex); + } else { + consumers.remove(topicName.toString(), consumer); + unsubscribeFuture.complete(null); + } + }); + unsubscribeList.add(unsubscribeFuture); + partialRemoved.add(topicName.getPartitionedTopicName()); + partialRemovedForLog.add(topicName.toString()); + } + } + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] remove topics. {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), + partialRemovedForLog); + } + + // Remove partitioned topics in memory. + return FutureUtil.waitForAll(unsubscribeList).handle((__, ex) -> { + List removedPartitionedTopicsForLog = new ArrayList<>(); + for (String groupedTopicRemoved : partialRemoved) { + Integer partitions = partitionedTopics.get(groupedTopicRemoved); + if (partitions != null) { + boolean allPartitionsHasBeenRemoved = true; + for (int i = 0; i < partitions; i++) { + if (consumers.containsKey( + TopicName.get(groupedTopicRemoved).getPartition(i).toString())) { + allPartitionsHasBeenRemoved = false; + break; + } + } + if (allPartitionsHasBeenRemoved) { + removedPartitionedTopicsForLog.add(String.format("%s with %s partitions", + groupedTopicRemoved, partitions)); + partitionedTopics.remove(groupedTopicRemoved, partitions); + } + } + } + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] remove partitioned topics because all partitions have been" + + " removed. {}", PatternMultiTopicsConsumerImpl.this.getSubscription(), + removedPartitionedTopicsForLog); + } return null; }); - return removeFuture; } /** @@ -271,29 +307,90 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) */ @Override public CompletableFuture onTopicsAdded(Collection addedTopics) { - CompletableFuture addFuture = new CompletableFuture<>(); - if (addedTopics.isEmpty()) { - addFuture.complete(null); - return addFuture; + return CompletableFuture.completedFuture(null); } - - Set addTopicPartitionedName = addedTopics.stream() - .map(addTopicName -> TopicName.get(addTopicName).getPartitionedTopicName()) - .collect(Collectors.toSet()); - - List> futures = Lists.newArrayListWithExpectedSize(partitionedTopics.size()); - addTopicPartitionedName.forEach(partitionedTopic -> futures.add( - subscribeAsync(partitionedTopic, - false /* createTopicIfDoesNotExist */))); - FutureUtil.waitForAll(futures) - .thenAccept(finalFuture -> addFuture.complete(null)) - .exceptionally(ex -> { - log.warn("[{}] Failed to subscribe to topics: {}", topic, ex.getMessage()); - addFuture.completeExceptionally(ex); - return null; + List> futures = Lists.newArrayListWithExpectedSize(addedTopics.size()); + /** + * Three normal cases: + * 1. Expand partitions. + * 2. Non-partitioned topic, but has been subscribing. + * 3. Non-partitioned topic or Partitioned topic, but has not been subscribing. + * Two unexpected cases: + * Error-1: Received adding non-partitioned topic event, but has subscribed a partitioned topic with the + * same name. + * Error-2: Received adding partitioned topic event, but has subscribed a non-partitioned topic with the + * same name. + * + * Note: The events that triggered by {@link TopicsPartitionChangedListener} after expanding partitions has + * been disabled through "conf.setAutoUpdatePartitions(false)" when creating + * {@link PatternMultiTopicsConsumerImpl}. + */ + Set groupedTopics = new HashSet<>(); + List expendPartitionsForLog = new ArrayList<>(); + for (String tp : addedTopics) { + TopicName topicName = TopicName.get(tp); + groupedTopics.add(topicName.getPartitionedTopicName()); + } + for (String tp : addedTopics) { + TopicName topicName = TopicName.get(tp); + // Case 1: Expand partitions. + if (partitionedTopics.containsKey(topicName.getPartitionedTopicName())) { + if (consumers.containsKey(topicName.toString())) { + // Already subscribed. + } else if (topicName.getPartitionIndex() < 0) { + // Error-1: Received adding non-partitioned topic event, but has subscribed a partitioned topic + // with the same name. + log.error("Pattern consumer [{}] skip to subscribe to the non-partitioned topic {}, because has" + + "subscribed a partitioned topic with the same name", + PatternMultiTopicsConsumerImpl.this.getSubscription(), topicName.toString()); + } else { + if (topicName.getPartitionIndex() + 1 + > partitionedTopics.get(topicName.getPartitionedTopicName())) { + partitionedTopics.put(topicName.getPartitionedTopicName(), + topicName.getPartitionIndex() + 1); + } + expendPartitionsForLog.add(topicName.toString()); + CompletableFuture consumerFuture = subscribeAsync(topicName.toString(), + PartitionedTopicMetadata.NON_PARTITIONED); + consumerFuture.whenComplete((__, ex) -> { + if (ex != null) { + log.warn("Pattern consumer [{}] Failed to subscribe to topics: {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), topicName, ex); + } + }); + futures.add(consumerFuture); + } + groupedTopics.remove(topicName.getPartitionedTopicName()); + } else if (consumers.containsKey(topicName.toString())) { + // Case-2: Non-partitioned topic, but has been subscribing. + groupedTopics.remove(topicName.getPartitionedTopicName()); + } else if (consumers.containsKey(topicName.getPartitionedTopicName()) + && topicName.getPartitionIndex() >= 0) { + // Error-2: Received adding partitioned topic event, but has subscribed a non-partitioned topic + // with the same name. + log.error("Pattern consumer [{}] skip to subscribe to the partitioned topic {}, because has" + + "subscribed a non-partitioned topic with the same name", + PatternMultiTopicsConsumerImpl.this.getSubscription(), topicName); + groupedTopics.remove(topicName.getPartitionedTopicName()); + } + } + // Case 3: Non-partitioned topic or Partitioned topic, which has not been subscribed. + for (String partitionedTopic : groupedTopics) { + CompletableFuture consumerFuture = subscribeAsync(partitionedTopic, false); + consumerFuture.whenComplete((__, ex) -> { + if (ex != null) { + log.warn("Pattern consumer [{}] Failed to subscribe to topics: {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), partitionedTopic, ex); + } }); - return addFuture; + futures.add(consumerFuture); + } + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] add topics. expend partitions {}, new subscribing {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), expendPartitionsForLog, groupedTopics); + } + return FutureUtil.waitForAll(futures); } } @@ -313,7 +410,7 @@ public CompletableFuture closeAsync() { closeFutures.add(watcher.closeAsync()); } } - closeFutures.add(super.closeAsync()); + closeFutures.add(updateTaskQueue.cancelAllAndWaitForTheRunningTask().thenCompose(__ -> super.closeAsync())); return FutureUtil.waitForAll(closeFutures); } @@ -322,5 +419,11 @@ Timeout getRecheckPatternTimeout() { return recheckPatternTimeout; } + protected void handleSubscribeOneTopicError(String topicName, + Throwable error, + CompletableFuture subscribeFuture) { + subscribeFuture.completeExceptionally(error); + } + private static final Logger log = LoggerFactory.getLogger(PatternMultiTopicsConsumerImpl.class); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index f4afb2931cc9e..120bdeb569c69 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -585,12 +585,13 @@ private CompletableFuture> patternTopicSubscribeAsync(ConsumerCo lookup.getTopicsUnderNamespace(namespaceName, subscriptionMode, regex, null) .thenAccept(getTopicsResult -> { if (log.isDebugEnabled()) { - log.debug("Get topics under namespace {}, topics.size: {}," - + " topicsHash: {}, changed: {}, filtered: {}", + log.debug("Pattern consumer [{}] get topics under namespace {}, topics.size: {}," + + " topicsHash: {}, changed: {}, filtered: {}", conf.getSubscriptionName(), namespaceName, getTopicsResult.getTopics().size(), getTopicsResult.getTopicsHash(), getTopicsResult.isChanged(), getTopicsResult.isFiltered()); getTopicsResult.getTopics().forEach(topicName -> - log.debug("Get topics under namespace {}, topic: {}", namespaceName, topicName)); + log.debug("Pattern consumer [{}] get topics under namespace {}, topic: {}", + conf.getSubscriptionName(), namespaceName, topicName)); } List topicsList = getTopicsResult.getTopics(); @@ -598,6 +599,14 @@ private CompletableFuture> patternTopicSubscribeAsync(ConsumerCo topicsList = TopicList.filterTopics(getTopicsResult.getTopics(), pattern); } conf.getTopicNames().addAll(topicsList); + + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] initialize topics. {}", conf.getSubscriptionName(), + getTopicsResult.getNonPartitionedOrPartitionTopics()); + } + + // Pattern consumer has his unique check mechanism, so do not need the feature "autoUpdatePartitions". + conf.setAutoUpdatePartitions(false); ConsumerBase consumer = new PatternMultiTopicsConsumerImpl<>(pattern, getTopicsResult.getTopicsHash(), PulsarClientImpl.this, diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java index 15922d1180ce0..0007f98b253a0 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java @@ -43,7 +43,7 @@ public class TopicListWatcher extends HandlerState implements ConnectionHandler. AtomicLongFieldUpdater .newUpdater(TopicListWatcher.class, "createWatcherDeadline"); - private final PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener; + private final PatternConsumerUpdateQueue patternConsumerUpdateQueue; private final String name; private final ConnectionHandler connectionHandler; private final Pattern topicsPattern; @@ -63,13 +63,13 @@ public class TopicListWatcher extends HandlerState implements ConnectionHandler. /*** * @param topicsPattern The regexp for the topic name(not contains partition suffix). */ - public TopicListWatcher(PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener, + public TopicListWatcher(PatternConsumerUpdateQueue patternConsumerUpdateQueue, PulsarClientImpl client, Pattern topicsPattern, long watcherId, NamespaceName namespace, String topicsHash, CompletableFuture watcherFuture, Runnable recheckTopicsChangeAfterReconnect) { super(client, topicsPattern.pattern()); - this.topicsChangeListener = topicsChangeListener; + this.patternConsumerUpdateQueue = patternConsumerUpdateQueue; this.name = "Watcher(" + topicsPattern + ")"; this.connectionHandler = new ConnectionHandler(this, new BackoffBuilder() @@ -277,13 +277,7 @@ private void cleanupAtClose(CompletableFuture closeFuture, Throwable excep } public void handleCommandWatchTopicUpdate(CommandWatchTopicUpdate update) { - List deleted = update.getDeletedTopicsList(); - if (!deleted.isEmpty()) { - topicsChangeListener.onTopicsRemoved(deleted); - } - List added = update.getNewTopicsList(); - if (!added.isEmpty()) { - topicsChangeListener.onTopicsAdded(added); - } + patternConsumerUpdateQueue.appendTopicsRemovedOp(update.getDeletedTopicsList()); + patternConsumerUpdateQueue.appendTopicsAddedOp(update.getNewTopicsList()); } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java new file mode 100644 index 0000000000000..01f0be6a85ef6 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java @@ -0,0 +1,247 @@ +/* + * 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.impl; + +import static org.mockito.ArgumentMatchers.anyCollection; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import io.netty.util.HashedWheelTimer; +import java.io.Closeable; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import lombok.AllArgsConstructor; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.util.FutureUtil; +import org.awaitility.Awaitility; +import org.testng.annotations.Test; + +@Test(groups = "utils") +public class PatternConsumerUpdateQueueTest { + + private QueueInstance createInstance(CompletableFuture customizedRecheckFuture, + CompletableFuture customizedPartialUpdateFuture, + CompletableFuture customizedConsumerInitFuture) { + return createInstance(customizedRecheckFuture, customizedPartialUpdateFuture, customizedConsumerInitFuture, + null, null); + } + + private QueueInstance createInstance(CompletableFuture customizedRecheckFuture, + CompletableFuture customizedPartialUpdateFuture, + CompletableFuture customizedConsumerInitFuture, + Collection successTopics, + Collection errorTopics) { + HashedWheelTimer timer = new HashedWheelTimer(new ExecutorProvider.ExtendedThreadFactory("timer-x", + Thread.currentThread().isDaemon()), 1, TimeUnit.MILLISECONDS); + PulsarClientImpl client = mock(PulsarClientImpl.class); + when(client.timer()).thenReturn(timer); + + PatternMultiTopicsConsumerImpl patternConsumer = mock(PatternMultiTopicsConsumerImpl.class); + when(patternConsumer.recheckTopicsChange()).thenReturn(customizedRecheckFuture); + when(patternConsumer.getClient()).thenReturn(client); + if (customizedConsumerInitFuture != null) { + when(patternConsumer.getSubscribeFuture()).thenReturn(customizedConsumerInitFuture); + } else { + when(patternConsumer.getSubscribeFuture()).thenReturn(CompletableFuture.completedFuture(null)); + } + + PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener = + mock(PatternMultiTopicsConsumerImpl.TopicsChangedListener.class); + if (successTopics == null && errorTopics == null) { + when(topicsChangeListener.onTopicsAdded(anyCollection())).thenReturn(customizedPartialUpdateFuture); + when(topicsChangeListener.onTopicsRemoved(anyCollection())).thenReturn(customizedPartialUpdateFuture); + } else { + CompletableFuture ex = FutureUtil.failedFuture(new RuntimeException("mock error")); + when(topicsChangeListener.onTopicsAdded(successTopics)).thenReturn(customizedPartialUpdateFuture); + when(topicsChangeListener.onTopicsRemoved(successTopics)).thenReturn(customizedPartialUpdateFuture); + when(topicsChangeListener.onTopicsAdded(errorTopics)).thenReturn(ex); + when(topicsChangeListener.onTopicsRemoved(errorTopics)).thenReturn(ex); + } + + PatternConsumerUpdateQueue queue = new PatternConsumerUpdateQueue(patternConsumer, topicsChangeListener); + return new QueueInstance(queue, patternConsumer, topicsChangeListener); + } + + private QueueInstance createInstance() { + CompletableFuture completedFuture = CompletableFuture.completedFuture(null); + return createInstance(completedFuture, completedFuture, completedFuture); + } + + @AllArgsConstructor + private static class QueueInstance implements Closeable { + private PatternConsumerUpdateQueue queue; + private PatternMultiTopicsConsumerImpl mockedConsumer; + private PatternMultiTopicsConsumerImpl.TopicsChangedListener mockedListener; + + @Override + public void close() { + mockedConsumer.getClient().timer().stop(); + } + } + + @Test + public void testTopicsChangedEvents() { + QueueInstance instance = createInstance(); + + Collection topics = Arrays.asList("a"); + for (int i = 0; i < 10; i++) { + instance.queue.appendTopicsAddedOp(topics); + instance.queue.appendTopicsRemovedOp(topics); + } + Awaitility.await().untilAsserted(() -> { + verify(instance.mockedListener, times(10)).onTopicsAdded(topics); + verify(instance.mockedListener, times(10)).onTopicsRemoved(topics); + }); + + // cleanup. + instance.close(); + } + + @Test + public void testRecheckTask() { + QueueInstance instance = createInstance(); + + for (int i = 0; i < 10; i++) { + instance.queue.appendRecheckOp(); + } + + Awaitility.await().untilAsserted(() -> { + verify(instance.mockedConsumer, times(10)).recheckTopicsChange(); + }); + + // cleanup. + instance.close(); + } + + @Test + public void testDelayedRecheckTask() { + CompletableFuture recheckFuture = new CompletableFuture<>(); + CompletableFuture partialUpdateFuture = CompletableFuture.completedFuture(null); + CompletableFuture consumerInitFuture = CompletableFuture.completedFuture(null); + QueueInstance instance = createInstance(recheckFuture, partialUpdateFuture, consumerInitFuture); + + for (int i = 0; i < 10; i++) { + instance.queue.appendRecheckOp(); + } + + recheckFuture.complete(null); + Awaitility.await().untilAsserted(() -> { + // The first task will be running, and never completed until all tasks have been added. + // Since the first was started, the second one will not be skipped. + // The others after the second task will be skipped. + // So the times that called "recheckTopicsChange" will be 2. + verify(instance.mockedConsumer, times(2)).recheckTopicsChange(); + }); + + // cleanup. + instance.close(); + } + + @Test + public void testCompositeTasks() { + CompletableFuture recheckFuture = new CompletableFuture<>(); + CompletableFuture partialUpdateFuture = CompletableFuture.completedFuture(null); + CompletableFuture consumerInitFuture = CompletableFuture.completedFuture(null); + QueueInstance instance = createInstance(recheckFuture, partialUpdateFuture, consumerInitFuture); + + Collection topics = Arrays.asList("a"); + for (int i = 0; i < 10; i++) { + instance.queue.appendRecheckOp(); + instance.queue.appendTopicsAddedOp(topics); + instance.queue.appendTopicsRemovedOp(topics); + } + recheckFuture.complete(null); + Awaitility.await().untilAsserted(() -> { + // The first task will be running, and never completed until all tasks have been added. + // Since the first was started, the second one will not be skipped. + // The others after the second task will be skipped. + // So the times that called "recheckTopicsChange" will be 2. + verify(instance.mockedConsumer, times(2)).recheckTopicsChange(); + // The tasks after the second "recheckTopicsChange" will be skipped due to there is a previous + // "recheckTopicsChange" that has not been executed. + // The tasks between the fist "recheckTopicsChange" and the second "recheckTopicsChange" will be skipped + // due to there is a following "recheckTopicsChange". + verify(instance.mockedListener, times(0)).onTopicsAdded(topics); + verify(instance.mockedListener, times(0)).onTopicsRemoved(topics); + }); + + // cleanup. + instance.close(); + } + + @Test + public void testErrorTask() { + CompletableFuture immediatelyCompleteFuture = CompletableFuture.completedFuture(null); + Collection successTopics = Arrays.asList("a"); + Collection errorTopics = Arrays.asList(UUID.randomUUID().toString()); + QueueInstance instance = createInstance(immediatelyCompleteFuture, immediatelyCompleteFuture, + immediatelyCompleteFuture, successTopics, errorTopics); + + instance.queue.appendTopicsAddedOp(successTopics); + instance.queue.appendTopicsRemovedOp(successTopics); + instance.queue.appendTopicsAddedOp(errorTopics); + instance.queue.appendTopicsAddedOp(successTopics); + instance.queue.appendTopicsRemovedOp(successTopics); + + Awaitility.await().atMost(Duration.ofSeconds(60)).untilAsserted(() -> { + verify(instance.mockedListener, times(2)).onTopicsAdded(successTopics); + verify(instance.mockedListener, times(2)).onTopicsRemoved(successTopics); + verify(instance.mockedListener, times(1)).onTopicsAdded(errorTopics); + // After an error task will push a recheck task to offset. + verify(instance.mockedConsumer, times(1)).recheckTopicsChange(); + }); + + // cleanup. + instance.close(); + } + + @Test + public void testFailedSubscribe() { + CompletableFuture immediatelyCompleteFuture = CompletableFuture.completedFuture(null); + CompletableFuture consumerInitFuture = new CompletableFuture<>(); + Collection successTopics = Arrays.asList("a"); + Collection errorTopics = Arrays.asList(UUID.randomUUID().toString()); + QueueInstance instance = createInstance(immediatelyCompleteFuture, immediatelyCompleteFuture, + consumerInitFuture, successTopics, errorTopics); + + instance.queue.appendTopicsAddedOp(successTopics); + instance.queue.appendTopicsRemovedOp(successTopics); + instance.queue.appendTopicsAddedOp(errorTopics); + instance.queue.appendTopicsAddedOp(successTopics); + instance.queue.appendTopicsRemovedOp(successTopics); + + // Consumer init failed after multi topics changes. + // All the topics changes events should be skipped. + consumerInitFuture.completeExceptionally(new RuntimeException("mocked ex")); + Awaitility.await().untilAsserted(() -> { + verify(instance.mockedListener, times(0)).onTopicsAdded(successTopics); + verify(instance.mockedListener, times(0)).onTopicsRemoved(successTopics); + verify(instance.mockedListener, times(0)).onTopicsAdded(errorTopics); + verify(instance.mockedConsumer, times(0)).recheckTopicsChange(); + }); + + // cleanup. + instance.close(); + } +} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java index 116a69b63e4ec..3dfb23f31954a 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java @@ -61,7 +61,7 @@ public void testChangedUnfilteredResponse() { "persistent://tenant/my-ns/non-matching"), null, false, true), mockListener, - Collections.emptyList()); + Collections.emptyList(), ""); verify(mockListener).onTopicsAdded(Sets.newHashSet( "persistent://tenant/my-ns/name-1", "persistent://tenant/my-ns/name-2")); @@ -80,7 +80,7 @@ public void testChangedFilteredResponse() { "persistent://tenant/my-ns/name-2"), "TOPICS_HASH", true, true), mockListener, - Arrays.asList("persistent://tenant/my-ns/name-0")); + Arrays.asList("persistent://tenant/my-ns/name-0"), ""); verify(mockListener).onTopicsAdded(Sets.newHashSet( "persistent://tenant/my-ns/name-1", "persistent://tenant/my-ns/name-2")); @@ -99,7 +99,7 @@ public void testUnchangedResponse() { "persistent://tenant/my-ns/name-2"), "TOPICS_HASH", true, false), mockListener, - Arrays.asList("persistent://tenant/my-ns/name-0")); + Arrays.asList("persistent://tenant/my-ns/name-0"), ""); verify(mockListener, never()).onTopicsAdded(any()); verify(mockListener, never()).onTopicsRemoved(any()); verify(mockTopicsHashSetter).accept("TOPICS_HASH"); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java index 74a71f3da850d..7daf316c4c576 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java @@ -30,6 +30,7 @@ import org.apache.pulsar.common.api.proto.CommandWatchTopicUpdate; import org.apache.pulsar.common.naming.NamespaceName; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyCollection; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; @@ -68,8 +69,17 @@ public void setup() { thenReturn(clientCnxFuture.thenApply(clientCnx -> Pair.of(clientCnx, false))); when(client.getConnection(any(), any(), anyInt())).thenReturn(clientCnxFuture); when(connectionPool.getConnection(any(), any(), anyInt())).thenReturn(clientCnxFuture); + + CompletableFuture completedFuture = CompletableFuture.completedFuture(null); + PatternMultiTopicsConsumerImpl patternConsumer = mock(PatternMultiTopicsConsumerImpl.class); + when(patternConsumer.getSubscribeFuture()).thenReturn(completedFuture); + when(patternConsumer.recheckTopicsChange()).thenReturn(completedFuture); + when(listener.onTopicsAdded(anyCollection())).thenReturn(completedFuture); + when(listener.onTopicsRemoved(anyCollection())).thenReturn(completedFuture); + PatternConsumerUpdateQueue queue = new PatternConsumerUpdateQueue(patternConsumer, listener); + watcherFuture = new CompletableFuture<>(); - watcher = new TopicListWatcher(listener, client, + watcher = new TopicListWatcher(queue, client, Pattern.compile(topic), 7, NamespaceName.get("tenant/ns"), null, watcherFuture, () -> {}); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java b/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java index 80f16e6c36717..26a295264fcae 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java @@ -18,9 +18,12 @@ */ package org.apache.pulsar.common.lookup; +import com.google.re2j.Pattern; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; +import java.util.Set; import lombok.Getter; import lombok.ToString; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; @@ -119,4 +122,25 @@ public List getTopics() { return topics; } } + + public GetTopicsResult filterTopics(Pattern topicsPattern) { + List topicsFiltered = TopicList.filterTopics(getTopics(), topicsPattern); + // If nothing changed. + if (topicsFiltered.equals(getTopics())) { + GetTopicsResult newObj = new GetTopicsResult(nonPartitionedOrPartitionTopics, null, true, true); + newObj.topics = topics; + return newObj; + } + // Filtered some topics. + Set topicsFilteredSet = new HashSet<>(topicsFiltered); + List newTps = new ArrayList<>(); + for (String tp: nonPartitionedOrPartitionTopics) { + if (topicsFilteredSet.contains(TopicName.get(tp).getPartitionedTopicName())) { + newTps.add(tp); + } + } + GetTopicsResult newObj = new GetTopicsResult(newTps, null, true, true); + newObj.topics = topicsFiltered; + return newObj; + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java index e051e01495dbe..d264eab9574ef 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java @@ -23,6 +23,7 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.util.concurrent.UncheckedExecutionException; +import com.google.re2j.Pattern; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.List; import java.util.Objects; @@ -102,6 +103,14 @@ public static boolean isValid(String topic) { } } + public static String getPartitionPattern(String topic) { + return "^" + Pattern.quote(get(topic).getPartitionedTopicName().toString()) + "-partition-[0-9]+$"; + } + + public static String getPattern(String topic) { + return "^" + Pattern.quote(get(topic).getPartitionedTopicName().toString()) + "$"; + } + @SuppressFBWarnings("DCN_NULLPOINTER_EXCEPTION") private TopicName(String completeTopicName) { try { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java index 0628d494af3af..454eee0f966c5 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java @@ -54,6 +54,9 @@ public class FutureUtil { * @return a new CompletableFuture that is completed when all of the given CompletableFutures complete */ public static CompletableFuture waitForAll(Collection> futures) { + if (futures == null || futures.isEmpty()) { + return CompletableFuture.completedFuture(null); + } return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])); } From b473a7b0e12b1bd15e7856e6d35fc47a034685a9 Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Tue, 9 Jul 2024 11:44:45 +0800 Subject: [PATCH 367/580] [improve][broker] log exception in MessageDeduplication#takeSnapshot (#22994) Co-authored-by: fanjianye --- .../pulsar/broker/service/persistent/MessageDeduplication.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java index a4879f2e9520a..e8d19d2e2eca1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java @@ -470,7 +470,8 @@ public void markDeleteComplete(Object ctx) { @Override public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { - log.warn("[{}] Failed to store new deduplication snapshot at {}", topic.getName(), position); + log.warn("[{}] Failed to store new deduplication snapshot at {}", + topic.getName(), position, exception); snapshotTaking.set(false); future.completeExceptionally(exception); } From 1f3449736e614428ea4d625e48cafa09b35e608d Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 10 Jul 2024 10:28:47 +0800 Subject: [PATCH 368/580] [fix][admin] Fix half deletion when attempt to topic with a incorrect API (#23002) --- .../admin/impl/PersistentTopicsBase.java | 12 +++- .../broker/admin/v2/PersistentTopics.java | 14 ++++- .../broker/admin/AdminTopicApiTest.java | 61 +++++++++++++++++++ 3 files changed, 85 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 93e4234559ecc..747031df7a0af 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -742,7 +742,17 @@ protected void internalDeletePartitionedTopic(AsyncResponse asyncResponse, .thenCompose(partitionedMeta -> { final int numPartitions = partitionedMeta.partitions; if (numPartitions < 1) { - return CompletableFuture.completedFuture(null); + return pulsar().getNamespaceService().checkNonPartitionedTopicExists(topicName) + .thenApply(exists -> { + if (exists) { + throw new RestException(Response.Status.CONFLICT, + String.format("%s is a non-partitioned topic. Instead of calling" + + " delete-partitioned-topic please call delete.", topicName)); + } else { + throw new RestException(Status.NOT_FOUND, + String.format("Topic %s not found.", topicName)); + } + }); } return internalRemovePartitionsAuthenticationPoliciesAsync() .thenCompose(unused -> internalRemovePartitionsTopicAsync(numPartitions, force)); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 0a8bf22c42d91..a8e5e7a3ce77b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -1167,7 +1167,17 @@ public void deleteTopic( @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - internalDeleteTopicAsync(authoritative, force) + + getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExistsAsync(topicName).thenAccept(exists -> { + if (exists) { + RestException restException = new RestException(Response.Status.CONFLICT, + String.format("%s is a partitioned topic, instead of calling delete topic, please call" + + " delete-partitioned-topic.", topicName)); + resumeAsyncResponseExceptionally(asyncResponse, restException); + return; + } + internalDeleteTopicAsync(authoritative, force) .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { Throwable t = FutureUtil.unwrapCompletionException(ex); @@ -1186,6 +1196,8 @@ public void deleteTopic( resumeAsyncResponseExceptionally(asyncResponse, ex); return null; }); + }); + } @GET diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTopicApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTopicApiTest.java index a1ed427161619..0a334cd7e819e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTopicApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTopicApiTest.java @@ -23,6 +23,8 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.time.Duration; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -32,6 +34,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -40,12 +43,14 @@ import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.stats.NonPersistentTopicStatsImpl; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.util.FutureUtil; +import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -71,6 +76,62 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + @Test + public void testDeleteNonExistTopic() throws Exception { + // Case 1: call delete for a partitioned topic. + final String topic1 = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createPartitionedTopic(topic1, 2); + admin.schemas().createSchemaAsync(topic1, Schema.STRING.getSchemaInfo()); + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.schemas().getAllSchemas(topic1).size(), 1); + }); + try { + admin.topics().delete(topic1); + fail("expected a 409 error"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("please call delete-partitioned-topic")); + } + Awaitility.await().pollDelay(Duration.ofSeconds(2)).untilAsserted(() -> { + assertEquals(admin.schemas().getAllSchemas(topic1).size(), 1); + }); + // cleanup. + admin.topics().deletePartitionedTopic(topic1, false); + + // Case 2: call delete-partitioned-topi for a non-partitioned topic. + final String topic2 = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic2); + admin.schemas().createSchemaAsync(topic2, Schema.STRING.getSchemaInfo()); + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.schemas().getAllSchemas(topic2).size(), 1); + }); + try { + admin.topics().deletePartitionedTopic(topic2); + fail("expected a 409 error"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("Instead of calling delete-partitioned-topic please call delete")); + } + Awaitility.await().pollDelay(Duration.ofSeconds(2)).untilAsserted(() -> { + assertEquals(admin.schemas().getAllSchemas(topic2).size(), 1); + }); + // cleanup. + admin.topics().delete(topic2, false); + + // Case 3: delete topic does not exist. + final String topic3 = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + try { + admin.topics().delete(topic3); + fail("expected a 404 error"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("not found")); + } + try { + admin.topics().deletePartitionedTopic(topic3); + fail("expected a 404 error"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("not found")); + } + } + @Test public void testPeekMessages() throws Exception { @Cleanup From 7c0e82739215fbae9e21270d4c70c9a52dd3e403 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 10 Jul 2024 18:08:48 +0800 Subject: [PATCH 369/580] [fix][broker]Fix lookupService.getTopicsUnderNamespace can not work with a quote pattern (#23014) --- .../impl/PatternTopicsConsumerImplTest.java | 56 ++++++++++++++++++ .../pulsar/common/topics/TopicList.java | 20 +++++-- .../pulsar/common/topics/TopicListTest.java | 58 ++++++++++++++++++- 3 files changed, 128 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java index 9c19fadffb137..4823426c8b83a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java @@ -27,6 +27,8 @@ import com.google.common.collect.Lists; import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -49,6 +51,7 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.metrics.InstrumentProvider; import org.apache.pulsar.common.api.proto.BaseCommand; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; @@ -1114,4 +1117,57 @@ public void testTopicDeletion() throws Exception { assertEquals(pulsar.getBrokerService().getTopicIfExists(baseTopicName + "-1").join(), Optional.empty()); assertTrue(pulsar.getBrokerService().getTopicIfExists(baseTopicName + "-2").join().isPresent()); } + + @Test(dataProvider = "partitioned") + public void testPatternQuote(boolean partitioned) throws Exception { + final NamespaceName namespace = NamespaceName.get("public/default"); + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final PulsarClientImpl client = (PulsarClientImpl) pulsarClient; + final LookupService lookup = client.getLookup(); + List expectedRes = new ArrayList<>(); + if (partitioned) { + admin.topics().createPartitionedTopic(topicName, 2); + expectedRes.add(TopicName.get(topicName).getPartition(0).toString()); + expectedRes.add(TopicName.get(topicName).getPartition(1).toString()); + Collections.sort(expectedRes); + } else { + admin.topics().createNonPartitionedTopic(topicName); + expectedRes.add(topicName); + } + + // Verify 1: "java.util.regex.Pattern.quote". + String pattern1 = java.util.regex.Pattern.quote(topicName); + List res1 = lookup.getTopicsUnderNamespace(namespace, CommandGetTopicsOfNamespace.Mode.PERSISTENT, + pattern1, null).join().getNonPartitionedOrPartitionTopics(); + Collections.sort(res1); + assertEquals(res1, expectedRes); + + // Verify 2: "com.google.re2j.Pattern.quote" + String pattern2 = com.google.re2j.Pattern.quote(topicName); + List res2 = lookup.getTopicsUnderNamespace(namespace, CommandGetTopicsOfNamespace.Mode.PERSISTENT, + pattern2, null).join().getNonPartitionedOrPartitionTopics(); + Collections.sort(res2); + assertEquals(res2, expectedRes); + + // Verify 3: "java.util.regex.Pattern.quote" & "^$" + String pattern3 = "^" + java.util.regex.Pattern.quote(topicName) + "$"; + List res3 = lookup.getTopicsUnderNamespace(namespace, CommandGetTopicsOfNamespace.Mode.PERSISTENT, + pattern3, null).join().getNonPartitionedOrPartitionTopics(); + Collections.sort(res3); + assertEquals(res3, expectedRes); + + // Verify 4: "com.google.re2j.Pattern.quote" & "^$" + String pattern4 = "^" + com.google.re2j.Pattern.quote(topicName) + "$"; + List res4 = lookup.getTopicsUnderNamespace(namespace, CommandGetTopicsOfNamespace.Mode.PERSISTENT, + pattern4, null).join().getNonPartitionedOrPartitionTopics(); + Collections.sort(res4); + assertEquals(res4, expectedRes); + + // cleanup. + if (partitioned) { + admin.topics().deletePartitionedTopic(topicName, false); + } else { + admin.topics().delete(topicName, false); + } + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/topics/TopicList.java b/pulsar-common/src/main/java/org/apache/pulsar/common/topics/TopicList.java index e8a485b844df5..9e24483df8239 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/topics/TopicList.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/topics/TopicList.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.common.topics; +import com.google.common.annotations.VisibleForTesting; import com.google.common.hash.Hashing; import com.google.re2j.Pattern; import java.nio.charset.StandardCharsets; @@ -28,6 +29,7 @@ import java.util.stream.Collectors; import lombok.experimental.UtilityClass; import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; @UtilityClass @@ -83,15 +85,23 @@ public static Set minus(Collection list1, Collection lis return s1; } - private static String removeTopicDomainScheme(String originalRegexp) { + @VisibleForTesting + static String removeTopicDomainScheme(String originalRegexp) { if (!originalRegexp.toString().contains(SCHEME_SEPARATOR)) { return originalRegexp; } - String removedTopicDomain = SCHEME_SEPARATOR_PATTERN.split(originalRegexp.toString())[1]; - if (originalRegexp.contains("^")) { - return String.format("^%s", removedTopicDomain); + String[] parts = SCHEME_SEPARATOR_PATTERN.split(originalRegexp.toString()); + String prefix = parts[0]; + String removedTopicDomain = parts[1]; + if (prefix.equals(TopicDomain.persistent.value()) || prefix.equals(TopicDomain.non_persistent.value())) { + prefix = ""; + } else if (prefix.endsWith(TopicDomain.non_persistent.value())) { + prefix = prefix.substring(0, prefix.length() - TopicDomain.non_persistent.value().length()); + } else if (prefix.endsWith(TopicDomain.persistent.value())){ + prefix = prefix.substring(0, prefix.length() - TopicDomain.persistent.value().length()); } else { - return removedTopicDomain; + throw new IllegalArgumentException("Does not support topic domain: " + prefix); } + return String.format("%s%s", prefix, removedTopicDomain); } } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/topics/TopicListTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/topics/TopicListTest.java index a83ef2ac8c719..7bcdacb2e9b20 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/topics/TopicListTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/topics/TopicListTest.java @@ -30,6 +30,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; public class TopicListTest { @@ -107,5 +108,60 @@ public void testCalculateHash() { } - + @Test + public void testRemoveTopicDomainScheme() { + // persistent. + final String tpName1 = "persistent://public/default/tp"; + String res1 = TopicList.removeTopicDomainScheme(tpName1); + assertEquals(res1, "public/default/tp"); + + // non-persistent + final String tpName2 = "non-persistent://public/default/tp"; + String res2 = TopicList.removeTopicDomainScheme(tpName2); + assertEquals(res2, "public/default/tp"); + + // without topic domain. + final String tpName3 = "public/default/tp"; + String res3 = TopicList.removeTopicDomainScheme(tpName3); + assertEquals(res3, "public/default/tp"); + + // persistent & "java.util.regex.Pattern.quote". + final String tpName4 = java.util.regex.Pattern.quote(tpName1); + String res4 = TopicList.removeTopicDomainScheme(tpName4); + assertEquals(res4, java.util.regex.Pattern.quote("public/default/tp")); + + // persistent & "java.util.regex.Pattern.quote" & "^$". + final String tpName5 = "^" + java.util.regex.Pattern.quote(tpName1) + "$"; + String res5 = TopicList.removeTopicDomainScheme(tpName5); + assertEquals(res5, "^" + java.util.regex.Pattern.quote("public/default/tp") + "$"); + + // persistent & "com.google.re2j.Pattern.quote". + final String tpName6 = Pattern.quote(tpName1); + String res6 = TopicList.removeTopicDomainScheme(tpName6); + assertEquals(res6, Pattern.quote("public/default/tp")); + + // non-persistent & "java.util.regex.Pattern.quote". + final String tpName7 = java.util.regex.Pattern.quote(tpName2); + String res7 = TopicList.removeTopicDomainScheme(tpName7); + assertEquals(res7, java.util.regex.Pattern.quote("public/default/tp")); + + // non-persistent & "com.google.re2j.Pattern.quote". + final String tpName8 = Pattern.quote(tpName2); + String res8 = TopicList.removeTopicDomainScheme(tpName8); + assertEquals(res8, Pattern.quote("public/default/tp")); + + // non-persistent & "com.google.re2j.Pattern.quote" & "^$". + final String tpName9 = "^" + Pattern.quote(tpName2) + "$"; + String res9 = TopicList.removeTopicDomainScheme(tpName9); + assertEquals(res9, "^" + Pattern.quote("public/default/tp") + "$"); + + // wrong topic domain. + final String tpName10 = "xx://public/default/tp"; + try { + TopicList.removeTopicDomainScheme(tpName10); + fail("Does not support the topic domain xx"); + } catch (Exception ex) { + // expected error. + } + } } From c160cc9c3d44c1df073d63b325b45b9bc9bff8c7 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Thu, 11 Jul 2024 17:17:53 +0800 Subject: [PATCH 370/580] [improve] [pip] PIP-364: Introduce a new load balance algorithm AvgShedder (#22949) Co-authored-by: Kai Wang Co-authored-by: Yunze Xu --- conf/broker.conf | 19 ++ .../pulsar/broker/ServiceConfiguration.java | 36 +- .../broker/loadbalance/impl/AvgShedder.java | 318 ++++++++++++++++++ .../impl/ModularLoadManagerImpl.java | 16 +- .../ModularLoadManagerStrategyTest.java | 43 +++ .../loadbalance/impl/AvgShedderTest.java | 283 ++++++++++++++++ 6 files changed, 711 insertions(+), 4 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedder.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedderTest.java diff --git a/conf/broker.conf b/conf/broker.conf index 5c5d8d42817e9..b715c4e515bc8 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1414,6 +1414,25 @@ loadBalancerBundleUnloadMinThroughputThreshold=10 # Time to wait for the unloading of a namespace bundle namespaceBundleUnloadingTimeoutMs=60000 +# configuration for AvgShedder, a new shedding and placement strategy +# The low threshold for the difference between the highest and lowest loaded brokers. +loadBalancerAvgShedderLowThreshold = 15 + +# The high threshold for the difference between the highest and lowest loaded brokers. +loadBalancerAvgShedderHighThreshold = 40 + +# The number of times the low threshold is triggered before the bundle is unloaded. +loadBalancerAvgShedderHitCountLowThreshold = 8 + +# The number of times the high threshold is triggered before the bundle is unloaded. +loadBalancerAvgShedderHitCountHighThreshold = 2 + +# In the UniformLoadShedder and AvgShedder strategy, the maximum unload ratio. +# For AvgShedder, recommend to set to 0.5, so that it will distribute the load evenly +# between the highest and lowest brokers. +maxUnloadPercentage = 0.2 + + ### --- Load balancer extension --- ### # Option to enable the debug mode for the load balancer logics. 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 73bf2316b8287..aba3ad3a669f5 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 @@ -2395,21 +2395,51 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece @FieldContext( dynamic = true, category = CATEGORY_LOAD_BALANCER, - doc = "In the UniformLoadShedder strategy, the minimum message that triggers unload." + doc = "The low threshold for the difference between the highest and lowest loaded brokers." + ) + private int loadBalancerAvgShedderLowThreshold = 15; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "The high threshold for the difference between the highest and lowest loaded brokers." + ) + private int loadBalancerAvgShedderHighThreshold = 40; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "The number of times the low threshold is triggered before the bundle is unloaded." + ) + private int loadBalancerAvgShedderHitCountLowThreshold = 8; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "The number of times the high threshold is triggered before the bundle is unloaded." + ) + private int loadBalancerAvgShedderHitCountHighThreshold = 2; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "In the UniformLoadShedder and AvgShedder strategy, the minimum message that triggers unload." ) private int minUnloadMessage = 1000; @FieldContext( dynamic = true, category = CATEGORY_LOAD_BALANCER, - doc = "In the UniformLoadShedder strategy, the minimum throughput that triggers unload." + doc = "In the UniformLoadShedder and AvgShedder strategy, the minimum throughput that triggers unload." ) private int minUnloadMessageThroughput = 1 * 1024 * 1024; @FieldContext( dynamic = true, category = CATEGORY_LOAD_BALANCER, - doc = "In the UniformLoadShedder strategy, the maximum unload ratio." + doc = "In the UniformLoadShedder and AvgShedder strategy, the maximum unload ratio." + + "For AvgShedder, recommend to set to 0.5, so that it will distribute the load " + + "evenly between the highest and lowest brokers." ) private double maxUnloadPercentage = 0.2; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedder.java new file mode 100644 index 0000000000000..39ff242fc6c17 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedder.java @@ -0,0 +1,318 @@ +/* + * 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.loadbalance.impl; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; +import com.google.common.hash.Hashing; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Random; +import java.util.Set; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.mutable.MutableDouble; +import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.LoadData; +import org.apache.pulsar.broker.loadbalance.LoadSheddingStrategy; +import org.apache.pulsar.broker.loadbalance.ModularLoadManagerStrategy; +import org.apache.pulsar.policies.data.loadbalancer.BrokerData; +import org.apache.pulsar.policies.data.loadbalancer.BundleData; +import org.apache.pulsar.policies.data.loadbalancer.LocalBrokerData; +import org.apache.pulsar.policies.data.loadbalancer.TimeAverageMessageData; + +@Slf4j +public class AvgShedder implements LoadSheddingStrategy, ModularLoadManagerStrategy { + // map bundle to broker. + private final Map bundleBrokerMap = new HashMap<>(); + // map broker to Scores. scores:0-100 + private final Map brokerScoreMap = new HashMap<>(); + // map broker hit count for high threshold/low threshold + private final Map brokerHitCountForHigh = new HashMap<>(); + private final Map brokerHitCountForLow = new HashMap<>(); + private static final double MB = 1024 * 1024; + + @Override + public Multimap findBundlesForUnloading(LoadData loadData, ServiceConfiguration conf) { + // result returned by shedding, map broker to bundles. + Multimap selectedBundlesCache = ArrayListMultimap.create(); + + // configuration for shedding. + final double minThroughputThreshold = conf.getMinUnloadMessageThroughput(); + final double minMsgThreshold = conf.getMinUnloadMessage(); + final double maxUnloadPercentage = conf.getMaxUnloadPercentage(); + final double lowThreshold = conf.getLoadBalancerAvgShedderLowThreshold(); + final double highThreshold = conf.getLoadBalancerAvgShedderHighThreshold(); + final int hitCountHighThreshold = conf.getLoadBalancerAvgShedderHitCountHighThreshold(); + final int hitCountLowThreshold = conf.getLoadBalancerAvgShedderHitCountLowThreshold(); + if (log.isDebugEnabled()) { + log.debug("highThreshold:{}, lowThreshold:{}, hitCountHighThreshold:{}, hitCountLowThreshold:{}, " + + "minMsgThreshold:{}, minThroughputThreshold:{}", + highThreshold, lowThreshold, hitCountHighThreshold, hitCountLowThreshold, + minMsgThreshold, minThroughputThreshold); + } + + List brokers = calculateScoresAndSort(loadData, conf); + log.info("sorted broker list:{}", brokers); + + // find broker pairs for shedding. + List> pairs = findBrokerPairs(brokers, lowThreshold, highThreshold); + log.info("brokerHitCountForHigh:{}, brokerHitCountForLow:{}", brokerHitCountForHigh, brokerHitCountForLow); + if (pairs.isEmpty()) { + if (log.isDebugEnabled()) { + log.debug("there is no any overload broker, no need to shedding bundles."); + } + brokerHitCountForHigh.clear(); + brokerHitCountForLow.clear(); + return selectedBundlesCache; + } + + // choosing bundles to unload. + for (Pair pair : pairs) { + String overloadedBroker = pair.getRight(); + String underloadedBroker = pair.getLeft(); + + // check hit count for high threshold and low threshold. + if (!(brokerHitCountForHigh.computeIfAbsent(underloadedBroker, __ -> new MutableInt(0)) + .intValue() >= hitCountHighThreshold) + && !(brokerHitCountForHigh.computeIfAbsent(overloadedBroker, __ -> new MutableInt(0)) + .intValue() >= hitCountHighThreshold) + && !(brokerHitCountForLow.computeIfAbsent(underloadedBroker, __ -> new MutableInt(0)) + .intValue() >= hitCountLowThreshold) + && !(brokerHitCountForLow.computeIfAbsent(overloadedBroker, __ -> new MutableInt(0)) + .intValue() >= hitCountLowThreshold)) { + continue; + } + + // if hit, remove entry. + brokerHitCountForHigh.remove(underloadedBroker); + brokerHitCountForHigh.remove(overloadedBroker); + brokerHitCountForLow.remove(underloadedBroker); + brokerHitCountForLow.remove(overloadedBroker); + + // select bundle for unloading. + selectBundleForUnloading(loadData, overloadedBroker, underloadedBroker, minThroughputThreshold, + minMsgThreshold, maxUnloadPercentage, selectedBundlesCache); + } + return selectedBundlesCache; + } + + private void selectBundleForUnloading(LoadData loadData, String overloadedBroker, String underloadedBroker, + double minThroughputThreshold, double minMsgThreshold, + double maxUnloadPercentage, Multimap selectedBundlesCache) { + // calculate how much throughput to unload. + LocalBrokerData minLocalBrokerData = loadData.getBrokerData().get(underloadedBroker).getLocalData(); + LocalBrokerData maxLocalBrokerData = loadData.getBrokerData().get(overloadedBroker).getLocalData(); + + double minMsgRate = minLocalBrokerData.getMsgRateIn() + minLocalBrokerData.getMsgRateOut(); + double maxMsgRate = maxLocalBrokerData.getMsgRateIn() + maxLocalBrokerData.getMsgRateOut(); + + double minThroughput = minLocalBrokerData.getMsgThroughputIn() + minLocalBrokerData.getMsgThroughputOut(); + double maxThroughput = maxLocalBrokerData.getMsgThroughputIn() + maxLocalBrokerData.getMsgThroughputOut(); + + double msgRequiredFromUnloadedBundles = (maxMsgRate - minMsgRate) * maxUnloadPercentage; + double throughputRequiredFromUnloadedBundles = (maxThroughput - minThroughput) * maxUnloadPercentage; + + boolean isMsgRateToOffload; + MutableDouble trafficMarkedToOffload = new MutableDouble(0); + + if (msgRequiredFromUnloadedBundles > minMsgThreshold) { + isMsgRateToOffload = true; + trafficMarkedToOffload.setValue(msgRequiredFromUnloadedBundles); + } else if (throughputRequiredFromUnloadedBundles > minThroughputThreshold) { + isMsgRateToOffload = false; + trafficMarkedToOffload.setValue(throughputRequiredFromUnloadedBundles); + } else { + log.info( + "broker:[{}] is planning to shed bundles to broker:[{}],but the throughput {} MByte/s is " + + "less than minimumThroughputThreshold {} MByte/s, and the msgRate {} rate/s" + + " is also less than minimumMsgRateThreshold {} rate/s, skipping bundle unload.", + overloadedBroker, underloadedBroker, throughputRequiredFromUnloadedBundles / MB, + minThroughputThreshold / MB, msgRequiredFromUnloadedBundles, minMsgThreshold); + return; + } + + if (maxLocalBrokerData.getBundles().size() == 1) { + log.warn("HIGH USAGE WARNING : Sole namespace bundle {} is overloading broker {}. " + + "No Load Shedding will be done on this broker", + maxLocalBrokerData.getBundles().iterator().next(), overloadedBroker); + } else if (maxLocalBrokerData.getBundles().isEmpty()) { + log.warn("Broker {} is overloaded despite having no bundles", overloadedBroker); + } + + // do shedding + log.info( + "broker:[{}] is planning to shed bundles to broker:[{}]. " + + "maxBroker stat:scores:{}, throughput:{}, msgRate:{}. " + + "minBroker stat:scores:{}, throughput:{}, msgRate:{}. " + + "isMsgRateToOffload:{}, trafficMarkedToOffload:{}", + overloadedBroker, underloadedBroker, brokerScoreMap.get(overloadedBroker), maxThroughput, + maxMsgRate, brokerScoreMap.get(underloadedBroker), minThroughput, minMsgRate, + isMsgRateToOffload, trafficMarkedToOffload); + + loadData.getBundleDataForLoadShedding().entrySet().stream().filter(e -> + maxLocalBrokerData.getBundles().contains(e.getKey()) + ).filter(e -> + !loadData.getRecentlyUnloadedBundles().containsKey(e.getKey()) + ).map((e) -> { + BundleData bundleData = e.getValue(); + TimeAverageMessageData shortTermData = bundleData.getShortTermData(); + double traffic = isMsgRateToOffload + ? shortTermData.getMsgRateIn() + shortTermData.getMsgRateOut() + : shortTermData.getMsgThroughputIn() + shortTermData.getMsgThroughputOut(); + return Pair.of(e, traffic); + }).sorted((e1, e2) -> + Double.compare(e2.getRight(), e1.getRight()) + ).forEach(e -> { + Map.Entry bundle = e.getLeft(); + double traffic = e.getRight(); + if (traffic > 0 && traffic <= trafficMarkedToOffload.getValue()) { + selectedBundlesCache.put(overloadedBroker, bundle.getKey()); + bundleBrokerMap.put(bundle.getValue(), underloadedBroker); + trafficMarkedToOffload.add(-traffic); + if (log.isDebugEnabled()) { + log.debug("Found bundle to unload:{}, isMsgRateToOffload:{}, traffic:{}", + bundle, isMsgRateToOffload, traffic); + } + } + }); + } + + @Override + public void onActiveBrokersChange(Set activeBrokers) { + LoadSheddingStrategy.super.onActiveBrokersChange(activeBrokers); + } + + private List calculateScoresAndSort(LoadData loadData, ServiceConfiguration conf) { + brokerScoreMap.clear(); + + // calculate scores of brokers. + for (Map.Entry entry : loadData.getBrokerData().entrySet()) { + LocalBrokerData localBrokerData = entry.getValue().getLocalData(); + String broker = entry.getKey(); + Double score = calculateScores(localBrokerData, conf); + brokerScoreMap.put(broker, score); + if (log.isDebugEnabled()) { + log.info("broker:{}, scores:{}, throughput:{}, messageRate:{}", broker, score, + localBrokerData.getMsgThroughputIn() + localBrokerData.getMsgThroughputOut(), + localBrokerData.getMsgRateIn() + localBrokerData.getMsgRateOut()); + } + } + + // sort brokers by scores. + return brokerScoreMap.entrySet().stream().sorted((o1, o2) -> (int) (o1.getValue() - o2.getValue())) + .map(Map.Entry::getKey).toList(); + } + + private Double calculateScores(LocalBrokerData localBrokerData, final ServiceConfiguration conf) { + return localBrokerData.getMaxResourceUsageWithWeight( + conf.getLoadBalancerCPUResourceWeight(), + conf.getLoadBalancerDirectMemoryResourceWeight(), + conf.getLoadBalancerBandwidthInResourceWeight(), + conf.getLoadBalancerBandwidthOutResourceWeight()) * 100; + } + + private List> findBrokerPairs(List brokers, + double lowThreshold, double highThreshold) { + List> pairs = new LinkedList<>(); + int i = 0, j = brokers.size() - 1; + while (i <= j) { + String maxBroker = brokers.get(j); + String minBroker = brokers.get(i); + if (brokerScoreMap.get(maxBroker) - brokerScoreMap.get(minBroker) < lowThreshold) { + brokerHitCountForHigh.remove(maxBroker); + brokerHitCountForHigh.remove(minBroker); + + brokerHitCountForLow.remove(maxBroker); + brokerHitCountForLow.remove(minBroker); + } else { + pairs.add(Pair.of(minBroker, maxBroker)); + if (brokerScoreMap.get(maxBroker) - brokerScoreMap.get(minBroker) < highThreshold) { + brokerHitCountForLow.computeIfAbsent(minBroker, k -> new MutableInt(0)).increment(); + brokerHitCountForLow.computeIfAbsent(maxBroker, k -> new MutableInt(0)).increment(); + + brokerHitCountForHigh.remove(maxBroker); + brokerHitCountForHigh.remove(minBroker); + } else { + brokerHitCountForLow.computeIfAbsent(minBroker, k -> new MutableInt(0)).increment(); + brokerHitCountForLow.computeIfAbsent(maxBroker, k -> new MutableInt(0)).increment(); + + brokerHitCountForHigh.computeIfAbsent(minBroker, k -> new MutableInt(0)).increment(); + brokerHitCountForHigh.computeIfAbsent(maxBroker, k -> new MutableInt(0)).increment(); + } + } + i++; + j--; + } + return pairs; + } + + @Override + public Optional selectBroker(Set candidates, BundleData bundleToAssign, LoadData loadData, + ServiceConfiguration conf) { + final var brokerToUnload = bundleBrokerMap.getOrDefault(bundleToAssign, null); + if (brokerToUnload == null || !candidates.contains(bundleBrokerMap.get(bundleToAssign))) { + // cluster initializing or broker is shutdown + if (log.isDebugEnabled()) { + if (!bundleBrokerMap.containsKey(bundleToAssign)) { + log.debug("cluster is initializing"); + } else { + log.debug("expected broker:{} is shutdown, candidates:{}", bundleBrokerMap.get(bundleToAssign), + candidates); + } + } + String broker = getExpectedBroker(candidates, bundleToAssign); + bundleBrokerMap.put(bundleToAssign, broker); + return Optional.of(broker); + } else { + return Optional.of(brokerToUnload); + } + } + + private static String getExpectedBroker(Collection brokers, BundleData bundle) { + List sortedBrokers = new ArrayList<>(brokers); + Collections.sort(sortedBrokers); + + try { + // use random number as input of hashing function to avoid special case that, + // if there is 4 brokers running in the cluster,and add broker5,and shutdown broker3, + // then all bundles belonging to broker3 will be loaded on the same broker. + final long hashcode = Hashing.crc32().hashString(String.valueOf(new Random().nextInt()), + StandardCharsets.UTF_8).padToLong(); + final int index = (int) (Math.abs(hashcode) % sortedBrokers.size()); + if (log.isDebugEnabled()) { + log.debug("Assignment details: brokers={}, bundle={}, hashcode={}, index={}", + sortedBrokers, bundle, hashcode, index); + } + return sortedBrokers.get(index); + } catch (Throwable e) { + // theoretically this logic branch should not be executed + log.error("Bundle format of {} is invalid", bundle, e); + return sortedBrokers.get(Math.abs(bundle.hashCode()) % sortedBrokers.size()); + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index 08c9483e87063..8f095b7d84df8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -270,7 +270,21 @@ public void initialize(final PulsarService pulsar) { () -> LoadManagerShared.refreshBrokerToFailureDomainMap(pulsar, brokerToFailureDomainMap)); }); - loadSheddingStrategy = createLoadSheddingStrategy(); + if (placementStrategy instanceof LoadSheddingStrategy) { + // if the placement strategy is also a load shedding strategy + // we need to check two strategies are the same + if (!conf.getLoadBalancerLoadSheddingStrategy().equals( + conf.getLoadBalancerPlacementStrategy())) { + throw new IllegalArgumentException("The load shedding strategy: " + + conf.getLoadBalancerLoadSheddingStrategy() + + " can't work with the placement strategy: " + + conf.getLoadBalancerPlacementStrategy()); + } + // bind the load shedding strategy and the placement strategy + loadSheddingStrategy = (LoadSheddingStrategy) placementStrategy; + } else { + loadSheddingStrategy = createLoadSheddingStrategy(); + } } public void handleDataNotification(Notification t) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java index f5bd0f46a5ec1..53ddde8856c63 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.loadbalance; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; import java.lang.reflect.Field; import java.util.Arrays; @@ -34,6 +35,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.impl.AvgShedder; import org.apache.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate; import org.apache.pulsar.broker.loadbalance.impl.LeastResourceUsageWithWeight; import org.apache.pulsar.broker.loadbalance.impl.RoundRobinBrokerSelector; @@ -47,6 +49,47 @@ @Test(groups = "broker") public class ModularLoadManagerStrategyTest { + public void testAvgShedderWithPreassignedBroker() throws Exception { + ModularLoadManagerStrategy strategy = new AvgShedder(); + Field field = AvgShedder.class.getDeclaredField("bundleBrokerMap"); + field.setAccessible(true); + Map bundleBrokerMap = (Map) field.get(strategy); + BundleData bundleData = new BundleData(); + // assign bundle to broker1 in bundleBrokerMap. + bundleBrokerMap.put(bundleData, "1"); + assertEquals(strategy.selectBroker(Set.of("1", "2", "3"), bundleData, null, null), Optional.of("1")); + assertEquals(bundleBrokerMap.get(bundleData), "1"); + + // remove broker1 in candidates, only broker2 is candidate. + assertEquals(strategy.selectBroker(Set.of("2"), bundleData, null, null), Optional.of("2")); + assertEquals(bundleBrokerMap.get(bundleData), "2"); + } + + public void testAvgShedderWithoutPreassignedBroker() throws Exception { + ModularLoadManagerStrategy strategy = new AvgShedder(); + Field field = AvgShedder.class.getDeclaredField("bundleBrokerMap"); + field.setAccessible(true); + Map bundleBrokerMap = (Map) field.get(strategy); + BundleData bundleData = new BundleData(); + Set candidates = new HashSet<>(); + candidates.add("1"); + candidates.add("2"); + candidates.add("3"); + + // select broker from candidates randomly. + Optional selectedBroker = strategy.selectBroker(candidates, bundleData, null, null); + assertTrue(selectedBroker.isPresent()); + assertTrue(candidates.contains(selectedBroker.get())); + assertEquals(bundleBrokerMap.get(bundleData), selectedBroker.get()); + + // remove original broker in candidates + candidates.remove(selectedBroker.get()); + selectedBroker = strategy.selectBroker(candidates, bundleData, null, null); + assertTrue(selectedBroker.isPresent()); + assertTrue(candidates.contains(selectedBroker.get())); + assertEquals(bundleBrokerMap.get(bundleData), selectedBroker.get()); + } + // Test that least long term message rate works correctly. public void testLeastLongTermMessageRate() { BundleData bundleData = new BundleData(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedderTest.java new file mode 100644 index 0000000000000..215e3d766a927 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedderTest.java @@ -0,0 +1,283 @@ +/* + * 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.loadbalance.impl; + +import com.google.common.collect.Multimap; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.LoadData; +import org.apache.pulsar.policies.data.loadbalancer.BrokerData; +import org.apache.pulsar.policies.data.loadbalancer.BundleData; +import org.apache.pulsar.policies.data.loadbalancer.LocalBrokerData; +import org.apache.pulsar.policies.data.loadbalancer.ResourceUsage; +import org.apache.pulsar.policies.data.loadbalancer.TimeAverageBrokerData; +import org.apache.pulsar.policies.data.loadbalancer.TimeAverageMessageData; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.fail; + +@Test(groups = "broker") +public class AvgShedderTest { + private AvgShedder avgShedder; + private final ServiceConfiguration conf; + + public AvgShedderTest() { + conf = new ServiceConfiguration(); + } + + @BeforeMethod + public void setup() { + avgShedder = new AvgShedder(); + } + + private BrokerData initBrokerData() { + LocalBrokerData localBrokerData = new LocalBrokerData(); + localBrokerData.setCpu(new ResourceUsage()); + localBrokerData.setMemory(new ResourceUsage()); + localBrokerData.setBandwidthIn(new ResourceUsage()); + localBrokerData.setBandwidthOut(new ResourceUsage()); + BrokerData brokerData = new BrokerData(localBrokerData); + TimeAverageBrokerData timeAverageBrokerData = new TimeAverageBrokerData(); + brokerData.setTimeAverageData(timeAverageBrokerData); + return brokerData; + } + + @Test + public void testHitHighThreshold() { + LoadData loadData = new LoadData(); + BrokerData brokerData1 = initBrokerData(); + BrokerData brokerData2 = initBrokerData(); + BrokerData brokerData3 = initBrokerData(); + loadData.getBrokerData().put("broker1", brokerData1); + loadData.getBrokerData().put("broker2", brokerData2); + loadData.getBrokerData().put("broker3", brokerData3); + // AvgShedder will distribute the load evenly between the highest and lowest brokers + conf.setMaxUnloadPercentage(0.5); + + // Set the high threshold to 40% and hit count high threshold to 2 + int hitCountForHighThreshold = 2; + conf.setLoadBalancerAvgShedderHighThreshold(40); + conf.setLoadBalancerAvgShedderHitCountHighThreshold(hitCountForHighThreshold); + brokerData1.getLocalData().setCpu(new ResourceUsage(80, 100)); + brokerData2.getLocalData().setCpu(new ResourceUsage(30, 100)); + brokerData1.getLocalData().setMsgRateIn(10000); + brokerData1.getLocalData().setMsgRateOut(10000); + brokerData2.getLocalData().setMsgRateIn(1000); + brokerData2.getLocalData().setMsgRateOut(1000); + + // broker3 is in the middle + brokerData3.getLocalData().setCpu(new ResourceUsage(50, 100)); + brokerData3.getLocalData().setMsgRateIn(5000); + brokerData3.getLocalData().setMsgRateOut(5000); + + // expect to shed bundles with message rate(in+out) ((10000+10000)-(1000+1000))/2 = 9000 + // each bundle with 450 msg rate in and 450 msg rate out + // so 9000/(450+450)=10 bundles will be shed + for (int i = 0; i < 11; i++) { + brokerData1.getLocalData().getBundles().add("bundle-" + i); + BundleData bundle = new BundleData(); + TimeAverageMessageData timeAverageMessageData = new TimeAverageMessageData(); + timeAverageMessageData.setMsgRateIn(450); + timeAverageMessageData.setMsgRateOut(450); + // as AvgShedder map BundleData to broker, the hashCode of different BundleData should be different + // so we need to set some different fields to make the hashCode different + timeAverageMessageData.setNumSamples(i); + bundle.setShortTermData(timeAverageMessageData); + loadData.getBundleData().put("bundle-" + i, bundle); + } + + // do shedding for the first time, expect to shed nothing because hit count is not enough + Multimap bundlesToUnload = avgShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 0); + + // do shedding for the second time, expect to shed 10 bundles + bundlesToUnload = avgShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 10); + + // assert that all the bundles are shed from broker1 + for (String broker : bundlesToUnload.keys()) { + assertEquals(broker, "broker1"); + } + // assert that all the bundles are shed to broker2 + for (String bundle : bundlesToUnload.values()) { + BundleData bundleData = loadData.getBundleData().get(bundle); + assertEquals(avgShedder.selectBroker(loadData.getBrokerData().keySet(), bundleData, loadData, conf).get(), "broker2"); + } + } + + @Test + public void testHitLowThreshold() { + LoadData loadData = new LoadData(); + BrokerData brokerData1 = initBrokerData(); + BrokerData brokerData2 = initBrokerData(); + BrokerData brokerData3 = initBrokerData(); + loadData.getBrokerData().put("broker1", brokerData1); + loadData.getBrokerData().put("broker2", brokerData2); + loadData.getBrokerData().put("broker3", brokerData3); + // AvgShedder will distribute the load evenly between the highest and lowest brokers + conf.setMaxUnloadPercentage(0.5); + + // Set the low threshold to 20% and hit count low threshold to 6 + int hitCountForLowThreshold = 6; + conf.setLoadBalancerAvgShedderLowThreshold(20); + conf.setLoadBalancerAvgShedderHitCountLowThreshold(hitCountForLowThreshold); + brokerData1.getLocalData().setCpu(new ResourceUsage(60, 100)); + brokerData2.getLocalData().setCpu(new ResourceUsage(40, 100)); + brokerData1.getLocalData().setMsgRateIn(10000); + brokerData1.getLocalData().setMsgRateOut(10000); + brokerData2.getLocalData().setMsgRateIn(1000); + brokerData2.getLocalData().setMsgRateOut(1000); + + // broker3 is in the middle + brokerData3.getLocalData().setCpu(new ResourceUsage(50, 100)); + brokerData3.getLocalData().setMsgRateIn(5000); + brokerData3.getLocalData().setMsgRateOut(5000); + + // expect to shed bundles with message rate(in+out) ((10000+10000)-(1000+1000))/2 = 9000 + // each bundle with 450 msg rate in and 450 msg rate out + // so 9000/(450+450)=10 bundles will be shed + for (int i = 0; i < 11; i++) { + brokerData1.getLocalData().getBundles().add("bundle-" + i); + BundleData bundle = new BundleData(); + TimeAverageMessageData timeAverageMessageData = new TimeAverageMessageData(); + timeAverageMessageData.setMsgRateIn(450); + timeAverageMessageData.setMsgRateOut(450); + // as AvgShedder map BundleData to broker, the hashCode of different BundleData should be different + // so we need to set some different fields to make the hashCode different + timeAverageMessageData.setNumSamples(i); + bundle.setShortTermData(timeAverageMessageData); + loadData.getBundleData().put("bundle-" + i, bundle); + } + + // do shedding for (lowCountForHighThreshold - 1) times, expect to shed nothing because hit count is not enough + Multimap bundlesToUnload; + for (int i = 0; i < hitCountForLowThreshold - 1; i++) { + bundlesToUnload = avgShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 0); + } + + // do shedding for the last time, expect to shed 10 bundles + bundlesToUnload = avgShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 10); + + // assert that all the bundles are shed from broker1 + for (String broker : bundlesToUnload.keys()) { + assertEquals(broker, "broker1"); + } + // assert that all the bundles are shed to broker2 + for (String bundle : bundlesToUnload.values()) { + BundleData bundleData = loadData.getBundleData().get(bundle); + assertEquals(avgShedder.selectBroker(loadData.getBrokerData().keySet(), bundleData, loadData, conf).get(), "broker2"); + } + } + + @Test + public void testSheddingMultiplePairs() { + LoadData loadData = new LoadData(); + BrokerData brokerData1 = initBrokerData(); + BrokerData brokerData2 = initBrokerData(); + BrokerData brokerData3 = initBrokerData(); + BrokerData brokerData4 = initBrokerData(); + loadData.getBrokerData().put("broker1", brokerData1); + loadData.getBrokerData().put("broker2", brokerData2); + loadData.getBrokerData().put("broker3", brokerData3); + loadData.getBrokerData().put("broker4", brokerData4); + // AvgShedder will distribute the load evenly between the highest and lowest brokers + conf.setMaxUnloadPercentage(0.5); + + // Set the high threshold to 40% and hit count high threshold to 2 + int hitCountForHighThreshold = 2; + conf.setLoadBalancerAvgShedderHighThreshold(40); + conf.setLoadBalancerAvgShedderHitCountHighThreshold(hitCountForHighThreshold); + + // pair broker1 and broker2 + brokerData1.getLocalData().setCpu(new ResourceUsage(80, 100)); + brokerData2.getLocalData().setCpu(new ResourceUsage(30, 100)); + brokerData1.getLocalData().setMsgRateIn(10000); + brokerData1.getLocalData().setMsgRateOut(10000); + brokerData2.getLocalData().setMsgRateIn(1000); + brokerData2.getLocalData().setMsgRateOut(1000); + + // pair broker3 and broker4 + brokerData3.getLocalData().setCpu(new ResourceUsage(75, 100)); + brokerData3.getLocalData().setMsgRateIn(10000); + brokerData3.getLocalData().setMsgRateOut(10000); + brokerData4.getLocalData().setCpu(new ResourceUsage(35, 100)); + brokerData4.getLocalData().setMsgRateIn(1000); + brokerData4.getLocalData().setMsgRateOut(1000); + + // expect to shed bundles with message rate(in+out) ((10000+10000)-(1000+1000))/2 = 9000 + // each bundle with 450 msg rate in and 450 msg rate out + // so 9000/(450+450)=10 bundles will be shed + for (int i = 0; i < 11; i++) { + brokerData1.getLocalData().getBundles().add("bundle1-" + i); + brokerData3.getLocalData().getBundles().add("bundle3-" + i); + + BundleData bundle = new BundleData(); + TimeAverageMessageData timeAverageMessageData = new TimeAverageMessageData(); + timeAverageMessageData.setMsgRateIn(450); + timeAverageMessageData.setMsgRateOut(450); + // as AvgShedder map BundleData to broker, the hashCode of different BundleData should be different + // so we need to set some different fields to make the hashCode different + timeAverageMessageData.setNumSamples(i); + bundle.setShortTermData(timeAverageMessageData); + loadData.getBundleData().put("bundle1-" + i, bundle); + + bundle = new BundleData(); + timeAverageMessageData = new TimeAverageMessageData(); + timeAverageMessageData.setMsgRateIn(450); + timeAverageMessageData.setMsgRateOut(450); + timeAverageMessageData.setNumSamples(i+11); + bundle.setShortTermData(timeAverageMessageData); + loadData.getBundleData().put("bundle3-" + i, bundle); + } + + // do shedding for the first time, expect to shed nothing because hit count is not enough + Multimap bundlesToUnload = avgShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 0); + + // do shedding for the second time, expect to shed 10*2=20 bundles + bundlesToUnload = avgShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 20); + + // assert that half of the bundles are shed from broker1, and the other half are shed from broker3 + for (String broker : bundlesToUnload.keys()) { + if (broker.equals("broker1")) { + assertEquals(bundlesToUnload.get(broker).size(), 10); + } else if (broker.equals("broker3")) { + assertEquals(bundlesToUnload.get(broker).size(), 10); + } else { + fail(); + } + } + + // assert that all the bundles from broker1 are shed to broker2, and all the bundles from broker3 are shed to broker4 + for (String bundle : bundlesToUnload.values()) { + BundleData bundleData = loadData.getBundleData().get(bundle); + if (bundle.startsWith("bundle1-")) { + assertEquals(avgShedder.selectBroker(loadData.getBrokerData().keySet(), bundleData, loadData, conf).get(), "broker2"); + } else if (bundle.startsWith("bundle3-")) { + assertEquals(avgShedder.selectBroker(loadData.getBrokerData().keySet(), bundleData, loadData, conf).get(), "broker4"); + } else { + fail(); + } + } + } +} From 88ebe785dbdab239104981453a9bd0e4a7e896d3 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 15 Jul 2024 10:04:48 +0800 Subject: [PATCH 371/580] [fix][broker] Fix stuck when enable topic level replication and build remote admin fails (#23028) --- .../pulsar/broker/admin/AdminResource.java | 15 ++++++-- .../broker/service/OneWayReplicatorTest.java | 35 +++++++++++++++++++ 2 files changed, 47 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index 45455f16d4dc1..1f43aeaa668bc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -48,6 +48,7 @@ import org.apache.pulsar.broker.service.plugin.InvalidEntryFilterException; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.internal.TopicsImpl; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; @@ -630,7 +631,7 @@ private void internalCreatePartitionedTopicToReplicatedClustersInBackground(int }); } - protected Map> internalCreatePartitionedTopicToReplicatedClustersInBackground( + protected Map> internalCreatePartitionedTopicToReplicatedClustersInBackground ( Set clusters, int numPartitions) { final String shortTopicName = topicName.getPartitionedTopicName(); Map> tasksForAllClusters = new HashMap<>(); @@ -649,9 +650,17 @@ protected Map> internalCreatePartitionedTopicToR createRemoteTopicFuture.completeExceptionally(new RestException(ex1)); return; } + PulsarAdmin remotePulsarAdmin; + try { + remotePulsarAdmin = pulsar().getBrokerService().getClusterPulsarAdmin(cluster, clusterData); + } catch (Exception ex) { + log.error("[{}] [{}] An un-expected error occurs when trying to create remote pulsar admin for" + + " cluster {}", clientAppId(), topicName, cluster, ex); + createRemoteTopicFuture.completeExceptionally(new RestException(ex)); + return; + } // Get cluster data success. - TopicsImpl topics = - (TopicsImpl) pulsar().getBrokerService().getClusterPulsarAdmin(cluster, clusterData).topics(); + TopicsImpl topics = (TopicsImpl) remotePulsarAdmin.topics(); topics.createPartitionedTopicAsync(shortTopicName, numPartitions, true, null) .whenComplete((ignore, ex2) -> { if (ex2 == null) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index 80091c9e5eb2c..9aad26530df5b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -38,8 +38,10 @@ import java.time.Duration; import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.Iterator; import java.util.Optional; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; @@ -58,6 +60,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.resources.ClusterResources; import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -75,7 +78,9 @@ import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; @@ -947,6 +952,36 @@ protected void disableReplication(String topic) throws Exception { admin1.topics().setReplicationClusters(topic, Arrays.asList(cluster1, cluster2)); } + @Test(timeOut = 30 * 1000) + public void testCreateRemoteAdminFailed() throws Exception { + final TenantInfo tenantInfo = admin1.tenants().getTenantInfo(defaultTenant); + final String ns1 = defaultTenant + "/ns_" + UUID.randomUUID().toString().replace("-", ""); + final String randomClusterName = "c_" + UUID.randomUUID().toString().replace("-", ""); + final String topic = BrokerTestUtil.newUniqueName(ns1 + "/tp"); + admin1.namespaces().createNamespace(ns1); + admin1.topics().createPartitionedTopic(topic, 2); + + // Inject a wrong cluster data which with empty fields. + ClusterResources clusterResources = broker1.getPulsar().getPulsarResources().getClusterResources(); + clusterResources.createCluster(randomClusterName, ClusterData.builder().build()); + Set allowedClusters = new HashSet<>(tenantInfo.getAllowedClusters()); + allowedClusters.add(randomClusterName); + admin1.tenants().updateTenant(defaultTenant, TenantInfo.builder().adminRoles(tenantInfo.getAdminRoles()) + .allowedClusters(allowedClusters).build()); + + // Verify. + try { + admin1.topics().setReplicationClusters(topic, Arrays.asList(cluster1, randomClusterName)); + fail("Expected a error due to empty fields"); + } catch (Exception ex) { + // Expected an error. + } + + // cleanup. + admin1.topics().deletePartitionedTopic(topic); + admin1.tenants().updateTenant(defaultTenant, tenantInfo); + } + @Test public void testConfigReplicationStartAt() throws Exception { // Initialize. From a8ce990a72c3024fafe689f3bc3c5127583021e6 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 15 Jul 2024 23:01:47 +0800 Subject: [PATCH 372/580] [fix][broker] Replication stuck when partitions count between two clusters is not the same (#22983) --- .../broker/service/AbstractReplicator.java | 5 + .../persistent/PersistentReplicator.java | 6 + .../service/AbstractReplicatorTest.java | 7 +- .../broker/service/OneWayReplicatorTest.java | 91 ++++++++++++++ .../OneWayReplicatorUsingGlobalZKTest.java | 6 + .../api/NonPartitionedTopicExpectedTest.java | 118 ++++++++++++++++++ .../pulsar/client/impl/PulsarClientImpl.java | 46 +++++-- .../impl/conf/ProducerConfigurationData.java | 2 + 8 files changed, 271 insertions(+), 10 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPartitionedTopicExpectedTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index 8552a9f09e93b..424263720f012 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -37,6 +37,7 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.ProducerBuilderImpl; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.TopicName; @@ -184,6 +185,10 @@ public void startProducer() { } log.info("[{}] Starting replicator", replicatorId); + // Force only replicate messages to a non-partitioned topic, to avoid auto-create a partitioned topic on + // the remote cluster. + ProducerBuilderImpl builderImpl = (ProducerBuilderImpl) producerBuilder; + builderImpl.getConf().setNonPartitionedTopicExpected(true); producerBuilder.createAsync().thenAccept(producer -> { setProducerAndTriggerReadEntries(producer); }).exceptionally(ex -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index 54b8993784e29..33e883ab9406a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -154,6 +154,12 @@ protected void setProducerAndTriggerReadEntries(Producer producer) { Pair changeStateRes; changeStateRes = compareSetAndGetState(Starting, Started); if (changeStateRes.getLeft()) { + if (!(producer instanceof ProducerImpl)) { + log.error("[{}] The partitions count between two clusters is not the same, the replicator can not be" + + " created successfully: {}", replicatorId, state); + doCloseProducerAsync(producer, () -> {}); + throw new ClassCastException(producer.getClass().getName() + " can not be cast to ProducerImpl"); + } this.producer = (ProducerImpl) producer; HAVE_PENDING_READ_UPDATER.set(this, FALSE); // Trigger a new read. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java index 64d3088b20622..7415a40ad5553 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java @@ -39,10 +39,11 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.ConnectionPool; +import org.apache.pulsar.client.impl.ProducerBuilderImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; @@ -71,7 +72,8 @@ public void testRetryStartProducerStoppedByTopicRemove() throws Exception { when(localClient.getCnxPool()).thenReturn(connectionPool); final PulsarClientImpl remoteClient = mock(PulsarClientImpl.class); when(remoteClient.getCnxPool()).thenReturn(connectionPool); - final ProducerBuilder producerBuilder = mock(ProducerBuilder.class); + final ProducerConfigurationData producerConf = new ProducerConfigurationData(); + final ProducerBuilderImpl producerBuilder = mock(ProducerBuilderImpl.class); final ConcurrentOpenHashMap>> topics = new ConcurrentOpenHashMap<>(); when(broker.executor()).thenReturn(eventLoopGroup); when(broker.getTopics()).thenReturn(topics); @@ -87,6 +89,7 @@ public void testRetryStartProducerStoppedByTopicRemove() throws Exception { when(producerBuilder.sendTimeout(anyInt(), any())).thenReturn(producerBuilder); when(producerBuilder.maxPendingMessages(anyInt())).thenReturn(producerBuilder); when(producerBuilder.producerName(anyString())).thenReturn(producerBuilder); + when(producerBuilder.getConf()).thenReturn(producerConf); // Mock create producer fail. when(producerBuilder.create()).thenThrow(new RuntimeException("mocked ex")); when(producerBuilder.createAsync()) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index 9aad26530df5b..1745d4dc90f3b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -40,6 +40,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Optional; import java.util.Set; import java.util.UUID; @@ -50,7 +51,9 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; +import java.util.function.Predicate; import java.util.function.Supplier; +import java.util.stream.Collectors; import lombok.AllArgsConstructor; import lombok.Data; import lombok.SneakyThrows; @@ -79,9 +82,11 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; @@ -1069,4 +1074,90 @@ public void testConfigReplicationStartAt() throws Exception { admin1.topics().delete(topic3, false); admin2.topics().delete(topic3, false); } + + @DataProvider(name = "replicationModes") + public Object[][] replicationModes() { + return new Object[][]{ + {ReplicationMode.OneWay}, + {ReplicationMode.DoubleWay} + }; + } + + protected enum ReplicationMode { + OneWay, + DoubleWay; + } + + @Test(dataProvider = "replicationModes") + public void testDifferentTopicCreationRule(ReplicationMode replicationMode) throws Exception { + String ns = defaultTenant + "/" + UUID.randomUUID().toString().replace("-", ""); + admin1.namespaces().createNamespace(ns); + admin2.namespaces().createNamespace(ns); + + // Set topic auto-creation rule. + // c1: no-partitioned topic + // c2: partitioned topic with 2 partitions. + AutoTopicCreationOverride autoTopicCreation = + AutoTopicCreationOverrideImpl.builder().allowAutoTopicCreation(true) + .topicType("partitioned").defaultNumPartitions(2).build(); + admin2.namespaces().setAutoTopicCreation(ns, autoTopicCreation); + Awaitility.await().untilAsserted(() -> { + assertEquals(admin2.namespaces().getAutoTopicCreationAsync(ns).join().getDefaultNumPartitions(), 2); + // Trigger system topic __change_event's initialize. + pulsar2.getTopicPoliciesService().getTopicPoliciesAsync(TopicName.get("persistent://" + ns + "/1")); + }); + + // Create non-partitioned topic. + // Enable replication. + final String tp = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp_"); + admin1.topics().createNonPartitionedTopic(tp); + admin1.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1, cluster2))); + if (replicationMode.equals(ReplicationMode.DoubleWay)) { + admin2.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1, cluster2))); + } + + // Trigger and wait for replicator starts. + Producer p1 = client1.newProducer(Schema.STRING).topic(tp).create(); + p1.send("msg-1"); + p1.close(); + Awaitility.await().untilAsserted(() -> { + PersistentTopic persistentTopic = (PersistentTopic) broker1.getTopic(tp, false).join().get(); + assertFalse(persistentTopic.getReplicators().isEmpty()); + }); + + // Verify: the topics are the same between two clusters. + Predicate topicNameFilter = t -> { + TopicName topicName = TopicName.get(t); + if (!topicName.getNamespace().equals(ns)) { + return false; + } + return t.startsWith(tp); + }; + Awaitility.await().untilAsserted(() -> { + List topics1 = pulsar1.getBrokerService().getTopics().keys() + .stream().filter(topicNameFilter).collect(Collectors.toList()); + List topics2 = pulsar2.getBrokerService().getTopics().keys() + .stream().filter(topicNameFilter).collect(Collectors.toList()); + Collections.sort(topics1); + Collections.sort(topics2); + assertEquals(topics1, topics2); + }); + + // cleanup. + admin1.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1))); + if (replicationMode.equals(ReplicationMode.DoubleWay)) { + admin2.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster2))); + } + Awaitility.await().untilAsserted(() -> { + PersistentTopic persistentTopic = (PersistentTopic) broker1.getTopic(tp, false).join().get(); + assertTrue(persistentTopic.getReplicators().isEmpty()); + if (replicationMode.equals(ReplicationMode.DoubleWay)) { + assertTrue(persistentTopic.getReplicators().isEmpty()); + } + }); + admin1.topics().delete(tp, false); + admin2.topics().delete(tp, false); + admin1.namespaces().deleteNamespace(ns); + admin2.namespaces().deleteNamespace(ns); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java index 31e94f435f0f6..34810bbe9057b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java @@ -161,4 +161,10 @@ public void testConfigReplicationStartAt() throws Exception { pulsar1.getConfiguration().getReplicationStartAt().equalsIgnoreCase("latest"); }); } + + @Test(enabled = false) + @Override + public void testDifferentTopicCreationRule(ReplicationMode replicationMode) throws Exception { + super.testDifferentTopicCreationRule(replicationMode); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPartitionedTopicExpectedTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPartitionedTopicExpectedTest.java new file mode 100644 index 0000000000000..7b0edd314d055 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPartitionedTopicExpectedTest.java @@ -0,0 +1,118 @@ +/* + * 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.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.impl.ProducerBuilderImpl; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; +import org.apache.pulsar.common.policies.data.TopicType; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +public class NonPartitionedTopicExpectedTest extends ProducerConsumerBase { + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testWhenNonPartitionedTopicExists() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic); + ProducerBuilderImpl producerBuilder = + (ProducerBuilderImpl) pulsarClient.newProducer(Schema.STRING).topic(topic); + producerBuilder.getConf().setNonPartitionedTopicExpected(true); + // Verify: create successfully. + Producer producer = producerBuilder.create(); + // cleanup. + producer.close(); + admin.topics().delete(topic, false); + } + + @Test + public void testWhenPartitionedTopicExists() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createPartitionedTopic(topic, 2); + ProducerBuilderImpl producerBuilder = + (ProducerBuilderImpl) pulsarClient.newProducer(Schema.STRING).topic(topic); + producerBuilder.getConf().setNonPartitionedTopicExpected(true); + // Verify: failed to create. + try { + producerBuilder.create(); + Assert.fail("expected an error since producer expected a non-partitioned topic"); + } catch (Exception ex) { + // expected an error. + log.error("expected error", ex); + } + // cleanup. + admin.topics().deletePartitionedTopic(topic, false); + } + + @DataProvider(name = "topicTypes") + public Object[][] topicTypes() { + return new Object[][]{ + {TopicType.PARTITIONED}, + {TopicType.NON_PARTITIONED} + }; + } + + @Test(dataProvider = "topicTypes") + public void testWhenTopicNotExists(TopicType topicType) throws Exception { + final String namespace = "public/default"; + final String topic = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/tp"); + final TopicName topicName = TopicName.get(topic); + AutoTopicCreationOverride.Builder policyBuilder = AutoTopicCreationOverride.builder() + .topicType(topicType.toString()).allowAutoTopicCreation(true); + if (topicType.equals(TopicType.PARTITIONED)) { + policyBuilder.defaultNumPartitions(2); + } + AutoTopicCreationOverride policy = policyBuilder.build(); + admin.namespaces().setAutoTopicCreation(namespace, policy); + + ProducerBuilderImpl producerBuilder = + (ProducerBuilderImpl) pulsarClient.newProducer(Schema.STRING).topic(topic); + producerBuilder.getConf().setNonPartitionedTopicExpected(true); + // Verify: create successfully. + Producer producer = producerBuilder.create(); + // Verify: only create non-partitioned topic. + Assert.assertFalse(pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(topicName)); + Assert.assertTrue(pulsar.getNamespaceService().checkNonPartitionedTopicExists(topicName).join()); + + // cleanup. + producer.close(); + admin.topics().delete(topic, false); + admin.namespaces().removeAutoTopicCreation(namespace); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 120bdeb569c69..4585b5328129b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -49,9 +49,11 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; import lombok.Builder; import lombok.Getter; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -382,26 +384,55 @@ public CompletableFuture> createProducerAsync(ProducerConfigurat } + private CompletableFuture checkPartitions(String topic, boolean forceNoPartitioned, + @Nullable String producerNameForLog) { + CompletableFuture checkPartitions = new CompletableFuture<>(); + getPartitionedTopicMetadata(topic, !forceNoPartitioned).thenAccept(metadata -> { + if (forceNoPartitioned && metadata.partitions > 0) { + String errorMsg = String.format("Can not create the producer[%s] for the topic[%s] that contains %s" + + " partitions, but the producer does not support for a partitioned topic.", + producerNameForLog, topic, metadata.partitions); + log.error(errorMsg); + checkPartitions.completeExceptionally( + new PulsarClientException.NotConnectedException(errorMsg)); + } else { + checkPartitions.complete(metadata.partitions); + } + }).exceptionally(ex -> { + Throwable actEx = FutureUtil.unwrapCompletionException(ex); + if (forceNoPartitioned && actEx instanceof PulsarClientException.NotFoundException + || actEx instanceof PulsarClientException.TopicDoesNotExistException + || actEx instanceof PulsarAdminException.NotFoundException) { + checkPartitions.complete(0); + } else { + checkPartitions.completeExceptionally(ex); + } + return null; + }); + return checkPartitions; + } + private CompletableFuture> createProducerAsync(String topic, ProducerConfigurationData conf, Schema schema, ProducerInterceptors interceptors) { CompletableFuture> producerCreatedFuture = new CompletableFuture<>(); - getPartitionedTopicMetadata(topic, true).thenAccept(metadata -> { + + + checkPartitions(topic, conf.isNonPartitionedTopicExpected(), conf.getProducerName()).thenAccept(partitions -> { if (log.isDebugEnabled()) { - log.debug("[{}] Received topic metadata. partitions: {}", topic, metadata.partitions); + log.debug("[{}] Received topic metadata. partitions: {}", topic, partitions); } ProducerBase producer; - if (metadata.partitions > 0) { + if (partitions > 0) { producer = newPartitionedProducerImpl(topic, conf, schema, interceptors, producerCreatedFuture, - metadata); + partitions); } else { producer = newProducerImpl(topic, -1, conf, schema, interceptors, producerCreatedFuture, Optional.empty()); } - producers.add(producer); }).exceptionally(ex -> { log.warn("[{}] Failed to get partitioned topic metadata: {}", topic, ex.getMessage()); @@ -422,7 +453,6 @@ private CompletableFuture> createProducerAsync(String topic, * @param schema topic schema * @param interceptors producer interceptors * @param producerCreatedFuture future for signaling completion of async producer creation - * @param metadata partitioned topic metadata * @param message type class * @return new PartitionedProducerImpl instance */ @@ -432,8 +462,8 @@ protected PartitionedProducerImpl newPartitionedProducerImpl(String topic ProducerInterceptors interceptors, CompletableFuture> producerCreatedFuture, - PartitionedTopicMetadata metadata) { - return new PartitionedProducerImpl<>(PulsarClientImpl.this, topic, conf, metadata.partitions, + int partitions) { + return new PartitionedProducerImpl<>(PulsarClientImpl.this, topic, conf, partitions, producerCreatedFuture, schema, interceptors); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ProducerConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ProducerConfigurationData.java index 581b3d8a1635e..6ec738bbf4c8d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ProducerConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ProducerConfigurationData.java @@ -204,6 +204,8 @@ public class ProducerConfigurationData implements Serializable, Cloneable { private SortedMap properties = new TreeMap<>(); + private boolean isNonPartitionedTopicExpected; + @ApiModelProperty( name = "initialSubscriptionName", value = "Use this configuration to automatically create an initial subscription when creating a topic." From 5c6602cbb3660a696bf960f2847aac1a2ae037d2 Mon Sep 17 00:00:00 2001 From: Aurora Twinkle Date: Tue, 16 Jul 2024 10:47:55 +0800 Subject: [PATCH 373/580] [improve][pip] PIP-359: Support custom message listener executor for specific subscription (#22902) Co-authored-by: duanlinlin --- pip/pip-359.md | 216 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 216 insertions(+) create mode 100644 pip/pip-359.md diff --git a/pip/pip-359.md b/pip/pip-359.md new file mode 100644 index 0000000000000..52a76193d6cf2 --- /dev/null +++ b/pip/pip-359.md @@ -0,0 +1,216 @@ +# PIP-359: Support custom message listener executor for specific subscription +Implementation PR: [#22861](https://github.com/apache/pulsar/pull/22861) + +# Background knowledge +In the current Pulsar client versions, from the user's perspective, when using a Pulsar Consumer, +we have two main options to consume messages: +1. Pull mode, by calling `consumer.recieve()`(or `consumer.recieveAsync()`) +```java +public class ConsumerExample { + public static void main(String[] args) throws PulsarClientException { + PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl("pulsar://localhost:6650") + .build(); + Consumer consumer = pulsarClient.newConsumer(Schema.INT64) + .topic("persistent://public/default/my-topic") + .subscriptionName("my-subscription") + .subscribe(); + do { + Message message = consumer.receive(); + consumer.acknowledge(message); + } while (true); + + } +} + +``` +2. Push mode, by registering a `MessageListener` interface, when building the Consumer. +When this method is used, we can't also use `consumer.receive()`(or `consumer.recieveAsync()`). +In the push mode, the MessageListener instance is called by the consumer, hence it is +doing that with a thread taken from its own internal `ExecutorService` (i.e. thread pool). +The problem comes when we build and use multiple Consumers from the same PulsarClient. It +so happens that those consumers will share the same thread pool to call the Message Listeners. +One can be slower from the other. + +```java +public class ConsumerExample { + public static void main(String[] args) throws PulsarClientException { + PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl("pulsar://localhost:6650") + .build(); + Consumer consumer = pulsarClient.newConsumer(Schema.INT64) + .topic("persistent://public/default/my-topic") + .subscriptionName("my-subscription2") + .messageListener((consumer, message) -> { + // process message + consumer.acknowledgeAsync(message); + }) + .subscribe(); + } +} +``` + +# Motivation + +As [Background knowledge](#background-knowledge) mentioned, when using asynchronous consumer, +by registering a MessageListener interface, there is a problem of different consumer groups +affecting each other, leading to unnecessary consumption delays. +Therefore, for this scenario, this PIP prepare to support specific a message +listener executor of consumer latitudes to solve such problem. +# Goals +1. Improve consumer message listener isolation, solve the consumption delay problem caused by +mutual influence of different consumers from the same PulsarClient instance. + +## In Scope +If this PIP is accepted, it will help Pulsar solve the problem of different consumers +from same `PulsarClient` affecting each other in the asynchronous consumption mode(`MessageListener`). + +## Out of Scope +This PIP will not build the plugin library mentioned in [PR](https://github.com/apache/pulsar/pull/22902#issuecomment-2169962642), +we will open a new PIP in the future to do this + + +# Detailed Design + +## Design & Implementation Details + +1. Add an interface `MessageListenerExecutor`, responsible for executing message listener callback tasks. +Users can customize the implementation to determine in which thread the message listener task is executed. +For example, in the situation described in [Motivation](#motivation) part, users can implement the +interface with an independent underlying thread pool to ensure that the message listener task of each +consumer is executed in a separate thread. The caller would be responsible for the life cycle of the +Executor, and it would be used only for this specific consumer. + ```java + public interface MessageListenerExecutor { + + /** + * select a thread by message(if necessary, for example, + * Key_Shared SubscriptionType, maybe need select thread + * by message order key to ensure order) to execute the runnable! + * + * @param message the message + * @param runnable the runnable to execute + */ + void execute(Message message, Runnable runnable); + } + ``` +2. Add an optional config `messageListenerExecutor` in `ConsumerBuilder`, then +users can pass their implementations. + ```java + ConsumerBuilder messageListenerExecutor(MessageListenerExecutor messageListenerExecutor); + ``` + +### Why need an interface like `MessageListenerExecutor` +Some people may wonder why not just use `java.util.concurrent.ExecutorService`, +but define an interface like `MessageListenerExecutor`. + +The reason is that: + +For sequential consumption scenarios, we need to ensure that messages with the same +key or the same partition are processed by the same thread to ensure order. If we +use `java.util.concurrent.ExecutorService`, refer to the following figure, we will not be able to make such guarantees, +because for ExecutorService, which thread to execute the task is not controlled by the user. +![](https://github.com/AuroraTwinkle/pulsar/assets/25919180/232854d6-01f2-4821-b2df-34d01dda1992) +![](https://github.com/AuroraTwinkle/pulsar/assets/25919180/204f5622-1e5a-4e73-b86b-15220bfb06d6) +### Interface implementation suggestions +When implementing the `MessageListenerExecutor` interface, you should consider the following points. +1. if you need to ensure the order of message processing, +you can select the thread by the message order key or `msg.getTopicName()`(partition topic name), +to ensure that the messages of the same order key (or partition) are processed in same thread. + +### Usage Example +```java + private void startConsumerWithMessageListener(String topic, String subscriptionName) throws PulsarClientException { + // for example: key_shared + MessageListenerExecutor keySharedExecutor = getKeySharedMessageListenerExecutor(subscriptionName); + Consumer keySharedconsumer = + pulsarClient.newConsumer(Schema.INT64) + .topic(topic) + .subscriptionName(subscriptionName) + // set and then message lister will be executed in the executor + .messageListener((c1, msg) -> { + log.info("Received message [{}] in the listener", msg.getValue()); + c1.acknowledgeAsync(msg); + }) + .messageListenerExecutor(keySharedExecutor) + .subscribe(); + + + // for example: partition_ordered + MessageListenerExecutor partitionOrderedExecutor = getPartitionOrderdMessageListenerExecutor(subscriptionName); + Consumer partitionOrderedConsumer = + pulsarClient.newConsumer(Schema.INT64) + .topic(topic) + .subscriptionName(subscriptionName) + // set and then message lister will be executed in the executor + .messageListener((c1, msg) -> { + log.info("Received message [{}] in the listener", msg.getValue()); + c1.acknowledgeAsync(msg); + }) + .messageListenerExecutor(partitionOrderedExecutor) + .subscribe(); + + // for example: out-of-order + ExecutorService executorService = Executors.newFixedThreadPool(10); + Consumer outOfOrderConsumer = + pulsarClient.newConsumer(Schema.INT64) + .topic(topic) + .subscriptionName(subscriptionName) + // not set and then message lister will be executed in the default executor + .messageListener((c1, msg) -> { + log.info("Received message [{}] in the listener", msg.getValue()); + c1.acknowledgeAsync(msg); + }) + .messageListenerExecutor((message, runnable) -> executorService.execute(runnable)) + .subscribe(); +} + +private static MessageListenerExecutor getKeySharedMessageListenerExecutor(String subscriptionName) { + ExecutorProvider executorProvider = new ExecutorProvider(10, subscriptionName + "listener-executor-"); + + return (message, runnable) -> { + byte[] key = "".getBytes(StandardCharsets.UTF_8); + if (message.hasKey()) { + key = message.getKeyBytes(); + } else if (message.hasOrderingKey()) { + key = message.getOrderingKey(); + } + // select a thread by message key to execute the runnable! + // that say, the message listener task with same order key + // will be executed by the same thread + ExecutorService executorService = executorProvider.getExecutor(key); + // executorService is a SingleThreadExecutor + executorService.execute(runnable); + }; +} + +private static MessageListenerExecutor getPartitionOrderdMessageListenerExecutor(String subscriptionName) { + ExecutorProvider executorProvider = new ExecutorProvider(10, subscriptionName + "listener-executor-"); + + return (message, runnable) -> { + // select a thread by partition topic name to execute the runnable! + // that say, the message listener task from the same partition topic + // will be executed by the same thread + ExecutorService executorService = executorProvider.getExecutor(message.getTopicName().getBytes()); + // executorService is a SingleThreadExecutor + executorService.execute(runnable); + }; +} + +``` +## Public-facing Changes + +### Public API + +1. Add an optional config `messageListenerExecutor` in `ConsumerBuilder` +```java +ConsumerBuilder messageListenerExecutor(MessageListenerExecutor messageListenerExecutor); +``` + +# Backward & Forward Compatibility +You can do upgrading or reverting normally, no specified steps are needed to do. + +# Links + +* Mailing List discussion thread: https://lists.apache.org/thread/8nhqfdhkglsg5bgx6z7c1nho7z7l596l +* Mailing List voting thread: https://lists.apache.org/thread/oo3jdvq3b6bv6p4n7x7sdvypw4gp6hpk From 23163f97b7e971e8e6cc07a6e20899c49e435bef Mon Sep 17 00:00:00 2001 From: moinessim <47434700+moinessim@users.noreply.github.com> Date: Tue, 16 Jul 2024 10:52:58 -0500 Subject: [PATCH 374/580] [feat][ws]Add support for initialSubscriptionPosition in Websocket Consumer (#23013) Co-authored-by: Moises Nessim --- .../org/apache/pulsar/websocket/ConsumerHandler.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ConsumerHandler.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ConsumerHandler.java index f07c2aa57066c..b93c4b215108e 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ConsumerHandler.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ConsumerHandler.java @@ -44,6 +44,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.AlreadyClosedException; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.ConsumerBuilderImpl; @@ -431,6 +432,14 @@ protected ConsumerBuilder getConsumerConfiguration(PulsarClient client) builder.subscriptionMode(SubscriptionMode.valueOf(queryParams.get("subscriptionMode"))); } + if (queryParams.containsKey("subscriptionInitialPosition")) { + final String subscriptionInitialPosition = queryParams.get("subscriptionInitialPosition"); + checkArgument( + Enums.getIfPresent(SubscriptionInitialPosition.class, subscriptionInitialPosition).isPresent(), + "Invalid subscriptionInitialPosition %s", subscriptionInitialPosition); + builder.subscriptionInitialPosition(SubscriptionInitialPosition.valueOf(subscriptionInitialPosition)); + } + if (queryParams.containsKey("receiverQueueSize")) { builder.receiverQueueSize(Math.min(Integer.parseInt(queryParams.get("receiverQueueSize")), 1000)); } From aa757ac721edab7605aa464eb0debed846bb4cb0 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 16 Jul 2024 09:52:44 -0700 Subject: [PATCH 375/580] [improve][proxy] Add debug logs for proxy cnx management (#23037) --- .../org/apache/pulsar/proxy/server/ProxyConnection.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java index 594d6cbc3bb59..d58fe46e0063a 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java @@ -169,6 +169,10 @@ public void channelRegistered(ChannelHandlerContext ctx) throws Exception { ProxyService.ACTIVE_CONNECTIONS.inc(); SocketAddress rmAddress = ctx.channel().remoteAddress(); ConnectionController.State state = connectionController.increaseConnection(rmAddress); + if (LOG.isDebugEnabled()) { + LOG.debug("Active connection count={} for cnx {} with state {}", ProxyService.ACTIVE_CONNECTIONS.get(), + rmAddress, state); + } if (!state.equals(ConnectionController.State.OK)) { ctx.writeAndFlush(Commands.newError(-1, ServerError.NotAllowedError, state.equals(ConnectionController.State.REACH_MAX_CONNECTION) @@ -184,6 +188,9 @@ public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { super.channelUnregistered(ctx); connectionController.decreaseConnection(ctx.channel().remoteAddress()); ProxyService.ACTIVE_CONNECTIONS.dec(); + if (LOG.isDebugEnabled()) { + LOG.debug("Decreasing active connection count={} ", ProxyService.ACTIVE_CONNECTIONS.get()); + } } @Override From fe5dafdf5168d17f690ffdd37d458c58a8dc0abf Mon Sep 17 00:00:00 2001 From: congbo <39078850+congbobo184@users.noreply.github.com> Date: Wed, 17 Jul 2024 20:11:01 +0800 Subject: [PATCH 376/580] [fix][test]Fix flaky test increase the delay time (#23046) --- .../client/impl/TransactionEndToEndTest.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java index 56cf053314053..812f8fd571cac 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java @@ -1654,24 +1654,24 @@ public void testDelayedTransactionMessages() throws Exception { for (int i = 0; i < 10; i++) { producer.newMessage(transaction) .value("msg-" + i) - .deliverAfter(5, TimeUnit.SECONDS) + .deliverAfter(7, TimeUnit.SECONDS) .sendAsync(); } producer.flush(); transaction.commit().get(); - - // Failover consumer will receive the messages immediately while - // the shared consumer will get them after the delay - Message msg = sharedConsumer.receive(waitTimeForCannotReceiveMsgInSec, TimeUnit.SECONDS); - assertNull(msg); - + Message msg; for (int i = 0; i < 10; i++) { msg = failoverConsumer.receive(waitTimeForCanReceiveMsgInSec, TimeUnit.SECONDS); assertEquals(msg.getValue(), "msg-" + i); } + // Failover consumer will receive the messages immediately while + // the shared consumer will get them after the delay + msg = sharedConsumer.receive(waitTimeForCannotReceiveMsgInSec, TimeUnit.SECONDS); + assertNull(msg); + Set receivedMsgs = new TreeSet<>(); for (int i = 0; i < 10; i++) { msg = sharedConsumer.receive(waitTimeForCanReceiveMsgInSec, TimeUnit.SECONDS); From e51d3e2d5890114725cba54de47344b2b03d0756 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 17 Jul 2024 14:37:39 +0200 Subject: [PATCH 377/580] [fix] Upgrade to Oxia 0.3.1 (#23048) --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index f46b18347c1eb..af50d818c4e7a 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -480,8 +480,8 @@ The Apache Software License, Version 2.0 * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia - - io.streamnative.oxia-oxia-client-api-0.3.0.jar - - io.streamnative.oxia-oxia-client-0.3.0.jar + - io.streamnative.oxia-oxia-client-api-0.3.1.jar + - io.streamnative.oxia-oxia-client-0.3.1.jar * OpenHFT - net.openhft-zero-allocation-hashing-0.16.jar * Java JSON WebTokens diff --git a/pom.xml b/pom.xml index 93e2e24c0558b..c497ea12e838b 100644 --- a/pom.xml +++ b/pom.xml @@ -252,7 +252,7 @@ flexible messaging model and an intuitive client API. 4.5.13 4.4.15 0.7.7 - 0.3.0 + 0.3.1 2.0 1.10.12 5.5.0 From 59136a0ffa0b833411b8af4b7ef9b9c7eb74f909 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 17 Jul 2024 10:06:39 -0700 Subject: [PATCH 378/580] [feat][misc] PIP-264: Add OpenTelemetry HTTP rate limiting filter metric (#23042) --- .../pulsar/broker/web/RateLimitingFilter.java | 27 +++++++++++++++++-- .../apache/pulsar/broker/web/WebService.java | 3 ++- .../pulsar/broker/web/WebServiceTest.java | 26 +++++++++++++++++- .../worker/PulsarWorkerOpenTelemetry.java | 4 ++- .../functions/worker/rest/WorkerServer.java | 5 +++- .../apache/pulsar/proxy/server/WebServer.java | 5 +++- .../proxy/stats/PulsarProxyOpenTelemetry.java | 4 ++- 7 files changed, 66 insertions(+), 8 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/RateLimitingFilter.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/RateLimitingFilter.java index 502b691fa34b0..0618df6609c49 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/RateLimitingFilter.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/RateLimitingFilter.java @@ -19,6 +19,10 @@ package org.apache.pulsar.broker.web; import com.google.common.util.concurrent.RateLimiter; +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.Meter; import io.prometheus.client.Counter; import java.io.IOException; import javax.servlet.Filter; @@ -33,15 +37,32 @@ public class RateLimitingFilter implements Filter { private final RateLimiter limiter; - public RateLimitingFilter(double rateLimit) { - limiter = RateLimiter.create(rateLimit); + public static final String RATE_LIMIT_REQUEST_COUNT_METRIC_NAME = + "pulsar.web.filter.rate_limit.request.count"; + private final LongCounter rateLimitRequestCounter; + + public static final AttributeKey RATE_LIMIT_RESULT = + AttributeKey.stringKey("pulsar.web.filter.rate_limit.result"); + public enum Result { + ACCEPTED, + REJECTED; + public final Attributes attributes = Attributes.of(RATE_LIMIT_RESULT, name().toLowerCase()); } + @Deprecated private static final Counter httpRejectedRequests = Counter.build() .name("pulsar_broker_http_rejected_requests") .help("Counter of HTTP requests rejected by rate limiting") .register(); + public RateLimitingFilter(double rateLimit, Meter meter) { + limiter = RateLimiter.create(rateLimit); + rateLimitRequestCounter = meter.counterBuilder(RATE_LIMIT_REQUEST_COUNT_METRIC_NAME) + .setDescription("Counter of HTTP requests processed by the rate limiting filter.") + .setUnit("{request}") + .build(); + } + @Override public void init(FilterConfig filterConfig) throws ServletException { } @@ -50,9 +71,11 @@ public void init(FilterConfig filterConfig) throws ServletException { public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) throws IOException, ServletException { if (limiter.tryAcquire()) { + rateLimitRequestCounter.add(1, Result.ACCEPTED.attributes); chain.doFilter(request, response); } else { httpRejectedRequests.inc(); + rateLimitRequestCounter.add(1, Result.REJECTED.attributes); HttpServletResponse httpResponse = (HttpServletResponse) response; httpResponse.sendError(429, "Too Many Requests"); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java index c969f40ad4382..d95e88661ae8c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java @@ -250,7 +250,8 @@ private static class FilterInitializer { if (config.isHttpRequestsLimitEnabled()) { filterHolders.add(new FilterHolder( - new RateLimitingFilter(config.getHttpRequestsMaxPerSecond()))); + new RateLimitingFilter(config.getHttpRequestsMaxPerSecond(), + pulsarService.getOpenTelemetry().getMeter()))); } // wait until the PulsarService is ready to serve incoming requests diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index 30644237a7405..08041d72c7e44 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -18,10 +18,10 @@ */ package org.apache.pulsar.broker.web; +import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; -import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -61,6 +61,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.broker.web.RateLimitingFilter.Result; import org.apache.pulsar.broker.web.WebExecutorThreadPoolStats.LimitType; import org.apache.pulsar.broker.web.WebExecutorThreadPoolStats.UsageType; import org.apache.pulsar.client.admin.PulsarAdmin; @@ -270,12 +271,29 @@ public void testTlsAuthDisallowInsecure() throws Exception { public void testRateLimiting() throws Exception { setupEnv(false, false, false, false, 10.0, false); + // setupEnv makes a HTTP call to create the cluster. + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, RateLimitingFilter.RATE_LIMIT_REQUEST_COUNT_METRIC_NAME, + Result.ACCEPTED.attributes, 1); + assertThat(metrics).noneSatisfy(metricData -> assertThat(metricData) + .hasName(RateLimitingFilter.RATE_LIMIT_REQUEST_COUNT_METRIC_NAME) + .hasLongSumSatisfying( + sum -> sum.hasPointsSatisfying(point -> point.hasAttributes(Result.REJECTED.attributes)))); + // Make requests without exceeding the max rate for (int i = 0; i < 5; i++) { makeHttpRequest(false, false); Thread.sleep(200); } + metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, RateLimitingFilter.RATE_LIMIT_REQUEST_COUNT_METRIC_NAME, + Result.ACCEPTED.attributes, 6); + assertThat(metrics).noneSatisfy(metricData -> assertThat(metricData) + .hasName(RateLimitingFilter.RATE_LIMIT_REQUEST_COUNT_METRIC_NAME) + .hasLongSumSatisfying( + sum -> sum.hasPointsSatisfying(point -> point.hasAttributes(Result.REJECTED.attributes)))); + try { for (int i = 0; i < 500; i++) { makeHttpRequest(false, false); @@ -285,6 +303,12 @@ public void testRateLimiting() throws Exception { } catch (IOException e) { assertTrue(e.getMessage().contains("429")); } + + metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, RateLimitingFilter.RATE_LIMIT_REQUEST_COUNT_METRIC_NAME, + Result.ACCEPTED.attributes, value -> assertThat(value).isGreaterThan(6)); + assertMetricLongSumValue(metrics, RateLimitingFilter.RATE_LIMIT_REQUEST_COUNT_METRIC_NAME, + Result.REJECTED.attributes, value -> assertThat(value).isPositive()); } @Test diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerOpenTelemetry.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerOpenTelemetry.java index be7c15dfd85e0..6673a89659a65 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerOpenTelemetry.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerOpenTelemetry.java @@ -27,6 +27,8 @@ public class PulsarWorkerOpenTelemetry implements Closeable { public static final String SERVICE_NAME = "pulsar-function-worker"; + public static final String INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.function_worker"; + private final OpenTelemetryService openTelemetryService; @Getter @@ -38,7 +40,7 @@ public PulsarWorkerOpenTelemetry(WorkerConfig workerConfig) { .serviceName(SERVICE_NAME) .serviceVersion(PulsarVersion.getVersion()) .build(); - meter = openTelemetryService.getOpenTelemetry().getMeter("org.apache.pulsar.function_worker"); + meter = openTelemetryService.getOpenTelemetry().getMeter(INSTRUMENTATION_SCOPE_NAME); } @Override diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java index 583d8ce558b08..1d8c66a57df53 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.functions.worker.rest; +import io.opentelemetry.api.OpenTelemetry; import io.prometheus.client.jetty.JettyStatisticsCollector; import java.util.ArrayList; import java.util.EnumSet; @@ -30,6 +31,7 @@ import org.apache.pulsar.broker.web.JettyRequestLogFactory; import org.apache.pulsar.broker.web.RateLimitingFilter; import org.apache.pulsar.broker.web.WebExecutorThreadPool; +import org.apache.pulsar.functions.worker.PulsarWorkerOpenTelemetry; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.rest.api.v2.WorkerApiV2Resource; @@ -219,7 +221,8 @@ private static class FilterInitializer { if (config.isHttpRequestsLimitEnabled()) { filterHolders.add(new FilterHolder( - new RateLimitingFilter(config.getHttpRequestsMaxPerSecond()))); + new RateLimitingFilter(config.getHttpRequestsMaxPerSecond(), + OpenTelemetry.noop().getMeter(PulsarWorkerOpenTelemetry.INSTRUMENTATION_SCOPE_NAME)))); } if (config.isAuthenticationEnabled()) { diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java index 478b911eb23cf..ad94f1b65a092 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java @@ -19,6 +19,7 @@ package org.apache.pulsar.proxy.server; import static org.apache.pulsar.proxy.server.AdminProxyHandler.INIT_PARAM_REQUEST_BUFFER_SIZE; +import io.opentelemetry.api.OpenTelemetry; import io.prometheus.client.jetty.JettyStatisticsCollector; import java.io.IOException; import java.net.URI; @@ -37,6 +38,7 @@ import org.apache.pulsar.broker.web.RateLimitingFilter; import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.apache.pulsar.proxy.stats.PulsarProxyOpenTelemetry; import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; import org.eclipse.jetty.server.Connector; @@ -191,7 +193,8 @@ private static class FilterInitializer { if (config.isHttpRequestsLimitEnabled()) { filterHolders.add(new FilterHolder( - new RateLimitingFilter(config.getHttpRequestsMaxPerSecond()))); + new RateLimitingFilter(config.getHttpRequestsMaxPerSecond(), + OpenTelemetry.noop().getMeter(PulsarProxyOpenTelemetry.INSTRUMENTATION_SCOPE_NAME)))); } if (config.isAuthenticationEnabled()) { diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java index 14bbc649466bb..2748e2c3df5b0 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/PulsarProxyOpenTelemetry.java @@ -28,6 +28,8 @@ public class PulsarProxyOpenTelemetry implements Closeable { public static final String SERVICE_NAME = "pulsar-proxy"; + public static final String INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.proxy"; + private final OpenTelemetryService openTelemetryService; @Getter @@ -39,7 +41,7 @@ public PulsarProxyOpenTelemetry(ProxyConfiguration config) { .serviceName(SERVICE_NAME) .serviceVersion(PulsarVersion.getVersion()) .build(); - meter = openTelemetryService.getOpenTelemetry().getMeter("org.apache.pulsar.proxy"); + meter = openTelemetryService.getOpenTelemetry().getMeter(INSTRUMENTATION_SCOPE_NAME); } @Override From d7e8ea16e6682df9a9354cda25cf4f1f9cb54429 Mon Sep 17 00:00:00 2001 From: Yuri Mizushima Date: Fri, 19 Jul 2024 12:37:41 +0900 Subject: [PATCH 379/580] [fix][broker] Introduce the last sent position to fix message ordering issues in Key_Shared (PIP-282) (#21953) --- .../mledger/impl/ManagedCursorImpl.java | 13 + .../mledger/impl/ManagedLedgerImpl.java | 2 +- .../pulsar/broker/service/Consumer.java | 10 +- ...tStickyKeyDispatcherMultipleConsumers.java | 195 +++++++-- .../persistent/PersistentSubscription.java | 19 +- .../pulsar/broker/admin/AdminApiTest.java | 196 ++++++++- ...ckyKeyDispatcherMultipleConsumersTest.java | 357 ++++++++++++++++ .../broker/stats/ConsumerStatsTest.java | 2 +- .../client/api/KeySharedSubscriptionTest.java | 399 +++++++++++++++++- .../common/policies/data/ConsumerStats.java | 4 +- .../policies/data/SubscriptionStats.java | 6 + .../data/stats/ConsumerStatsImpl.java | 10 +- .../data/stats/SubscriptionStatsImpl.java | 6 + 13 files changed, 1158 insertions(+), 61 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 4ef9678f3e180..f99ee957e025a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -3472,6 +3472,19 @@ public LongPairRangeSet getIndividuallyDeletedMessagesSet() { return individualDeletedMessages; } + public Position processIndividuallyDeletedMessagesAndGetMarkDeletedPosition( + LongPairRangeSet.RangeProcessor processor) { + final Position mdp; + lock.readLock().lock(); + try { + mdp = markDeletePosition; + individualDeletedMessages.forEach(processor); + } finally { + lock.readLock().unlock(); + } + return mdp; + } + public boolean isMessageDeleted(Position position) { lock.readLock().lock(); try { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index b7734906f7553..209bf57b24f0f 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -3497,7 +3497,7 @@ private CompletableFuture completeLedgerInfoForOffloaded(long ledgerId, UU * the position range * @return the count of entries */ - long getNumberOfEntries(Range range) { + public long getNumberOfEntries(Range range) { Position fromPosition = range.lowerEndpoint(); boolean fromIncluded = range.lowerBoundType() == BoundType.CLOSED; Position toPosition = range.upperEndpoint(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index 02e21c44c9179..dca64395d8674 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -145,7 +145,7 @@ public class Consumer { private static final double avgPercent = 0.9; private boolean preciseDispatcherFlowControl; - private Position readPositionWhenJoining; + private Position lastSentPositionWhenJoining; private final String clientAddress; // IP address only, no port number included private final MessageId startMessageId; private final boolean isAcknowledgmentAtBatchIndexLevelEnabled; @@ -931,8 +931,8 @@ public ConsumerStatsImpl getStats() { stats.unackedMessages = unackedMessages; stats.blockedConsumerOnUnackedMsgs = blockedConsumerOnUnackedMsgs; stats.avgMessagesPerEntry = getAvgMessagesPerEntry(); - if (readPositionWhenJoining != null) { - stats.readPositionWhenJoining = readPositionWhenJoining.toString(); + if (lastSentPositionWhenJoining != null) { + stats.lastSentPositionWhenJoining = lastSentPositionWhenJoining.toString(); } return stats; } @@ -1166,8 +1166,8 @@ public boolean isPreciseDispatcherFlowControl() { return preciseDispatcherFlowControl; } - public void setReadPositionWhenJoining(Position readPositionWhenJoining) { - this.readPositionWhenJoining = readPositionWhenJoining; + public void setLastSentPositionWhenJoining(Position lastSentPositionWhenJoining) { + this.lastSentPositionWhenJoining = lastSentPositionWhenJoining; } public int getMaxUnackedMessages() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 766f45ad9908c..91cec1f8e9071 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -34,9 +34,12 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import javax.annotation.Nullable; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.collections4.MapUtils; import org.apache.pulsar.broker.ServiceConfiguration; @@ -55,6 +58,8 @@ import org.apache.pulsar.common.api.proto.KeySharedMeta; import org.apache.pulsar.common.api.proto.KeySharedMode; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; +import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,12 +78,22 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi */ private final LinkedHashMap recentlyJoinedConsumers; + /** + * The lastSentPosition and the individuallySentPositions are not thread safe. + */ + @Nullable + private Position lastSentPosition; + private final LongPairRangeSet individuallySentPositions; + private static final LongPairRangeSet.LongPairConsumer positionRangeConverter = PositionFactory::create; + PersistentStickyKeyDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor, Subscription subscription, ServiceConfiguration conf, KeySharedMeta ksm) { super(topic, cursor, subscription, ksm.isAllowOutOfOrderDelivery()); this.allowOutOfOrderDelivery = ksm.isAllowOutOfOrderDelivery(); this.recentlyJoinedConsumers = allowOutOfOrderDelivery ? null : new LinkedHashMap<>(); + this.individuallySentPositions = + allowOutOfOrderDelivery ? null : new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter); this.keySharedMode = ksm.getKeySharedMode(); switch (this.keySharedMode) { case AUTO_SPLIT: @@ -124,15 +139,18 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { }) ).thenRun(() -> { synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) { - Position readPositionWhenJoining = cursor.getReadPosition(); - consumer.setReadPositionWhenJoining(readPositionWhenJoining); - // If this was the 1st consumer, or if all the messages are already acked, then we - // don't need to do anything special - if (!allowOutOfOrderDelivery - && recentlyJoinedConsumers != null - && consumerList.size() > 1 - && cursor.getNumberOfEntriesSinceFirstNotAckedMessage() > 1) { - recentlyJoinedConsumers.put(consumer, readPositionWhenJoining); + if (!allowOutOfOrderDelivery) { + final Position lastSentPositionWhenJoining = updateIfNeededAndGetLastSentPosition(); + if (lastSentPositionWhenJoining != null) { + consumer.setLastSentPositionWhenJoining(lastSentPositionWhenJoining); + // If this was the 1st consumer, or if all the messages are already acked, then we + // don't need to do anything special + if (recentlyJoinedConsumers != null + && consumerList.size() > 1 + && cursor.getNumberOfEntriesSinceFirstNotAckedMessage() > 1) { + recentlyJoinedConsumers.put(consumer, lastSentPositionWhenJoining); + } + } } } }); @@ -148,10 +166,16 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE // eventually causing all consumers to get stuck. selector.removeConsumer(consumer); super.removeConsumer(consumer); - if (recentlyJoinedConsumers != null) { + if (!allowOutOfOrderDelivery && recentlyJoinedConsumers != null) { recentlyJoinedConsumers.remove(consumer); if (consumerList.size() == 1) { recentlyJoinedConsumers.clear(); + } else if (consumerList.isEmpty()) { + // The subscription removes consumers if rewind or reset cursor operations are called. + // The dispatcher must clear lastSentPosition and individuallySentPositions because + // these operations trigger re-sending messages. + lastSentPosition = null; + individuallySentPositions.clear(); } if (removeConsumersFromRecentJoinedConsumers() || !redeliveryMessages.isEmpty()) { readMoreEntries(); @@ -193,9 +217,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis return false; } - // A corner case that we have to retry a readMoreEntries in order to preserver order delivery. - // This may happen when consumer closed. See issue #12885 for details. if (!allowOutOfOrderDelivery) { + // A corner case that we have to retry a readMoreEntries in order to preserver order delivery. + // This may happen when consumer closed. See issue #12885 for details. NavigableSet messagesToReplayNow = this.getMessagesToReplayNow(1); if (messagesToReplayNow != null && !messagesToReplayNow.isEmpty()) { Position replayPosition = messagesToReplayNow.first(); @@ -229,6 +253,24 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } } } + + // Update if the markDeletePosition move forward + updateIfNeededAndGetLastSentPosition(); + + // Should not access to individualDeletedMessages from outside managed cursor + // because it doesn't guarantee thread safety. + if (lastSentPosition == null) { + if (cursor.getMarkDeletedPosition() != null) { + lastSentPosition = ((ManagedCursorImpl) cursor) + .processIndividuallyDeletedMessagesAndGetMarkDeletedPosition(range -> { + final Position lower = range.lowerEndpoint(); + final Position upper = range.upperEndpoint(); + individuallySentPositions.addOpenClosed(lower.getLedgerId(), lower.getEntryId(), + upper.getLedgerId(), upper.getEntryId()); + return true; + }); + } + } } final Map> groupedEntries = localGroupedEntries.get(); @@ -280,12 +322,24 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } if (messagesForC > 0) { - // remove positions first from replay list first : sendMessages recycles entries - if (readType == ReadType.Replay) { - for (int i = 0; i < messagesForC; i++) { - Entry entry = entriesWithSameKey.get(i); + final ManagedLedgerImpl managedLedger = ((ManagedLedgerImpl) cursor.getManagedLedger()); + for (int i = 0; i < messagesForC; i++) { + final Entry entry = entriesWithSameKey.get(i); + // remove positions first from replay list first : sendMessages recycles entries + if (readType == ReadType.Replay) { redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId()); } + // Add positions to individuallySentPositions if necessary + if (!allowOutOfOrderDelivery) { + final Position position = entry.getPosition(); + // Store to individuallySentPositions even if lastSentPosition is null + if ((lastSentPosition == null || position.compareTo(lastSentPosition) > 0) + && !individuallySentPositions.contains(position.getLedgerId(), position.getEntryId())) { + final Position previousPosition = managedLedger.getPreviousPosition(position); + individuallySentPositions.addOpenClosed(previousPosition.getLedgerId(), + previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); + } + } } SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); @@ -311,6 +365,61 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } } + // Update the last sent position and remove ranges from individuallySentPositions if necessary + if (!allowOutOfOrderDelivery && lastSentPosition != null) { + final ManagedLedgerImpl managedLedger = ((ManagedLedgerImpl) cursor.getManagedLedger()); + com.google.common.collect.Range range = individuallySentPositions.firstRange(); + + // If the upper bound is before the last sent position, we need to move ahead as these + // individuallySentPositions are now irrelevant. + if (range != null && range.upperEndpoint().compareTo(lastSentPosition) <= 0) { + individuallySentPositions.removeAtMost(lastSentPosition.getLedgerId(), + lastSentPosition.getEntryId()); + range = individuallySentPositions.firstRange(); + } + + if (range != null) { + // If the lowerBound is ahead of the last sent position, + // verify if there are any entries in-between. + if (range.lowerEndpoint().compareTo(lastSentPosition) <= 0 || managedLedger + .getNumberOfEntries(com.google.common.collect.Range.openClosed(lastSentPosition, + range.lowerEndpoint())) <= 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] Found a position range to last sent: {}", name, range); + } + Position newLastSentPosition = range.upperEndpoint(); + Position positionAfterNewLastSent = managedLedger + .getNextValidPosition(newLastSentPosition); + // sometime ranges are connected but belongs to different ledgers + // so, they are placed sequentially + // eg: (2:10..3:15] can be returned as (2:10..2:15],[3:0..3:15]. + // So, try to iterate over connected range and found the last non-connected range + // which gives new last sent position. + final Position lastConfirmedEntrySnapshot = managedLedger.getLastConfirmedEntry(); + if (lastConfirmedEntrySnapshot != null) { + while (positionAfterNewLastSent.compareTo(lastConfirmedEntrySnapshot) <= 0) { + if (individuallySentPositions.contains(positionAfterNewLastSent.getLedgerId(), + positionAfterNewLastSent.getEntryId())) { + range = individuallySentPositions.rangeContaining( + positionAfterNewLastSent.getLedgerId(), positionAfterNewLastSent.getEntryId()); + newLastSentPosition = range.upperEndpoint(); + positionAfterNewLastSent = managedLedger.getNextValidPosition(newLastSentPosition); + // check if next valid position is also deleted and part of the deleted-range + continue; + } + break; + } + } + + if (lastSentPosition.compareTo(newLastSentPosition) < 0) { + lastSentPosition = newLastSentPosition; + } + individuallySentPositions.removeAtMost(lastSentPosition.getLedgerId(), + lastSentPosition.getEntryId()); + } + } + } + // acquire message-dispatch permits for already delivered messages acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); @@ -351,10 +460,10 @@ private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List= 0) { + if ((entries.get(i)).compareTo(maxLastSentPosition) > 0) { // We have already crossed the divider line. All messages in the list are now // newer than what we can currently dispatch to this consumer return i; @@ -416,11 +525,9 @@ private boolean removeConsumersFromRecentJoinedConsumers() { boolean hasConsumerRemovedFromTheRecentJoinedConsumers = false; Position mdp = cursor.getMarkDeletedPosition(); if (mdp != null) { - Position nextPositionOfTheMarkDeletePosition = - ((ManagedLedgerImpl) cursor.getManagedLedger()).getNextValidPosition(mdp); while (itr.hasNext()) { Map.Entry entry = itr.next(); - if (entry.getValue().compareTo(nextPositionOfTheMarkDeletePosition) <= 0) { + if (entry.getValue().compareTo(mdp) <= 0) { itr.remove(); hasConsumerRemovedFromTheRecentJoinedConsumers = true; } else { @@ -431,6 +538,18 @@ private boolean removeConsumersFromRecentJoinedConsumers() { return hasConsumerRemovedFromTheRecentJoinedConsumers; } + @Nullable + private synchronized Position updateIfNeededAndGetLastSentPosition() { + if (lastSentPosition == null) { + return null; + } + final Position mdp = cursor.getMarkDeletedPosition(); + if (mdp != null && mdp.compareTo(lastSentPosition) > 0) { + lastSentPosition = mdp; + } + return lastSentPosition; + } + @Override protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { if (isDispatcherStuckOnReplays) { @@ -551,6 +670,30 @@ public LinkedHashMap getRecentlyJoinedConsumers() { return recentlyJoinedConsumers; } + public synchronized String getLastSentPosition() { + if (lastSentPosition == null) { + return null; + } + return lastSentPosition.toString(); + } + + @VisibleForTesting + public Position getLastSentPositionField() { + return lastSentPosition; + } + + public synchronized String getIndividuallySentPositions() { + if (individuallySentPositions == null) { + return null; + } + return individuallySentPositions.toString(); + } + + @VisibleForTesting + public LongPairRangeSet getIndividuallySentPositionsField() { + return individuallySentPositions; + } + public Map> getConsumerKeyHashRanges() { return selector.getConsumerKeyHashRanges(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index a1d51668ca808..77aa5f82c3914 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -1305,9 +1305,26 @@ public SubscriptionStatsImpl getStats(GetStatsOptions getStatsOptions) { .getRecentlyJoinedConsumers(); if (recentlyJoinedConsumers != null && recentlyJoinedConsumers.size() > 0) { recentlyJoinedConsumers.forEach((k, v) -> { - subStats.consumersAfterMarkDeletePosition.put(k.consumerName(), v.toString()); + // The dispatcher allows same name consumers + final StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append("consumerName=").append(k.consumerName()) + .append(", consumerId=").append(k.consumerId()); + if (k.cnx() != null) { + stringBuilder.append(", address=").append(k.cnx().clientAddress()); + } + subStats.consumersAfterMarkDeletePosition.put(stringBuilder.toString(), v.toString()); }); } + final String lastSentPosition = ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher) + .getLastSentPosition(); + if (lastSentPosition != null) { + subStats.lastSentPosition = lastSentPosition; + } + final String individuallySentPositions = ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher) + .getIndividuallySentPositions(); + if (individuallySentPositions != null) { + subStats.individuallySentPositions = individuallySentPositions; + } } subStats.nonContiguousDeletedMessagesRanges = cursor.getTotalNonContiguousDeletedMessagesRange(); subStats.nonContiguousDeletedMessagesRangesSerializedSize = diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index 1c83941d6e721..5432b8a430d63 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.admin; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -56,6 +58,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; import javax.ws.rs.client.InvocationCallback; import javax.ws.rs.client.WebTarget; import javax.ws.rs.core.Response.Status; @@ -65,6 +68,7 @@ import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarServerException; @@ -75,6 +79,8 @@ import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.broker.testcontext.SpyConfig; import org.apache.pulsar.client.admin.GetStatsOptions; @@ -139,7 +145,10 @@ import org.apache.pulsar.common.policies.data.TopicHashPositions; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.Codec; +import org.apache.pulsar.common.util.Murmur3_32Hash; import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; +import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.compaction.Compactor; import org.apache.pulsar.compaction.PulsarCompactionServiceFactory; import org.awaitility.Awaitility; @@ -3449,8 +3458,8 @@ public void testGetTtlDurationDefaultInSeconds() throws Exception { } @Test - public void testGetReadPositionWhenJoining() throws Exception { - final String topic = "persistent://prop-xyz/ns1/testGetReadPositionWhenJoining-" + UUID.randomUUID().toString(); + public void testGetLastSentPositionWhenJoining() throws Exception { + final String topic = "persistent://prop-xyz/ns1/testGetLastSentPositionWhenJoining-" + UUID.randomUUID().toString(); final String subName = "my-sub"; @Cleanup Producer producer = pulsarClient.newProducer() @@ -3458,34 +3467,189 @@ public void testGetReadPositionWhenJoining() throws Exception { .enableBatching(false) .create(); + @Cleanup + final Consumer consumer1 = pulsarClient.newConsumer() + .topic(topic) + .subscriptionType(SubscriptionType.Key_Shared) + .subscriptionName(subName) + .subscribe(); + final int messages = 10; MessageIdImpl messageId = null; for (int i = 0; i < messages; i++) { messageId = (MessageIdImpl) producer.send(("Hello Pulsar - " + i).getBytes()); + consumer1.receive(); } - List> consumers = new ArrayList<>(); - for (int i = 0; i < 2; i++) { - Consumer consumer = pulsarClient.newConsumer() - .topic(topic) - .subscriptionType(SubscriptionType.Key_Shared) - .subscriptionName(subName) - .subscribe(); - consumers.add(consumer); - } + @Cleanup + final Consumer consumer2 = pulsarClient.newConsumer() + .topic(topic) + .subscriptionType(SubscriptionType.Key_Shared) + .subscriptionName(subName) + .subscribe(); TopicStats stats = admin.topics().getStats(topic); Assert.assertEquals(stats.getSubscriptions().size(), 1); SubscriptionStats subStats = stats.getSubscriptions().get(subName); Assert.assertNotNull(subStats); Assert.assertEquals(subStats.getConsumers().size(), 2); - ConsumerStats consumerStats = subStats.getConsumers().get(0); - Assert.assertEquals(consumerStats.getReadPositionWhenJoining(), - PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId() + 1).toString()); + ConsumerStats consumerStats = subStats.getConsumers().stream() + .filter(s -> s.getConsumerName().equals(consumer2.getConsumerName())).findFirst().get(); + Assert.assertEquals(consumerStats.getLastSentPositionWhenJoining(), + PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()).toString()); + } + + @Test + public void testGetLastSentPosition() throws Exception { + final String topic = "persistent://prop-xyz/ns1/testGetLastSentPosition-" + UUID.randomUUID().toString(); + final String subName = "my-sub"; + @Cleanup + final Producer producer = pulsarClient.newProducer() + .topic(topic) + .enableBatching(false) + .create(); + final AtomicInteger counter = new AtomicInteger(); + @Cleanup + final Consumer consumer = pulsarClient.newConsumer() + .topic(topic) + .subscriptionType(SubscriptionType.Key_Shared) + .subscriptionName(subName) + .messageListener((c, msg) -> { + try { + c.acknowledge(msg); + counter.getAndIncrement(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }) + .subscribe(); + + TopicStats stats = admin.topics().getStats(topic); + Assert.assertEquals(stats.getSubscriptions().size(), 1); + SubscriptionStats subStats = stats.getSubscriptions().get(subName); + Assert.assertNotNull(subStats); + Assert.assertNull(subStats.getLastSentPosition()); - for (Consumer consumer : consumers) { - consumer.close(); + final int messages = 10; + MessageIdImpl messageId = null; + for (int i = 0; i < messages; i++) { + messageId = (MessageIdImpl) producer.send(("Hello Pulsar - " + i).getBytes()); } + + Awaitility.await().untilAsserted(() -> assertEquals(counter.get(), messages)); + + stats = admin.topics().getStats(topic); + Assert.assertEquals(stats.getSubscriptions().size(), 1); + subStats = stats.getSubscriptions().get(subName); + Assert.assertNotNull(subStats); + Assert.assertEquals(subStats.getLastSentPosition(), PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()).toString()); + } + + @Test + public void testGetIndividuallySentPositions() throws Exception { + // The producer sends messages with two types of keys. + // The dispatcher sends keyA messages to consumer1. + // Consumer1 will not receive any messages. Its receiver queue size is 1. + // Consumer2 will receive and ack any messages immediately. + + final String topic = "persistent://prop-xyz/ns1/testGetIndividuallySentPositions-" + UUID.randomUUID().toString(); + final String subName = "my-sub"; + @Cleanup + final Producer producer = pulsarClient.newProducer() + .topic(topic) + .enableBatching(false) + .create(); + + final String consumer1Name = "c1"; + final String consumer2Name = "c2"; + + @Cleanup + final Consumer consumer1 = pulsarClient.newConsumer() + .topic(topic) + .consumerName(consumer1Name) + .receiverQueueSize(1) + .subscriptionType(SubscriptionType.Key_Shared) + .subscriptionName(subName) + .subscribe(); + + final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topic).get().get().getSubscription(subName).getDispatcher(); + final String keyA = "key-a"; + final String keyB = "key-b"; + final int hashA = Murmur3_32Hash.getInstance().makeHash(keyA.getBytes()); + + final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); + selectorField.setAccessible(true); + final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); + selectorField.set(dispatcher, selector); + + // the selector returns consumer1 if keyA + doAnswer((invocationOnMock -> { + final int hash = invocationOnMock.getArgument(0); + + final String consumerName = hash == hashA ? consumer1Name : consumer2Name; + return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); + })).when(selector).select(anyInt()); + + final AtomicInteger consumer2AckCounter = new AtomicInteger(); + @Cleanup + final Consumer consumer2 = pulsarClient.newConsumer() + .topic(topic) + .consumerName(consumer2Name) + .subscriptionType(SubscriptionType.Key_Shared) + .subscriptionName(subName) + .messageListener((c, msg) -> { + try { + c.acknowledge(msg); + consumer2AckCounter.getAndIncrement(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }) + .subscribe(); + + final LongPairRangeSet.LongPairConsumer positionRangeConverter = PositionFactory::create; + final LongPairRangeSet expectedIndividuallySentPositions = new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter); + + TopicStats stats = admin.topics().getStats(topic); + Assert.assertEquals(stats.getSubscriptions().size(), 1); + SubscriptionStats subStats = stats.getSubscriptions().get(subName); + Assert.assertNotNull(subStats); + Assert.assertEquals(subStats.getIndividuallySentPositions(), expectedIndividuallySentPositions.toString()); + + final Function sendFn = (key) -> { + try { + return (MessageIdImpl) producer.newMessage().key(key).value(("msg").getBytes()).send(); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } + }; + final List messageIdList = new ArrayList<>(); + + // the dispatcher can send keyA message, but then consumer1's receiver queue will be full + messageIdList.add(sendFn.apply(keyA)); + + // the dispatcher can send messages other than keyA + messageIdList.add(sendFn.apply(keyA)); + messageIdList.add(sendFn.apply(keyB)); + messageIdList.add(sendFn.apply(keyA)); + messageIdList.add(sendFn.apply(keyB)); + messageIdList.add(sendFn.apply(keyB)); + + assertEquals(messageIdList.size(), 6); + Awaitility.await().untilAsserted(() -> assertEquals(consumer2AckCounter.get(), 3)); + + // set expected value + expectedIndividuallySentPositions.addOpenClosed(messageIdList.get(1).getLedgerId(), messageIdList.get(1).getEntryId(), + messageIdList.get(2).getLedgerId(), messageIdList.get(2).getEntryId()); + expectedIndividuallySentPositions.addOpenClosed(messageIdList.get(3).getLedgerId(), messageIdList.get(3).getEntryId(), + messageIdList.get(5).getLedgerId(), messageIdList.get(5).getEntryId()); + + stats = admin.topics().getStats(topic); + Assert.assertEquals(stats.getSubscriptions().size(), 1); + subStats = stats.getSubscriptions().get(subName); + Assert.assertNotNull(subStats); + Assert.assertEquals(subStats.getIndividuallySentPositions(), expectedIndividuallySentPositions.toString()); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index a70b3ce7a42f6..1a205d0f686d5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -35,14 +35,19 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.google.common.collect.BoundType; +import com.google.common.collect.Range; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.ChannelPromise; import io.netty.channel.EventLoopGroup; import java.lang.reflect.Field; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Queue; @@ -50,12 +55,14 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; @@ -72,11 +79,14 @@ import org.apache.pulsar.common.policies.data.HierarchyTopicPolicies; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.Markers; +import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; +import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.awaitility.Awaitility; import org.mockito.ArgumentCaptor; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Test(groups = "broker") @@ -84,6 +94,7 @@ public class PersistentStickyKeyDispatcherMultipleConsumersTest { private PulsarService pulsarMock; private BrokerService brokerMock; + private ManagedLedgerImpl ledgerMock; private ManagedCursorImpl cursorMock; private Consumer consumerMock; private PersistentTopic topicMock; @@ -135,9 +146,44 @@ public void setup() throws Exception { doReturn(topicName).when(topicMock).getName(); doReturn(topicPolicies).when(topicMock).getHierarchyTopicPolicies(); + ledgerMock = mock(ManagedLedgerImpl.class); + doAnswer((invocationOnMock -> { + final Position position = invocationOnMock.getArgument(0); + if (position.getEntryId() > 0) { + return PositionFactory.create(position.getLedgerId(), position.getEntryId() - 1); + } else { + fail("Undefined behavior on mock"); + return PositionFactory.EARLIEST; + } + })).when(ledgerMock).getPreviousPosition(any(Position.class)); + doAnswer((invocationOnMock -> { + final Position position = invocationOnMock.getArgument(0); + return PositionFactory.create(position.getLedgerId(), position.getEntryId() < 0 ? 0 : position.getEntryId() + 1); + })).when(ledgerMock).getNextValidPosition(any(Position.class)); + doAnswer((invocationOnMock -> { + final Range range = invocationOnMock.getArgument(0); + Position fromPosition = range.lowerEndpoint(); + boolean fromIncluded = range.lowerBoundType() == BoundType.CLOSED; + Position toPosition = range.upperEndpoint(); + boolean toIncluded = range.upperBoundType() == BoundType.CLOSED; + + long count = 0; + + if (fromPosition.getLedgerId() == toPosition.getLedgerId()) { + // If the 2 positions are in the same ledger + count = toPosition.getEntryId() - fromPosition.getEntryId() - 1; + count += fromIncluded ? 1 : 0; + count += toIncluded ? 1 : 0; + } else { + fail("Undefined behavior on mock"); + } + return count; + })).when(ledgerMock).getNumberOfEntries(any()); + cursorMock = mock(ManagedCursorImpl.class); doReturn(null).when(cursorMock).getLastIndividualDeletedRange(); doReturn(subscriptionName).when(cursorMock).getName(); + doReturn(ledgerMock).when(cursorMock).getManagedLedger(); consumerMock = mock(Consumer.class); channelMock = mock(ChannelPromise.class); @@ -465,6 +511,317 @@ public void testMessageRedelivery() throws Exception { allEntries.forEach(entry -> entry.release()); } + + + @DataProvider(name = "initializeLastSentPosition") + private Object[][] initialLastSentPositionProvider() { + return new Object[][] { { false }, { true } }; + } + + @Test(dataProvider = "initializeLastSentPosition") + public void testLastSentPositionAndIndividuallySentPositions(final boolean initializeLastSentPosition) throws Exception { + final Position initialLastSentPosition = PositionFactory.create(1, 10); + final LongPairRangeSet expectedIndividuallySentPositions + = new ConcurrentOpenLongPairRangeSet<>(4096, PositionFactory::create); + + final Field lastSentPositionField = PersistentStickyKeyDispatcherMultipleConsumers.class + .getDeclaredField("lastSentPosition"); + lastSentPositionField.setAccessible(true); + final LongPairRangeSet individuallySentPositions = persistentDispatcher.getIndividuallySentPositionsField(); + final Supplier clearPosition = () -> { + try { + lastSentPositionField.set(persistentDispatcher, initializeLastSentPosition ? initialLastSentPosition : null); + individuallySentPositions.clear(); + expectedIndividuallySentPositions.clear(); + } catch (Throwable e) { + return e; + } + return null; + }; + if (!initializeLastSentPosition) { + doReturn(initialLastSentPosition).when(cursorMock).getMarkDeletedPosition(); + doAnswer(invocationOnMock -> { + // skip copy operation + return initialLastSentPosition; + }).when(cursorMock).processIndividuallyDeletedMessagesAndGetMarkDeletedPosition(any()); + } + + // Assume the range sequence is [1:0, 1:19], [2:0, 2:19], ..., [10:0, 10:19] + doAnswer((invocationOnMock -> { + final Position position = invocationOnMock.getArgument(0); + if (position.getEntryId() > 0) { + return PositionFactory.create(position.getLedgerId(), position.getEntryId() - 1); + } else if (position.getLedgerId() > 0) { + return PositionFactory.create(position.getLedgerId() - 1, 19); + } else { + throw new NullPointerException(); + } + })).when(ledgerMock).getPreviousPosition(any(Position.class)); + doAnswer((invocationOnMock -> { + final Position position = invocationOnMock.getArgument(0); + if (position.getEntryId() < 19) { + return PositionFactory.create(position.getLedgerId(), position.getEntryId() + 1); + } else { + return PositionFactory.create(position.getLedgerId() + 1, 0); + } + })).when(ledgerMock).getNextValidPosition(any(Position.class)); + doReturn(PositionFactory.create(10, 19)).when(ledgerMock).getLastConfirmedEntry(); + doAnswer((invocationOnMock -> { + final Range range = invocationOnMock.getArgument(0); + Position fromPosition = range.lowerEndpoint(); + boolean fromIncluded = range.lowerBoundType() == BoundType.CLOSED; + Position toPosition = range.upperEndpoint(); + boolean toIncluded = range.upperBoundType() == BoundType.CLOSED; + + if (fromPosition.getLedgerId() == toPosition.getLedgerId()) { + // If the 2 positions are in the same ledger + long count = toPosition.getEntryId() - fromPosition.getEntryId() - 1; + count += fromIncluded ? 1 : 0; + count += toIncluded ? 1 : 0; + return count; + } else { + long count = 0; + // If the from & to are pointing to different ledgers, then we need to : + // 1. Add the entries in the ledger pointed by toPosition + count += toPosition.getEntryId(); + count += toIncluded ? 1 : 0; + + // 2. Add the entries in the ledger pointed by fromPosition + count += 20 - (fromPosition.getEntryId() + 1); + count += fromIncluded ? 1 : 0; + + // 3. Add the whole ledgers entries in between + for (long i = fromPosition.getLedgerId() + 1; i < toPosition.getLedgerId(); i++) { + count += 20; + } + + return count; + } + })).when(ledgerMock).getNumberOfEntries(any()); + assertEquals(ledgerMock.getNextValidPosition(PositionFactory.create(1, 0)), PositionFactory.create(1, 1)); + assertEquals(ledgerMock.getNextValidPosition(PositionFactory.create(1, 19)), PositionFactory.create(2, 0)); + assertEquals(ledgerMock.getPreviousPosition(PositionFactory.create(2, 0)), PositionFactory.create(1, 19)); + assertThrows(NullPointerException.class, () -> ledgerMock.getPreviousPosition(PositionFactory.create(0, 0))); + assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( + PositionFactory.create(1, 0), PositionFactory.create(1, 0))), 0); + assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( + PositionFactory.create(1, -1), PositionFactory.create(1, 9))), 10); + assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( + PositionFactory.create(1, 19), PositionFactory.create(2, -1))), 0); + assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( + PositionFactory.create(1, 19), PositionFactory.create(2, 9))), 10); + assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( + PositionFactory.create(1, -1), PositionFactory.create(3, 19))), 60); + + // Add a consumer + final Consumer consumer1 = mock(Consumer.class); + doReturn("consumer1").when(consumer1).consumerName(); + when(consumer1.getAvailablePermits()).thenReturn(1000); + doReturn(true).when(consumer1).isWritable(); + doReturn(channelMock).when(consumer1).sendMessages(anyList(), any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); + persistentDispatcher.addConsumer(consumer1); + + /* + On single ledger + */ + + // Expected individuallySentPositions (isp): [(1:-1, 1:8]] (init) -> [(1:-1, 1:9]] (update) -> [] (remove) + // Expected lastSentPosition (lsp): 1:10 (init) -> 1:10 (remove) + // upper bound and the new entry are less than initial last sent position + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, -1, 1, 8); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(1, 9, createMessage("test", 1))), true); + assertTrue(individuallySentPositions.isEmpty()); + assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); + + // isp: [(1:-1, 1:9]] -> [(1:-1, 1:10]] -> [] + // lsp: 1:10 -> 1:10 + // upper bound is less than initial last sent position + // upper bound and the new entry are less than or equal to initial last sent position + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, -1, 1, 9); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(1, 10, createMessage("test", 1))), true); + assertTrue(individuallySentPositions.isEmpty()); + assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); + + // isp: [(1:-1, 1:2], (1:3, 1:4], (1:5, 1:6]] -> [(1:-1, 1:2], (1:3, 1:4], (1:5, 1:6], (1:9, 1:10]] -> [] + // lsp: 1:10 -> 1:10 + // upper bound and the new entry are less than or equal to initial last sent position + // individually sent positions has multiple ranges + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, -1, 1, 2); + individuallySentPositions.addOpenClosed(1, 3, 1, 4); + individuallySentPositions.addOpenClosed(1, 5, 1, 6); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(1, 10, createMessage("test", 1))), true); + assertTrue(individuallySentPositions.isEmpty()); + assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); + + // isp: [(1:-1, 1:10]] -> [(1:-1, 1:11]] -> [] + // lsp: 1:10 -> 1:11 + // upper bound is less than or equal to initial last sent position + // the new entry is next position of initial last sent position + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, -1, 1, 10); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); + assertTrue(individuallySentPositions.isEmpty()); + assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 11).toString()); + + // isp: [(1:-1, 1:9]] -> [(1:-1, 1:9], (1:10, 1:11]] -> [] + // lsp: 1:10 -> 1:11 + // upper bound is less than initial last sent position + // the new entry is next position of initial last sent position + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, -1, 1, 9); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); + assertTrue(individuallySentPositions.isEmpty()); + assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 11).toString()); + + // isp: [(1:11, 1:15]] -> [(1:10, 1:15]] -> [] + // lsp: 1:10 -> 1:15 + // upper bound is greater than initial last sent position + // the range doesn't contain next position of initial last sent position + // the new entry is next position of initial last sent position + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, 11, 1, 15); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); + assertTrue(individuallySentPositions.isEmpty()); + assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 15).toString()); + + // isp: [(1:11, 1:15]] -> [(1:10, 1:16]] -> [] + // lsp: 1:10 -> 1:16 + // upper bound is greater than initial last sent position + // the range doesn't contain next position of initial last sent position + // the new entries contain next position of initial last sent position + // first of the new entries is less than initial last sent position + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, 11, 1, 15); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(1, 9, createMessage("test", 1)), + EntryImpl.create(1, 11, createMessage("test", 2)), + EntryImpl.create(1, 16, createMessage("test", 3))), true); + assertTrue(individuallySentPositions.isEmpty()); + assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 16).toString()); + + // isp: [(1:11, 1:15]] -> [(1:11, 1:15]] -> [(1:11, 1:15]] + // lsp: 1:10 -> 1:10 + // upper bound is greater than initial last sent position + // the range doesn't contain next position of initial last sent position + // the new entry isn't next position of initial last sent position + // the range contains the new entry + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, 11, 1, 15); + expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 15); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(1, 15, createMessage("test", 1))), true); + assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); + assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); + + // isp: [(1:11, 1:15]] -> [(1:11, 1:16]] -> [(1:11, 1:16]] + // lsp: 1:10 -> 1:10 + // upper bound is greater than initial last sent position + // the range doesn't contain next position of initial last sent position + // the new entry isn't next position of initial last sent position + // the range doesn't contain the new entry + // the new entry is next position of upper bound + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, 11, 1, 15); + expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 16); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(1, 16, createMessage("test", 1))), true); + assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); + assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); + + // isp: [(1:11, 1:15]] -> [(1:11, 1:15], (1:16, 1:17]] -> [(1:11, 1:15], (1:16, 1:17]] + // lsp: 1:10 -> 1:10 + // upper bound is greater than initial last sent position + // the range doesn't contain next position of initial last sent position + // the new entry isn't next position of initial last sent position + // the range doesn't contain the new entry + // the new entry isn't next position of upper bound + // the new entry is same ledger + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, 11, 1, 15); + expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 15); + expectedIndividuallySentPositions.addOpenClosed(1, 16, 1, 17); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(1, 17, createMessage("test", 1))), true); + assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); + assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); + + /* + On multiple contiguous ledgers + */ + + // isp: [(1:11, 1:18]] -> [(1:11, 1:18], (2:-1, 2:0]] -> [(1:11, 1:18], (2:-1, 2:0]] + // lsp: 1:10 -> 1:10 + // upper bound is greater than initial last sent position + // the range doesn't contain next position of initial last sent position + // the new entry isn't next position of initial last sent position + // the range doesn't contain the new entry + // the new entry isn't next position of upper bound + // the new entry isn't same ledger + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, 11, 1, 18); + expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 18); + expectedIndividuallySentPositions.addOpenClosed(2, -1, 2, 0); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(2, 0, createMessage("test", 1))), true); + assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); + assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); + + // isp: [(1:11, 1:19], (2:-1, 2:0]] -> [(1:10, 1:19], (2:-1, 2:0]] -> [] + // lsp: 1:10 -> 2:0 + // upper bound is greater than initial last sent position + // the range doesn't contain next position of initial last sent position + // the new entry is next position of initial last sent position + // the new entry isn't same ledger + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, 11, 1, 19); + individuallySentPositions.addOpenClosed(2, -1, 2, 0); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); + assertTrue(individuallySentPositions.isEmpty()); + assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(2, 0).toString()); + + // isp: [(1:11, 1:19], (2:-1, 2:19], (3:-1, 3:0]] -> [(1:10, 1:19], (2:-1, 2:19], (3:-1, 3:0]] -> [] + // lsp: 1:10 -> 3:0 + // upper bound is greater than initial last sent position + // the range doesn't contain next position of initial last sent position + // the new entry is next position of initial last sent position + // the new entry isn't same ledger + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, 11, 1, 19); + individuallySentPositions.addOpenClosed(2, -1, 2, 19); + individuallySentPositions.addOpenClosed(3, -1, 3, 0); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); + assertTrue(individuallySentPositions.isEmpty()); + assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(3, 0).toString()); + + // isp: [(1:11, 1:19], (2:-1, 2:0]] -> [(1:11, 1:19], (2:-1, 2:1]] -> [(1:11, 1:19], (2:-1, 2:1]] + // lsp: 1:10 -> 1:10 + // upper bound is greater than initial last sent position + // the range doesn't contain next position of initial last sent position + // the new entry isn't next position of initial last sent position + // the new entry isn't same ledger + assertNull(clearPosition.get()); + individuallySentPositions.addOpenClosed(1, 11, 1, 19); + individuallySentPositions.addOpenClosed(2, -1, 2, 0); + expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 19); + expectedIndividuallySentPositions.addOpenClosed(2, -1, 2, 1); + persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, + Arrays.asList(EntryImpl.create(2, 1, createMessage("test", 1))), true); + assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); + assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); + } + private ByteBuf createMessage(String message, int sequenceId) { return createMessage(message, sequenceId, "testKey"); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index 5b2998216e8e1..14403765105b9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -233,7 +233,7 @@ public void testConsumerStatsOutput() throws Exception { "unackedMessages", "avgMessagesPerEntry", "blockedConsumerOnUnackedMsgs", - "readPositionWhenJoining", + "lastSentPositionWhenJoining", "lastAckedTime", "lastAckedTimestamp", "lastConsumedTime", diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index 92c51da64d39d..e8fd537831673 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -18,6 +18,9 @@ */ package org.apache.pulsar.client.api; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -26,6 +29,7 @@ import static org.testng.Assert.fail; import com.google.common.collect.Lists; import java.io.IOException; +import java.lang.reflect.Field; import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; @@ -33,6 +37,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -49,6 +54,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; import java.util.stream.Collectors; import lombok.Cleanup; import org.apache.bookkeeper.mledger.Position; @@ -56,17 +62,24 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentStickyKeyDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.MessageRedeliveryController; +import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.impl.ConsumerImpl; +import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.api.proto.KeySharedMode; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.util.Murmur3_32Hash; +import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; +import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.awaitility.Awaitility; import org.mockito.Mockito; import org.slf4j.Logger; @@ -1096,13 +1109,21 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr final String topicName = "persistent://public/default/change-allow-ooo-delivery-" + UUID.randomUUID(); final String subName = "my-sub"; - Consumer consumer = pulsarClient.newConsumer() + final Consumer consumer1 = pulsarClient.newConsumer() .topic(topicName) .subscriptionName(subName) .subscriptionType(SubscriptionType.Key_Shared) .keySharedPolicy(KeySharedPolicy.autoSplitHashRange().setAllowOutOfOrderDelivery(true)) .subscribe(); + @Cleanup + final Producer producer = pulsarClient.newProducer() + .topic(topicName) + .enableBatching(false) + .create(); + producer.send("message".getBytes()); + Awaitility.await().untilAsserted(() -> assertNotNull(consumer1.receive(100, TimeUnit.MILLISECONDS))); + CompletableFuture> future = pulsar.getBrokerService().getTopicIfExists(topicName); assertTrue(future.isDone()); assertTrue(future.get().isPresent()); @@ -1110,14 +1131,18 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr PersistentStickyKeyDispatcherMultipleConsumers dispatcher = (PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subName).getDispatcher(); assertTrue(dispatcher.isAllowOutOfOrderDelivery()); - consumer.close(); + assertNull(dispatcher.getLastSentPositionField()); + assertNull(dispatcher.getIndividuallySentPositionsField()); + consumer1.close(); - consumer = pulsarClient.newConsumer() + final Consumer consumer2 = pulsarClient.newConsumer() .topic(topicName) .subscriptionName(subName) .subscriptionType(SubscriptionType.Key_Shared) .keySharedPolicy(KeySharedPolicy.autoSplitHashRange().setAllowOutOfOrderDelivery(false)) .subscribe(); + producer.send("message".getBytes()); + Awaitility.await().untilAsserted(() -> assertNotNull(consumer2.receive(100, TimeUnit.MILLISECONDS))); future = pulsar.getBrokerService().getTopicIfExists(topicName); assertTrue(future.isDone()); @@ -1125,7 +1150,9 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr topic = future.get().get(); dispatcher = (PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subName).getDispatcher(); assertFalse(dispatcher.isAllowOutOfOrderDelivery()); - consumer.close(); + assertNotNull(dispatcher.getLastSentPositionField()); + assertNotNull(dispatcher.getIndividuallySentPositionsField()); + consumer2.close(); } @Test(timeOut = 30_000) @@ -1199,6 +1226,370 @@ public void testCheckConsumersWithSameName() throws Exception { l.await(); } + @DataProvider(name = "preSend") + private Object[][] preSendProvider() { + return new Object[][] { { false }, { true } }; + } + + @Test(timeOut = 30_000, dataProvider = "preSend") + public void testCheckBetweenSkippingAndRecentlyJoinedConsumers(boolean preSend) throws Exception { + conf.setSubscriptionKeySharedUseConsistentHashing(true); + + final String topicName = "persistent://public/default/recently-joined-consumers-" + UUID.randomUUID(); + final String subName = "my-sub"; + + @Cleanup + final Producer p = pulsarClient.newProducer(Schema.STRING) + .topic(topicName) + .create(); + if (preSend) { + // verify that the test succeeds even if the topic has a message + p.send("msg"); + } + + final Supplier> cb = () -> pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Latest) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() + .setAllowOutOfOrderDelivery(false)); + + // create 2 consumers + final String c1ConsumerName = "c1"; + @Cleanup + final Consumer c1 = cb.get().consumerName(c1ConsumerName).receiverQueueSize(1).subscribe(); + @Cleanup + final Consumer c2 = cb.get().consumerName("c2").receiverQueueSize(1000).subscribe(); + + final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); + final Field recentlyJoinedConsumersField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("recentlyJoinedConsumers"); + recentlyJoinedConsumersField.setAccessible(true); + final LinkedHashMap recentlyJoinedConsumers = (LinkedHashMap) recentlyJoinedConsumersField.get(dispatcher); + final String keyA = "key-a"; + final int hashA = Murmur3_32Hash.getInstance().makeHash(keyA.getBytes()); + final Map hashConsumerMap = new HashMap<>(); + hashConsumerMap.put(hashA, c1.getConsumerName()); + + // enforce the selector will return c1 if keyA + final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); + selectorField.setAccessible(true); + final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); + selectorField.set(dispatcher, selector); + doAnswer((invocationOnMock -> { + final int hash = invocationOnMock.getArgument(0); + final String consumerName = hashConsumerMap.getOrDefault(hash, c2.getConsumerName()); + return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); + })).when(selector).select(anyInt()); + + // send and receive + Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getConsumers().stream().filter(c -> c.getConsumerName().equals(c1ConsumerName)).findFirst().get().getAvailablePermits(), 1)); + final MessageIdImpl msg0Id = (MessageIdImpl) p.newMessage().key(keyA).value("msg-0").send(); + Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getConsumers().stream().filter(c -> c.getConsumerName().equals(c1ConsumerName)).findFirst().get().getAvailablePermits(), 0)); + + final MessageIdImpl msg1Id = (MessageIdImpl) p.newMessage().key(keyA).value("msg-1").send(); + Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getMsgBacklog(), 2)); + + final Field redeliveryMessagesField = PersistentDispatcherMultipleConsumers.class + .getDeclaredField("redeliveryMessages"); + redeliveryMessagesField.setAccessible(true); + final MessageRedeliveryController redeliveryMessages = (MessageRedeliveryController) redeliveryMessagesField.get(dispatcher); + + final Set replayMsgSet = redeliveryMessages.getMessagesToReplayNow(3); + assertEquals(replayMsgSet.size(), 1); + final Position replayMsg = replayMsgSet.stream().findAny().get(); + assertEquals(replayMsg, PositionFactory.create(msg1Id.getLedgerId(), msg1Id.getEntryId())); + + // add c3 + final String c3ConsumerName = "c3"; + hashConsumerMap.put(hashA, c3ConsumerName); + @Cleanup + final Consumer c3 = cb.get().consumerName(c3ConsumerName).subscribe(); + final List> c3Msgs = new ArrayList<>(); + final org.apache.pulsar.broker.service.Consumer c3Broker = dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(c3ConsumerName)).findFirst().get(); + assertEquals(recentlyJoinedConsumers.get(c3Broker), PositionFactory.create(msg0Id.getLedgerId(), msg0Id.getEntryId())); + + // None of messages are sent to c3. + Message c3Msg = c3.receive(100, TimeUnit.MILLISECONDS); + assertNull(c3Msg); + + // Disconnect c1 + c1.close(); + + c3Msg = c3.receive(100, TimeUnit.MILLISECONDS); + assertNotNull(c3Msg); + c3Msgs.add(c3Msg); + // The mark delete position will move forward. Then remove c3 from recentlyJoinedConsumers. + c3.acknowledge(c3Msg); + Awaitility.await().untilAsserted(() -> assertNull(recentlyJoinedConsumers.get(c3Broker))); + c3Msg = c3.receive(100, TimeUnit.MILLISECONDS); + assertNotNull(c3Msg); + c3Msgs.add(c3Msg); + c3.acknowledge(c3Msg); + + // check ordering + assertTrue(c3Msgs.get(0).getMessageId().compareTo(c3Msgs.get(1).getMessageId()) < 0); + } + + @Test(timeOut = 30_000) + public void testLastSentPositionWhenRecreatingDispatcher() throws Exception { + // The lastSentPosition and individuallySentPositions should be initialized + // by the markDeletedPosition and individuallyDeletedMessages. + final String topicName = "persistent://public/default/rewind-" + UUID.randomUUID(); + final String subName = "my-sub"; + + final int numMessages = 9; + final List keys = Arrays.asList("key-a", "key-b", "key-c"); + final AtomicInteger receiveCounter = new AtomicInteger(); + final AtomicInteger ackCounter = new AtomicInteger(); + + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topicName) + .enableBatching(false) + .create(); + + final Supplier> cb = () -> pulsarClient.newConsumer(Schema.INT32) + .topic(topicName) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() + .setAllowOutOfOrderDelivery(false)); + + @Cleanup + final Consumer c1 = cb.get().messageListener((c, msg) -> { + if (keys.get(0).equals(msg.getKey())) { + try { + c.acknowledge(msg); + ackCounter.getAndIncrement(); + } catch (PulsarClientException e) { + fail(e.getMessage()); + } + } + receiveCounter.getAndIncrement(); + }).subscribe(); + + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); + LongPairRangeSet individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); + final ManagedCursorImpl cursor = (ManagedCursorImpl) ((PersistentSubscription) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName)).getCursor(); + final ManagedLedgerImpl ledger = (ManagedLedgerImpl) cursor.getManagedLedger(); + + MessageIdImpl msgId = null; + for (int i = 0; i < numMessages; i++) { + msgId = (MessageIdImpl) producer.newMessage().key(keys.get(i % keys.size())).value(i).send(); + } + + // wait for consumption + Awaitility.await().untilAsserted(() -> assertEquals(receiveCounter.get(), numMessages)); + assertEquals(ackCounter.get(), numMessages / keys.size()); + assertEquals(dispatcher.getLastSentPositionField(), PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId())); + assertTrue(individuallySentPositionsField.isEmpty()); + receiveCounter.set(0); + ackCounter.set(0); + + // create expected values + final Position expectedLastSentPosition = ledger.getNextValidPosition(cursor.getMarkDeletedPosition()); + final ConcurrentOpenLongPairRangeSet + expectedIndividuallySentPositions = new ConcurrentOpenLongPairRangeSet<>(4096, PositionFactory::create); + cursor.getIndividuallyDeletedMessagesSet().forEach(range -> { + final Position lower = range.lowerEndpoint(); + final Position upper = range.upperEndpoint(); + expectedIndividuallySentPositions.addOpenClosed(lower.getLedgerId(), lower.getEntryId(), upper.getLedgerId(), upper.getEntryId()); + return true; + }); + + // modify subscription type to close current dispatcher + admin.topics().createSubscription(topicName, "sub-alt", MessageId.earliest); + c1.close(); + @Cleanup + final Consumer c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topicName) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Exclusive) + .subscribe(); + c2.close(); + assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getType(), SubscriptionType.Exclusive.toString()); + + @Cleanup + final Consumer c3 = cb.get().receiverQueueSize(0).subscribe(); + dispatcher = (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); + individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); + + assertNull(dispatcher.getLastSentPositionField()); + assertTrue(individuallySentPositionsField.isEmpty()); + + assertNotNull(c3.receive()); + + // validate the individuallySentPosition is initialized by the individuallyDeletedMessages + // if it is not initialized expectedly, it has sent-hole of key-c messages because key-c messages are not scheduled to be dispatched to some consumer(already acked). + assertEquals(dispatcher.getLastSentPositionField(), expectedLastSentPosition); + assertEquals(individuallySentPositionsField.toString(), expectedIndividuallySentPositions.toString()); + } + + @Test(timeOut = 30_000) + public void testLastSentPositionWhenResettingCursor() throws Exception { + // The lastSentPosition and individuallySentPositions should be cleared if reset-cursor operation is executed. + final String nsName = "public/default"; + final String topicName = "persistent://" + nsName + "/reset-cursor-" + UUID.randomUUID(); + final String subName = "my-sub"; + + final int numMessages = 10; + final List keys = Arrays.asList("key-a", "key-b"); + final AtomicInteger ackCounter = new AtomicInteger(); + + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topicName) + .enableBatching(false) + .create(); + + final Supplier> cb = () -> pulsarClient.newConsumer(Schema.INT32) + .topic(topicName) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(0) + .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() + .setAllowOutOfOrderDelivery(false)); + + @Cleanup + final Consumer c1 = cb.get().consumerName("c1").subscribe(); + @Cleanup + final Consumer c2 = cb.get().consumerName("c2").subscribe(); + + // set retention policy + admin.namespaces().setRetention(nsName, new RetentionPolicies(1, 1024 * 1024)); + + // enforce the selector will return c1 if keys.get(0) + final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); + final int hashA = Murmur3_32Hash.getInstance().makeHash(keys.get(0).getBytes()); + final Map hashConsumerMap = new HashMap<>(); + hashConsumerMap.put(hashA, c1.getConsumerName()); + final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); + selectorField.setAccessible(true); + final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); + selectorField.set(dispatcher, selector); + doAnswer((invocationOnMock -> { + final int hash = invocationOnMock.getArgument(0); + final String consumerName = hashConsumerMap.getOrDefault(hash, c2.getConsumerName()); + return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); + })).when(selector).select(anyInt()); + + for (int i = 0; i < numMessages; i++) { + producer.newMessage().key(keys.get(i % keys.size())).value(i).send(); + } + + // consume some messages + for (int i = 0; i < numMessages / keys.size(); i++) { + final Message msg = c2.receive(); + if (msg != null) { + c2.acknowledge(msg); + ackCounter.getAndIncrement(); + } + } + assertEquals(ackCounter.get(), numMessages / keys.size()); + + // store current lastSentPosition for comparison + final LongPairRangeSet individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); + assertNotNull(dispatcher.getLastSentPositionField()); + assertFalse(individuallySentPositionsField.isEmpty()); + + // reset cursor and receive a message + admin.topics().resetCursor(topicName, subName, MessageId.earliest, true); + + // validate the lastSentPosition and individuallySentPositions are cleared after resetting cursor + assertNull(dispatcher.getLastSentPositionField()); + assertTrue(individuallySentPositionsField.isEmpty()); + } + + @Test(timeOut = 30_000) + public void testLastSentPositionWhenSkipping() throws Exception { + // The lastSentPosition and individuallySentPositions should be updated if skip operation is executed. + // There are updated to follow the new markDeletedPosition. + final String topicName = "persistent://public/default/skip-" + UUID.randomUUID(); + final String subName = "my-sub"; + + final int numMessages = 10; + final List keys = Arrays.asList("key-a", "key-b"); + final int numSkip = 2; + final AtomicInteger ackCounter = new AtomicInteger(); + + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topicName) + .enableBatching(false) + .create(); + + final Supplier> cb = () -> pulsarClient.newConsumer(Schema.INT32) + .topic(topicName) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() + .setAllowOutOfOrderDelivery(false)) + .receiverQueueSize(0); + + @Cleanup + final Consumer c1 = cb.get().consumerName("c1").subscribe(); + @Cleanup + final Consumer c2 = cb.get().consumerName("c2").subscribe(); + + // enforce the selector will return c1 if keys.get(0) + final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); + final int hashA = Murmur3_32Hash.getInstance().makeHash(keys.get(0).getBytes()); + final Map hashConsumerMap = new HashMap<>(); + hashConsumerMap.put(hashA, c1.getConsumerName()); + final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); + selectorField.setAccessible(true); + final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); + selectorField.set(dispatcher, selector); + doAnswer((invocationOnMock -> { + final int hash = invocationOnMock.getArgument(0); + final String consumerName = hashConsumerMap.getOrDefault(hash, c2.getConsumerName()); + return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); + })).when(selector).select(anyInt()); + + final List positionList = new ArrayList<>(); + for (int i = 0; i < numMessages; i++) { + final MessageIdImpl msgId = (MessageIdImpl) producer.newMessage().key(keys.get(i % keys.size())).value(i).send(); + positionList.add(PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId())); + } + + // consume some messages + for (int i = 0; i < numSkip; i++) { + final Message msg = c2.receive(); + if (msg != null) { + c2.acknowledge(msg); + ackCounter.getAndIncrement(); + } + } + assertEquals(ackCounter.get(), numSkip); + final ManagedCursorImpl managedCursor = ((ManagedCursorImpl) ((PersistentSubscription) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName)).getCursor()); + Awaitility.await().untilAsserted(() -> assertEquals(managedCursor.getIndividuallyDeletedMessagesSet().size(), 2)); + + // store current lastSentPosition for comparison + final Position lastSentPositionBeforeSkip = dispatcher.getLastSentPositionField(); + final LongPairRangeSet individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); + assertNotNull(lastSentPositionBeforeSkip); + assertFalse(individuallySentPositionsField.isEmpty()); + + // skip messages and receive a message + admin.topics().skipMessages(topicName, subName, numSkip); + final MessageIdImpl msgIdAfterSkip = (MessageIdImpl) c1.receive().getMessageId(); + final Position positionAfterSkip = PositionFactory.create(msgIdAfterSkip.getLedgerId(), + msgIdAfterSkip.getEntryId()); + assertEquals(positionAfterSkip, positionList.get(4)); + + // validate the lastSentPosition is updated to the new markDeletedPosition + // validate the individuallySentPositions is updated expectedly (removeAtMost the new markDeletedPosition) + final Position lastSentPosition = dispatcher.getLastSentPositionField(); + assertNotNull(lastSentPosition); + assertTrue(lastSentPosition.compareTo(lastSentPositionBeforeSkip) > 0); + assertEquals(lastSentPosition, positionList.get(4)); + assertTrue(individuallySentPositionsField.isEmpty()); + } private KeySharedMode getKeySharedModeOfSubscription(Topic topic, String subscription) { if (TopicName.get(topic.getName()).getDomain().equals(TopicDomain.persistent)) { diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java index d2d3600df96ed..5f2cf7b209ee9 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java @@ -72,8 +72,8 @@ public interface ConsumerStats { /** Flag to verify if consumer is blocked due to reaching threshold of unacked messages. */ boolean isBlockedConsumerOnUnackedMsgs(); - /** The read position of the cursor when the consumer joining. */ - String getReadPositionWhenJoining(); + /** The last sent position of the cursor when the consumer joining. */ + String getLastSentPositionWhenJoining(); /** Address of this consumer. */ String getAddress(); diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index d4850adaa6f22..cabef1ca9602d 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -118,6 +118,12 @@ public interface SubscriptionStats { /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ Map getConsumersAfterMarkDeletePosition(); + /** The last sent position of the cursor. This is for Key_Shared subscription. */ + String getLastSentPosition(); + + /** Set of individually sent ranges. This is for Key_Shared subscription. */ + String getIndividuallySentPositions(); + /** SubscriptionProperties (key/value strings) associated with this subscribe. */ Map getSubscriptionProperties(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java index de36b330b7f1a..b4c5d21e6926e 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java @@ -77,8 +77,8 @@ public class ConsumerStatsImpl implements ConsumerStats { /** Flag to verify if consumer is blocked due to reaching threshold of unacked messages. */ public boolean blockedConsumerOnUnackedMsgs; - /** The read position of the cursor when the consumer joining. */ - public String readPositionWhenJoining; + /** The last sent position of the cursor when the consumer joining. */ + public String lastSentPositionWhenJoining; /** Address of this consumer. */ private String address; @@ -113,7 +113,7 @@ public ConsumerStatsImpl add(ConsumerStatsImpl stats) { this.availablePermits += stats.availablePermits; this.unackedMessages += stats.unackedMessages; this.blockedConsumerOnUnackedMsgs = stats.blockedConsumerOnUnackedMsgs; - this.readPositionWhenJoining = stats.readPositionWhenJoining; + this.lastSentPositionWhenJoining = stats.lastSentPositionWhenJoining; return this; } @@ -141,8 +141,8 @@ public void setClientVersion(String clientVersion) { this.clientVersion = clientVersion; } - public String getReadPositionWhenJoining() { - return readPositionWhenJoining; + public String getLastSentPositionWhenJoining() { + return lastSentPositionWhenJoining; } public String getLastAckedTime() { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java index a8ea0060629a0..ab4d07c7ae486 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java @@ -126,6 +126,12 @@ public class SubscriptionStatsImpl implements SubscriptionStats { /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ public Map consumersAfterMarkDeletePosition; + /** The last sent position of the cursor. This is for Key_Shared subscription. */ + public String lastSentPosition; + + /** Set of individually sent ranges. This is for Key_Shared subscription. */ + public String individuallySentPositions; + /** The number of non-contiguous deleted messages ranges. */ public int nonContiguousDeletedMessagesRanges; From d08e2e08b43e86f27f8206c7e189234163bfd795 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Sun, 21 Jul 2024 12:11:06 +0800 Subject: [PATCH 380/580] [improve][io] The JDBC connector supports JSON substructure schema (#23043) --- pulsar-io/jdbc/core/pom.xml | 13 ++-- .../io/jdbc/BaseJdbcAutoSchemaSink.java | 5 +- .../io/jdbc/BaseJdbcAutoSchemaSinkTest.java | 68 +++++++++++++++++++ 3 files changed, 78 insertions(+), 8 deletions(-) diff --git a/pulsar-io/jdbc/core/pom.xml b/pulsar-io/jdbc/core/pom.xml index 8d15388a3faf7..3fe31b3e0ac97 100644 --- a/pulsar-io/jdbc/core/pom.xml +++ b/pulsar-io/jdbc/core/pom.xml @@ -44,6 +44,12 @@ ${project.version} + + ${project.groupId} + pulsar-client-original + ${project.version} + + org.apache.avro avro @@ -71,13 +77,6 @@ provided - - ${project.groupId} - pulsar-client-original - ${project.version} - test - - \ No newline at end of file diff --git a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java index 3655688c0f3ad..c1f44cf37efdf 100644 --- a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java +++ b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java @@ -33,6 +33,7 @@ import org.apache.pulsar.client.api.schema.GenericObject; import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.client.api.schema.KeyValueSchema; +import org.apache.pulsar.client.impl.schema.generic.GenericJsonRecord; import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.functions.api.Record; @@ -173,7 +174,7 @@ private static void setColumnNull(PreparedStatement statement, int index, int ty } - private static void setColumnValue(PreparedStatement statement, int index, Object value) throws Exception { + protected void setColumnValue(PreparedStatement statement, int index, Object value) throws Exception { log.debug("Setting column value, statement: {}, index: {}, value: {}", statement, index, value); @@ -193,6 +194,8 @@ private static void setColumnValue(PreparedStatement statement, int index, Objec statement.setShort(index, (Short) value); } else if (value instanceof ByteString) { statement.setBytes(index, ((ByteString) value).toByteArray()); + } else if (value instanceof GenericJsonRecord) { + statement.setString(index, ((GenericJsonRecord) value).getJsonNode().toString()); } else { throw new Exception("Not supported value type, need to add it. " + value.getClass()); } diff --git a/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java b/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java index c088dd3c42c32..8cb6219deb89e 100644 --- a/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java +++ b/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java @@ -18,13 +18,23 @@ */ package org.apache.pulsar.io.jdbc; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import java.lang.reflect.Field; +import java.sql.PreparedStatement; +import java.util.Arrays; +import java.util.List; import java.util.function.Function; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; import org.apache.avro.util.Utf8; import org.apache.pulsar.client.api.schema.GenericObject; import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; +import org.apache.pulsar.client.impl.schema.generic.GenericJsonSchema; import org.apache.pulsar.functions.api.Record; import org.testng.Assert; import org.testng.annotations.Test; @@ -169,4 +179,62 @@ public GenericRecord getValue() { } + @Test + @SuppressWarnings("unchecked") + public void testSubFieldJsonArray() throws Exception { + BaseJdbcAutoSchemaSink baseJdbcAutoSchemaSink = new BaseJdbcAutoSchemaSink() {}; + + Field field = JdbcAbstractSink.class.getDeclaredField("jdbcSinkConfig"); + field.setAccessible(true); + JdbcSinkConfig jdbcSinkConfig = new JdbcSinkConfig(); + jdbcSinkConfig.setNullValueAction(JdbcSinkConfig.NullValueAction.FAIL); + field.set(baseJdbcAutoSchemaSink, jdbcSinkConfig); + + TStates tStates = new TStates("tstats", Arrays.asList( + new PC("brand1", "model1"), + new PC("brand2", "model2") + )); + org.apache.pulsar.client.api.Schema jsonSchema = org.apache.pulsar.client.api.Schema.JSON(TStates.class); + GenericJsonSchema genericJsonSchema = new GenericJsonSchema(jsonSchema.getSchemaInfo()); + byte[] encode = jsonSchema.encode(tStates); + GenericRecord genericRecord = genericJsonSchema.decode(encode); + + AutoConsumeSchema autoConsumeSchema = new AutoConsumeSchema(); + autoConsumeSchema.setSchema(org.apache.pulsar.client.api.Schema.JSON(TStates.class)); + Record record = new Record() { + @Override + public org.apache.pulsar.client.api.Schema getSchema() { + return genericJsonSchema; + } + + @Override + public GenericRecord getValue() { + return genericRecord; + } + }; + JdbcAbstractSink.Mutation mutation = baseJdbcAutoSchemaSink.createMutation((Record) record); + PreparedStatement mockPreparedStatement = mock(PreparedStatement.class); + baseJdbcAutoSchemaSink.setColumnValue(mockPreparedStatement, 0, mutation.getValues().apply("state")); + baseJdbcAutoSchemaSink.setColumnValue(mockPreparedStatement, 1, mutation.getValues().apply("pcList")); + verify(mockPreparedStatement).setString(0, "tstats"); + verify(mockPreparedStatement).setString(1, "[{\"brand\":\"brand1\",\"model\":\"model1\"},{\"brand\":\"brand2\",\"model\":\"model2\"}]"); + } + + @Data + @AllArgsConstructor + @NoArgsConstructor + private static class TStates { + public String state; + public List pcList; + } + + @Data + @AllArgsConstructor + @NoArgsConstructor + private static class PC { + public String brand; + public String model; + } + + } \ No newline at end of file From c50f4afeae610d3f1994aa2dd53c761589bbb4e2 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sun, 21 Jul 2024 19:44:52 +0800 Subject: [PATCH 381/580] [fix] [broker] fix compile error for PersistentStickyKeyDispatcherMultipleConsumers (#23055) --- .../ConcurrentOpenLongPairRangeSet.java | 420 ++++++++++++++++++ 1 file changed, 420 insertions(+) create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java new file mode 100644 index 0000000000000..6e45401978546 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java @@ -0,0 +1,420 @@ +/* + * 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.collections; + +import static java.util.Objects.requireNonNull; +import com.google.common.collect.BoundType; +import com.google.common.collect.Range; +import java.util.ArrayList; +import java.util.BitSet; +import java.util.List; +import java.util.Map.Entry; +import java.util.NavigableMap; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.commons.lang.mutable.MutableInt; + +/** + * A Concurrent set comprising zero or more ranges of type {@link LongPair}. This can be alternative of + * {@link com.google.common.collect.RangeSet} and can be used if {@code range} type is {@link LongPair} + * + *

+ * Usage:
+ * a. This can be used if one doesn't want to create object for every new inserted {@code range}
+ * b. It creates {@link BitSet} for every unique first-key of the range.
+ * So, this rangeSet is not suitable for large number of unique keys.
+ * 
+ */ +public class ConcurrentOpenLongPairRangeSet> implements LongPairRangeSet { + + protected final NavigableMap rangeBitSetMap = new ConcurrentSkipListMap<>(); + private boolean threadSafe = true; + private final int bitSetSize; + private final LongPairConsumer consumer; + + // caching place-holder for cpu-optimization to avoid calculating ranges again + private volatile int cachedSize = 0; + private volatile String cachedToString = "[]"; + private volatile boolean updatedAfterCachedForSize = true; + private volatile boolean updatedAfterCachedForToString = true; + + public ConcurrentOpenLongPairRangeSet(LongPairConsumer consumer) { + this(1024, true, consumer); + } + + public ConcurrentOpenLongPairRangeSet(int size, LongPairConsumer consumer) { + this(size, true, consumer); + } + + public ConcurrentOpenLongPairRangeSet(int size, boolean threadSafe, LongPairConsumer consumer) { + this.threadSafe = threadSafe; + this.bitSetSize = size; + this.consumer = consumer; + } + + /** + * Adds the specified range to this {@code RangeSet} (optional operation). That is, for equal range sets a and b, + * the result of {@code a.add(range)} is that {@code a} will be the minimal range set for which both + * {@code a.enclosesAll(b)} and {@code a.encloses(range)}. + * + *

Note that {@code range} will merge given {@code range} with any ranges in the range set that are + * {@linkplain Range#isConnected(Range) connected} with it. Moreover, if {@code range} is empty, this is a no-op. + */ + @Override + public void addOpenClosed(long lowerKey, long lowerValueOpen, long upperKey, long upperValue) { + long lowerValue = lowerValueOpen + 1; + if (lowerKey != upperKey) { + // (1) set lower to last in lowerRange.getKey() + if (isValid(lowerKey, lowerValue)) { + BitSet rangeBitSet = rangeBitSetMap.get(lowerKey); + // if lower and upper has different key/ledger then set ranges for lower-key only if + // a. bitSet already exist and given value is not the last value in the bitset. + // it will prevent setting up values which are not actually expected to set + // eg: (2:10..4:10] in this case, don't set any value for 2:10 and set [4:0..4:10] + if (rangeBitSet != null && (rangeBitSet.previousSetBit(rangeBitSet.size()) > lowerValueOpen)) { + int lastValue = rangeBitSet.previousSetBit(rangeBitSet.size()); + rangeBitSet.set((int) lowerValue, (int) Math.max(lastValue, lowerValue) + 1); + } + } + // (2) set 0th-index to upper-index in upperRange.getKey() + if (isValid(upperKey, upperValue)) { + BitSet rangeBitSet = rangeBitSetMap.computeIfAbsent(upperKey, (key) -> createNewBitSet()); + if (rangeBitSet != null) { + rangeBitSet.set(0, (int) upperValue + 1); + } + } + // No-op if values are not valid eg: if lower == LongPair.earliest or upper == LongPair.latest then nothing + // to set + } else { + long key = lowerKey; + BitSet rangeBitSet = rangeBitSetMap.computeIfAbsent(key, (k) -> createNewBitSet()); + rangeBitSet.set((int) lowerValue, (int) upperValue + 1); + } + updatedAfterCachedForSize = true; + updatedAfterCachedForToString = true; + } + + private boolean isValid(long key, long value) { + return key != LongPair.earliest.getKey() && value != LongPair.earliest.getValue() + && key != LongPair.latest.getKey() && value != LongPair.latest.getValue(); + } + + @Override + public boolean contains(long key, long value) { + + BitSet rangeBitSet = rangeBitSetMap.get(key); + if (rangeBitSet != null) { + return rangeBitSet.get(getSafeEntry(value)); + } + return false; + } + + @Override + public Range rangeContaining(long key, long value) { + BitSet rangeBitSet = rangeBitSetMap.get(key); + if (rangeBitSet != null) { + if (!rangeBitSet.get(getSafeEntry(value))) { + // if position is not part of any range then return null + return null; + } + int lowerValue = rangeBitSet.previousClearBit(getSafeEntry(value)) + 1; + final T lower = consumer.apply(key, lowerValue); + final T upper = consumer.apply(key, + Math.max(rangeBitSet.nextClearBit(getSafeEntry(value)) - 1, lowerValue)); + return Range.closed(lower, upper); + } + return null; + } + + @Override + public void removeAtMost(long key, long value) { + this.remove(Range.atMost(new LongPair(key, value))); + } + + @Override + public boolean isEmpty() { + if (rangeBitSetMap.isEmpty()) { + return true; + } + for (BitSet rangeBitSet : rangeBitSetMap.values()) { + if (!rangeBitSet.isEmpty()) { + return false; + } + } + return true; + } + + @Override + public void clear() { + rangeBitSetMap.clear(); + updatedAfterCachedForSize = true; + updatedAfterCachedForToString = true; + } + + @Override + public Range span() { + if (rangeBitSetMap.isEmpty()) { + return null; + } + Entry firstSet = rangeBitSetMap.firstEntry(); + Entry lastSet = rangeBitSetMap.lastEntry(); + int first = firstSet.getValue().nextSetBit(0); + int last = lastSet.getValue().previousSetBit(lastSet.getValue().size()); + return Range.openClosed(consumer.apply(firstSet.getKey(), first - 1), consumer.apply(lastSet.getKey(), last)); + } + + @Override + public List> asRanges() { + List> ranges = new ArrayList<>(); + forEach((range) -> { + ranges.add(range); + return true; + }); + return ranges; + } + + @Override + public void forEach(RangeProcessor action) { + forEach(action, consumer); + } + + @Override + public void forEach(RangeProcessor action, LongPairConsumer consumerParam) { + forEachRawRange((lowerKey, lowerValue, upperKey, upperValue) -> { + Range range = Range.openClosed( + consumerParam.apply(lowerKey, lowerValue), + consumerParam.apply(upperKey, upperValue) + ); + return action.process(range); + }); + } + + @Override + public void forEachRawRange(RawRangeProcessor processor) { + AtomicBoolean completed = new AtomicBoolean(false); + rangeBitSetMap.forEach((key, set) -> { + if (completed.get()) { + return; + } + if (set.isEmpty()) { + return; + } + int first = set.nextSetBit(0); + int last = set.previousSetBit(set.size()); + int currentClosedMark = first; + while (currentClosedMark != -1 && currentClosedMark <= last) { + int nextOpenMark = set.nextClearBit(currentClosedMark); + if (!processor.processRawRange(key, currentClosedMark - 1, + key, nextOpenMark - 1)) { + completed.set(true); + break; + } + currentClosedMark = set.nextSetBit(nextOpenMark); + } + }); + } + + + @Override + public Range firstRange() { + if (rangeBitSetMap.isEmpty()) { + return null; + } + Entry firstSet = rangeBitSetMap.firstEntry(); + int lower = firstSet.getValue().nextSetBit(0); + int upper = Math.max(lower, firstSet.getValue().nextClearBit(lower) - 1); + return Range.openClosed(consumer.apply(firstSet.getKey(), lower - 1), consumer.apply(firstSet.getKey(), upper)); + } + + @Override + public Range lastRange() { + if (rangeBitSetMap.isEmpty()) { + return null; + } + Entry lastSet = rangeBitSetMap.lastEntry(); + int upper = lastSet.getValue().previousSetBit(lastSet.getValue().size()); + int lower = Math.min(lastSet.getValue().previousClearBit(upper), upper); + return Range.openClosed(consumer.apply(lastSet.getKey(), lower), consumer.apply(lastSet.getKey(), upper)); + } + + @Override + public int cardinality(long lowerKey, long lowerValue, long upperKey, long upperValue) { + NavigableMap subMap = rangeBitSetMap.subMap(lowerKey, true, upperKey, true); + MutableInt v = new MutableInt(0); + subMap.forEach((key, bitset) -> { + if (key == lowerKey || key == upperKey) { + BitSet temp = (BitSet) bitset.clone(); + // Trim the bitset index which < lowerValue + if (key == lowerKey) { + temp.clear(0, (int) Math.max(0, lowerValue)); + } + // Trim the bitset index which > upperValue + if (key == upperKey) { + temp.clear((int) Math.min(upperValue + 1, temp.length()), temp.length()); + } + v.add(temp.cardinality()); + } else { + v.add(bitset.cardinality()); + } + }); + return v.intValue(); + } + + @Override + public int size() { + if (updatedAfterCachedForSize) { + MutableInt size = new MutableInt(0); + + // ignore result because we just want to count + forEachRawRange((lowerKey, lowerValue, upperKey, upperValue) -> { + size.increment(); + return true; + }); + + cachedSize = size.intValue(); + updatedAfterCachedForSize = false; + } + return cachedSize; + } + + @Override + public String toString() { + if (updatedAfterCachedForToString) { + StringBuilder toString = new StringBuilder(); + AtomicBoolean first = new AtomicBoolean(true); + if (toString != null) { + toString.append("["); + } + forEach((range) -> { + if (!first.get()) { + toString.append(","); + } + toString.append(range); + first.set(false); + return true; + }); + toString.append("]"); + cachedToString = toString.toString(); + updatedAfterCachedForToString = false; + } + return cachedToString; + } + + /** + * Adds the specified range to this {@code RangeSet} (optional operation). That is, for equal range sets a and b, + * the result of {@code a.add(range)} is that {@code a} will be the minimal range set for which both + * {@code a.enclosesAll(b)} and {@code a.encloses(range)}. + * + *

Note that {@code range} will merge given {@code range} with any ranges in the range set that are + * {@linkplain Range#isConnected(Range) connected} with it. Moreover, if {@code range} is empty/invalid, this is a + * no-op. + */ + public void add(Range range) { + LongPair lowerEndpoint = range.hasLowerBound() ? range.lowerEndpoint() : LongPair.earliest; + LongPair upperEndpoint = range.hasUpperBound() ? range.upperEndpoint() : LongPair.latest; + + long lowerValueOpen = (range.hasLowerBound() && range.lowerBoundType().equals(BoundType.CLOSED)) + ? getSafeEntry(lowerEndpoint) - 1 + : getSafeEntry(lowerEndpoint); + long upperValueClosed = (range.hasUpperBound() && range.upperBoundType().equals(BoundType.CLOSED)) + ? getSafeEntry(upperEndpoint) + : getSafeEntry(upperEndpoint) + 1; + + // #addOpenClosed doesn't create bitSet for lower-key because it avoids setting up values for non-exist items + // into the key-ledger. so, create bitSet and initialize so, it can't be ignored at #addOpenClosed + rangeBitSetMap.computeIfAbsent(lowerEndpoint.getKey(), (key) -> createNewBitSet()) + .set((int) lowerValueOpen + 1); + this.addOpenClosed(lowerEndpoint.getKey(), lowerValueOpen, upperEndpoint.getKey(), upperValueClosed); + } + + public boolean contains(LongPair position) { + requireNonNull(position, "argument can't be null"); + return contains(position.getKey(), position.getValue()); + } + + public void remove(Range range) { + LongPair lowerEndpoint = range.hasLowerBound() ? range.lowerEndpoint() : LongPair.earliest; + LongPair upperEndpoint = range.hasUpperBound() ? range.upperEndpoint() : LongPair.latest; + + long lower = (range.hasLowerBound() && range.lowerBoundType().equals(BoundType.CLOSED)) + ? getSafeEntry(lowerEndpoint) + : getSafeEntry(lowerEndpoint) + 1; + long upper = (range.hasUpperBound() && range.upperBoundType().equals(BoundType.CLOSED)) + ? getSafeEntry(upperEndpoint) + : getSafeEntry(upperEndpoint) - 1; + + // if lower-bound is not set then remove all the keys less than given upper-bound range + if (lowerEndpoint.equals(LongPair.earliest)) { + // remove all keys with + rangeBitSetMap.forEach((key, set) -> { + if (key < upperEndpoint.getKey()) { + rangeBitSetMap.remove(key); + } + }); + } + + // if upper-bound is not set then remove all the keys greater than given lower-bound range + if (upperEndpoint.equals(LongPair.latest)) { + // remove all keys with + rangeBitSetMap.forEach((key, set) -> { + if (key > lowerEndpoint.getKey()) { + rangeBitSetMap.remove(key); + } + }); + } + + // remove all the keys between two endpoint keys + rangeBitSetMap.forEach((key, set) -> { + if (lowerEndpoint.getKey() == upperEndpoint.getKey() && key == upperEndpoint.getKey()) { + set.clear((int) lower, (int) upper + 1); + } else { + // eg: remove-range: [(3,5) - (5,5)] -> Delete all items from 3,6->3,N,4.*,5,0->5,5 + if (key == lowerEndpoint.getKey()) { + // remove all entries from given position to last position + set.clear((int) lower, set.previousSetBit(set.size())); + } else if (key == upperEndpoint.getKey()) { + // remove all entries from 0 to given position + set.clear(0, (int) upper + 1); + } else if (key > lowerEndpoint.getKey() && key < upperEndpoint.getKey()) { + rangeBitSetMap.remove(key); + } + } + // remove bit-set if set is empty + if (set.isEmpty()) { + rangeBitSetMap.remove(key); + } + }); + + updatedAfterCachedForSize = true; + updatedAfterCachedForToString = true; + } + + private int getSafeEntry(LongPair position) { + return (int) Math.max(position.getValue(), -1); + } + + private int getSafeEntry(long value) { + return (int) Math.max(value, -1); + } + + private BitSet createNewBitSet() { + return this.threadSafe ? new ConcurrentBitSet(bitSetSize) : new BitSet(bitSetSize); + } + +} \ No newline at end of file From 3e4f338e91877fb2e4592aa9abc3aced6d4e50c7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 22 Jul 2024 08:45:23 +0800 Subject: [PATCH 382/580] [improve] [broker] high CPU usage caused by list topics under namespace (#23049) --- .../broker/namespace/NamespaceService.java | 23 +++++++++++++++++++ .../pulsar/broker/service/ServerCnx.java | 4 ++-- .../pulsar/broker/service/ServerCnxTest.java | 2 ++ 3 files changed, 27 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 2a1584df961f7..ec4c907234ab6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -40,6 +40,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -55,6 +56,7 @@ import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.ListUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -104,6 +106,7 @@ import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies; import org.apache.pulsar.common.stats.MetricsUtil; +import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataCache; @@ -187,6 +190,9 @@ public class NamespaceService implements AutoCloseable { .register(); private final DoubleHistogram lookupLatencyHistogram; + private ConcurrentHashMap>> inProgressQueryUserTopics = + new ConcurrentHashMap<>(); + /** * Default constructor. */ @@ -1509,6 +1515,23 @@ public CompletableFuture> getListOfTopics(NamespaceName namespaceNa } } + public CompletableFuture> getListOfUserTopics(NamespaceName namespaceName, Mode mode) { + String key = String.format("%s://%s", mode, namespaceName); + final MutableBoolean initializedByCurrentThread = new MutableBoolean(); + CompletableFuture> queryRes = inProgressQueryUserTopics.computeIfAbsent(key, k -> { + initializedByCurrentThread.setTrue(); + return getListOfTopics(namespaceName, mode).thenApplyAsync(list -> { + return TopicList.filterSystemTopic(list); + }, pulsar.getExecutor()); + }); + if (initializedByCurrentThread.getValue()) { + queryRes.whenComplete((ignore, ex) -> { + inProgressQueryUserTopics.remove(key, queryRes); + }); + } + return queryRes; + } + public CompletableFuture> getAllPartitions(NamespaceName namespaceName) { return getPartitions(namespaceName, TopicDomain.persistent) .thenCombine(getPartitions(namespaceName, TopicDomain.non_persistent), 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 4933aee974d08..9bca80c41bb49 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 @@ -2459,11 +2459,11 @@ protected void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGet if (lookupSemaphore.tryAcquire()) { isNamespaceOperationAllowed(namespaceName, NamespaceOperation.GET_TOPICS).thenApply(isAuthorized -> { if (isAuthorized) { - getBrokerService().pulsar().getNamespaceService().getListOfTopics(namespaceName, mode) + getBrokerService().pulsar().getNamespaceService().getListOfUserTopics(namespaceName, mode) .thenAccept(topics -> { boolean filterTopics = false; // filter system topic - List filteredTopics = TopicList.filterSystemTopic(topics); + List filteredTopics = topics; if (enableSubscriptionPatternEvaluation && topicsPattern.isPresent()) { if (topicsPattern.get().length() <= maxSubscriptionPatternLength) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index 27afedd6b101e..58c6b96a0f346 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -229,6 +229,8 @@ public void setup() throws Exception { doReturn(CompletableFuture.completedFuture(true)).when(namespaceService).checkTopicOwnership(any()); doReturn(CompletableFuture.completedFuture(topics)).when(namespaceService).getListOfTopics( NamespaceName.get("use", "ns-abc"), CommandGetTopicsOfNamespace.Mode.ALL); + doReturn(CompletableFuture.completedFuture(topics)).when(namespaceService).getListOfUserTopics( + NamespaceName.get("use", "ns-abc"), CommandGetTopicsOfNamespace.Mode.ALL); doReturn(CompletableFuture.completedFuture(topics)).when(namespaceService).getListOfPersistentTopics( NamespaceName.get("use", "ns-abc")); From 3ed37afa0ae03e0df04378d82c3b62863dc510ab Mon Sep 17 00:00:00 2001 From: congbo <39078850+congbobo184@users.noreply.github.com> Date: Mon, 22 Jul 2024 15:20:26 +0800 Subject: [PATCH 383/580] [improve][broker] don't do load shedding when metadata service not available (#23040) ### Motivation don't do load shedding when metadata service not available. if unload bundle when metadata service not available, these topics which in this bundle can't recover the current ledger and send read again ### Modifications 1. check metadata service state when do load shedding task 2. do not interrupt the task are doing in the same time --- .../apache/pulsar/broker/PulsarService.java | 3 ++- .../broker/loadbalance/LoadSheddingTask.java | 12 +++++++++++- .../loadbalance/SimpleLoadManagerImplTest.java | 18 +++++++++++++++++- 3 files changed, 30 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 848484fe3763d..c623f5d4e5b0d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -1320,7 +1320,8 @@ private synchronized void startLoadBalancerTasks() { if (isRunning()) { long resourceQuotaUpdateInterval = TimeUnit.MINUTES .toMillis(getConfiguration().getLoadBalancerResourceQuotaUpdateIntervalMinutes()); - loadSheddingTask = new LoadSheddingTask(loadManager, loadManagerExecutor, config); + loadSheddingTask = new LoadSheddingTask(loadManager, loadManagerExecutor, + config, getManagedLedgerFactory()); loadSheddingTask.start(); loadResourceQuotaTask = loadManagerExecutor.scheduleAtFixedRate( new LoadResourceQuotaUpdaterTask(loadManager), resourceQuotaUpdateInterval, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadSheddingTask.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadSheddingTask.java index eb7eacec60815..25a0a2752d11b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadSheddingTask.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadSheddingTask.java @@ -22,6 +22,8 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.pulsar.broker.ServiceConfiguration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,12 +42,16 @@ public class LoadSheddingTask implements Runnable { private volatile ScheduledFuture future; + private final ManagedLedgerFactory factory; + public LoadSheddingTask(AtomicReference loadManager, ScheduledExecutorService loadManagerExecutor, - ServiceConfiguration config) { + ServiceConfiguration config, + ManagedLedgerFactory factory) { this.loadManager = loadManager; this.loadManagerExecutor = loadManagerExecutor; this.config = config; + this.factory = factory; } @Override @@ -53,6 +59,10 @@ public void run() { if (isCancel) { return; } + if (factory instanceof ManagedLedgerFactoryImpl + && !((ManagedLedgerFactoryImpl) factory).isMetadataServiceAvailable()) { + return; + } try { loadManager.get().doLoadShedding(); } catch (Exception e) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java index 8f7aa17d0d7bf..acf096751d769 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java @@ -20,6 +20,7 @@ import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgsRecordingInvocations; import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -43,6 +44,7 @@ import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.commons.lang3.SystemUtils; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; @@ -489,7 +491,21 @@ public void testTask() throws Exception { task1.run(); verify(loadManager, times(1)).writeResourceQuotasToZooKeeper(); - LoadSheddingTask task2 = new LoadSheddingTask(atomicLoadManager, null, null); + LoadSheddingTask task2 = new LoadSheddingTask(atomicLoadManager, null, null, null); + task2.run(); + verify(loadManager, times(1)).doLoadShedding(); + } + + @Test + public void testMetadataServiceNotAvailable() { + LoadManager loadManager = mock(LoadManager.class); + AtomicReference atomicLoadManager = new AtomicReference<>(loadManager); + ManagedLedgerFactoryImpl factory = mock(ManagedLedgerFactoryImpl.class); + doReturn(false).when(factory).isMetadataServiceAvailable(); + LoadSheddingTask task2 = new LoadSheddingTask(atomicLoadManager, null, null, factory); + task2.run(); + verify(loadManager, times(0)).doLoadShedding(); + doReturn(true).when(factory).isMetadataServiceAvailable(); task2.run(); verify(loadManager, times(1)).doLoadShedding(); } From 6fa3bcfe082e6662733928aa49bee1fcce217c80 Mon Sep 17 00:00:00 2001 From: congbo <39078850+congbobo184@users.noreply.github.com> Date: Mon, 22 Jul 2024 16:43:38 +0800 Subject: [PATCH 384/580] [improve][broker] GetPartitionMetadata fail also can produce messages (#23050) ### Motivation GetPartitionMetadata fail also can produce messages - 'autoUpdatePartitionsInterval' will get partition metadata and will regularly detect partition changes - if GetPartitionMetadata will return ServiceNotReady, client receive ServiceNotReady will close cnx - if close the current cnx, all producers and consumers witch use this cnx will close and reconnect (https://github.com/apache/pulsar/blob/5c6602cbb3660a696bf960f2847aac1a2ae037d2/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java#L323-L345) - this will influence a lot of producers and consumers and if current time the zk not available and bundle cache not exist the topic's bundle metadata, the client can't send messages to broker because the producer lookup will fail ### Modifications GetPartitionMetadata return MetadataError when throw MetadataStoreException --- .../pulsar/broker/service/ServerCnx.java | 5 +- .../broker/zookeeper/ZKReconnectTest.java | 87 +++++++++++++++++++ 2 files changed, 91 insertions(+), 1 deletion(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZKReconnectTest.java 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 9bca80c41bb49..260552c55c0d7 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 @@ -176,6 +176,7 @@ import org.apache.pulsar.common.util.netty.NettyChannelUtil; import org.apache.pulsar.common.util.netty.NettyFutureUtil; import org.apache.pulsar.functions.utils.Exceptions; +import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.transaction.coordinator.exceptions.CoordinatorException; import org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStore; @@ -663,7 +664,9 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa log.warn("Failed to get Partitioned Metadata [{}] {}: {}", remoteAddress, topicName, ex.getMessage(), ex); ServerError error = ServerError.ServiceNotReady; - if (ex instanceof RestException restException){ + if (ex instanceof MetadataStoreException) { + error = ServerError.MetadataError; + } else if (ex instanceof RestException restException){ int responseCode = restException.getResponse().getStatus(); if (responseCode == NOT_FOUND.getStatusCode()){ error = ServerError.TopicNotFound; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZKReconnectTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZKReconnectTest.java new file mode 100644 index 0000000000000..7b9e4beec6bdf --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZKReconnectTest.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.broker.zookeeper; + +import com.google.common.collect.Sets; +import org.apache.pulsar.broker.MetadataSessionExpiredPolicy; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.zookeeper.KeeperException; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.concurrent.TimeUnit; + + +@Test +public class ZKReconnectTest extends MockedPulsarServiceBaseTest { + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + this.conf.setZookeeperSessionExpiredPolicy(MetadataSessionExpiredPolicy.reconnect); + this.internalSetup(); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + admin.tenants().createTenant("public", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); + admin.namespaces().createNamespace("public/default"); + admin.namespaces().setNamespaceReplicationClusters("public/default", Sets.newHashSet("test")); + } + + @Test + public void testGetPartitionMetadataFailAlsoCanProduceMessage() throws Exception { + + pulsarClient = PulsarClient.builder(). + serviceUrl(pulsar.getBrokerServiceUrl()) + .build(); + + String topic = "testGetPartitionMetadataFailAlsoCanProduceMessage"; + admin.topics().createPartitionedTopic(topic, 5); + Producer producer = pulsarClient.newProducer() + .autoUpdatePartitionsInterval(1, TimeUnit.SECONDS).topic(topic).create(); + + this.mockZooKeeper.setAlwaysFail(KeeperException.Code.SESSIONEXPIRED); + + // clear cache + pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .getCache().delete("/admin/partitioned-topics/public/default/persistent" + + "/testGetPartitionMetadataFailAlsoCanProduceMessage"); + pulsar.getNamespaceService().getOwnershipCache().invalidateLocalOwnerCache(); + + // autoUpdatePartitions 1 second + TimeUnit.SECONDS.sleep(3); + + // also can send message + producer.send("test".getBytes()); + this.mockZooKeeper.unsetAlwaysFail(); + producer.send("test".getBytes()); + producer.close(); + } + + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + this.internalCleanup(); + } +} From 81aed6c75eba99fb62172b986b0c59e693e6f4b9 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 22 Jul 2024 17:40:30 +0800 Subject: [PATCH 385/580] [improve] [broker] Improve CPU resources usege of TopicName Cache (#23052) Co-authored-by: Zixuan Liu --- conf/broker.conf | 8 +++++ .../pulsar/broker/ServiceConfiguration.java | 15 +++++++++ .../pulsar/broker/service/BrokerService.java | 10 ++++++ .../pulsar/broker/PulsarServiceTest.java | 33 +++++++++++++++++++ .../pulsar/broker/service/StandaloneTest.java | 2 ++ .../naming/ServiceConfigurationTest.java | 13 ++++++++ .../configurations/pulsar_broker_test.conf | 2 ++ .../pulsar_broker_test_standalone.conf | 2 ++ .../pulsar/common/naming/TopicName.java | 33 +++++++++---------- 9 files changed, 101 insertions(+), 17 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index b715c4e515bc8..3c956bdd86dab 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -159,6 +159,14 @@ skipBrokerShutdownOnOOM=false # Factory class-name to create topic with custom workflow topicFactoryClassName= +# Max capacity of the topic name cache. -1 means unlimited cache; 0 means broker will clear all cache +# per "maxSecondsToClearTopicNameCache", it does not mean broker will not cache TopicName. +topicNameCacheMaxCapacity=100000 + +# A Specifies the minimum number of seconds that the topic name stays in memory, to avoid clear cache frequently when +# there are too many topics are in use. +maxSecondsToClearTopicNameCache=7200 + # Enable backlog quota check. Enforces action on topic when the quota is reached backlogQuotaCheckEnabled=true 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 aba3ad3a669f5..2d2765287c0e0 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 @@ -594,6 +594,21 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece ) private boolean backlogQuotaCheckEnabled = true; + @FieldContext( + dynamic = true, + category = CATEGORY_POLICIES, + doc = "Max capacity of the topic name cache. -1 means unlimited cache; 0 means broker will clear all cache" + + " per maxSecondsToClearTopicNameCache, it does not mean broker will not cache TopicName." + ) + private int topicNameCacheMaxCapacity = 100_000; + + @FieldContext( + category = CATEGORY_POLICIES, + doc = "A Specifies the minimum number of seconds that the topic name stays in memory, to avoid clear cache" + + " frequently when there are too many topics are in use." + ) + private int maxSecondsToClearTopicNameCache = 3600 * 2; + @FieldContext( category = CATEGORY_POLICIES, doc = "Whether to enable precise time based backlog quota check. " 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 6ecd0a1ba6075..c0f44838ac680 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 @@ -625,6 +625,16 @@ public void start() throws Exception { this.updateBrokerDispatchThrottlingMaxRate(); this.startCheckReplicationPolicies(); this.startDeduplicationSnapshotMonitor(); + this.startClearInvalidateTopicNameCacheTask(); + } + + protected void startClearInvalidateTopicNameCacheTask() { + final int maxSecondsToClearTopicNameCache = pulsar.getConfiguration().getMaxSecondsToClearTopicNameCache(); + inactivityMonitor.scheduleAtFixedRate( + () -> TopicName.clearIfReachedMaxCapacity(pulsar.getConfiguration().getTopicNameCacheMaxCapacity()), + maxSecondsToClearTopicNameCache, + maxSecondsToClearTopicNameCache, + TimeUnit.SECONDS); } protected void startStatsUpdater(int statsUpdateInitialDelayInSecs, int statsUpdateFrequencyInSecs) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java index daa4393db55fd..3bbf423da6ef3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java @@ -24,11 +24,14 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import static org.testng.AssertJUnit.assertSame; +import java.util.ArrayList; +import java.util.List; import java.util.Optional; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; import org.testng.annotations.AfterMethod; @@ -56,6 +59,8 @@ protected void doInitConf() throws Exception { super.doInitConf(); conf.setBrokerServicePortTls(Optional.of(0)); conf.setWebServicePortTls(Optional.of(0)); + conf.setTopicNameCacheMaxCapacity(5000); + conf.setMaxSecondsToClearTopicNameCache(5); if (useStaticPorts) { conf.setBrokerServicePortTls(Optional.of(6651)); conf.setBrokerServicePort(Optional.of(6660)); @@ -187,6 +192,34 @@ public void testDynamicBrokerPort() throws Exception { assertEquals(pulsar.getWebServiceAddressTls(), "https://localhost:" + pulsar.getWebService().getListenPortHTTPS().get()); } + @Test + public void testTopicCacheConfiguration() throws Exception { + cleanup(); + setup(); + assertEquals(conf.getTopicNameCacheMaxCapacity(), 5000); + assertEquals(conf.getMaxSecondsToClearTopicNameCache(), 5); + + List topicNameCached = new ArrayList<>(); + for (int i = 0; i < 20; i++) { + topicNameCached.add(TopicName.get("public/default/tp_" + i)); + } + + // Verify: the cache does not clear since it is not larger than max capacity. + Thread.sleep(10 * 1000); + for (int i = 0; i < 20; i++) { + assertTrue(topicNameCached.get(i) == TopicName.get("public/default/tp_" + i)); + } + + // Update max capacity. + admin.brokers().updateDynamicConfiguration("topicNameCacheMaxCapacity", "10"); + + // Verify: the cache were cleared. + Thread.sleep(10 * 1000); + for (int i = 0; i < 20; i++) { + assertFalse(topicNameCached.get(i) == TopicName.get("public/default/tp_" + i)); + } + } + @Test public void testBacklogAndRetentionCheck() throws PulsarServerException { ServiceConfiguration config = new ServiceConfiguration(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java index b99f8d5338f60..e95b9410f4d12 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java @@ -63,5 +63,7 @@ public void testInitialize() throws Exception { assertEquals(standalone.getConfig().getAdvertisedListeners(), "internal:pulsar://192.168.1.11:6660,internal:pulsar+ssl://192.168.1.11:6651"); assertEquals(standalone.getConfig().isDispatcherPauseOnAckStatePersistentEnabled(), true); + assertEquals(standalone.getConfig().getMaxSecondsToClearTopicNameCache(), 1); + assertEquals(standalone.getConfig().getTopicNameCacheMaxCapacity(), 200); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java index ebeaffc48e4b9..c64c54d2d191c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java @@ -74,6 +74,8 @@ public void testInit() throws Exception { assertEquals(config.getBacklogQuotaDefaultLimitGB(), 0.05); assertEquals(config.getHttpMaxRequestHeaderSize(), 1234); assertEquals(config.isDispatcherPauseOnAckStatePersistentEnabled(), true); + assertEquals(config.getMaxSecondsToClearTopicNameCache(), 1); + assertEquals(config.getTopicNameCacheMaxCapacity(), 200); OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.create(config.getProperties()); assertEquals(offloadPolicies.getManagedLedgerOffloadedReadPriority().getValue(), "bookkeeper-first"); } @@ -375,4 +377,15 @@ public void testAllowAutoTopicCreationType() throws Exception { conf = PulsarConfigurationLoader.create(properties, ServiceConfiguration.class); assertEquals(conf.getAllowAutoTopicCreationType(), TopicType.NON_PARTITIONED); } + + @Test + public void testTopicNameCacheConfiguration() throws Exception { + ServiceConfiguration conf; + final Properties properties = new Properties(); + properties.setProperty("maxSecondsToClearTopicNameCache", "2"); + properties.setProperty("topicNameCacheMaxCapacity", "100"); + conf = PulsarConfigurationLoader.create(properties, ServiceConfiguration.class); + assertEquals(conf.getMaxSecondsToClearTopicNameCache(), 2); + assertEquals(conf.getTopicNameCacheMaxCapacity(), 100); + } } diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf index ddda30d0a4bd9..f344a3e3f63da 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf @@ -104,3 +104,5 @@ transactionPendingAckBatchedWriteEnabled=true transactionPendingAckBatchedWriteMaxRecords=44 transactionPendingAckBatchedWriteMaxSize=55 transactionPendingAckBatchedWriteMaxDelayInMillis=66 +topicNameCacheMaxCapacity=200 +maxSecondsToClearTopicNameCache=1 diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf index 812c8dc9748f9..c520512e77bf9 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf @@ -95,3 +95,5 @@ supportedNamespaceBundleSplitAlgorithms=[range_equally_divide] defaultNamespaceBundleSplitAlgorithm=topic_count_equally_divide maxMessagePublishBufferSizeInMB=-1 dispatcherPauseOnAckStatePersistentEnabled=true +topicNameCacheMaxCapacity=200 +maxSecondsToClearTopicNameCache=1 diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java index d264eab9574ef..dd24c9a971210 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java @@ -19,16 +19,11 @@ package org.apache.pulsar.common.naming; import com.google.common.base.Splitter; -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; -import com.google.common.util.concurrent.UncheckedExecutionException; import com.google.re2j.Pattern; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.List; import java.util.Objects; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.util.Codec; @@ -54,13 +49,17 @@ public class TopicName implements ServiceUnitId { private final int partitionIndex; - private static final LoadingCache cache = CacheBuilder.newBuilder().maximumSize(100000) - .expireAfterAccess(30, TimeUnit.MINUTES).build(new CacheLoader() { - @Override - public TopicName load(String name) throws Exception { - return new TopicName(name); - } - }); + private static final ConcurrentHashMap cache = new ConcurrentHashMap<>(); + + public static void clearIfReachedMaxCapacity(int maxCapacity) { + if (maxCapacity < 0) { + // Unlimited cache. + return; + } + if (cache.size() > maxCapacity) { + cache.clear(); + } + } public static TopicName get(String domain, NamespaceName namespaceName, String topic) { String name = domain + "://" + namespaceName.toString() + '/' + topic; @@ -79,11 +78,11 @@ public static TopicName get(String domain, String tenant, String cluster, String } public static TopicName get(String topic) { - try { - return cache.get(topic); - } catch (ExecutionException | UncheckedExecutionException e) { - throw (RuntimeException) e.getCause(); + TopicName tp = cache.get(topic); + if (tp != null) { + return tp; } + return cache.computeIfAbsent(topic, k -> new TopicName(k)); } public static TopicName getPartitionedTopicName(String topic) { From fca9c5c392cb72fa15f8d9211e39e1c55afd3281 Mon Sep 17 00:00:00 2001 From: congbo <39078850+congbobo184@users.noreply.github.com> Date: Tue, 23 Jul 2024 10:44:53 +0800 Subject: [PATCH 386/580] [improve][client] Add exception handle for client send error (#23038) ### Motivation - producer send messages return error will close the current cnx - if close the current cnx, all producers and consumers witch use this cnx will close and reconnect (https://github.com/apache/pulsar/blob/5c6602cbb3660a696bf960f2847aac1a2ae037d2/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java#L323-L345) - this will influence a lot of producers and consumers - we only close this producer and reconnect enough, don't need to close this cnx ### Modifications receive send_error, close current producer connection, then it will reconnect automatically --- .../pulsar/client/impl/ClientCnxTest.java | 48 +++++++++++++++++++ .../apache/pulsar/client/impl/ClientCnx.java | 6 +-- 2 files changed, 50 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java index df6b1b8a8f92f..57d709e9768c3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ClientCnxTest.java @@ -25,6 +25,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -32,9 +33,12 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.api.proto.ServerError; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.protocol.Commands; import org.awaitility.Awaitility; +import org.awaitility.core.ConditionTimeoutException; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -130,6 +134,50 @@ public void testClientVersion() throws Exception { } @Test + public void testCnxReceiveSendError() throws Exception { + final String topicOne = "persistent://" + NAMESPACE + "/testCnxReceiveSendError-one"; + final String topicTwo = "persistent://" + NAMESPACE + "/testCnxReceiveSendError-two"; + + PulsarClient client = PulsarClient.builder().serviceUrl(lookupUrl.toString()).connectionsPerBroker(1).build(); + Producer producerOne = client.newProducer(Schema.STRING) + .topic(topicOne) + .create(); + Producer producerTwo = client.newProducer(Schema.STRING) + .topic(topicTwo) + .create(); + ClientCnx cnxOne = ((ProducerImpl) producerOne).getClientCnx(); + ClientCnx cnxTwo = ((ProducerImpl) producerTwo).getClientCnx(); + + // simulate a sending error + cnxOne.handleSendError(Commands.newSendErrorCommand(((ProducerImpl) producerOne).producerId, + 10, ServerError.PersistenceError, "persistent error").getSendError()); + + // two producer use the same cnx + Assert.assertEquals(cnxOne, cnxTwo); + + // the cnx will not change + try { + Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> + (((ProducerImpl) producerOne).getClientCnx() != null + && !cnxOne.equals(((ProducerImpl) producerOne).getClientCnx())) + || !cnxTwo.equals(((ProducerImpl) producerTwo).getClientCnx())); + Assert.fail(); + } catch (Throwable e) { + Assert.assertTrue(e instanceof ConditionTimeoutException); + } + + // two producer use the same cnx + Assert.assertEquals(((ProducerImpl) producerTwo).getClientCnx(), + ((ProducerImpl) producerOne).getClientCnx()); + + // producer also can send message + producerOne.send("test"); + producerTwo.send("test"); + producerTwo.close(); + producerOne.close(); + client.close(); + } + public void testSupportsGetPartitionedMetadataWithoutAutoCreation() throws Exception { final String topic = BrokerTestUtil.newUniqueName( "persistent://" + NAMESPACE + "/tp"); admin.topics().createNonPartitionedTopic(topic); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index 6f343a2ee5855..24163c631ffe9 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -783,11 +783,9 @@ protected void handleSendError(CommandSendError sendError) { case NotAllowedError: producers.get(producerId).recoverNotAllowedError(sequenceId, sendError.getMessage()); break; - default: - // By default, for transient error, let the reconnection logic - // to take place and re-establish the produce again - ctx.close(); + // don't close this ctx, otherwise it will close all consumers and producers which use this ctx + producers.get(producerId).connectionClosed(this, Optional.empty(), Optional.empty()); } } From c9c5bb49b05118429426a4589cd8f57c47980318 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Tue, 23 Jul 2024 12:05:25 -0700 Subject: [PATCH 387/580] [feat][misc] PIP-264: Add OpenTelemetry messaging rate limit metrics (#23035) --- .../impl/ManagedLedgerFactoryImpl.java | 2 +- .../impl/cache/InflightReadsLimiter.java | 49 ++++++++- .../cache/RangeEntryCacheManagerImpl.java | 5 +- .../impl/cache/InflightReadsLimiterTest.java | 85 ++++++++++++--- .../impl/cache/PendingReadsManagerTest.java | 36 +++---- .../pulsar/broker/service/BrokerService.java | 37 +++++-- .../pulsar/broker/service/ServerCnx.java | 4 +- .../service/ServerCnxThrottleTracker.java | 14 +-- .../MessagePublishBufferThrottleTest.java | 102 ++++++++++++++---- .../OpenTelemetryAttributes.java | 19 ++++ 10 files changed, 278 insertions(+), 75 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 00afb85a9d486..398575461d5bf 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -213,7 +213,7 @@ private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore, compressionConfigForManagedCursorInfo); this.config = config; this.mbean = new ManagedLedgerFactoryMBeanImpl(this); - this.entryCacheManager = new RangeEntryCacheManagerImpl(this); + this.entryCacheManager = new RangeEntryCacheManagerImpl(this, openTelemetry); this.statsTask = scheduledExecutor.scheduleWithFixedDelay(catchingAndLoggingThrowables(this::refreshStats), 0, config.getStatsPeriodSeconds(), TimeUnit.SECONDS); this.flushCursorsTask = scheduledExecutor.scheduleAtFixedRate(catchingAndLoggingThrowables(this::flushCursors), diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java index b946dc09a0c71..c87807b86631b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiter.java @@ -19,20 +19,37 @@ package org.apache.bookkeeper.mledger.impl.cache; import com.google.common.annotations.VisibleForTesting; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.metrics.ObservableLongCounter; import io.prometheus.client.Gauge; import lombok.AllArgsConstructor; import lombok.ToString; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.opentelemetry.Constants; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.InflightReadLimiterUtilization; +import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric; @Slf4j -public class InflightReadsLimiter { +public class InflightReadsLimiter implements AutoCloseable { + public static final String INFLIGHT_READS_LIMITER_LIMIT_METRIC_NAME = + "pulsar.broker.managed_ledger.inflight.read.limit"; + private final ObservableLongCounter inflightReadsLimitCounter; + + @PulsarDeprecatedMetric(newMetricName = INFLIGHT_READS_LIMITER_LIMIT_METRIC_NAME) + @Deprecated private static final Gauge PULSAR_ML_READS_BUFFER_SIZE = Gauge .build() .name("pulsar_ml_reads_inflight_bytes") .help("Estimated number of bytes retained by data read from storage or cache") .register(); + public static final String INFLIGHT_READS_LIMITER_USAGE_METRIC_NAME = + "pulsar.broker.managed_ledger.inflight.read.usage"; + private final ObservableLongCounter inflightReadsUsageCounter; + + @PulsarDeprecatedMetric(newMetricName = INFLIGHT_READS_LIMITER_USAGE_METRIC_NAME) + @Deprecated private static final Gauge PULSAR_ML_READS_AVAILABLE_BUFFER_SIZE = Gauge .build() .name("pulsar_ml_reads_available_inflight_bytes") @@ -42,7 +59,7 @@ public class InflightReadsLimiter { private final long maxReadsInFlightSize; private long remainingBytes; - public InflightReadsLimiter(long maxReadsInFlightSize) { + public InflightReadsLimiter(long maxReadsInFlightSize, OpenTelemetry openTelemetry) { if (maxReadsInFlightSize <= 0) { // set it to -1 in order to show in the metrics that the metric is not available PULSAR_ML_READS_BUFFER_SIZE.set(-1); @@ -50,6 +67,28 @@ public InflightReadsLimiter(long maxReadsInFlightSize) { } this.maxReadsInFlightSize = maxReadsInFlightSize; this.remainingBytes = maxReadsInFlightSize; + + var meter = openTelemetry.getMeter(Constants.BROKER_INSTRUMENTATION_SCOPE_NAME); + inflightReadsLimitCounter = meter.counterBuilder(INFLIGHT_READS_LIMITER_LIMIT_METRIC_NAME) + .setDescription("Maximum number of bytes that can be retained by managed ledger data read from storage " + + "or cache.") + .setUnit("By") + .buildWithCallback(measurement -> { + if (!isDisabled()) { + measurement.record(maxReadsInFlightSize); + } + }); + inflightReadsUsageCounter = meter.counterBuilder(INFLIGHT_READS_LIMITER_USAGE_METRIC_NAME) + .setDescription("Estimated number of bytes retained by managed ledger data read from storage or cache.") + .setUnit("By") + .buildWithCallback(measurement -> { + if (!isDisabled()) { + var freeBytes = getRemainingBytes(); + var usedBytes = maxReadsInFlightSize - freeBytes; + measurement.record(freeBytes, InflightReadLimiterUtilization.FREE.attributes); + measurement.record(usedBytes, InflightReadLimiterUtilization.USED.attributes); + } + }); } @VisibleForTesting @@ -57,6 +96,12 @@ public synchronized long getRemainingBytes() { return remainingBytes; } + @Override + public void close() { + inflightReadsLimitCounter.close(); + inflightReadsUsageCounter.close(); + } + @AllArgsConstructor @ToString static class Handle { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java index d5a3019855cb5..34be25df1f476 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java @@ -20,6 +20,7 @@ import com.google.common.collect.Lists; import io.netty.buffer.ByteBuf; +import io.opentelemetry.api.OpenTelemetry; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; @@ -56,10 +57,10 @@ public class RangeEntryCacheManagerImpl implements EntryCacheManager { private static final double evictionTriggerThresholdPercent = 0.98; - public RangeEntryCacheManagerImpl(ManagedLedgerFactoryImpl factory) { + public RangeEntryCacheManagerImpl(ManagedLedgerFactoryImpl factory, OpenTelemetry openTelemetry) { this.maxSize = factory.getConfig().getMaxCacheSize(); this.inflightReadsLimiter = new InflightReadsLimiter( - factory.getConfig().getManagedLedgerMaxReadsInFlightSize()); + factory.getConfig().getManagedLedgerMaxReadsInFlightSize(), openTelemetry); this.evictionTriggerThreshold = (long) (maxSize * evictionTriggerThresholdPercent); this.cacheEvictionWatermark = factory.getConfig().getCacheEvictionWatermark(); this.evictionPolicy = new EntryCacheDefaultEvictionPolicy(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java index 2b69581ca2c73..89bdda15afb4b 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/InflightReadsLimiterTest.java @@ -18,45 +18,79 @@ */ package org.apache.bookkeeper.mledger.impl.cache; +import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; +import static org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.InflightReadLimiterUtilization.FREE; +import static org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.InflightReadLimiterUtilization.USED; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; - +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.sdk.OpenTelemetrySdk; +import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdk; +import io.opentelemetry.sdk.testing.exporter.InMemoryMetricReader; +import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.tuple.Pair; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j public class InflightReadsLimiterTest { - @Test - public void testDisabled() throws Exception { - - InflightReadsLimiter limiter = new InflightReadsLimiter(0); - assertTrue(limiter.isDisabled()); - - limiter = new InflightReadsLimiter(-1); - assertTrue(limiter.isDisabled()); + @DataProvider + private static Object[][] isDisabled() { + return new Object[][] { + {0, true}, + {-1, true}, + {1, false}, + }; + } - limiter = new InflightReadsLimiter(1); - assertFalse(limiter.isDisabled()); + @Test(dataProvider = "isDisabled") + public void testDisabled(long maxReadsInFlightSize, boolean shouldBeDisabled) throws Exception { + var otel = buildOpenTelemetryAndReader(); + @Cleanup var openTelemetry = otel.getLeft(); + @Cleanup var metricReader = otel.getRight(); + + var limiter = new InflightReadsLimiter(maxReadsInFlightSize, openTelemetry); + assertEquals(limiter.isDisabled(), shouldBeDisabled); + + if (shouldBeDisabled) { + // Verify metrics are not present + var metrics = metricReader.collectAllMetrics(); + assertThat(metrics).noneSatisfy(metricData -> assertThat(metricData) + .hasName(InflightReadsLimiter.INFLIGHT_READS_LIMITER_LIMIT_METRIC_NAME)); + assertThat(metrics).noneSatisfy(metricData -> assertThat(metricData) + .hasName(InflightReadsLimiter.INFLIGHT_READS_LIMITER_USAGE_METRIC_NAME)); + } } @Test public void testBasicAcquireRelease() throws Exception { - InflightReadsLimiter limiter = new InflightReadsLimiter(100); + var otel = buildOpenTelemetryAndReader(); + @Cleanup var openTelemetry = otel.getLeft(); + @Cleanup var metricReader = otel.getRight(); + + InflightReadsLimiter limiter = new InflightReadsLimiter(100, openTelemetry); assertEquals(100, limiter.getRemainingBytes()); + assertLimiterMetrics(metricReader, 100, 0, 100); + InflightReadsLimiter.Handle handle = limiter.acquire(100, null); assertEquals(0, limiter.getRemainingBytes()); assertTrue(handle.success); assertEquals(handle.acquiredPermits, 100); assertEquals(1, handle.trials); + assertLimiterMetrics(metricReader, 100, 100, 0); + limiter.release(handle); assertEquals(100, limiter.getRemainingBytes()); + assertLimiterMetrics(metricReader, 100, 0, 100); } + @Test public void testNotEnoughPermits() throws Exception { - InflightReadsLimiter limiter = new InflightReadsLimiter(100); + InflightReadsLimiter limiter = new InflightReadsLimiter(100, OpenTelemetry.noop()); assertEquals(100, limiter.getRemainingBytes()); InflightReadsLimiter.Handle handle = limiter.acquire(100, null); assertEquals(0, limiter.getRemainingBytes()); @@ -86,7 +120,7 @@ public void testNotEnoughPermits() throws Exception { @Test public void testPartialAcquire() throws Exception { - InflightReadsLimiter limiter = new InflightReadsLimiter(100); + InflightReadsLimiter limiter = new InflightReadsLimiter(100, OpenTelemetry.noop()); assertEquals(100, limiter.getRemainingBytes()); InflightReadsLimiter.Handle handle = limiter.acquire(30, null); @@ -116,7 +150,7 @@ public void testPartialAcquire() throws Exception { @Test public void testTooManyTrials() throws Exception { - InflightReadsLimiter limiter = new InflightReadsLimiter(100); + InflightReadsLimiter limiter = new InflightReadsLimiter(100, OpenTelemetry.noop()); assertEquals(100, limiter.getRemainingBytes()); InflightReadsLimiter.Handle handle = limiter.acquire(30, null); @@ -169,4 +203,25 @@ public void testTooManyTrials() throws Exception { } + private Pair buildOpenTelemetryAndReader() { + var metricReader = InMemoryMetricReader.create(); + var openTelemetry = AutoConfiguredOpenTelemetrySdk.builder() + .addMeterProviderCustomizer((builder, __) -> builder.registerMetricReader(metricReader)) + .build() + .getOpenTelemetrySdk(); + return Pair.of(openTelemetry, metricReader); + } + + private void assertLimiterMetrics(InMemoryMetricReader metricReader, + long expectedLimit, long expectedUsed, long expectedFree) { + var metrics = metricReader.collectAllMetrics(); + assertThat(metrics).anySatisfy(metricData -> assertThat(metricData) + .hasName(InflightReadsLimiter.INFLIGHT_READS_LIMITER_LIMIT_METRIC_NAME) + .hasLongSumSatisfying(longSum -> longSum.hasPointsSatisfying(point -> point.hasValue(expectedLimit)))); + assertThat(metrics).anySatisfy(metricData -> assertThat(metricData) + .hasName(InflightReadsLimiter.INFLIGHT_READS_LIMITER_USAGE_METRIC_NAME) + .hasLongSumSatisfying(longSum -> longSum.hasPointsSatisfying( + point -> point.hasValue(expectedFree).hasAttributes(FREE.attributes), + point -> point.hasValue(expectedUsed).hasAttributes(USED.attributes)))); + } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java index 6f573ff8d75c8..01976f648aba4 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java @@ -18,8 +18,24 @@ */ package org.apache.bookkeeper.mledger.impl.cache; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.AssertJUnit.assertNotSame; +import static org.testng.AssertJUnit.assertSame; +import io.opentelemetry.api.OpenTelemetry; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.stream.Collectors; import lombok.Data; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.api.ReadHandle; @@ -30,7 +46,6 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; - import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.testng.annotations.AfterClass; @@ -38,23 +53,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.testng.Assert.assertEquals; -import static org.testng.AssertJUnit.assertNotSame; -import static org.testng.AssertJUnit.assertSame; - @Slf4j public class PendingReadsManagerTest { @@ -93,7 +91,7 @@ void setupMocks() { config.setReadEntryTimeoutSeconds(10000); when(rangeEntryCache.getName()).thenReturn("my-topic"); when(rangeEntryCache.getManagedLedgerConfig()).thenReturn(config); - inflighReadsLimiter = new InflightReadsLimiter(0); + inflighReadsLimiter = new InflightReadsLimiter(0, OpenTelemetry.noop()); when(rangeEntryCache.getPendingReadsLimiter()).thenReturn(inflighReadsLimiter); pendingReadsManager = new PendingReadsManager(rangeEntryCache); doAnswer(new Answer() { 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 c0f44838ac680..5ea055287ebcf 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 @@ -40,7 +40,9 @@ import io.netty.channel.socket.SocketChannel; import io.netty.handler.ssl.SslContext; import io.netty.util.concurrent.DefaultThreadFactory; +import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.ObservableLongUpDownCounter; +import io.prometheus.client.Gauge; import io.prometheus.client.Histogram; import java.io.Closeable; import java.io.IOException; @@ -182,6 +184,7 @@ import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.ConnectionRateLimitOperationName; import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric; import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStore; @@ -258,6 +261,15 @@ public class BrokerService implements Closeable { private final ObservableLongUpDownCounter pendingTopicLoadOperationsCounter; private final ObservableLongUpDownCounter pendingTopicLoadOperationsLimitCounter; + public static final String CONNECTION_RATE_LIMIT_COUNT_METRIC_NAME = "pulsar.broker.connection.rate_limit.count"; + private final LongCounter rateLimitedConnectionsCounter; + @PulsarDeprecatedMetric(newMetricName = CONNECTION_RATE_LIMIT_COUNT_METRIC_NAME) + @Deprecated + private static final Gauge throttledConnectionsGauge = Gauge.build() + .name("pulsar_broker_throttled_connections") + .help("Counter of connections throttled because of per-connection limit") + .register(); + private final ScheduledExecutorService inactivityMonitor; private final ScheduledExecutorService messageExpiryMonitor; private final ScheduledExecutorService compactionMonitor; @@ -301,7 +313,6 @@ public class BrokerService implements Closeable { private Channel listenChannelTls; private boolean preciseTopicPublishRateLimitingEnable; - private final LongAdder pausedConnections = new LongAdder(); private BrokerInterceptor interceptor; private final EntryFilterProvider entryFilterProvider; private TopicFactory topicFactory; @@ -456,6 +467,12 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws .buildWithCallback( measurement -> measurement.record(pulsar.getConfig().getMaxConcurrentTopicLoadRequest())); + this.rateLimitedConnectionsCounter = pulsar.getOpenTelemetry().getMeter() + .counterBuilder(BrokerService.CONNECTION_RATE_LIMIT_COUNT_METRIC_NAME) + .setDescription("The number of times a connection has been rate limited.") + .setUnit("{operation}") + .build(); + this.brokerEntryMetadataInterceptors = BrokerEntryMetadataUtils .loadBrokerEntryMetadataInterceptors(pulsar.getConfiguration().getBrokerEntryMetadataInterceptors(), BrokerService.class.getClassLoader()); @@ -3701,16 +3718,22 @@ public boolean isBrokerPayloadProcessorEnabled() { return !brokerEntryPayloadProcessors.isEmpty(); } - public void pausedConnections(int numberOfConnections) { - pausedConnections.add(numberOfConnections); + public void recordConnectionPaused() { + rateLimitedConnectionsCounter.add(1, ConnectionRateLimitOperationName.PAUSED.attributes); + } + + public void recordConnectionResumed() { + rateLimitedConnectionsCounter.add(1, ConnectionRateLimitOperationName.RESUMED.attributes); } - public void resumedConnections(int numberOfConnections) { - pausedConnections.add(-numberOfConnections); + public void recordConnectionThrottled() { + rateLimitedConnectionsCounter.add(1, ConnectionRateLimitOperationName.THROTTLED.attributes); + throttledConnectionsGauge.inc(); } - public long getPausedConnections() { - return pausedConnections.longValue(); + public void recordConnectionUnthrottled() { + rateLimitedConnectionsCounter.add(1, ConnectionRateLimitOperationName.UNTHROTTLED.attributes); + throttledConnectionsGauge.dec(); } @SuppressWarnings("unchecked") 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 260552c55c0d7..6690ab4af5fd1 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 @@ -298,8 +298,7 @@ enum State { Start, Connected, Failed, Connecting } - private final ServerCnxThrottleTracker throttleTracker = new ServerCnxThrottleTracker(this); - + private final ServerCnxThrottleTracker throttleTracker; public ServerCnx(PulsarService pulsar) { this(pulsar, null); @@ -348,6 +347,7 @@ public ServerCnx(PulsarService pulsar, String listenerName) { this.topicListService = new TopicListService(pulsar, this, enableSubscriptionPatternEvaluation, maxSubscriptionPatternLength); this.brokerInterceptor = this.service != null ? this.service.getInterceptor() : null; + this.throttleTracker = new ServerCnxThrottleTracker(this); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnxThrottleTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnxThrottleTracker.java index 7e55397022d5e..78bac024218d8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnxThrottleTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnxThrottleTracker.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service; -import io.prometheus.client.Gauge; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import lombok.extern.slf4j.Slf4j; @@ -38,10 +37,6 @@ */ @Slf4j final class ServerCnxThrottleTracker { - private static final Gauge throttledConnections = Gauge.build() - .name("pulsar_broker_throttled_connections") - .help("Counter of connections throttled because of per-connection limit") - .register(); private static final AtomicIntegerFieldUpdater THROTTLE_COUNT_UPDATER = AtomicIntegerFieldUpdater.newUpdater( @@ -59,6 +54,7 @@ final class ServerCnxThrottleTracker { private volatile int pendingSendRequestsExceeded; private volatile int publishBufferLimiting; + public ServerCnxThrottleTracker(ServerCnx serverCnx) { this.serverCnx = serverCnx; @@ -94,10 +90,10 @@ private void changeAutoRead(boolean autoRead) { } // update the metrics that track throttling if (autoRead) { - serverCnx.getBrokerService().resumedConnections(1); + serverCnx.getBrokerService().recordConnectionResumed(); } else if (isChannelActive()) { serverCnx.increasePublishLimitedTimesForTopics(); - serverCnx.getBrokerService().pausedConnections(1); + serverCnx.getBrokerService().recordConnectionPaused(); } } @@ -114,9 +110,9 @@ public void setPendingSendRequestsExceeded(boolean throttlingEnabled) { if (changed) { // update the metrics that track throttling due to pending send requests if (throttlingEnabled) { - throttledConnections.inc(); + serverCnx.getBrokerService().recordConnectionThrottled(); } else { - throttledConnections.dec(); + serverCnx.getBrokerService().recordConnectionUnthrottled(); } } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessagePublishBufferThrottleTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessagePublishBufferThrottleTest.java index 27f72eac94254..0faae14da08ba 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessagePublishBufferThrottleTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessagePublishBufferThrottleTest.java @@ -18,12 +18,18 @@ */ package org.apache.pulsar.broker.service; -import static org.testng.Assert.assertEquals; +import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.testng.Assert.fail; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import lombok.Cleanup; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes.ConnectionRateLimitOperationName; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -43,6 +49,12 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder pulsarTestContextBuilder) { + super.customizeMainPulsarTestContextBuilder(pulsarTestContextBuilder); + pulsarTestContextBuilder.enableOpenTelemetry(true); + } + @Test public void testMessagePublishBufferThrottleDisabled() throws Exception { conf.setMaxMessagePublishBufferSizeInMB(-1); @@ -52,7 +64,8 @@ public void testMessagePublishBufferThrottleDisabled() throws Exception { .topic(topic) .producerName("producer-name") .create(); - assertEquals(pulsar.getBrokerService().getPausedConnections(), 0); + assertRateLimitCounter(ConnectionRateLimitOperationName.PAUSED, 0); + assertRateLimitCounter(ConnectionRateLimitOperationName.RESUMED, 0); pulsarTestContext.getMockBookKeeper().addEntryDelay(1, TimeUnit.SECONDS); @@ -63,7 +76,8 @@ public void testMessagePublishBufferThrottleDisabled() throws Exception { } producer.flush(); - assertEquals(pulsar.getBrokerService().getPausedConnections(), 0); + assertRateLimitCounter(ConnectionRateLimitOperationName.PAUSED, 0); + assertRateLimitCounter(ConnectionRateLimitOperationName.RESUMED, 0); } @Test @@ -71,14 +85,14 @@ public void testMessagePublishBufferThrottleEnable() throws Exception { conf.setMaxMessagePublishBufferSizeInMB(1); super.baseSetup(); - assertEquals(pulsar.getBrokerService().getPausedConnections(), 0); final String topic = "persistent://prop/ns-abc/testMessagePublishBufferThrottleEnable"; Producer producer = pulsarClient.newProducer() .topic(topic) .producerName("producer-name") .create(); - assertEquals(pulsar.getBrokerService().getPausedConnections(), 0); + assertRateLimitCounter(ConnectionRateLimitOperationName.PAUSED, 0); + assertRateLimitCounter(ConnectionRateLimitOperationName.RESUMED, 0); pulsarTestContext.getMockBookKeeper().addEntryDelay(1, TimeUnit.SECONDS); @@ -87,23 +101,27 @@ public void testMessagePublishBufferThrottleEnable() throws Exception { producer.sendAsync(payload); } - Awaitility.await().untilAsserted( - () -> Assert.assertEquals(pulsar.getBrokerService().getPausedConnections(), 1L)); - assertEquals(pulsar.getBrokerService().getPausedConnections(), 1); + Awaitility.await().untilAsserted(() -> { + assertRateLimitCounter(ConnectionRateLimitOperationName.PAUSED, 0); + assertRateLimitCounter(ConnectionRateLimitOperationName.RESUMED, 0); + }); producer.flush(); - Awaitility.await().untilAsserted( - () -> Assert.assertEquals(pulsar.getBrokerService().getPausedConnections(), 0L)); - - assertEquals(pulsar.getBrokerService().getPausedConnections(), 0); + Awaitility.await().untilAsserted(() -> { + assertRateLimitCounter(ConnectionRateLimitOperationName.PAUSED, 0); + assertRateLimitCounter(ConnectionRateLimitOperationName.RESUMED, 0); + }); } @Test public void testBlockByPublishRateLimiting() throws Exception { conf.setMaxMessagePublishBufferSizeInMB(1); super.baseSetup(); - assertEquals(pulsar.getBrokerService().getPausedConnections(), 0); + + assertRateLimitCounter(ConnectionRateLimitOperationName.PAUSED, 0); + assertRateLimitCounter(ConnectionRateLimitOperationName.RESUMED, 0); + final String topic = "persistent://prop/ns-abc/testBlockByPublishRateLimiting"; Producer producer = pulsarClient.newProducer() .topic(topic) @@ -111,7 +129,8 @@ public void testBlockByPublishRateLimiting() throws Exception { .create(); Topic topicRef = pulsar.getBrokerService().getTopicReference(topic).get(); Assert.assertNotNull(topicRef); - assertEquals(pulsar.getBrokerService().getPausedConnections(), 0); + assertRateLimitCounter(ConnectionRateLimitOperationName.PAUSED, 0); + assertRateLimitCounter(ConnectionRateLimitOperationName.RESUMED, 0); pulsarTestContext.getMockBookKeeper().addEntryDelay(5, TimeUnit.SECONDS); @@ -121,13 +140,15 @@ public void testBlockByPublishRateLimiting() throws Exception { producer.sendAsync(payload); } - Awaitility.await().untilAsserted(() -> assertEquals(pulsar.getBrokerService().getPausedConnections(), 1)); + Awaitility.await().untilAsserted(() -> assertRateLimitCounter(ConnectionRateLimitOperationName.PAUSED, 1)); CompletableFuture flushFuture = producer.flushAsync(); // Block by publish rate. // After 1 second, the message buffer throttling will be lifted, but the rate limiting will still be in place. - assertEquals(pulsar.getBrokerService().getPausedConnections(), 1); + assertRateLimitCounter(ConnectionRateLimitOperationName.PAUSED, 1); + assertRateLimitCounter(ConnectionRateLimitOperationName.RESUMED, 0); + try { flushFuture.get(2, TimeUnit.SECONDS); fail("Should have timed out"); @@ -137,7 +158,52 @@ public void testBlockByPublishRateLimiting() throws Exception { flushFuture.join(); - Awaitility.await().untilAsserted(() -> - assertEquals(pulsar.getBrokerService().getPausedConnections(), 0)); + Awaitility.await().untilAsserted(() -> { + assertRateLimitCounter(ConnectionRateLimitOperationName.PAUSED, 10); + assertRateLimitCounter(ConnectionRateLimitOperationName.RESUMED, 10); + }); + } + + @Test + public void testConnectionThrottled() throws Exception { + super.baseSetup(); + + var topic = BrokerTestUtil.newUniqueName("persistent://prop/ns-abc/testSendThrottled"); + + assertRateLimitCounter(ConnectionRateLimitOperationName.THROTTLED, 0); + assertRateLimitCounter(ConnectionRateLimitOperationName.UNTHROTTLED, 0); + + @Cleanup + var producer = pulsarClient.newProducer(Schema.STRING) + .enableBatching(false) + .topic(topic) + .create(); + final int messages = 2000; + for (int i = 0; i < messages; i++) { + producer.sendAsync("Message - " + i); + } + producer.flush(); + + // Wait for the connection to be throttled and unthrottled. + Awaitility.await().untilAsserted(() -> { + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, BrokerService.CONNECTION_RATE_LIMIT_COUNT_METRIC_NAME, + ConnectionRateLimitOperationName.THROTTLED.attributes, value -> assertThat(value).isPositive()); + assertMetricLongSumValue(metrics, BrokerService.CONNECTION_RATE_LIMIT_COUNT_METRIC_NAME, + ConnectionRateLimitOperationName.UNTHROTTLED.attributes, value -> assertThat(value).isPositive()); + }); + } + + private void assertRateLimitCounter(ConnectionRateLimitOperationName connectionRateLimitState, int expectedCount) { + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + if (expectedCount == 0) { + assertThat(metrics).noneSatisfy(metricData -> assertThat(metricData) + .hasName(BrokerService.CONNECTION_RATE_LIMIT_COUNT_METRIC_NAME) + .hasLongSumSatisfying(sum -> sum.hasPointsSatisfying( + points -> points.hasAttributes(connectionRateLimitState.attributes)))); + } else { + assertMetricLongSumValue(metrics, BrokerService.CONNECTION_RATE_LIMIT_COUNT_METRIC_NAME, + connectionRateLimitState.attributes, expectedCount); + } } } diff --git a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java index 41358a72c0d90..6eb84e94bc61b 100644 --- a/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java +++ b/pulsar-opentelemetry/src/main/java/org/apache/pulsar/opentelemetry/OpenTelemetryAttributes.java @@ -112,6 +112,17 @@ public interface OpenTelemetryAttributes { */ AttributeKey PULSAR_CLIENT_VERSION = AttributeKey.stringKey("pulsar.client.version"); + AttributeKey PULSAR_CONNECTION_RATE_LIMIT_OPERATION_NAME = + AttributeKey.stringKey("pulsar.connection.rate_limit.operation.name"); + enum ConnectionRateLimitOperationName { + PAUSED, + RESUMED, + THROTTLED, + UNTHROTTLED; + public final Attributes attributes = + Attributes.of(PULSAR_CONNECTION_RATE_LIMIT_OPERATION_NAME, name().toLowerCase()); + } + /** * The status of the Pulsar transaction. */ @@ -197,6 +208,14 @@ enum ManagedCursorOperationStatus { public final Attributes attributes = Attributes.of(ML_CURSOR_OPERATION_STATUS, name().toLowerCase()); } + AttributeKey MANAGED_LEDGER_READ_INFLIGHT_USAGE = + AttributeKey.stringKey("pulsar.managed_ledger.inflight.read.usage.state"); + enum InflightReadLimiterUtilization { + USED, + FREE; + public final Attributes attributes = Attributes.of(MANAGED_LEDGER_READ_INFLIGHT_USAGE, name().toLowerCase()); + } + /** * The name of the remote cluster for a Pulsar replicator. */ From 1c53841cc7f585bdd8ff6702d74f37491d8cc9c6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 24 Jul 2024 14:40:03 +0800 Subject: [PATCH 388/580] [fix][broker] Handle BucketDelayedDeliveryTracker recover failed (#22735) --- .../BucketDelayedDeliveryTrackerFactory.java | 30 ++- .../delayed/DelayedDeliveryTracker.java | 47 ++++ ...InMemoryDelayedDeliveryTrackerFactory.java | 19 ++ .../bucket/BucketDelayedDeliveryTracker.java | 35 ++- ...ecoverDelayedDeliveryTrackerException.java | 25 ++ .../pulsar/broker/service/BrokerService.java | 28 +- ...PersistentDispatcherMultipleConsumers.java | 11 +- .../DelayedDeliveryTrackerFactoryTest.java | 242 ++++++++++++++++++ .../BucketDelayedDeliveryTrackerTest.java | 6 +- 9 files changed, 420 insertions(+), 23 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/RecoverDelayedDeliveryTrackerException.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java index 17d9795dd9082..11ad243e0c9d1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.delayed; +import com.google.common.annotations.VisibleForTesting; import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; import io.netty.util.concurrent.DefaultThreadFactory; @@ -33,10 +34,15 @@ import org.apache.pulsar.broker.delayed.bucket.BookkeeperBucketSnapshotStorage; import org.apache.pulsar.broker.delayed.bucket.BucketDelayedDeliveryTracker; import org.apache.pulsar.broker.delayed.bucket.BucketSnapshotStorage; +import org.apache.pulsar.broker.delayed.bucket.RecoverDelayedDeliveryTrackerException; +import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class BucketDelayedDeliveryTrackerFactory implements DelayedDeliveryTrackerFactory { + private static final Logger log = LoggerFactory.getLogger(BucketDelayedDeliveryTrackerFactory.class); BucketSnapshotStorage bucketSnapshotStorage; @@ -73,8 +79,28 @@ public void initialize(PulsarService pulsarService) throws Exception { @Override public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher) { - return new BucketDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, isDelayedDeliveryDeliverAtTimeStrict, - bucketSnapshotStorage, delayedDeliveryMinIndexCountPerBucket, + String topicName = dispatcher.getTopic().getName(); + String subscriptionName = dispatcher.getSubscription().getName(); + BrokerService brokerService = dispatcher.getTopic().getBrokerService(); + DelayedDeliveryTracker tracker; + + try { + tracker = newTracker0(dispatcher); + } catch (RecoverDelayedDeliveryTrackerException ex) { + log.warn("Failed to recover BucketDelayedDeliveryTracker, fallback to InMemoryDelayedDeliveryTracker." + + " topic {}, subscription {}", topicName, subscriptionName, ex); + // If failed to create BucketDelayedDeliveryTracker, fallback to InMemoryDelayedDeliveryTracker + brokerService.initializeFallbackDelayedDeliveryTrackerFactory(); + tracker = brokerService.getFallbackDelayedDeliveryTrackerFactory().newTracker(dispatcher); + } + return tracker; + } + + @VisibleForTesting + BucketDelayedDeliveryTracker newTracker0(PersistentDispatcherMultipleConsumers dispatcher) + throws RecoverDelayedDeliveryTrackerException { + return new BucketDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, + isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, delayedDeliveryMinIndexCountPerBucket, TimeUnit.SECONDS.toMillis(delayedDeliveryMaxTimeStepPerBucketSnapshotSegmentSeconds), delayedDeliveryMaxIndexesPerBucketSnapshotSegment, delayedDeliveryMaxNumBuckets); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java index 81ed4894dc6ad..981583120e78f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java @@ -85,4 +85,51 @@ public interface DelayedDeliveryTracker extends AutoCloseable { * Close the subscription tracker and release all resources. */ void close(); + + DelayedDeliveryTracker DISABLE = new DelayedDeliveryTracker() { + @Override + public boolean addMessage(long ledgerId, long entryId, long deliveryAt) { + return false; + } + + @Override + public boolean hasMessageAvailable() { + return false; + } + + @Override + public long getNumberOfDelayedMessages() { + return 0; + } + + @Override + public long getBufferMemoryUsage() { + return 0; + } + + @Override + public NavigableSet getScheduledMessages(int maxMessages) { + return null; + } + + @Override + public boolean shouldPauseAllDeliveries() { + return false; + } + + @Override + public void resetTickTime(long tickTime) { + + } + + @Override + public CompletableFuture clear() { + return null; + } + + @Override + public void close() { + + } + }; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java index e7dc3f18f4630..179cf74db4179 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.delayed; +import com.google.common.annotations.VisibleForTesting; import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; import io.netty.util.concurrent.DefaultThreadFactory; @@ -25,8 +26,11 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class InMemoryDelayedDeliveryTrackerFactory implements DelayedDeliveryTrackerFactory { + private static final Logger log = LoggerFactory.getLogger(InMemoryDelayedDeliveryTrackerFactory.class); private Timer timer; @@ -48,6 +52,21 @@ public void initialize(PulsarService pulsarService) { @Override public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher) { + String topicName = dispatcher.getTopic().getName(); + String subscriptionName = dispatcher.getSubscription().getName(); + DelayedDeliveryTracker tracker = DelayedDeliveryTracker.DISABLE; + try { + tracker = newTracker0(dispatcher); + } catch (Exception e) { + // it should never go here + log.warn("Failed to create InMemoryDelayedDeliveryTracker, topic {}, subscription {}", + topicName, subscriptionName, e); + } + return tracker; + } + + @VisibleForTesting + InMemoryDelayedDeliveryTracker newTracker0(PersistentDispatcherMultipleConsumers dispatcher) { return new InMemoryDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index 063793f2dd1fa..5a6df389aeddb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -106,22 +106,24 @@ public class BucketDelayedDeliveryTracker extends AbstractDelayedDeliveryTracker private CompletableFuture pendingLoad = null; public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, - Timer timer, long tickTimeMillis, - boolean isDelayedDeliveryDeliverAtTimeStrict, - BucketSnapshotStorage bucketSnapshotStorage, - long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, - int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) { + Timer timer, long tickTimeMillis, + boolean isDelayedDeliveryDeliverAtTimeStrict, + BucketSnapshotStorage bucketSnapshotStorage, + long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, + int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) + throws RecoverDelayedDeliveryTrackerException { this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegmentInMillis, maxIndexesPerBucketSnapshotSegment, maxNumBuckets); } public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, - Timer timer, long tickTimeMillis, Clock clock, - boolean isDelayedDeliveryDeliverAtTimeStrict, - BucketSnapshotStorage bucketSnapshotStorage, - long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, - int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) { + Timer timer, long tickTimeMillis, Clock clock, + boolean isDelayedDeliveryDeliverAtTimeStrict, + BucketSnapshotStorage bucketSnapshotStorage, + long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, + int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) + throws RecoverDelayedDeliveryTrackerException { super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict); this.minIndexCountPerBucket = minIndexCountPerBucket; this.timeStepPerBucketSnapshotSegmentInMillis = timeStepPerBucketSnapshotSegmentInMillis; @@ -134,10 +136,17 @@ public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispat new MutableBucket(dispatcher.getName(), dispatcher.getCursor(), FutureUtil.Sequencer.create(), bucketSnapshotStorage); this.stats = new BucketDelayedMessageIndexStats(); - this.numberDelayedMessages = recoverBucketSnapshot(); + + // Close the tracker if failed to recover. + try { + this.numberDelayedMessages = recoverBucketSnapshot(); + } catch (RecoverDelayedDeliveryTrackerException e) { + close(); + throw e; + } } - private synchronized long recoverBucketSnapshot() throws RuntimeException { + private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryTrackerException { ManagedCursor cursor = this.lastMutableBucket.getCursor(); Map cursorProperties = cursor.getCursorProperties(); if (MapUtils.isEmpty(cursorProperties)) { @@ -182,7 +191,7 @@ private synchronized long recoverBucketSnapshot() throws RuntimeException { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } - throw new RuntimeException(e); + throw new RecoverDelayedDeliveryTrackerException(e); } for (Map.Entry, CompletableFuture>> entry : futures.entrySet()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/RecoverDelayedDeliveryTrackerException.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/RecoverDelayedDeliveryTrackerException.java new file mode 100644 index 0000000000000..71a851100fe4e --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/RecoverDelayedDeliveryTrackerException.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.delayed.bucket; + +public class RecoverDelayedDeliveryTrackerException extends Exception { + public RecoverDelayedDeliveryTrackerException(Throwable cause) { + super(cause); + } +} 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 5ea055287ebcf..c62da22ac6827 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 @@ -108,6 +108,7 @@ import org.apache.pulsar.broker.cache.BundlesQuotas; import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerLoader; +import org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.intercept.ManagedLedgerInterceptorImpl; import org.apache.pulsar.broker.loadbalance.LoadManager; @@ -296,10 +297,11 @@ public class BrokerService implements Closeable { private final AtomicBoolean blockedDispatcherOnHighUnackedMsgs = new AtomicBoolean(false); private final ConcurrentOpenHashSet blockedDispatchers; private final ReadWriteLock lock = new ReentrantReadWriteLock(); - - @Getter @VisibleForTesting private final DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory; + // InMemoryDelayedDeliveryTrackerFactory is for the purpose of + // fallback if recover BucketDelayedDeliveryTracker failed. + private volatile DelayedDeliveryTrackerFactory fallbackDelayedDeliveryTrackerFactory; private final ServerBootstrap defaultServerBootstrap; private final List protocolHandlersWorkerGroups = new ArrayList<>(); @@ -865,6 +867,9 @@ public CompletableFuture closeAsync() { pendingLookupOperationsCounter.close(); try { delayedDeliveryTrackerFactory.close(); + if (fallbackDelayedDeliveryTrackerFactory != null) { + fallbackDelayedDeliveryTrackerFactory.close(); + } } catch (Exception e) { log.warn("Error in closing delayedDeliveryTrackerFactory", e); } @@ -3418,6 +3423,25 @@ public void unblockDispatchersOnUnAckMessages(List pair = + mockDelayedDeliveryTrackerFactoryAndDispatcher(); + BrokerService brokerService = pair.getLeft(); + PersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); + + // Since Mocked BucketDelayedDeliveryTrackerFactory.newTracker0() throws RecoverDelayedDeliveryTrackerException, + // the factory should be fallback to InMemoryDelayedDeliveryTrackerFactory + @Cleanup + DelayedDeliveryTracker tracker = brokerService.getDelayedDeliveryTrackerFactory().newTracker(dispatcher); + Assert.assertTrue(tracker instanceof InMemoryDelayedDeliveryTracker); + + DelayedDeliveryTrackerFactory fallbackFactory = brokerService.getFallbackDelayedDeliveryTrackerFactory(); + Assert.assertTrue(fallbackFactory instanceof InMemoryDelayedDeliveryTrackerFactory); + } + + + private Pair mockDelayedDeliveryTrackerFactoryAndDispatcher() + throws Exception { + BrokerService brokerService = Mockito.spy(pulsar.getBrokerService()); + + // Mock dispatcher + PersistentDispatcherMultipleConsumers dispatcher = Mockito.mock(PersistentDispatcherMultipleConsumers.class); + Mockito.doReturn("test").when(dispatcher).getName(); + // Mock BucketDelayedDeliveryTrackerFactory + @Cleanup + BucketDelayedDeliveryTrackerFactory factory = new BucketDelayedDeliveryTrackerFactory(); + factory = Mockito.spy(factory); + factory.initialize(pulsar); + Mockito.doThrow(new RecoverDelayedDeliveryTrackerException(new RuntimeException())) + .when(factory).newTracker0(Mockito.eq(dispatcher)); + // Mock brokerService + Mockito.doReturn(factory).when(brokerService).getDelayedDeliveryTrackerFactory(); + // Mock topic and subscription + PersistentTopic topic = Mockito.mock(PersistentTopic.class); + Mockito.doReturn(brokerService).when(topic).getBrokerService(); + Subscription subscription = Mockito.mock(Subscription.class); + Mockito.doReturn("topic").when(topic).getName(); + Mockito.doReturn("sub").when(subscription).getName(); + Mockito.doReturn(topic).when(dispatcher).getTopic(); + Mockito.doReturn(subscription).when(dispatcher).getSubscription(); + + return Pair.of(brokerService, dispatcher); + } + + @Test + public void testFallbackToInMemoryTrackerFactoryFailed() throws Exception { + Pair pair = + mockDelayedDeliveryTrackerFactoryAndDispatcher(); + BrokerService brokerService = pair.getLeft(); + PersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); + + // Mock InMemoryDelayedDeliveryTrackerFactory + @Cleanup + InMemoryDelayedDeliveryTrackerFactory factory = new InMemoryDelayedDeliveryTrackerFactory(); + factory = Mockito.spy(factory); + factory.initialize(pulsar); + // Mock InMemoryDelayedDeliveryTrackerFactory.newTracker0() throws RuntimeException + Mockito.doThrow(new RuntimeException()).when(factory).newTracker0(Mockito.eq(dispatcher)); + + // Mock brokerService to return mocked InMemoryDelayedDeliveryTrackerFactory + Mockito.doAnswer(inv -> null).when(brokerService).initializeFallbackDelayedDeliveryTrackerFactory(); + Mockito.doReturn(factory).when(brokerService).getFallbackDelayedDeliveryTrackerFactory(); + + // Since Mocked BucketDelayedDeliveryTrackerFactory.newTracker0() throws RecoverDelayedDeliveryTrackerException, + // and Mocked InMemoryDelayedDeliveryTrackerFactory.newTracker0() throws RuntimeException, + // the tracker instance should be DelayedDeliveryTracker.DISABLE + @Cleanup + DelayedDeliveryTracker tracker = brokerService.getDelayedDeliveryTrackerFactory().newTracker(dispatcher); + Assert.assertEquals(tracker, DelayedDeliveryTracker.DISABLE); + } + + // 1. Create BucketDelayedDeliveryTracker failed, fallback to InMemoryDelayedDeliveryTracker, + // 2. Publish delay messages + @Test(timeOut = 60_000) + public void testPublishDelayMessagesAndCreateBucketDelayDeliveryTrackerFailed() throws Exception { + String topicName = "persistent://public/default/" + UUID.randomUUID(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topicName) + .enableBatching(false) + .create(); + + // Mock BucketDelayedDeliveryTrackerFactory.newTracker0() throws RecoverDelayedDeliveryTrackerException + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); + topic = Mockito.spy(topic); + BrokerService brokerService = Mockito.spy(pulsar.getBrokerService()); + BucketDelayedDeliveryTrackerFactory factory = + (BucketDelayedDeliveryTrackerFactory) Mockito.spy(brokerService.getDelayedDeliveryTrackerFactory()); + Mockito.doThrow(new RecoverDelayedDeliveryTrackerException(new RuntimeException())) + .when(factory).newTracker0(Mockito.any()); + Mockito.doReturn(factory).when(brokerService).getDelayedDeliveryTrackerFactory(); + + // Return mocked BrokerService + Mockito.doReturn(brokerService).when(topic).getBrokerService(); + + // Set Mocked topic to BrokerService + Field topics = BrokerService.class.getDeclaredField("topics"); + topics.setAccessible(true); + @SuppressWarnings("unchecked") + ConcurrentOpenHashMap>> topicMap = + (ConcurrentOpenHashMap>>) topics.get(brokerService); + topicMap.put(topicName, CompletableFuture.completedFuture(Optional.of(topic))); + + // Create consumer + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName("sub") + .subscriptionType(SubscriptionType.Shared) + .messageListener((c, msg) -> { + try { + c.acknowledge(msg); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } + }) + .subscribe(); + + PersistentSubscription subscription = topic.getSubscription("sub"); + Dispatcher dispatcher = subscription.getDispatcher(); + Assert.assertTrue(dispatcher instanceof PersistentDispatcherMultipleConsumers); + + // Publish a delay message to initialize DelayedDeliveryTracker + producer.newMessage().value("test").deliverAfter(10_000, TimeUnit.MILLISECONDS).send(); + + // Get DelayedDeliveryTracker from Dispatcher + PersistentDispatcherMultipleConsumers dispatcher0 = (PersistentDispatcherMultipleConsumers) dispatcher; + Field trackerField = + PersistentDispatcherMultipleConsumers.class.getDeclaredField("delayedDeliveryTracker"); + trackerField.setAccessible(true); + + AtomicReference> reference = new AtomicReference<>(); + // Wait until DelayedDeliveryTracker is initialized + Awaitility.await().atMost(Duration.ofSeconds(20)).until(() -> { + @SuppressWarnings("unchecked") + Optional optional = + (Optional) trackerField.get(dispatcher0); + if (optional.isPresent()) { + reference.set(optional); + return true; + } + return false; + }); + + Optional optional = reference.get(); + Assert.assertTrue(optional.get() instanceof InMemoryDelayedDeliveryTracker); + + // Mock DelayedDeliveryTracker and Count the number of addMessage() calls + AtomicInteger counter = new AtomicInteger(0); + InMemoryDelayedDeliveryTracker tracker = (InMemoryDelayedDeliveryTracker) optional.get(); + tracker = Mockito.spy(tracker); + Mockito.doAnswer(inv -> { + counter.incrementAndGet(); + return inv.callRealMethod(); + }).when(tracker).addMessage(Mockito.anyLong(), Mockito.anyLong(), Mockito.anyLong()); + // Set Mocked InMemoryDelayedDeliveryTracker back to Dispatcher + trackerField.set(dispatcher0, Optional.of(tracker)); + + // Publish 10 delay messages, so the counter should be 10 + for (int i = 0; i < 10; i++) { + producer.newMessage().value("test") + .deliverAfter(10_000, TimeUnit.MILLISECONDS).send(); + } + + try { + Awaitility.await().atMost(Duration.ofSeconds(20)).until(() -> counter.get() == 10); + } finally { + consumer.close(); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java index d09249deb5be2..bf5a282a4ee6d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java @@ -183,7 +183,7 @@ public void testContainsMessage(BucketDelayedDeliveryTracker tracker) { } @Test(dataProvider = "delayedTracker", invocationCount = 10) - public void testRecoverSnapshot(BucketDelayedDeliveryTracker tracker) { + public void testRecoverSnapshot(BucketDelayedDeliveryTracker tracker) throws Exception { for (int i = 1; i <= 100; i++) { tracker.addMessage(i, i, i * 10); } @@ -266,7 +266,7 @@ public void testRoaringBitmapSerialize() { } @Test(dataProvider = "delayedTracker") - public void testMergeSnapshot(final BucketDelayedDeliveryTracker tracker) { + public void testMergeSnapshot(final BucketDelayedDeliveryTracker tracker) throws Exception { for (int i = 1; i <= 110; i++) { tracker.addMessage(i, i, i * 10); Awaitility.await().untilAsserted(() -> { @@ -319,7 +319,7 @@ public void testMergeSnapshot(final BucketDelayedDeliveryTracker tracker) { } @Test(dataProvider = "delayedTracker") - public void testWithBkException(final BucketDelayedDeliveryTracker tracker) { + public void testWithBkException(final BucketDelayedDeliveryTracker tracker) throws Exception { MockBucketSnapshotStorage mockBucketSnapshotStorage = (MockBucketSnapshotStorage) bucketSnapshotStorage; mockBucketSnapshotStorage.injectCreateException( new BucketSnapshotPersistenceException("Bookie operation timeout, op: Create entry")); From 38a5e91de9a71bd00d37739e15dbcc09e004ba2f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 24 Jul 2024 19:08:12 +0800 Subject: [PATCH 389/580] [fix] Fix compile failing (#23070) --- .../apache/pulsar/broker/delayed/DelayedDeliveryTracker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java index 981583120e78f..7c954879fe845 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java @@ -108,7 +108,7 @@ public long getBufferMemoryUsage() { } @Override - public NavigableSet getScheduledMessages(int maxMessages) { + public NavigableSet getScheduledMessages(int maxMessages) { return null; } From 55e468ee412fff9706f6badcd07b5c8f11ed8375 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Wed, 24 Jul 2024 11:19:40 -0700 Subject: [PATCH 390/580] [fix][broker] Do not try to clean owned bundles from inactive source brokers (ExtensibleLoadManagerImpl only) (#23064) --- .../extensions/ExtensibleLoadManagerImpl.java | 27 ++- .../channel/ServiceUnitStateChannelImpl.java | 178 +++++++----------- .../pulsar/broker/service/BrokerService.java | 9 + .../channel/ServiceUnitStateChannelTest.java | 68 +++++-- 4 files changed, 157 insertions(+), 125 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 4a7ba90aad919..a737a94b998ac 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -124,6 +124,9 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS private static final String ELECTION_ROOT = "/loadbalance/extension/leader"; + private static final Set INTERNAL_TOPICS = + Set.of(BROKER_LOAD_DATA_STORE_TOPIC, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TOPIC); + private PulsarService pulsar; private ServiceConfiguration conf; @@ -828,7 +831,8 @@ public void close() throws PulsarServerException { } public static boolean isInternalTopic(String topic) { - return topic.startsWith(TOPIC) + return INTERNAL_TOPICS.contains(topic) + || topic.startsWith(TOPIC) || topic.startsWith(BROKER_LOAD_DATA_STORE_TOPIC) || topic.startsWith(TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); } @@ -993,5 +997,26 @@ public void disableBroker() throws Exception { serviceUnitStateChannel.cleanOwnerships(); leaderElectionService.close(); brokerRegistry.unregister(); + // Close the internal topics (if owned any) after giving up the possible leader role, + // so that the subsequent lookups could hit the next leader. + closeInternalTopics(); + } + + private void closeInternalTopics() { + List> futures = new ArrayList<>(); + for (String name : INTERNAL_TOPICS) { + futures.add(pulsar.getBrokerService().getTopicIfExists(name) + .thenAccept(topicOptional -> topicOptional.ifPresent(topic -> topic.close(true))) + .exceptionally(__ -> { + log.warn("Failed to close internal topic:{}", name); + return null; + })); + } + try { + FutureUtil.waitForAll(futures) + .get(pulsar.getConfiguration().getNamespaceBundleUnloadingTimeoutMs(), TimeUnit.MILLISECONDS); + } catch (Throwable e) { + log.warn("Failed to wait for closing internal topics", e); + } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index 1688a892e237f..fc4968805f5c1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -638,20 +638,13 @@ public CompletableFuture publishAssignEventAsync(String serviceUnit, Str } private CompletableFuture publishOverrideEventAsync(String serviceUnit, - ServiceUnitStateData orphanData, ServiceUnitStateData override) { if (!validateChannelState(Started, true)) { throw new IllegalStateException("Invalid channel state:" + channelState.name()); } EventType eventType = EventType.Override; eventCounters.get(eventType).getTotal().incrementAndGet(); - return pubAsync(serviceUnit, override).whenComplete((__, e) -> { - if (e != null) { - eventCounters.get(eventType).getFailure().incrementAndGet(); - log.error("Failed to override serviceUnit:{} from orphanData:{} to overrideData:{}", - serviceUnit, orphanData, override, e); - } - }).thenApply(__ -> null); + return pubAsync(serviceUnit, override).thenApply(__ -> null); } public CompletableFuture publishUnloadEventAsync(Unload unload) { @@ -1307,24 +1300,49 @@ private void scheduleCleanup(String broker, long delayInSecs) { private void overrideOwnership(String serviceUnit, ServiceUnitStateData orphanData, String inactiveBroker) { final var version = getNextVersionId(orphanData); - final var override = selectBroker(serviceUnit, inactiveBroker).map(selectedBroker -> { - if (orphanData.state() == Splitting) { - return new ServiceUnitStateData(Splitting, orphanData.dstBroker(), selectedBroker, - Map.copyOf(orphanData.splitServiceUnitToDestBroker()), true, version); - } else { - return new ServiceUnitStateData(Owned, selectedBroker, inactiveBroker, true, version); - } - }).orElseGet(() -> new ServiceUnitStateData(Free, null, inactiveBroker, true, version)); - log.info("Overriding ownership serviceUnit:{} from orphanData:{} to overrideData:{}", - serviceUnit, orphanData, override); - publishOverrideEventAsync(serviceUnit, orphanData, override) - .exceptionally(e -> { - log.error( - "Failed to override the ownership serviceUnit:{} orphanData:{}. " - + "Failed to publish override event. totalCleanupErrorCnt:{}", - serviceUnit, orphanData, totalCleanupErrorCnt.incrementAndGet()); - return null; - }); + try { + selectBroker(serviceUnit, inactiveBroker) + .thenApply(selectedOpt -> + selectedOpt.map(selectedBroker -> { + if (orphanData.state() == Splitting) { + // if Splitting, set orphan.dstBroker() as dst to indicate where it was from. + // (The src broker runs handleSplitEvent.) + return new ServiceUnitStateData(Splitting, orphanData.dstBroker(), selectedBroker, + Map.copyOf(orphanData.splitServiceUnitToDestBroker()), true, version); + } else if (orphanData.state() == Owned) { + // if Owned, set orphan.dstBroker() as source to clean it up in case it is still + // alive. + return new ServiceUnitStateData(Owned, selectedBroker, + selectedBroker.equals(orphanData.dstBroker()) ? null : + orphanData.dstBroker(), + true, version); + } else { + // if Assigning or Releasing, set orphan.sourceBroker() as source + // to clean it up in case it is still alive. + return new ServiceUnitStateData(Owned, selectedBroker, + selectedBroker.equals(orphanData.sourceBroker()) ? null : + orphanData.sourceBroker(), + true, version); + } + // If no broker is selected(available), free the ownership. + // If the previous owner is still active, it will close the bundle(topic) ownership. + }).orElseGet(() -> new ServiceUnitStateData(Free, null, + orphanData.state() == Owned ? orphanData.dstBroker() : orphanData.sourceBroker(), + true, + version))) + .thenCompose(override -> { + log.info( + "Overriding inactiveBroker:{}, ownership serviceUnit:{} from orphanData:{} to " + + "overrideData:{}", + inactiveBroker, serviceUnit, orphanData, override); + return publishOverrideEventAsync(serviceUnit, override); + }).get(config.getMetadataStoreOperationTimeoutSeconds(), SECONDS); + } catch (Throwable e) { + log.error( + "Failed to override inactiveBroker:{} ownership serviceUnit:{} orphanData:{}. " + + "totalCleanupErrorCnt:{}", + inactiveBroker, serviceUnit, orphanData, totalCleanupErrorCnt.incrementAndGet(), e); + } } private void waitForCleanups(String broker, boolean excludeSystemTopics, int maxWaitTimeInMillis) { @@ -1440,60 +1458,13 @@ private synchronized void doCleanup(String broker) { } - private Optional selectBroker(String serviceUnit, String inactiveBroker) { - try { - return loadManager.selectAsync( - LoadManagerShared.getNamespaceBundle(pulsar, serviceUnit), - Set.of(inactiveBroker), LookupOptions.builder().build()) - .get(inFlightStateWaitingTimeInMillis, MILLISECONDS); - } catch (Throwable e) { - log.error("Failed to select a broker for serviceUnit:{}", serviceUnit); - } - return Optional.empty(); + private CompletableFuture> selectBroker(String serviceUnit, String inactiveBroker) { + return getLoadManager().selectAsync( + LoadManagerShared.getNamespaceBundle(pulsar, serviceUnit), + inactiveBroker == null ? Set.of() : Set.of(inactiveBroker), + LookupOptions.builder().build()); } - private Optional getRollForwardStateData(String serviceUnit, - String inactiveBroker, - long nextVersionId) { - Optional selectedBroker = selectBroker(serviceUnit, inactiveBroker); - if (selectedBroker.isEmpty()) { - return Optional.empty(); - } - return Optional.of(new ServiceUnitStateData(Owned, selectedBroker.get(), true, nextVersionId)); - } - - - private Optional getOverrideInFlightStateData( - String serviceUnit, ServiceUnitStateData orphanData, - Set availableBrokers) { - long nextVersionId = getNextVersionId(orphanData); - var state = orphanData.state(); - switch (state) { - case Assigning: { - return getRollForwardStateData(serviceUnit, orphanData.dstBroker(), nextVersionId); - } - case Splitting: { - return Optional.of(new ServiceUnitStateData(Splitting, - orphanData.dstBroker(), orphanData.sourceBroker(), - Map.copyOf(orphanData.splitServiceUnitToDestBroker()), - true, nextVersionId)); - } - case Releasing: { - if (availableBrokers.contains(orphanData.sourceBroker())) { - // rollback to the src - return Optional.of(new ServiceUnitStateData(Owned, orphanData.sourceBroker(), true, nextVersionId)); - } else { - return getRollForwardStateData(serviceUnit, orphanData.sourceBroker(), nextVersionId); - } - } - default: { - var msg = String.format("Failed to get the overrideStateData from serviceUnit=%s, orphanData=%s", - serviceUnit, orphanData); - log.error(msg); - throw new IllegalStateException(msg); - } - } - } @VisibleForTesting protected void monitorOwnerships(List brokers) { @@ -1521,7 +1492,7 @@ protected void monitorOwnerships(List brokers) { long startTime = System.nanoTime(); Set inactiveBrokers = new HashSet<>(); Set activeBrokers = new HashSet<>(brokers); - Map orphanServiceUnits = new HashMap<>(); + Map timedOutInFlightStateServiceUnits = new HashMap<>(); int serviceUnitTombstoneCleanupCnt = 0; int orphanServiceUnitCleanupCnt = 0; long totalCleanupErrorCntStart = totalCleanupErrorCnt.get(); @@ -1533,20 +1504,27 @@ protected void monitorOwnerships(List brokers) { String srcBroker = stateData.sourceBroker(); var state = stateData.state(); - if (isActiveState(state) && StringUtils.isNotBlank(srcBroker) && !activeBrokers.contains(srcBroker)) { + if (state == Owned && (StringUtils.isBlank(dstBroker) || !activeBrokers.contains(dstBroker))) { + inactiveBrokers.add(dstBroker); + continue; + } + + if (isInFlightState(state) && StringUtils.isNotBlank(srcBroker) && !activeBrokers.contains(srcBroker)) { inactiveBrokers.add(srcBroker); continue; } - if (isActiveState(state) && StringUtils.isNotBlank(dstBroker) && !activeBrokers.contains(dstBroker)) { + if (isInFlightState(state) && StringUtils.isNotBlank(dstBroker) && !activeBrokers.contains(dstBroker)) { inactiveBrokers.add(dstBroker); continue; } - if (isActiveState(state) && isInFlightState(state) + + if (isInFlightState(state) && now - stateData.timestamp() > inFlightStateWaitingTimeInMillis) { - orphanServiceUnits.put(serviceUnit, stateData); + timedOutInFlightStateServiceUnits.put(serviceUnit, stateData); continue; } + if (!isActiveState(state) && now - stateData.timestamp() > stateTombstoneDelayTimeInMillis) { log.info("Found semi-terminal states to tombstone" + " serviceUnit:{}, stateData:{}", serviceUnit, stateData); @@ -1562,37 +1540,21 @@ protected void monitorOwnerships(List brokers) { } } - // Skip cleaning orphan bundles if inactiveBrokers exist. This is a bigger problem. + if (!inactiveBrokers.isEmpty()) { for (String inactiveBroker : inactiveBrokers) { handleBrokerDeletionEvent(inactiveBroker); } - } else if (!orphanServiceUnits.isEmpty()) { - for (var etr : orphanServiceUnits.entrySet()) { + } + + // timedOutInFlightStateServiceUnits are the in-flight ones although their src and dst brokers are known to + // be active. + if (!timedOutInFlightStateServiceUnits.isEmpty()) { + for (var etr : timedOutInFlightStateServiceUnits.entrySet()) { var orphanServiceUnit = etr.getKey(); var orphanData = etr.getValue(); - var overrideData = getOverrideInFlightStateData( - orphanServiceUnit, orphanData, activeBrokers); - if (overrideData.isPresent()) { - log.info("Overriding in-flight state ownership serviceUnit:{} " - + "from orphanData:{} to overrideData:{}", - orphanServiceUnit, orphanData, overrideData); - publishOverrideEventAsync(orphanServiceUnit, orphanData, overrideData.get()) - .whenComplete((__, e) -> { - if (e != null) { - log.error("Failed cleaning the ownership orphanServiceUnit:{}, orphanData:{}, " - + "cleanupErrorCnt:{}.", - orphanServiceUnit, orphanData, - totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart, e); - } - }); - orphanServiceUnitCleanupCnt++; - } else { - log.warn("Failed get the overrideStateData from orphanServiceUnit:{}, orphanData:{}," - + " cleanupErrorCnt:{}. will retry..", - orphanServiceUnit, orphanData, - totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart); - } + overrideOwnership(orphanServiceUnit, orphanData, null); + orphanServiceUnitCleanupCnt++; } } 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 c62da22ac6827..5dec15fc19b89 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 @@ -2282,6 +2282,15 @@ private CompletableFuture unloadServiceUnit(NamespaceBundle serviceUnit topics.forEach((name, topicFuture) -> { TopicName topicName = TopicName.get(name); if (serviceUnit.includes(topicName)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar) + && ExtensibleLoadManagerImpl.isInternalTopic(topicName.toString())) { + if (ExtensibleLoadManagerImpl.debug(pulsar.getConfiguration(), log)) { + log.info("[{}] Skip unloading ExtensibleLoadManager internal topics. Such internal topic " + + "should be closed when shutting down the broker.", topicName); + } + return; + } + // Topic needs to be unloaded log.info("[{}] Unloading topic", topicName); if (topicFuture.isCompletedExceptionally()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java index 837aceca1416f..aef68aff9a262 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java @@ -110,6 +110,7 @@ public class ServiceUnitStateChannelTest extends MockedPulsarServiceBaseTest { private ServiceUnitStateChannel channel2; private String brokerId1; private String brokerId2; + private String brokerId3; private String bundle; private String bundle1; private String bundle2; @@ -161,6 +162,7 @@ protected void setup() throws Exception { FieldUtils.readDeclaredField(channel1, "brokerId", true); brokerId2 = (String) FieldUtils.readDeclaredField(channel2, "brokerId", true); + brokerId3 = "broker-3"; bundle = "public/default/0x00000000_0xffffffff"; bundle1 = "public/default/0x00000000_0xfffffff0"; @@ -1235,7 +1237,8 @@ public void splitTestWhenProducerFails() var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; - + doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) + .when(loadManager).selectAsync(any(), any(), any()); waitUntilStateWithMonitor(leader, bundle, Init); waitUntilStateWithMonitor(channel1, bundle, Init); waitUntilStateWithMonitor(channel2, bundle, Init); @@ -1426,6 +1429,8 @@ public void splitAndRetryFailureTest() throws Exception { assertEquals(3, count.get()); }); var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; + doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) + .when(loadManager).selectAsync(any(), any(), any()); ((ServiceUnitStateChannelImpl) leader) .monitorOwnerships(List.of(brokerId1, brokerId2)); @@ -1569,26 +1574,40 @@ public void testOverrideOrphanStateData() String broker = brokerId1; // test override states - String releasingBundle = "public/releasing/0xfffffff0_0xffffffff"; + String releasingBundle1 = "public/releasing1/0xfffffff0_0xffffffff"; + String releasingBundle2 = "public/releasing2/0xfffffff0_0xffffffff"; String splittingBundle = bundle; - String assigningBundle = "public/assigning/0xfffffff0_0xffffffff"; + String assigningBundle1 = "public/assigning1/0xfffffff0_0xffffffff"; + String assigningBundle2 = "public/assigning2/0xfffffff0_0xffffffff"; String freeBundle = "public/free/0xfffffff0_0xffffffff"; String deletedBundle = "public/deleted/0xfffffff0_0xffffffff"; - String ownedBundle = "public/owned/0xfffffff0_0xffffffff"; - overrideTableViews(releasingBundle, - new ServiceUnitStateData(Releasing, null, broker, 1)); + String ownedBundle1 = "public/owned1/0xfffffff0_0xffffffff"; + String ownedBundle2 = "public/owned2SourceBundle/0xfffffff0_0xffffffff"; + String ownedBundle3 = "public/owned3/0xfffffff0_0xffffffff"; + String inactiveBroker = "broker-inactive-1"; + overrideTableViews(releasingBundle1, + new ServiceUnitStateData(Releasing, broker, brokerId2, 1)); + overrideTableViews(releasingBundle2, + new ServiceUnitStateData(Releasing, brokerId2, brokerId3, 1)); overrideTableViews(splittingBundle, new ServiceUnitStateData(Splitting, null, broker, Map.of(childBundle1Range, Optional.empty(), childBundle2Range, Optional.empty()), 1)); - overrideTableViews(assigningBundle, + overrideTableViews(assigningBundle1, new ServiceUnitStateData(Assigning, broker, null, 1)); + overrideTableViews(assigningBundle2, + new ServiceUnitStateData(Assigning, broker, brokerId2, 1)); overrideTableViews(freeBundle, new ServiceUnitStateData(Free, null, broker, 1)); overrideTableViews(deletedBundle, new ServiceUnitStateData(Deleted, null, broker, 1)); - overrideTableViews(ownedBundle, + overrideTableViews(ownedBundle1, new ServiceUnitStateData(Owned, broker, null, 1)); + overrideTableViews(ownedBundle2, + new ServiceUnitStateData(Owned, broker, inactiveBroker, 1)); + overrideTableViews(ownedBundle3, + new ServiceUnitStateData(Owned, inactiveBroker, broker, 1)); + // test stable metadata state doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) @@ -1598,16 +1617,33 @@ public void testOverrideOrphanStateData() FieldUtils.writeDeclaredField(followerChannel, "inFlightStateWaitingTimeInMillis", -1, true); ((ServiceUnitStateChannelImpl) leaderChannel) - .monitorOwnerships(List.of(brokerId1, brokerId2)); + .monitorOwnerships(List.of(brokerId1, brokerId2, "broker-3")); - waitUntilNewOwner(channel2, releasingBundle, broker); - waitUntilNewOwner(channel2, childBundle11, broker); - waitUntilNewOwner(channel2, childBundle12, broker); - waitUntilNewOwner(channel2, assigningBundle, brokerId2); - waitUntilNewOwner(channel2, ownedBundle, broker); - assertEquals(Optional.empty(), channel2.getOwnerAsync(freeBundle).get()); - assertTrue(channel2.getOwnerAsync(deletedBundle).isCompletedExceptionally()); + ServiceUnitStateChannel finalLeaderChannel = leaderChannel; + Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> getCleanupJobs(finalLeaderChannel).isEmpty()); + + + waitUntilNewOwner(channel2, releasingBundle1, brokerId2); + waitUntilNewOwner(channel2, releasingBundle2, brokerId2); assertTrue(channel2.getOwnerAsync(splittingBundle).get().isEmpty()); + waitUntilNewOwner(channel2, childBundle11, brokerId2); + waitUntilNewOwner(channel2, childBundle12, brokerId2); + waitUntilNewOwner(channel2, assigningBundle1, brokerId2); + waitUntilNewOwner(channel2, assigningBundle2, brokerId2); + assertTrue(channel2.getOwnerAsync(freeBundle).get().isEmpty()); + assertTrue(channel2.getOwnerAsync(deletedBundle).isCompletedExceptionally()); + waitUntilNewOwner(channel2, ownedBundle1, broker); + waitUntilNewOwner(channel2, ownedBundle2, broker); + waitUntilNewOwner(channel2, ownedBundle3, brokerId2); + + validateMonitorCounters(leaderChannel, + 1, + 0, + 6, + 0, + 1, + 0, + 0); // clean-up FieldUtils.writeDeclaredField(channel1, From c7310e35d274541031907cccbf61a8817ac87ec0 Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Thu, 25 Jul 2024 10:55:28 +0800 Subject: [PATCH 391/580] [improve] [broker] Add subscription prefix for internal reader (#23044) ### Motivation We have many system topics, such as __change_events __transaction_buffer_snapshot __transaction_buffer_snapshot_indexes __transaction_buffer_snapshot_segments transaction_coordinator_assign _transaction_log __transaction_pending_ack In Pulsar Broker, we create an internal reader to fetch messages from those system topics. Due to we do not specify the subscription prefix, the reader will generate a random subscription name for each reader. In PIP-355, we introduced a broker-level metric named pulsar_broker_out_bytes_total, which separate the system subscription traffic bytes and user subscription traffic bytes. Due to the internal readers don't have a subscription prefix, we group the internal reader's traffic bytes into user subscription traffic. ### Modifications In this PR, we introduce a system subscription prefix named __system_reader and group the internal reader's traffic into system subscription traffic bytes in metric pulsar_broker_out_bytes_total. --- .../nonpersistent/NonPersistentTopic.java | 6 ++ .../service/persistent/PersistentTopic.java | 5 +- .../TopicPoliciesSystemTopicClient.java | 2 + ...onBufferSnapshotBaseSystemTopicClient.java | 2 + .../broker/stats/PrometheusMetricsTest.java | 70 ++++++++++++------- .../common/naming/SystemTopicNames.java | 5 ++ 6 files changed, 63 insertions(+), 27 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 3801ac7f3ee82..9456870589191 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -76,6 +76,7 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.KeySharedMeta; +import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.ClusterPolicies.ClusterUrl; @@ -1213,6 +1214,11 @@ public CompletableFuture unsubscribe(String subscriptionName) { SubscriptionStatsImpl stats = sub.getStats(getStatsOptions); bytesOutFromRemovedSubscriptions.add(stats.bytesOutCounter); msgOutFromRemovedSubscriptions.add(stats.msgOutCounter); + + if (isSystemCursor(subscriptionName) + || subscriptionName.startsWith(SystemTopicNames.SYSTEM_READER_PREFIX)) { + bytesOutFromRemovedSystemSubscriptions.add(stats.bytesOutCounter); + } } }, brokerService.executor()); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 07deb1168072a..3587dab775547 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -1410,7 +1410,8 @@ void removeSubscription(String subscriptionName) { bytesOutFromRemovedSubscriptions.add(stats.bytesOutCounter); msgOutFromRemovedSubscriptions.add(stats.msgOutCounter); - if (isSystemCursor(subscriptionName)) { + if (isSystemCursor(subscriptionName) + || subscriptionName.startsWith(SystemTopicNames.SYSTEM_READER_PREFIX)) { bytesOutFromRemovedSystemSubscriptions.add(stats.bytesOutCounter); } } @@ -2637,7 +2638,7 @@ public CompletableFuture asyncGetStats(GetStatsOptions topicMetricBean.value += v.value; }); - if (isSystemCursor(name)) { + if (isSystemCursor(name) || name.startsWith(SystemTopicNames.SYSTEM_READER_PREFIX)) { stats.bytesOutInternalCounter += subStats.bytesOutCounter; } }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java index b7cff2e08c2d0..ea3ac507d1128 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TopicPoliciesSystemTopicClient.java @@ -34,6 +34,7 @@ import org.apache.pulsar.client.internal.DefaultImplementation; import org.apache.pulsar.common.events.ActionType; import org.apache.pulsar.common.events.PulsarEvent; +import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,6 +70,7 @@ protected CompletableFuture> newWriterAsyncInternal() { protected CompletableFuture> newReaderAsyncInternal() { return client.newReader(avroSchema) .topic(topicName.toString()) + .subscriptionRolePrefix(SystemTopicNames.SYSTEM_READER_PREFIX) .startMessageId(MessageId.earliest) .readCompacted(true) .createAsync() diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java index 8efa983a64d73..4023cd88bef55 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/systopic/TransactionBufferSnapshotBaseSystemTopicClient.java @@ -28,6 +28,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; @Slf4j @@ -201,6 +202,7 @@ protected CompletableFuture> newWriterAsyncInternal() { protected CompletableFuture> newReaderAsyncInternal() { return client.newReader(Schema.AVRO(schemaType)) .topic(topicName.toString()) + .subscriptionRolePrefix(SystemTopicNames.SYSTEM_READER_PREFIX) .startMessageId(MessageId.earliest) .readCompacted(true) .createAsync() diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index 0d7f8eb0aa3e8..81c0acba44046 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -30,6 +30,7 @@ import static org.testng.Assert.fail; import com.google.common.base.Splitter; import com.google.common.collect.Multimap; +import com.google.common.collect.Sets; import io.jsonwebtoken.SignatureAlgorithm; import io.prometheus.client.Collector; import java.io.ByteArrayOutputStream; @@ -87,6 +88,9 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.compaction.Compactor; import org.apache.pulsar.compaction.PulsarCompactionServiceFactory; import org.apache.zookeeper.CreateMode; @@ -209,26 +213,31 @@ public void testPublishRateLimitedTimes() throws Exception { public void testBrokerMetrics() throws Exception { cleanup(); conf.setAdditionalSystemCursorNames(Set.of("test-cursor")); + conf.setTopicLevelPoliciesEnabled(true); + conf.setSystemTopicEnabled(true); setup(); - Producer p1 = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic1").create(); - Producer p2 = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic2").create(); + admin.tenants().createTenant("test-tenant", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); + admin.namespaces().createNamespace("test-tenant/test-ns", 4); + Producer p1 = pulsarClient.newProducer().topic("persistent://test-tenant/test-ns/my-topic1").create(); + Producer p2 = pulsarClient.newProducer().topic("persistent://test-tenant/test-ns/my-topic2").create(); // system topic - Producer p3 = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/__change_events").create(); + Producer p3 = pulsarClient.newProducer().topic("persistent://test-tenant/test-ns/__test-topic").create(); Consumer c1 = pulsarClient.newConsumer() - .topic("persistent://my-property/use/my-ns/my-topic1") + .topic("persistent://test-tenant/test-ns/my-topic1") .subscriptionName("test") .subscribe(); // additional system cursor Consumer c2 = pulsarClient.newConsumer() - .topic("persistent://my-property/use/my-ns/my-topic2") + .topic("persistent://test-tenant/test-ns/my-topic2") .subscriptionName("test-cursor") .subscribe(); Consumer c3 = pulsarClient.newConsumer() - .topic("persistent://my-property/use/my-ns/__change_events") + .topic("persistent://test-tenant/test-ns/__test-topic") .subscriptionName("test-v1") .subscribe(); @@ -250,7 +259,8 @@ public void testBrokerMetrics() throws Exception { c1.unsubscribe(); c2.unsubscribe(); - //admin.topics().unload("persistent://my-property/use/my-ns/my-topic1"); + admin.topicPolicies().setRetention("persistent://test-tenant/test-ns/my-topic2", + new RetentionPolicies(60, 1024)); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); @@ -263,33 +273,43 @@ public void testBrokerMetrics() throws Exception { List bytesOutTotal = (List) metrics.get("pulsar_broker_out_bytes_total"); List bytesInTotal = (List) metrics.get("pulsar_broker_in_bytes_total"); + List topicLevelBytesOutTotal = (List) metrics.get("pulsar_out_bytes_total"); + assertEquals(bytesOutTotal.size(), 2); assertEquals(bytesInTotal.size(), 2); + assertEquals(topicLevelBytesOutTotal.size(), 3); double systemOutBytes = 0.0; double userOutBytes = 0.0; - switch (bytesOutTotal.get(0).tags.get("system_subscription").toString()) { - case "true": - systemOutBytes = bytesOutTotal.get(0).value; - userOutBytes = bytesOutTotal.get(1).value; - case "false": - systemOutBytes = bytesOutTotal.get(1).value; - userOutBytes = bytesOutTotal.get(0).value; - } - double systemInBytes = 0.0; double userInBytes = 0.0; - switch (bytesInTotal.get(0).tags.get("system_topic").toString()) { - case "true": - systemInBytes = bytesInTotal.get(0).value; - userInBytes = bytesInTotal.get(1).value; - case "false": - systemInBytes = bytesInTotal.get(1).value; - userInBytes = bytesInTotal.get(0).value; + + for (Metric metric : bytesOutTotal) { + if (metric.tags.get("system_subscription").equals("true")) { + systemOutBytes = metric.value; + } else { + userOutBytes = metric.value; + } + } + + for (Metric metric : bytesInTotal) { + if (metric.tags.get("system_topic").equals("true")) { + systemInBytes = metric.value; + } else { + userInBytes = metric.value; + } + } + + double systemCursorOutBytes = 0.0; + for (Metric metric : topicLevelBytesOutTotal) { + if (metric.tags.get("subscription").startsWith(SystemTopicNames.SYSTEM_READER_PREFIX) + || metric.tags.get("subscription").equals(Compactor.COMPACTION_SUBSCRIPTION)) { + systemCursorOutBytes = metric.value; + } } - assertEquals(userOutBytes / 2, systemOutBytes); - assertEquals(userInBytes / 2, systemInBytes); + assertEquals(systemCursorOutBytes, systemInBytes); + assertEquals(userOutBytes / 2, systemOutBytes - systemCursorOutBytes); assertEquals(userOutBytes + systemOutBytes, userInBytes + systemInBytes); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java index 716d9bc31facb..9a3689912c926 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/SystemTopicNames.java @@ -51,6 +51,11 @@ public class SystemTopicNames { public static final String PENDING_ACK_STORE_CURSOR_NAME = "__pending_ack_state"; + /** + * Prefix for the system reader for all the system topics. + */ + public static final String SYSTEM_READER_PREFIX = "__system_reader"; + /** * The set of all local topic names declared above. */ From 47f204fb20d324d6794e3b57e06807af66b799bc Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Fri, 26 Jul 2024 11:40:33 +0800 Subject: [PATCH 392/580] [improve][pip] PIP-366: Support to specify different config for Configuration and Local Metadata Store (#23033) --- pip/pip-366.md | 70 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 70 insertions(+) create mode 100644 pip/pip-366.md diff --git a/pip/pip-366.md b/pip/pip-366.md new file mode 100644 index 0000000000000..78e7ad60de9b0 --- /dev/null +++ b/pip/pip-366.md @@ -0,0 +1,70 @@ +# PIP-366: Support to specify different config for Configuration and Local Metadata Store + +# Background knowledge + +Pulsar metadata store maintains all the metadata, configuration, and coordination of a Pulsar cluster, such as topic metadata, schema, broker load data, and so on. + +The metadata store of each Pulsar instance should contain the following two components: + +- A local metadata store ensemble (`metadataStoreUrl`) that stores cluster-specific configuration and coordination, such as which brokers are responsible for which topics as well as ownership metadata, broker load reports, and BookKeeper ledger metadata. +- A configuration store quorum (`configurationMetadataStoreUrl`) stores configuration for clusters, tenants, namespaces, topics, and other entities that need to be globally consistent. + +# Motivation + +When using Geo-Replication and global configuration store for configuration global consistency, the configuration store's config may be different from the local metadata store's config. For example, the configuration store may have a different set of ZooKeeper servers than the local metadata store. + +The global configuration store may deploy in a different data center, and the local metadata store may be deployed in the same data center as the Pulsar broker. In this case, the global configuration store may need to use TLS and authentication to protect the connection to metadata store server, while the local metadata store may not need to use TLS and authentication. + +However, the current implementation of Pulsar only supports configuring different metadata store url for the local metadata store and the configuration store. This limitation makes it impossible to support the above scenario. + +# Goals + +## In Scope + +- Support specifying different configurations for the local metadata store and the configuration store. + +# Detailed Design + +## Design & Implementation Details + +Pulsar support `metadataStoreConfigPath` configuration, but it only supports for `RocksdbMetadataStore`, and it is not able to specify different configuration for Configuration Metadata Store. + +```java + @FieldContext( + category = CATEGORY_SERVER, + doc = "Configuration file path for local metadata store. It's supported by RocksdbMetadataStore for now." + ) + private String metadataStoreConfigPath = null; +``` + +Therefore, we need to add a new configuration `configurationStoreConfigPath` for `ConfigurationMetadataStore`, and the `metadataStoreConfigPath` will be still use for `LocalMetadataStore`. + +```java + @FieldContext( + category = CATEGORY_SERVER, + doc = "Configuration file path for configuration metadata store." + ) + private String configurationStoreConfigPath = null; +``` + +When the `configurationStoreConfigPath` are not set, the `metadataStoreConfigPath` will be used as the configuration file path for the configuration store. + +For each metadata store implementation, we need pass the corresponding configuration file path to the metadata store. For example, for ZKMetadataStore, we can specify config when create the Zookeeper client. + +```java + protected ZooKeeper createZooKeeper() throws IOException { + return new ZooKeeper(connectString, sessionTimeoutMs, watcherManager, allowReadOnlyMode, /** Add the config here **/ new ZKClientConfig(configPath)); + } +``` + +# Backward & Forward Compatibility + +Fully compatible. + +# Links + + +* Mailing List discussion thread: https://lists.apache.org/thread/98ggo1zg1k7dbyx8wr9bc8onm10p16c6 +* Mailing List voting thread: https://lists.apache.org/thread/wm30dy9bkhxxmmcb0v9ftb56ckpknrfr From d4bbf10f58771e2d43e576dc3422e502834b1de4 Mon Sep 17 00:00:00 2001 From: Hideaki Oguni <22386882+izumo27@users.noreply.github.com> Date: Mon, 29 Jul 2024 16:29:59 +0900 Subject: [PATCH 393/580] [fix][client] Fix negative acknowledgement by messageId (#23060) --- .../apache/pulsar/client/impl/NegativeAcksTest.java | 13 ++++++++----- .../org/apache/pulsar/client/impl/ConsumerImpl.java | 2 +- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java index a6b77a1c72775..a41b7f05a8eb3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java @@ -134,7 +134,7 @@ public void testNegativeAcks(boolean batching, boolean usePartitions, Subscripti Set sentMessages = new HashSet<>(); final int N = 10; - for (int i = 0; i < N; i++) { + for (int i = 0; i < N * 2; i++) { String value = "test-" + i; producer.sendAsync(value); sentMessages.add(value); @@ -146,13 +146,18 @@ public void testNegativeAcks(boolean batching, boolean usePartitions, Subscripti consumer.negativeAcknowledge(msg); } + for (int i = 0; i < N; i++) { + Message msg = consumer.receive(); + consumer.negativeAcknowledge(msg.getMessageId()); + } + assertTrue(consumer instanceof ConsumerBase); assertEquals(((ConsumerBase) consumer).getUnAckedMessageTracker().size(), 0); Set receivedMessages = new HashSet<>(); // All the messages should be received again - for (int i = 0; i < N; i++) { + for (int i = 0; i < N * 2; i++) { Message msg = consumer.receive(); receivedMessages.add(msg.getValue()); consumer.acknowledge(msg); @@ -310,9 +315,7 @@ public void testNegativeAcksDeleteFromUnackedTracker() throws Exception { assertEquals(unAckedMessageTracker.size(), 0); negativeAcksTracker.close(); // negative batch message id - unAckedMessageTracker.add(batchMessageId); - unAckedMessageTracker.add(batchMessageId2); - unAckedMessageTracker.add(batchMessageId3); + unAckedMessageTracker.add(messageId); consumer.negativeAcknowledge(batchMessageId); consumer.negativeAcknowledge(batchMessageId2); consumer.negativeAcknowledge(batchMessageId3); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 6ddb0e1bc01db..1806d13493b2f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -811,7 +811,7 @@ public void negativeAcknowledge(MessageId messageId) { negativeAcksTracker.add(messageId); // Ensure the message is not redelivered for ack-timeout, since we did receive an "ack" - unAckedMessageTracker.remove(messageId); + unAckedMessageTracker.remove(MessageIdAdvUtils.discardBatch(messageId)); } @Override From e59cd05881bff11e4b127ed3496a02a0ce697fb7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 29 Jul 2024 18:47:33 +0800 Subject: [PATCH 394/580] [fix] [broker] Remove blocking calls from Subscription.getStats (#23088) --- .../persistent/PersistentSubscription.java | 51 +++++++---- .../service/persistent/PersistentTopic.java | 86 ++++++++++--------- 2 files changed, 80 insertions(+), 57 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 77aa5f82c3914..2dd890cfd2942 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -32,6 +32,8 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -1200,7 +1202,26 @@ public long estimateBacklogSize() { return cursor.getEstimatedSizeSinceMarkDeletePosition(); } + /** + * @deprecated please call {@link #getStatsAsync(GetStatsOptions)}. + */ + @Deprecated public SubscriptionStatsImpl getStats(GetStatsOptions getStatsOptions) { + // So far, there is no case hits this check. + if (getStatsOptions.isGetEarliestTimeInBacklog()) { + throw new IllegalArgumentException("Calling the sync method subscription.getStats with" + + " getEarliestTimeInBacklog, it may encountered a deadlock error."); + } + // The method "getStatsAsync" will be a sync method if the param "isGetEarliestTimeInBacklog" is false. + try { + return getStatsAsync(getStatsOptions).get(5, TimeUnit.SECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + // This error will never occur. + throw new RuntimeException(e); + } + } + + public CompletableFuture getStatsAsync(GetStatsOptions getStatsOptions) { SubscriptionStatsImpl subStats = new SubscriptionStatsImpl(); subStats.lastExpireTimestamp = lastExpireTimestamp; subStats.lastConsumedFlowTimestamp = lastConsumedFlowTimestamp; @@ -1273,21 +1294,6 @@ public SubscriptionStatsImpl getStats(GetStatsOptions getStatsOptions) { } else { subStats.backlogSize = -1; } - if (getStatsOptions.isGetEarliestTimeInBacklog()) { - if (subStats.msgBacklog > 0) { - ManagedLedgerImpl managedLedger = ((ManagedLedgerImpl) cursor.getManagedLedger()); - Position markDeletedPosition = cursor.getMarkDeletedPosition(); - long result = 0; - try { - result = managedLedger.getEarliestMessagePublishTimeOfPos(markDeletedPosition).get(); - } catch (InterruptedException | ExecutionException e) { - result = -1; - } - subStats.earliestMsgPublishTimeInBacklog = result; - } else { - subStats.earliestMsgPublishTimeInBacklog = -1; - } - } subStats.msgBacklogNoDelayed = subStats.msgBacklog - subStats.msgDelayed; subStats.msgRateExpired = expiryMonitor.getMessageExpiryRate(); subStats.totalMsgExpired = expiryMonitor.getTotalMessageExpired(); @@ -1329,7 +1335,20 @@ public SubscriptionStatsImpl getStats(GetStatsOptions getStatsOptions) { subStats.nonContiguousDeletedMessagesRanges = cursor.getTotalNonContiguousDeletedMessagesRange(); subStats.nonContiguousDeletedMessagesRangesSerializedSize = cursor.getNonContiguousDeletedMessagesRangeSerializedSize(); - return subStats; + if (!getStatsOptions.isGetEarliestTimeInBacklog()) { + return CompletableFuture.completedFuture(subStats); + } + if (subStats.msgBacklog > 0) { + ManagedLedgerImpl managedLedger = ((ManagedLedgerImpl) cursor.getManagedLedger()); + Position markDeletedPosition = cursor.getMarkDeletedPosition(); + return managedLedger.getEarliestMessagePublishTimeOfPos(markDeletedPosition).thenApply(v -> { + subStats.earliestMsgPublishTimeInBacklog = v; + return subStats; + }); + } else { + subStats.earliestMsgPublishTimeInBacklog = -1; + return CompletableFuture.completedFuture(subStats); + } } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 3587dab775547..42487d7239cc6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -2584,7 +2584,6 @@ public CompletableFuture asyncGetStats(boolean getPreciseBacklog @Override public CompletableFuture asyncGetStats(GetStatsOptions getStatsOptions) { - CompletableFuture statsFuture = new CompletableFuture<>(); TopicStatsImpl stats = new TopicStatsImpl(); ObjectObjectHashMap remotePublishersStats = new ObjectObjectHashMap<>(); @@ -2617,32 +2616,6 @@ public CompletableFuture asyncGetStats(GetStatsOptions stats.abortedTxnCount = txnBuffer.getAbortedTxnCount(); stats.committedTxnCount = txnBuffer.getCommittedTxnCount(); - subscriptions.forEach((name, subscription) -> { - SubscriptionStatsImpl subStats = subscription.getStats(getStatsOptions); - - stats.msgRateOut += subStats.msgRateOut; - stats.msgThroughputOut += subStats.msgThroughputOut; - stats.bytesOutCounter += subStats.bytesOutCounter; - stats.msgOutCounter += subStats.msgOutCounter; - stats.subscriptions.put(name, subStats); - stats.nonContiguousDeletedMessagesRanges += subStats.nonContiguousDeletedMessagesRanges; - stats.nonContiguousDeletedMessagesRangesSerializedSize += - subStats.nonContiguousDeletedMessagesRangesSerializedSize; - stats.delayedMessageIndexSizeInBytes += subStats.delayedMessageIndexSizeInBytes; - - subStats.bucketDelayedIndexStats.forEach((k, v) -> { - TopicMetricBean topicMetricBean = - stats.bucketDelayedIndexStats.computeIfAbsent(k, __ -> new TopicMetricBean()); - topicMetricBean.name = v.name; - topicMetricBean.labelsAndValues = v.labelsAndValues; - topicMetricBean.value += v.value; - }); - - if (isSystemCursor(name) || name.startsWith(SystemTopicNames.SYSTEM_READER_PREFIX)) { - stats.bytesOutInternalCounter += subStats.bytesOutCounter; - } - }); - replicators.forEach((cluster, replicator) -> { ReplicatorStatsImpl replicatorStats = replicator.computeStats(); @@ -2692,21 +2665,52 @@ public CompletableFuture asyncGetStats(GetStatsOptions return compactionRecord; }); - if (getStatsOptions.isGetEarliestTimeInBacklog() && stats.backlogSize != 0) { - ledger.getEarliestMessagePublishTimeInBacklog().whenComplete((earliestTime, e) -> { - if (e != null) { - log.error("[{}] Failed to get earliest message publish time in backlog", topic, e); - statsFuture.completeExceptionally(e); - } else { - stats.earliestMsgPublishTimeInBacklogs = earliestTime; - statsFuture.complete(stats); - } - }); - } else { - statsFuture.complete(stats); - } + Map> subscriptionFutures = new HashMap<>(); + subscriptions.forEach((name, subscription) -> { + subscriptionFutures.put(name, subscription.getStatsAsync(getStatsOptions)); + }); + return FutureUtil.waitForAll(subscriptionFutures.values()).thenCompose(ignore -> { + for (Map.Entry> e : subscriptionFutures.entrySet()) { + String name = e.getKey(); + SubscriptionStatsImpl subStats = e.getValue().join(); + stats.msgRateOut += subStats.msgRateOut; + stats.msgThroughputOut += subStats.msgThroughputOut; + stats.bytesOutCounter += subStats.bytesOutCounter; + stats.msgOutCounter += subStats.msgOutCounter; + stats.subscriptions.put(name, subStats); + stats.nonContiguousDeletedMessagesRanges += subStats.nonContiguousDeletedMessagesRanges; + stats.nonContiguousDeletedMessagesRangesSerializedSize += + subStats.nonContiguousDeletedMessagesRangesSerializedSize; + stats.delayedMessageIndexSizeInBytes += subStats.delayedMessageIndexSizeInBytes; + + subStats.bucketDelayedIndexStats.forEach((k, v) -> { + TopicMetricBean topicMetricBean = + stats.bucketDelayedIndexStats.computeIfAbsent(k, ignore2 -> new TopicMetricBean()); + topicMetricBean.name = v.name; + topicMetricBean.labelsAndValues = v.labelsAndValues; + topicMetricBean.value += v.value; + }); - return statsFuture; + if (isSystemCursor(name) || name.startsWith(SystemTopicNames.SYSTEM_READER_PREFIX)) { + stats.bytesOutInternalCounter += subStats.bytesOutCounter; + } + } + if (getStatsOptions.isGetEarliestTimeInBacklog() && stats.backlogSize != 0) { + CompletableFuture finalRes = ledger.getEarliestMessagePublishTimeInBacklog() + .thenApply((earliestTime) -> { + stats.earliestMsgPublishTimeInBacklogs = earliestTime; + return stats; + }); + // print error log. + finalRes.exceptionally(ex -> { + log.error("[{}] Failed to get earliest message publish time in backlog", topic, ex); + return null; + }); + return finalRes; + } else { + return CompletableFuture.completedFuture(stats); + } + }); } private Optional getCompactorMXBean() { From 679a3d49eefc2a82bbeba085c258b1f2b751f28a Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 29 Jul 2024 18:54:12 +0800 Subject: [PATCH 395/580] [improve] [broker] Check max producers/consumers limitation first before other ops to save resources (#23074) --- .../pulsar/broker/service/AbstractTopic.java | 20 +++-- .../pulsar/broker/service/ServerCnx.java | 19 ++++ .../pulsar/broker/admin/AdminApi2Test.java | 77 ++++++++++++---- .../broker/service/PersistentTopicTest.java | 45 ---------- .../pulsar/client/api/MaxProducerTest.java | 88 +++++++++++++++++++ 5 files changed, 181 insertions(+), 68 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/MaxProducerTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index fbf11f1d0ad62..f25dfef966bfd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -486,8 +486,18 @@ private PublishRate publishRateInBroker(ServiceConfiguration config) { return new PublishRate(config.getMaxPublishRatePerTopicInMessages(), config.getMaxPublishRatePerTopicInBytes()); } + public boolean isProducersExceeded(String producerName) { + String replicatorPrefix = brokerService.getPulsar().getConfig().getReplicatorPrefix() + "."; + boolean isRemote = producerName.startsWith(replicatorPrefix); + return isProducersExceeded(isRemote); + } + protected boolean isProducersExceeded(Producer producer) { - if (isSystemTopic() || producer.isRemote()) { + return isProducersExceeded(producer.isRemote()); + } + + protected boolean isProducersExceeded(boolean isRemote) { + if (isSystemTopic() || isRemote) { return false; } Integer maxProducers = topicPolicies.getMaxProducersPerTopic().get(); @@ -536,7 +546,7 @@ public int getNumberOfSameAddressProducers(final String clientAddress) { return count; } - protected boolean isConsumersExceededOnTopic() { + public boolean isConsumersExceededOnTopic() { if (isSystemTopic()) { return false; } @@ -973,12 +983,6 @@ protected void checkTopicFenced() throws BrokerServiceException { } protected CompletableFuture internalAddProducer(Producer producer) { - if (isProducersExceeded(producer)) { - log.warn("[{}] Attempting to add producer to topic which reached max producers limit", topic); - return CompletableFuture.failedFuture(new BrokerServiceException.ProducerBusyException( - "Topic '" + topic + "' reached max producers limit")); - } - if (isSameAddressProducersExceeded(producer)) { log.warn("[{}] Attempting to add producer to topic which reached max same address producers limit", topic); return CompletableFuture.failedFuture(new BrokerServiceException.ProducerBusyException( 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 6690ab4af5fd1..5df276e8f3dd5 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 @@ -1307,6 +1307,16 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { "Topic " + topicName + " does not exist")); } final Topic topic = optTopic.get(); + // Check max consumer limitation to avoid unnecessary ops wasting resources. For example: + // the new consumer reached max producer limitation, but pulsar did schema check first, + // it would waste CPU. + if (((AbstractTopic) topic).isConsumersExceededOnTopic()) { + log.warn("[{}] Attempting to add consumer to topic which reached max" + + " consumers limit", topic); + Throwable t = + new ConsumerBusyException("Topic reached max consumers limit"); + return FutureUtil.failedFuture(t); + } return service.isAllowAutoSubscriptionCreationAsync(topicName) .thenCompose(isAllowedAutoSubscriptionCreation -> { boolean rejectSubscriptionIfDoesNotExist = isDurable @@ -1545,6 +1555,15 @@ protected void handleProducer(final CommandProducer cmdProducer) { } service.getOrCreateTopic(topicName.toString()).thenCompose((Topic topic) -> { + // Check max producer limitation to avoid unnecessary ops wasting resources. For example: the new + // producer reached max producer limitation, but pulsar did schema check first, it would waste CPU + if (((AbstractTopic) topic).isProducersExceeded(producerName)) { + log.warn("[{}] Attempting to add producer to topic which reached max producers limit", topic); + String errorMsg = "Topic '" + topicName.toString() + "' reached max producers limit"; + Throwable t = new BrokerServiceException.ProducerBusyException(errorMsg); + return CompletableFuture.failedFuture(t); + } + // Before creating producer, check if backlog quota exceeded // on topic for size based limit and time based limit CompletableFuture backlogQuotaCheckFuture = CompletableFuture.allOf( diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java index 249dd3c4607be..40e2ca8cce905 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java @@ -22,6 +22,8 @@ import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; import static org.apache.pulsar.broker.resources.LoadBalanceResources.BUNDLE_DATA_BASE_PATH; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -52,6 +54,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import javax.ws.rs.NotAcceptableException; import javax.ws.rs.core.Response.Status; import lombok.AllArgsConstructor; @@ -70,6 +73,7 @@ import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; +import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -127,7 +131,13 @@ import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; +import org.apache.pulsar.common.protocol.schema.SchemaData; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; @@ -2870,34 +2880,40 @@ public void testMaxProducersPerTopicUnlimited() throws Exception { final String myNamespace = newUniqueName(defaultTenant + "/ns"); admin.namespaces().createNamespace(myNamespace, Set.of("test")); final String topic = "persistent://" + myNamespace + "/testMaxProducersPerTopicUnlimited"; + admin.topics().createNonPartitionedTopic(topic); + AtomicInteger schemaOpsCounter = injectSchemaCheckCounterForTopic(topic); //the policy is set to 0, so there will be no restrictions admin.namespaces().setMaxProducersPerTopic(myNamespace, 0); Awaitility.await().until(() -> admin.namespaces().getMaxProducersPerTopic(myNamespace) == 0); - List> producers = new ArrayList<>(); + List> producers = new ArrayList<>(); for (int i = 0; i < maxProducersPerTopic + 1; i++) { - Producer producer = pulsarClient.newProducer().topic(topic).create(); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); producers.add(producer); } + assertEquals(schemaOpsCounter.get(), maxProducersPerTopic + 1); admin.namespaces().removeMaxProducersPerTopic(myNamespace); Awaitility.await().until(() -> admin.namespaces().getMaxProducersPerTopic(myNamespace) == null); + try { @Cleanup - Producer producer = pulsarClient.newProducer().topic(topic).create(); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); fail("should fail"); } catch (PulsarClientException e) { String expectMsg = "Topic '" + topic + "' reached max producers limit"; assertTrue(e.getMessage().contains(expectMsg)); + assertEquals(schemaOpsCounter.get(), maxProducersPerTopic + 1); } //set the limit to 3 admin.namespaces().setMaxProducersPerTopic(myNamespace, 3); Awaitility.await().until(() -> admin.namespaces().getMaxProducersPerTopic(myNamespace) == 3); // should success - Producer producer = pulsarClient.newProducer().topic(topic).create(); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); producers.add(producer); + assertEquals(schemaOpsCounter.get(), maxProducersPerTopic + 2); try { @Cleanup Producer producer1 = pulsarClient.newProducer().topic(topic).create(); @@ -2905,14 +2921,39 @@ public void testMaxProducersPerTopicUnlimited() throws Exception { } catch (PulsarClientException e) { String expectMsg = "Topic '" + topic + "' reached max producers limit"; assertTrue(e.getMessage().contains(expectMsg)); + assertEquals(schemaOpsCounter.get(), maxProducersPerTopic + 2); } //clean up - for (Producer tempProducer : producers) { + for (Producer tempProducer : producers) { tempProducer.close(); } } + private AtomicInteger injectSchemaCheckCounterForTopic(String topicName) { + ConcurrentOpenHashMap>> topics = + WhiteboxImpl.getInternalState(pulsar.getBrokerService(), "topics"); + AbstractTopic topic = (AbstractTopic) topics.get(topicName).join().get(); + AbstractTopic spyTopic = Mockito.spy(topic); + AtomicInteger counter = new AtomicInteger(); + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + counter.incrementAndGet(); + return invocation.callRealMethod(); + } + }).when(spyTopic).addSchema(any(SchemaData.class)); + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + counter.incrementAndGet(); + return invocation.callRealMethod(); + } + }).when(spyTopic).addSchemaIfIdleOrCheckCompatible(any(SchemaData.class)); + topics.put(topicName, CompletableFuture.completedFuture(Optional.of(spyTopic))); + return counter; + } + @Test public void testMaxConsumersPerTopicUnlimited() throws Exception { restartClusterAfterTest(); @@ -2924,49 +2965,55 @@ public void testMaxConsumersPerTopicUnlimited() throws Exception { final String myNamespace = newUniqueName(defaultTenant + "/ns"); admin.namespaces().createNamespace(myNamespace, Set.of("test")); final String topic = "persistent://" + myNamespace + "/testMaxConsumersPerTopicUnlimited"; + admin.topics().createNonPartitionedTopic(topic); + AtomicInteger schemaOpsCounter = injectSchemaCheckCounterForTopic(topic); assertNull(admin.namespaces().getMaxConsumersPerTopic(myNamespace)); //the policy is set to 0, so there will be no restrictions admin.namespaces().setMaxConsumersPerTopic(myNamespace, 0); Awaitility.await().until(() -> admin.namespaces().getMaxConsumersPerTopic(myNamespace) == 0); - List> consumers = new ArrayList<>(); + List> consumers = new ArrayList<>(); for (int i = 0; i < maxConsumersPerTopic + 1; i++) { - Consumer consumer = - pulsarClient.newConsumer().subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); consumers.add(consumer); } + assertEquals(schemaOpsCounter.get(), maxConsumersPerTopic + 2); admin.namespaces().removeMaxConsumersPerTopic(myNamespace); Awaitility.await().until(() -> admin.namespaces().getMaxConsumersPerTopic(myNamespace) == null); try { @Cleanup - Consumer subscribe = - pulsarClient.newConsumer().subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); + Consumer subscribe = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); fail("should fail"); } catch (PulsarClientException e) { assertTrue(e.getMessage().contains("Topic reached max consumers limit")); + assertEquals(schemaOpsCounter.get(), maxConsumersPerTopic + 2); } //set the limit to 3 admin.namespaces().setMaxConsumersPerTopic(myNamespace, 3); Awaitility.await().until(() -> admin.namespaces().getMaxConsumersPerTopic(myNamespace) == 3); // should success - Consumer consumer = - pulsarClient.newConsumer().subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); consumers.add(consumer); + assertEquals(schemaOpsCounter.get(), maxConsumersPerTopic + 3); try { @Cleanup - Consumer subscribe = - pulsarClient.newConsumer().subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); + Consumer subscribe = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); fail("should fail"); } catch (PulsarClientException e) { assertTrue(e.getMessage().contains("Topic reached max consumers limit")); + assertEquals(schemaOpsCounter.get(), maxConsumersPerTopic + 3); } //clean up - for (Consumer subConsumer : consumers) { + for (Consumer subConsumer : consumers) { subConsumer.close(); } } 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 76f871a6c6035..8c21301c15b4c 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 @@ -509,51 +509,6 @@ public void testProducerOverwrite() { topic.getProducers().values().forEach(producer -> Assert.assertEquals(producer.getEpoch(), 3)); } - private void testMaxProducers() { - PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); - topic.initialize().join(); - String role = "appid1"; - // 1. add producer1 - Producer producer = new Producer(topic, serverCnx, 1 /* producer id */, "prod-name1", role, - false, null, SchemaVersion.Latest, 0, false, ProducerAccessMode.Shared, Optional.empty(), true); - topic.addProducer(producer, new CompletableFuture<>()); - assertEquals(topic.getProducers().size(), 1); - - // 2. add producer2 - Producer producer2 = new Producer(topic, serverCnx, 2 /* producer id */, "prod-name2", role, - false, null, SchemaVersion.Latest, 0, false, ProducerAccessMode.Shared, Optional.empty(), true); - topic.addProducer(producer2, new CompletableFuture<>()); - assertEquals(topic.getProducers().size(), 2); - - // 3. add producer3 but reached maxProducersPerTopic - try { - Producer producer3 = new Producer(topic, serverCnx, 3 /* producer id */, "prod-name3", role, - false, null, SchemaVersion.Latest, 0, false, ProducerAccessMode.Shared, Optional.empty(), true); - topic.addProducer(producer3, new CompletableFuture<>()).join(); - fail("should have failed"); - } catch (Exception e) { - assertEquals(e.getCause().getClass(), BrokerServiceException.ProducerBusyException.class); - } - } - - @Test - public void testMaxProducersForBroker() { - // set max clients - pulsarTestContext.getConfig().setMaxProducersPerTopic(2); - testMaxProducers(); - } - - @Test - public void testMaxProducersForNamespace() throws Exception { - // set max clients - Policies policies = new Policies(); - policies.max_producers_per_topic = 2; - pulsarTestContext.getPulsarResources().getNamespaceResources() - .createPolicies(TopicName.get(successTopicName).getNamespaceObject(), - policies); - testMaxProducers(); - } - private Producer getMockedProducerWithSpecificAddress(Topic topic, long producerId, InetAddress address) { final String producerNameBase = "producer"; final String role = "appid1"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MaxProducerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MaxProducerTest.java new file mode 100644 index 0000000000000..a34b05280c4f5 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MaxProducerTest.java @@ -0,0 +1,88 @@ +/* + * 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 static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.util.ArrayList; +import java.util.List; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class MaxProducerTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setMaxProducersPerTopic(2); + } + + @Test + public void testMaxProducersForBroker() throws Exception { + testMaxProducers(2); + } + + @Test + public void testMaxProducersForNamespace() throws Exception { + // set max clients + admin.namespaces().setMaxProducersPerTopic("public/default", 3); + testMaxProducers(3); + } + + private void testMaxProducers(int maxProducerExpected) throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topicName); + + List> producers = new ArrayList<>(); + for (int i = 0; i < maxProducerExpected; i++) { + producers.add(pulsarClient.newProducer().topic(topicName).create()); + } + + try { + pulsarClient.newProducer().topic(topicName).create(); + fail("should have failed"); + } catch (Exception e) { + assertTrue(e instanceof PulsarClientException.ProducerBusyException); + } + + // cleanup. + for (org.apache.pulsar.client.api.Producer p : producers) { + p.close(); + } + admin.topics().delete(topicName, false); + } +} From 76ed555e7e5ad60247e32fd2624fdb39ca537204 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 29 Jul 2024 19:00:35 +0800 Subject: [PATCH 396/580] [improve][build] Upgrade docker-maven-plugin to 0.45.0 (#23091) Signed-off-by: Zixuan Liu --- docker/pom.xml | 7 +++++++ docker/pulsar-all/pom.xml | 9 --------- docker/pulsar/pom.xml | 9 --------- pom.xml | 2 +- 4 files changed, 8 insertions(+), 19 deletions(-) diff --git a/docker/pom.xml b/docker/pom.xml index 90a845400d3e6..f35da13a96e95 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -75,5 +75,12 @@ + + docker-push + + false + linux/amd64,linux/arm64 + + diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 6a957d6f4623c..4f1e0425f0997 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -145,7 +145,6 @@ package build - tag push @@ -178,13 +177,5 @@ - - - docker-push - - false - - - diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 228c2b810313d..0317effc90815 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -72,7 +72,6 @@ package build - tag push @@ -123,13 +122,5 @@ - - - docker-push - - false - - - diff --git a/pom.xml b/pom.xml index c497ea12e838b..c2cdcee2ff385 100644 --- a/pom.xml +++ b/pom.xml @@ -167,7 +167,7 @@ flexible messaging model and an intuitive client API. 0.10.2 1.6.2 10.14.2 - 0.43.3 + 0.45.0 true 0.5.0 1.14.12 From a0bdf4585feb78b69179b260199dd9aaea6d793f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 29 Jul 2024 14:43:52 +0300 Subject: [PATCH 397/580] [improve][ci] Switch to use DEVELOCITY_ACCESS_KEY from GRADLE_ENTERPRISE_ACCESS_KEY (#23090) --- .github/workflows/ci-maven-cache-update.yaml | 2 +- .../workflows/ci-owasp-dependency-check.yaml | 2 +- .github/workflows/pulsar-ci-flaky.yaml | 2 +- .github/workflows/pulsar-ci.yaml | 20 +++++++++---------- .mvn/extensions.xml | 2 +- 5 files changed, 14 insertions(+), 14 deletions(-) diff --git a/.github/workflows/ci-maven-cache-update.yaml b/.github/workflows/ci-maven-cache-update.yaml index 5a0d4d840c655..a673a30843417 100644 --- a/.github/workflows/ci-maven-cache-update.yaml +++ b/.github/workflows/ci-maven-cache-update.yaml @@ -50,7 +50,7 @@ jobs: name: Update Maven dependency cache for ${{ matrix.name }} env: JOB_NAME: Update Maven dependency cache for ${{ matrix.name }} - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} runs-on: ${{ matrix.runs-on }} timeout-minutes: 45 diff --git a/.github/workflows/ci-owasp-dependency-check.yaml b/.github/workflows/ci-owasp-dependency-check.yaml index a70f4a82ff1af..a1c6dd594d3a2 100644 --- a/.github/workflows/ci-owasp-dependency-check.yaml +++ b/.github/workflows/ci-owasp-dependency-check.yaml @@ -34,7 +34,7 @@ jobs: name: Check ${{ matrix.branch }} env: JOB_NAME: Check ${{ matrix.branch }} - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} runs-on: ubuntu-22.04 timeout-minutes: 75 strategy: diff --git a/.github/workflows/pulsar-ci-flaky.yaml b/.github/workflows/pulsar-ci-flaky.yaml index a92e5cd26c35b..bfc5140943172 100644 --- a/.github/workflows/pulsar-ci-flaky.yaml +++ b/.github/workflows/pulsar-ci-flaky.yaml @@ -148,7 +148,7 @@ jobs: env: JOB_NAME: Flaky tests suite COLLECT_COVERAGE: "${{ needs.preconditions.outputs.collect_coverage }}" - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} TRACE_TEST_RESOURCE_CLEANUP: ${{ github.event_name == 'workflow_dispatch' && github.event.inputs.trace_test_resource_cleanup || 'off' }} TRACE_TEST_RESOURCE_CLEANUP_DIR: ${{ github.workspace }}/target/trace-test-resource-cleanup diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 828f876f13194..dd93003eecce6 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -147,7 +147,7 @@ jobs: name: Build and License check env: JOB_NAME: Build and License check - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} runs-on: ubuntu-22.04 timeout-minutes: 60 @@ -224,7 +224,7 @@ jobs: env: JOB_NAME: CI - Unit - ${{ matrix.name }} COLLECT_COVERAGE: "${{ needs.preconditions.outputs.collect_coverage }}" - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} TRACE_TEST_RESOURCE_CLEANUP: ${{ github.event_name == 'workflow_dispatch' && github.event.inputs.trace_test_resource_cleanup || 'off' }} TRACE_TEST_RESOURCE_CLEANUP_DIR: ${{ github.workspace }}/target/trace-test-resource-cleanup @@ -472,7 +472,7 @@ jobs: - linux/amd64 - linux/arm64 env: - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} IMAGE_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} steps: @@ -550,7 +550,7 @@ jobs: env: JOB_NAME: CI - Integration - ${{ matrix.name }} PULSAR_TEST_IMAGE_NAME: apachepulsar/java-test-image:latest - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} strategy: fail-fast: false @@ -828,7 +828,7 @@ jobs: needs: ['preconditions', 'build-and-license-check'] if: ${{ needs.preconditions.outputs.docs_only != 'true' }} env: - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} IMAGE_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} steps: @@ -951,7 +951,7 @@ jobs: env: JOB_NAME: CI - System - ${{ matrix.name }} PULSAR_TEST_IMAGE_NAME: apachepulsar/pulsar-test-latest-version:latest - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} strategy: fail-fast: false @@ -1181,7 +1181,7 @@ jobs: env: JOB_NAME: CI Flaky - System - ${{ matrix.name }} PULSAR_TEST_IMAGE_NAME: apachepulsar/pulsar-test-latest-version:latest - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} strategy: fail-fast: false @@ -1324,7 +1324,7 @@ jobs: needs: ['preconditions', 'integration-tests'] if: ${{ needs.preconditions.outputs.docs_only != 'true' }} env: - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} steps: - name: checkout @@ -1364,7 +1364,7 @@ jobs: contents: read security-events: write env: - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} CODEQL_LANGUAGE: java-kotlin steps: @@ -1425,7 +1425,7 @@ jobs: needs: [ 'preconditions', 'integration-tests' ] if: ${{ needs.preconditions.outputs.need_owasp == 'true' }} env: - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} CI_JDK_MAJOR_VERSION: ${{ needs.preconditions.outputs.jdk_major_version }} NIST_NVD_API_KEY: ${{ secrets.NIST_NVD_API_KEY }} steps: diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index 4a2117925f163..eb998dc3471b8 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -24,7 +24,7 @@ com.gradle develocity-maven-extension - 1.21.4 + 1.21.6 com.gradle From 40c8c23123e6c4e1c403445cc98e8fed6997f5b0 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 29 Jul 2024 20:48:15 +0800 Subject: [PATCH 398/580] [improve][broker] Reduce the CPU pressure from the transaction buffer in rolling restarts (#23062) --- .../apache/pulsar/broker/PulsarService.java | 2 +- .../SystemTopicTxnBufferSnapshotService.java | 15 +- ...ansactionBufferSnapshotServiceFactory.java | 26 +- ...SingleSnapshotAbortedTxnProcessorImpl.java | 73 +--- ...napshotSegmentAbortedTxnProcessorImpl.java | 394 +++++++----------- .../transaction/buffer/impl/TableView.java | 97 +++++ .../buffer/impl/TopicTransactionBuffer.java | 5 +- .../org/apache/pulsar/utils/SimpleCache.java | 83 ++++ .../TopicTransactionBufferRecoverTest.java | 46 +- .../apache/pulsar/utils/SimpleCacheTest.java | 81 ++++ 10 files changed, 486 insertions(+), 336 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TableView.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/utils/SimpleCache.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/utils/SimpleCacheTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index c623f5d4e5b0d..b23851a5ec464 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -992,7 +992,7 @@ public void start() throws PulsarServerException { MLTransactionMetadataStoreProvider.initBufferedWriterMetrics(getAdvertisedAddress()); MLPendingAckStoreProvider.initBufferedWriterMetrics(getAdvertisedAddress()); - this.transactionBufferSnapshotServiceFactory = new TransactionBufferSnapshotServiceFactory(getClient()); + this.transactionBufferSnapshotServiceFactory = new TransactionBufferSnapshotServiceFactory(this); this.transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-transaction-timer")); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java index bd1b90981695e..ba6cbee355775 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicTxnBufferSnapshotService.java @@ -22,12 +22,16 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarServerException; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.systopic.SystemTopicClientBase; -import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.broker.transaction.buffer.impl.TableView; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.events.EventType; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; @@ -42,6 +46,8 @@ public class SystemTopicTxnBufferSnapshotService { protected final EventType systemTopicType; private final ConcurrentHashMap> refCountedWriterMap; + @Getter + private final TableView tableView; // The class ReferenceCountedWriter will maintain the reference count, // when the reference count decrement to 0, it will be removed from writerFutureMap, the writer will be closed. @@ -95,13 +101,16 @@ public synchronized void release() { } - public SystemTopicTxnBufferSnapshotService(PulsarClient client, EventType systemTopicType, - Class schemaType) { + public SystemTopicTxnBufferSnapshotService(PulsarService pulsar, EventType systemTopicType, + Class schemaType) throws PulsarServerException { + final var client = (PulsarClientImpl) pulsar.getClient(); this.namespaceEventsSystemTopicFactory = new NamespaceEventsSystemTopicFactory(client); this.systemTopicType = systemTopicType; this.schemaType = schemaType; this.clients = new ConcurrentHashMap<>(); this.refCountedWriterMap = new ConcurrentHashMap<>(); + this.tableView = new TableView<>(this::createReader, + client.getConfiguration().getOperationTimeoutMs(), pulsar.getExecutor()); } public CompletableFuture> createReader(TopicName topicName) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java index 4b8548fae47c7..d54f65572f594 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TransactionBufferSnapshotServiceFactory.java @@ -18,12 +18,15 @@ */ package org.apache.pulsar.broker.service; +import lombok.Getter; +import org.apache.pulsar.broker.PulsarServerException; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot; import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes; import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment; -import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.common.events.EventType; +@Getter public class TransactionBufferSnapshotServiceFactory { private SystemTopicTxnBufferSnapshotService txnBufferSnapshotService; @@ -33,29 +36,16 @@ public class TransactionBufferSnapshotServiceFactory { private SystemTopicTxnBufferSnapshotService txnBufferSnapshotIndexService; - public TransactionBufferSnapshotServiceFactory(PulsarClient pulsarClient) { - this.txnBufferSnapshotSegmentService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, + public TransactionBufferSnapshotServiceFactory(PulsarService pulsar) throws PulsarServerException { + this.txnBufferSnapshotSegmentService = new SystemTopicTxnBufferSnapshotService<>(pulsar, EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS, TransactionBufferSnapshotSegment.class); - this.txnBufferSnapshotIndexService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, + this.txnBufferSnapshotIndexService = new SystemTopicTxnBufferSnapshotService<>(pulsar, EventType.TRANSACTION_BUFFER_SNAPSHOT_INDEXES, TransactionBufferSnapshotIndexes.class); - this.txnBufferSnapshotService = new SystemTopicTxnBufferSnapshotService<>(pulsarClient, + this.txnBufferSnapshotService = new SystemTopicTxnBufferSnapshotService<>(pulsar, EventType.TRANSACTION_BUFFER_SNAPSHOT, TransactionBufferSnapshot.class); } - public SystemTopicTxnBufferSnapshotService getTxnBufferSnapshotIndexService() { - return this.txnBufferSnapshotIndexService; - } - - public SystemTopicTxnBufferSnapshotService - getTxnBufferSnapshotSegmentService() { - return this.txnBufferSnapshotSegmentService; - } - - public SystemTopicTxnBufferSnapshotService getTxnBufferSnapshotService() { - return this.txnBufferSnapshotService; - } - public void close() throws Exception { if (this.txnBufferSnapshotIndexService != null) { this.txnBufferSnapshotIndexService.close(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java index 5c9075e9a3867..1649349e3e6f6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java @@ -21,26 +21,19 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.collections4.map.LinkedMap; -import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService.ReferenceCountedWriter; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; import org.apache.pulsar.broker.transaction.buffer.metadata.AbortTxnMetadata; import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot; -import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.transaction.TxnID; -import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TransactionBufferStats; -import org.apache.pulsar.common.util.FutureUtil; @Slf4j public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor { @@ -91,48 +84,27 @@ public boolean checkAbortedTransaction(TxnID txnID) { return aborts.containsKey(txnID); } - private long getSystemClientOperationTimeoutMs() throws Exception { - PulsarClientImpl pulsarClient = (PulsarClientImpl) topic.getBrokerService().getPulsar().getClient(); - return pulsarClient.getConfiguration().getOperationTimeoutMs(); - } - @Override public CompletableFuture recoverFromSnapshot() { - return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory() - .getTxnBufferSnapshotService() - .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> { - try { - Position startReadCursorPosition = null; - while (reader.hasMoreEvents()) { - Message message = reader.readNextAsync() - .get(getSystemClientOperationTimeoutMs(), TimeUnit.MILLISECONDS); - if (topic.getName().equals(message.getKey())) { - TransactionBufferSnapshot transactionBufferSnapshot = message.getValue(); - if (transactionBufferSnapshot != null) { - handleSnapshot(transactionBufferSnapshot); - startReadCursorPosition = PositionFactory.create( - transactionBufferSnapshot.getMaxReadPositionLedgerId(), - transactionBufferSnapshot.getMaxReadPositionEntryId()); - } - } - } - return CompletableFuture.completedFuture(startReadCursorPosition); - } catch (TimeoutException ex) { - Throwable t = FutureUtil.unwrapCompletionException(ex); - String errorMessage = String.format("[%s] Transaction buffer recover fail by read " - + "transactionBufferSnapshot timeout!", topic.getName()); - log.error(errorMessage, t); - return FutureUtil.failedFuture( - new BrokerServiceException.ServiceUnitNotReadyException(errorMessage, t)); - } catch (Exception ex) { - log.error("[{}] Transaction buffer recover fail when read " - + "transactionBufferSnapshot!", topic.getName(), ex); - return FutureUtil.failedFuture(ex); - } finally { - closeReader(reader); - } - }, topic.getBrokerService().getPulsar().getTransactionExecutorProvider() - .getExecutor(this)); + final var future = new CompletableFuture(); + final var pulsar = topic.getBrokerService().getPulsar(); + pulsar.getTransactionExecutorProvider().getExecutor(this).execute(() -> { + try { + final var snapshot = pulsar.getTransactionBufferSnapshotServiceFactory().getTxnBufferSnapshotService() + .getTableView().readLatest(topic.getName()); + if (snapshot != null) { + handleSnapshot(snapshot); + final var startReadCursorPosition = PositionFactory.create(snapshot.getMaxReadPositionLedgerId(), + snapshot.getMaxReadPositionEntryId()); + future.complete(startReadCursorPosition); + } else { + future.complete(null); + } + } catch (Throwable e) { + future.completeExceptionally(e); + } + }); + return future; } @Override @@ -191,13 +163,6 @@ public synchronized CompletableFuture closeAsync() { return CompletableFuture.completedFuture(null); } - private void closeReader(SystemTopicClient.Reader reader) { - reader.closeAsync().exceptionally(e -> { - log.error("[{}]Transaction buffer reader close error!", topic.getName(), e); - return null; - }); - } - private void handleSnapshot(TransactionBufferSnapshot snapshot) { if (snapshot.getAborts() != null) { snapshot.getAborts().forEach(abortTxnMetadata -> diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java index e94e7a047797a..4ca27f77a87f5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java @@ -24,11 +24,11 @@ import java.util.LinkedList; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.CompletionStage; import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.function.Supplier; @@ -54,7 +54,6 @@ import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata; import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment; import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData; -import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.MessageIdImpl; @@ -228,220 +227,129 @@ public CompletableFuture takeAbortedTxnsSnapshot(Position maxReadPosition) @Override public CompletableFuture recoverFromSnapshot() { - return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory() - .getTxnBufferSnapshotIndexService() - .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> { - Position startReadCursorPosition = null; - TransactionBufferSnapshotIndexes persistentSnapshotIndexes = null; - try { - /* - Read the transaction snapshot segment index. -

- The processor can get the sequence ID, unsealed transaction IDs, - segment index list and max read position in the snapshot segment index. - Then we can traverse the index list to read all aborted transaction IDs - in segments to aborts. -

- */ - while (reader.hasMoreEvents()) { - Message message = reader.readNextAsync() - .get(getSystemClientOperationTimeoutMs(), TimeUnit.MILLISECONDS); - if (topic.getName().equals(message.getKey())) { - TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue(); - if (transactionBufferSnapshotIndexes != null) { - persistentSnapshotIndexes = transactionBufferSnapshotIndexes; - startReadCursorPosition = PositionFactory.create( - transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(), - transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId()); - } - } - } - } catch (TimeoutException ex) { - Throwable t = FutureUtil.unwrapCompletionException(ex); - String errorMessage = String.format("[%s] Transaction buffer recover fail by read " - + "transactionBufferSnapshot timeout!", topic.getName()); - log.error(errorMessage, t); - return FutureUtil.failedFuture( - new BrokerServiceException.ServiceUnitNotReadyException(errorMessage, t)); - } catch (Exception ex) { - log.error("[{}] Transaction buffer recover fail when read " - + "transactionBufferSnapshot!", topic.getName(), ex); - return FutureUtil.failedFuture(ex); - } finally { - closeReader(reader); - } - Position finalStartReadCursorPosition = startReadCursorPosition; - TransactionBufferSnapshotIndexes finalPersistentSnapshotIndexes = persistentSnapshotIndexes; - if (persistentSnapshotIndexes == null) { - return recoverOldSnapshot(); - } else { - this.unsealedTxnIds = convertTypeToTxnID(persistentSnapshotIndexes - .getSnapshot().getAborts()); - } - //Read snapshot segment to recover aborts. - ArrayList> completableFutures = new ArrayList<>(); - CompletableFuture openManagedLedgerAndHandleSegmentsFuture = new CompletableFuture<>(); - AtomicBoolean hasInvalidIndex = new AtomicBoolean(false); - AsyncCallbacks.OpenReadOnlyManagedLedgerCallback callback = new AsyncCallbacks - .OpenReadOnlyManagedLedgerCallback() { - @Override - public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl readOnlyManagedLedger, - Object ctx) { - finalPersistentSnapshotIndexes.getIndexList().forEach(index -> { - CompletableFuture handleSegmentFuture = new CompletableFuture<>(); - completableFutures.add(handleSegmentFuture); - readOnlyManagedLedger.asyncReadEntry( - PositionFactory.create(index.getSegmentLedgerID(), - index.getSegmentEntryID()), - new AsyncCallbacks.ReadEntryCallback() { - @Override - public void readEntryComplete(Entry entry, Object ctx) { - handleSnapshotSegmentEntry(entry); - indexes.put(PositionFactory.create( - index.abortedMarkLedgerID, - index.abortedMarkEntryID), - index); - entry.release(); - handleSegmentFuture.complete(null); - } - - @Override - public void readEntryFailed(ManagedLedgerException exception, Object ctx) { - /* - The logic flow of deleting expired segment is: -

- 1. delete segment - 2. update segment index -

- If the worker delete segment successfully - but failed to update segment index, - the segment can not be read according to the index. - We update index again if there are invalid indexes. - */ - if (((ManagedLedgerImpl) topic.getManagedLedger()) - .ledgerExists(index.getAbortedMarkLedgerID())) { - log.error("[{}] Failed to read snapshot segment [{}:{}]", - topic.getName(), index.segmentLedgerID, - index.segmentEntryID, exception); - handleSegmentFuture.completeExceptionally(exception); - } else { - hasInvalidIndex.set(true); - } - } - - @Override - public String toString() { - return String.format("Transaction buffer [%s] recover from snapshot", - SnapshotSegmentAbortedTxnProcessorImpl.this.topic.getName()); - } - }, null); - }); - openManagedLedgerAndHandleSegmentsFuture.complete(null); - } + final var pulsar = topic.getBrokerService().getPulsar(); + final var future = new CompletableFuture(); + pulsar.getTransactionExecutorProvider().getExecutor(this).execute(() -> { + try { + final var indexes = pulsar.getTransactionBufferSnapshotServiceFactory() + .getTxnBufferSnapshotIndexService().getTableView().readLatest(topic.getName()); + if (indexes == null) { + // Try recovering from the old format snapshot + future.complete(recoverOldSnapshot()); + return; + } + final var snapshot = indexes.getSnapshot(); + final var startReadCursorPosition = PositionFactory.create(snapshot.getMaxReadPositionLedgerId(), + snapshot.getMaxReadPositionEntryId()); + this.unsealedTxnIds = convertTypeToTxnID(snapshot.getAborts()); + // Read snapshot segment to recover aborts + final var snapshotSegmentTopicName = TopicName.get(TopicDomain.persistent.toString(), + TopicName.get(topic.getName()).getNamespaceObject(), + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS); + readSegmentEntries(snapshotSegmentTopicName, indexes); + if (!this.indexes.isEmpty()) { + // If there is no segment index, the persistent worker will write segment begin from 0. + persistentWorker.sequenceID.set(this.indexes.get(this.indexes.lastKey()).sequenceID + 1); + } + unsealedTxnIds.forEach(txnID -> aborts.put(txnID, txnID)); + future.complete(startReadCursorPosition); + } catch (Throwable throwable) { + future.completeExceptionally(throwable); + } + }); + return future; + } - @Override - public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx) { - log.error("[{}] Failed to open readOnly managed ledger", topic, exception); - openManagedLedgerAndHandleSegmentsFuture.completeExceptionally(exception); - } - }; - - TopicName snapshotSegmentTopicName = TopicName.get(TopicDomain.persistent.toString(), - TopicName.get(topic.getName()).getNamespaceObject(), - SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS); - this.topic.getBrokerService().getPulsar().getManagedLedgerFactory() - .asyncOpenReadOnlyManagedLedger(snapshotSegmentTopicName - .getPersistenceNamingEncoding(), callback, - topic.getManagedLedger().getConfig(), - null); - /* - Wait the processor recover completely and then allow TB - to recover the messages after the startReadCursorPosition. - */ - return openManagedLedgerAndHandleSegmentsFuture - .thenCompose((ignore) -> FutureUtil.waitForAll(completableFutures)) - .thenCompose((i) -> { - /* - Update the snapshot segment index if there exist invalid indexes. - */ - if (hasInvalidIndex.get()) { - persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, - () -> persistentWorker.updateSnapshotIndex( - finalPersistentSnapshotIndexes.getSnapshot())); - } - /* - If there is no segment index, the persistent worker will write segment begin from 0. - */ - if (indexes.size() != 0) { - persistentWorker.sequenceID.set(indexes.get(indexes.lastKey()).sequenceID + 1); - } - /* - Append the aborted txn IDs in the index metadata - can keep the order of the aborted txn in the aborts. - So that we can trim the expired snapshot segment in aborts - according to the latest transaction IDs in the segmentIndex. - */ - unsealedTxnIds.forEach(txnID -> aborts.put(txnID, txnID)); - return CompletableFuture.completedFuture(finalStartReadCursorPosition); - }).exceptionally(ex -> { - log.error("[{}] Failed to recover snapshot segment", this.topic.getName(), ex); - return null; - }); - - }, topic.getBrokerService().getPulsar().getTransactionExecutorProvider() - .getExecutor(this)); + private void readSegmentEntries(TopicName topicName, TransactionBufferSnapshotIndexes indexes) throws Exception { + final var managedLedger = openReadOnlyManagedLedger(topicName); + boolean hasInvalidIndex = false; + for (var index : indexes.getIndexList()) { + final var position = PositionFactory.create(index.getSegmentLedgerID(), index.getSegmentEntryID()); + final var abortedPosition = PositionFactory.create(index.abortedMarkLedgerID, index.abortedMarkEntryID); + try { + final var entry = readEntry(managedLedger, position); + try { + handleSnapshotSegmentEntry(entry); + this.indexes.put(abortedPosition, index); + } finally { + entry.release(); + } + } catch (Throwable throwable) { + if (((ManagedLedgerImpl) topic.getManagedLedger()) + .ledgerExists(index.getAbortedMarkLedgerID())) { + log.error("[{}] Failed to read snapshot segment [{}:{}]", + topic.getName(), index.segmentLedgerID, + index.segmentEntryID, throwable); + throw throwable; + } else { + hasInvalidIndex = true; + } + } + } + if (hasInvalidIndex) { + // Update the snapshot segment index if there exist invalid indexes. + persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, + () -> persistentWorker.updateSnapshotIndex(indexes.getSnapshot())); + } + } + + private ReadOnlyManagedLedgerImpl openReadOnlyManagedLedger(TopicName topicName) throws Exception { + final var future = new CompletableFuture(); + final var callback = new AsyncCallbacks.OpenReadOnlyManagedLedgerCallback() { + @Override + public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl managedLedger, Object ctx) { + future.complete(managedLedger); + } + + @Override + public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + + @Override + public String toString() { + return String.format("Transaction buffer [%s] recover from snapshot", + SnapshotSegmentAbortedTxnProcessorImpl.this.topic.getName()); + } + }; + topic.getBrokerService().getPulsar().getManagedLedgerFactory().asyncOpenReadOnlyManagedLedger( + topicName.getPersistenceNamingEncoding(), callback, topic.getManagedLedger().getConfig(), null); + return wait(future, "open read only ml for " + topicName); + } + + private Entry readEntry(ReadOnlyManagedLedgerImpl managedLedger, Position position) throws Exception { + final var future = new CompletableFuture(); + managedLedger.asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { + @Override + public void readEntryComplete(Entry entry, Object ctx) { + future.complete(entry); + } + + @Override + public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + }, null); + return wait(future, "read entry from " + position); } // This method will be deprecated and removed in version 4.x.0 - private CompletableFuture recoverOldSnapshot() { - return topic.getBrokerService().getPulsar().getPulsarResources().getTopicResources() - .listPersistentTopicsAsync(NamespaceName.get(TopicName.get(topic.getName()).getNamespace())) - .thenCompose(topics -> { - if (!topics.contains(TopicDomain.persistent + "://" - + TopicName.get(topic.getName()).getNamespace() + "/" - + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT)) { - return CompletableFuture.completedFuture(null); - } else { - return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory() - .getTxnBufferSnapshotService() - .createReader(TopicName.get(topic.getName())).thenComposeAsync(snapshotReader -> { - Position startReadCursorPositionInOldSnapshot = null; - try { - while (snapshotReader.hasMoreEvents()) { - Message message = snapshotReader.readNextAsync() - .get(getSystemClientOperationTimeoutMs(), TimeUnit.MILLISECONDS); - if (topic.getName().equals(message.getKey())) { - TransactionBufferSnapshot transactionBufferSnapshot = - message.getValue(); - if (transactionBufferSnapshot != null) { - handleOldSnapshot(transactionBufferSnapshot); - startReadCursorPositionInOldSnapshot = PositionFactory.create( - transactionBufferSnapshot.getMaxReadPositionLedgerId(), - transactionBufferSnapshot.getMaxReadPositionEntryId()); - } - } - } - } catch (TimeoutException ex) { - Throwable t = FutureUtil.unwrapCompletionException(ex); - String errorMessage = String.format("[%s] Transaction buffer recover fail by " - + "read transactionBufferSnapshot timeout!", topic.getName()); - log.error(errorMessage, t); - return FutureUtil.failedFuture(new BrokerServiceException - .ServiceUnitNotReadyException(errorMessage, t)); - } catch (Exception ex) { - log.error("[{}] Transaction buffer recover fail when read " - + "transactionBufferSnapshot!", topic.getName(), ex); - return FutureUtil.failedFuture(ex); - } finally { - assert snapshotReader != null; - closeReader(snapshotReader); - } - return CompletableFuture.completedFuture(startReadCursorPositionInOldSnapshot); - }, - topic.getBrokerService().getPulsar().getTransactionExecutorProvider() - .getExecutor(this)); - } - }); + private Position recoverOldSnapshot() throws Exception { + final var pulsar = topic.getBrokerService().getPulsar(); + final var topicName = TopicName.get(topic.getName()); + final var topics = wait(pulsar.getPulsarResources().getTopicResources().listPersistentTopicsAsync( + NamespaceName.get(topicName.getNamespace())), "list persistent topics"); + if (!topics.contains(TopicDomain.persistent + "://" + topicName.getNamespace() + "/" + + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT)) { + return null; + } + final var snapshot = pulsar.getTransactionBufferSnapshotServiceFactory().getTxnBufferSnapshotService() + .getTableView().readLatest(topic.getName()); + if (snapshot == null) { + return null; + } + handleOldSnapshot(snapshot); + return PositionFactory.create(snapshot.getMaxReadPositionLedgerId(), snapshot.getMaxReadPositionEntryId()); } // This method will be deprecated and removed in version 4.x.0 @@ -509,9 +417,17 @@ private long getSystemClientOperationTimeoutMs() throws Exception { return pulsarClient.getConfiguration().getOperationTimeoutMs(); } - private void closeReader(SystemTopicClient.Reader reader) { + private R wait(CompletableFuture future, String msg) throws Exception { + try { + return future.get(getSystemClientOperationTimeoutMs(), TimeUnit.MILLISECONDS); + } catch (ExecutionException e) { + throw new CompletionException("Failed to " + msg, e.getCause()); + } + } + + private void closeReader(SystemTopicClient.Reader reader) { reader.closeAsync().exceptionally(e -> { - log.error("[{}]Transaction buffer snapshot reader close error!", topic.getName(), e); + log.warn("[{}] Failed to close reader: {}", topic.getName(), e.getMessage()); return null; }); } @@ -838,25 +754,37 @@ private CompletableFuture clearSnapshotSegmentAndIndexes() { *

*/ private CompletableFuture clearAllSnapshotSegments() { - return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory() - .getTxnBufferSnapshotSegmentService() - .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> { - try { - while (reader.hasMoreEvents()) { - Message message = reader.readNextAsync() - .get(getSystemClientOperationTimeoutMs(), TimeUnit.MILLISECONDS); - if (topic.getName().equals(message.getValue().getTopicName())) { - snapshotSegmentsWriter.getFuture().get().write(message.getKey(), null); - } + final var future = new CompletableFuture(); + final var pulsar = topic.getBrokerService().getPulsar(); + pulsar.getTransactionExecutorProvider().getExecutor(this).execute(() -> { + try { + final var reader = wait(pulsar.getTransactionBufferSnapshotServiceFactory() + .getTxnBufferSnapshotSegmentService().createReader(TopicName.get(topic.getName())) + , "create reader"); + try { + while (wait(reader.hasMoreEventsAsync(), "has more events")) { + final var message = wait(reader.readNextAsync(), "read next"); + if (topic.getName().equals(message.getValue().getTopicName())) { + snapshotSegmentsWriter.getFuture().get().write(message.getKey(), null); } - return CompletableFuture.completedFuture(null); - } catch (Exception ex) { - log.error("[{}] Transaction buffer clear snapshot segments fail!", topic.getName(), ex); - return FutureUtil.failedFuture(ex); - } finally { - closeReader(reader); } - }); + future.complete(null); + } finally { + closeReader(reader); + } + } catch (Throwable throwable) { + future.completeExceptionally(throwable); + } + }); + return future; + } + + private R wait(CompletableFuture future, String msg) throws Exception { + try { + return future.get(getSystemClientOperationTimeoutMs(), TimeUnit.MILLISECONDS); + } catch (ExecutionException e) { + throw new CompletionException("Failed to " + msg, e.getCause()); + } } synchronized CompletableFuture closeAsync() { @@ -882,4 +810,4 @@ private List convertTypeToTxnIDData(List abortedTxns) { return segment; } -} \ No newline at end of file +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TableView.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TableView.java new file mode 100644 index 0000000000000..7608a393cc980 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TableView.java @@ -0,0 +1,97 @@ +/* + * 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.transaction.buffer.impl; + +import static org.apache.pulsar.broker.systopic.SystemTopicClient.Reader; +import com.google.common.annotations.VisibleForTesting; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.utils.SimpleCache; + +/** + * Compared with the more generic {@link org.apache.pulsar.client.api.TableView}, this table view + * - Provides just a single public method that reads the latest value synchronously. + * - Maintains multiple long-lived readers that will be expired after some time (1 minute by default). + */ +@Slf4j +public class TableView { + + // Remove the cached reader and snapshots if there is no refresh request in 1 minute + private static final long CACHE_EXPIRE_TIMEOUT_MS = 60 * 1000L; + private static final long CACHE_EXPIRE_CHECK_FREQUENCY_MS = 3000L; + @VisibleForTesting + protected final Function>> readerCreator; + private final Map snapshots = new ConcurrentHashMap<>(); + private final long clientOperationTimeoutMs; + private final SimpleCache> readers; + + public TableView(Function>> readerCreator, long clientOperationTimeoutMs, + ScheduledExecutorService executor) { + this.readerCreator = readerCreator; + this.clientOperationTimeoutMs = clientOperationTimeoutMs; + this.readers = new SimpleCache<>(executor, CACHE_EXPIRE_TIMEOUT_MS, CACHE_EXPIRE_CHECK_FREQUENCY_MS); + } + + public T readLatest(String topic) throws Exception { + final var reader = getReader(topic); + while (wait(reader.hasMoreEventsAsync(), "has more events")) { + final var msg = wait(reader.readNextAsync(), "read message"); + if (msg.getKey() != null) { + if (msg.getValue() != null) { + snapshots.put(msg.getKey(), msg.getValue()); + } else { + snapshots.remove(msg.getKey()); + } + } + } + return snapshots.get(topic); + } + + @VisibleForTesting + protected Reader getReader(String topic) { + final var topicName = TopicName.get(topic); + return readers.get(topicName.getNamespaceObject(), () -> { + try { + return wait(readerCreator.apply(topicName), "create reader"); + } catch (Exception e) { + throw new RuntimeException(e); + } + }, __ -> __.closeAsync().exceptionally(e -> { + log.warn("Failed to close reader {}", e.getMessage()); + return null; + })); + } + + private R wait(CompletableFuture future, String msg) throws Exception { + try { + return future.get(clientOperationTimeoutMs, TimeUnit.MILLISECONDS); + } catch (ExecutionException e) { + throw new CompletionException("Failed to " + msg, e.getCause()); + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index b4662e5fa83ed..7561457d11f8e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -632,7 +632,7 @@ public void run() { this, topic.getName()); return; } - abortedTxnProcessor.recoverFromSnapshot().thenAcceptAsync(startReadCursorPosition -> { + abortedTxnProcessor.recoverFromSnapshot().thenAccept(startReadCursorPosition -> { //Transaction is not use for this topic, so just make maxReadPosition as LAC. if (startReadCursorPosition == null) { callBack.noNeedToRecover(); @@ -678,8 +678,7 @@ public void run() { closeCursor(SUBSCRIPTION_NAME); callBack.recoverComplete(); - }, topic.getBrokerService().getPulsar().getTransactionExecutorProvider() - .getExecutor(this)).exceptionally(e -> { + }).exceptionally(e -> { callBack.recoverExceptionally(e.getCause()); log.error("[{}]Transaction buffer failed to recover snapshot!", topic.getName(), e); return null; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/utils/SimpleCache.java b/pulsar-broker/src/main/java/org/apache/pulsar/utils/SimpleCache.java new file mode 100644 index 0000000000000..6a3a6721198e1 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/utils/SimpleCache.java @@ -0,0 +1,83 @@ +/* + * 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.utils; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.function.Supplier; +import lombok.RequiredArgsConstructor; + +public class SimpleCache { + + private final Map> cache = new HashMap<>(); + private final long timeoutMs; + + @RequiredArgsConstructor + private class ExpirableValue { + + private final V value; + private final Consumer expireCallback; + private long deadlineMs; + + boolean tryExpire() { + if (System.currentTimeMillis() >= deadlineMs) { + expireCallback.accept(value); + return true; + } else { + return false; + } + } + + void updateDeadline() { + deadlineMs = System.currentTimeMillis() + timeoutMs; + } + } + + public SimpleCache(final ScheduledExecutorService scheduler, final long timeoutMs, final long frequencyMs) { + this.timeoutMs = timeoutMs; + scheduler.scheduleAtFixedRate(() -> { + synchronized (SimpleCache.this) { + final var keys = new HashSet(); + cache.forEach((key, value) -> { + if (value.tryExpire()) { + keys.add(key); + } + }); + cache.keySet().removeAll(keys); + } + }, frequencyMs, frequencyMs, TimeUnit.MILLISECONDS); + } + + public synchronized V get(final K key, final Supplier valueSupplier, final Consumer expireCallback) { + final var value = cache.get(key); + if (value != null) { + value.updateDeadline(); + return value.value; + } + + final var newValue = new ExpirableValue<>(valueSupplier.get(), expireCallback); + newValue.updateDeadline(); + cache.put(key, newValue); + return newValue.value; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index e4240bce700bd..8ab9d58f57076 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -66,6 +66,7 @@ import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; import org.apache.pulsar.broker.transaction.buffer.impl.SingleSnapshotAbortedTxnProcessorImpl; +import org.apache.pulsar.broker.transaction.buffer.impl.TableView; import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot; import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex; @@ -90,7 +91,6 @@ import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.events.EventType; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; @@ -582,6 +582,19 @@ private void checkSnapshotCount(TopicName topicName, boolean hasSnapshot, reader.close(); } + static class MockTableView extends TableView { + + public MockTableView(PulsarService pulsar) { + super(topic -> pulsar.getTransactionBufferSnapshotServiceFactory().getTxnBufferSnapshotService() + .createReader(topic), 30000L, pulsar.getExecutor()); + } + + @Override + public SystemTopicClient.Reader getReader(String topic) { + return readerCreator.apply(TopicName.get(topic)).join(); + } + } + @Test(timeOut=30000) public void testTransactionBufferRecoverThrowException() throws Exception { String topic = NAMESPACE1 + "/testTransactionBufferRecoverThrowPulsarClientException"; @@ -612,6 +625,7 @@ public void testTransactionBufferRecoverThrowException() throws Exception { doReturn(CompletableFuture.completedFuture(reader)) .when(systemTopicTxnBufferSnapshotService).createReader(any()); doReturn(refCounterWriter).when(systemTopicTxnBufferSnapshotService).getReferenceWriter(any()); + doReturn(new MockTableView(pulsarServiceList.get(0))).when(systemTopicTxnBufferSnapshotService).getTableView(); TransactionBufferSnapshotServiceFactory transactionBufferSnapshotServiceFactory = mock(TransactionBufferSnapshotServiceFactory.class); doReturn(systemTopicTxnBufferSnapshotService) @@ -663,7 +677,8 @@ private void checkCloseTopic(PulsarClient pulsarClient, PersistentTopic originalTopic, Field field, Producer producer) throws Exception { - field.set(getPulsarServiceList().get(0), transactionBufferSnapshotServiceFactory); + final var pulsar = getPulsarServiceList().get(0); + field.set(pulsar, transactionBufferSnapshotServiceFactory); // recover again will throw then close topic new TopicTransactionBuffer(originalTopic); @@ -674,7 +689,7 @@ private void checkCloseTopic(PulsarClient pulsarClient, assertTrue((boolean) close.get(originalTopic)); }); - field.set(getPulsarServiceList().get(0), transactionBufferSnapshotServiceFactoryOriginal); + field.set(pulsar, transactionBufferSnapshotServiceFactoryOriginal); Transaction txn = pulsarClient.newTransaction() .withTransactionTimeout(5, TimeUnit.SECONDS) @@ -684,29 +699,11 @@ private void checkCloseTopic(PulsarClient pulsarClient, txn.commit().get(); } - - @Test - public void testTransactionBufferNoSnapshotCloseReader() throws Exception{ - String topic = NAMESPACE1 + "/test"; - @Cleanup - Producer producer = pulsarClient.newProducer(Schema.STRING).producerName("testTxnTimeOut_producer") - .topic(topic).sendTimeout(0, TimeUnit.SECONDS).enableBatching(false).create(); - - admin.topics().unload(topic); - - // unload success, all readers have been closed except for the compaction sub - producer.send("test"); - TopicStats stats = admin.topics().getStats(NAMESPACE1 + "/" + TRANSACTION_BUFFER_SNAPSHOT); - - // except for the compaction sub - assertEquals(stats.getSubscriptions().size(), 1); - assertTrue(stats.getSubscriptions().keySet().contains("__compaction")); - } - @Test public void testTransactionBufferIndexSystemTopic() throws Exception { + final var pulsar = pulsarServiceList.get(0); SystemTopicTxnBufferSnapshotService transactionBufferSnapshotIndexService = - new TransactionBufferSnapshotServiceFactory(pulsarClient).getTxnBufferSnapshotIndexService(); + new TransactionBufferSnapshotServiceFactory(pulsar).getTxnBufferSnapshotIndexService(); SystemTopicClient.Writer indexesWriter = transactionBufferSnapshotIndexService.getReferenceWriter( @@ -766,9 +763,10 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { BrokerService brokerService = pulsarService.getBrokerService(); // create snapshot segment writer + final var pulsar = pulsarServiceList.get(0); SystemTopicTxnBufferSnapshotService transactionBufferSnapshotSegmentService = - new TransactionBufferSnapshotServiceFactory(pulsarClient).getTxnBufferSnapshotSegmentService(); + new TransactionBufferSnapshotServiceFactory(pulsar).getTxnBufferSnapshotSegmentService(); SystemTopicClient.Writer segmentWriter = transactionBufferSnapshotSegmentService diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/utils/SimpleCacheTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/utils/SimpleCacheTest.java new file mode 100644 index 0000000000000..c590eda171804 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/utils/SimpleCacheTest.java @@ -0,0 +1,81 @@ +/* + * 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.utils; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +public class SimpleCacheTest { + + private final ScheduledExecutorService executor = Executors.newScheduledThreadPool(1); + + @AfterClass + public void shutdown() { + executor.shutdown(); + } + + @Test + public void testConcurrentUpdate() throws Exception { + final var cache = new SimpleCache(executor, 10000L, 10000L); + final var pool = Executors.newFixedThreadPool(2); + final var latch = new CountDownLatch(2); + for (int i = 0; i < 2; i++) { + final var value = i + 100; + pool.execute(() -> { + try { + Thread.sleep(100); + } catch (InterruptedException ignored) { + } + cache.get(0, () -> value, __ -> {}); + latch.countDown(); + }); + } + latch.await(); + final var value = cache.get(0, () -> -1, __ -> {}); + Assert.assertTrue(value == 100 || value == 101); + pool.shutdown(); + } + + @Test + public void testExpire() throws InterruptedException { + final var cache = new SimpleCache(executor, 500L, 5); + final var expiredValues = Collections.synchronizedSet(new HashSet()); + + final var allKeys = IntStream.range(0, 5).boxed().collect(Collectors.toSet()); + allKeys.forEach(key -> cache.get(key, () -> key + 100, expiredValues::add)); + + Thread.sleep(400L); + final var recentAccessedKey = Set.of(1, 2); + recentAccessedKey.forEach(key -> cache.get(key, () -> -1, expiredValues::add)); // access these keys + + Thread.sleep(300L); + recentAccessedKey.forEach(key -> Assert.assertEquals(key + 100, cache.get(key, () -> -1, __ -> {}))); + allKeys.stream().filter(key -> !recentAccessedKey.contains(key)) + .forEach(key -> Assert.assertEquals(-1, cache.get(key, () -> -1, __ -> {}))); + } +} From 77b6378ae8b9ac83962f71063ad44d6ac57f8e32 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 29 Jul 2024 22:02:29 +0800 Subject: [PATCH 399/580] [improve][broker] Optimize the performance of individual acknowledgments (#23072) --- .../pulsar/broker/service/Consumer.java | 151 ++++++++---------- 1 file changed, 69 insertions(+), 82 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index dca64395d8674..7f46e8969eb53 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -47,6 +47,7 @@ import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.tuple.MutablePair; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.authentication.AuthenticationDataSubscription; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -531,14 +532,16 @@ public CompletableFuture messageAcked(CommandAck ack) { //this method is for individual ack not carry the transaction private CompletableFuture individualAckNormal(CommandAck ack, Map properties) { - List positionsAcked = new ArrayList<>(); + List> positionsAcked = new ArrayList<>(); long totalAckCount = 0; for (int i = 0; i < ack.getMessageIdsCount(); i++) { MessageIdData msgId = ack.getMessageIdAt(i); Position position; - long ackedCount = 0; - long batchSize = getBatchSize(msgId); - Consumer ackOwnerConsumer = getAckOwnerConsumer(msgId.getLedgerId(), msgId.getEntryId()); + Pair ackOwnerConsumerAndBatchSize = + getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(), msgId.getEntryId()); + Consumer ackOwnerConsumer = ackOwnerConsumerAndBatchSize.getLeft(); + long ackedCount; + long batchSize = ackOwnerConsumerAndBatchSize.getRight(); if (msgId.getAckSetsCount() > 0) { long[] ackSets = new long[msgId.getAckSetsCount()]; for (int j = 0; j < msgId.getAckSetsCount(); j++) { @@ -557,28 +560,32 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map completableFuture = new CompletableFuture<>(); completableFuture.complete(totalAckCount); if (isTransactionEnabled() && Subscription.isIndividualAckMode(subType)) { - completableFuture.whenComplete((v, e) -> positionsAcked.forEach(position -> { + completableFuture.whenComplete((v, e) -> positionsAcked.forEach(positionPair -> { + Consumer ackOwnerConsumer = positionPair.getLeft(); + Position position = positionPair.getRight(); //check if the position can remove from the consumer pending acks. // the bit set is empty in pending ack handle. if (AckSetStateUtil.hasAckSet(position)) { if (((PersistentSubscription) subscription) .checkIsCanDeleteConsumerPendingAck(position)) { - removePendingAcks(position); + removePendingAcks(ackOwnerConsumer, position); } } })); @@ -590,7 +597,7 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map individualAckWithTransaction(CommandAck ack) { // Individual ack - List> positionsAcked = new ArrayList<>(); + List>> positionsAcked = new ArrayList<>(); if (!isTransactionEnabled()) { return FutureUtil.failedFuture( new BrokerServiceException.NotAllowedException("Server don't support transaction ack!")); @@ -600,20 +607,23 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { for (int i = 0; i < ack.getMessageIdsCount(); i++) { MessageIdData msgId = ack.getMessageIdAt(i); Position position = AckSetStateUtil.createPositionWithAckSet(msgId.getLedgerId(), msgId.getEntryId(), null); + Consumer ackOwnerConsumer = getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(), + msgId.getEntryId()).getLeft(); // acked count at least one - long ackedCount = 0; - long batchSize = 0; + long ackedCount; + long batchSize; if (msgId.hasBatchSize()) { batchSize = msgId.getBatchSize(); // ack batch messages set ackeCount = batchSize ackedCount = msgId.getBatchSize(); - positionsAcked.add(new MutablePair<>(position, msgId.getBatchSize())); + positionsAcked.add(Pair.of(ackOwnerConsumer, new MutablePair<>(position, msgId.getBatchSize()))); } else { // ack no batch message set ackedCount = 1 + batchSize = 0; ackedCount = 1; - positionsAcked.add(new MutablePair<>(position, (int) batchSize)); + positionsAcked.add(Pair.of(ackOwnerConsumer, new MutablePair<>(position, (int) batchSize))); } - Consumer ackOwnerConsumer = getAckOwnerConsumer(msgId.getLedgerId(), msgId.getEntryId()); + if (msgId.getAckSetsCount() > 0) { long[] ackSets = new long[msgId.getAckSetsCount()]; for (int j = 0; j < msgId.getAckSetsCount(); j++) { @@ -625,7 +635,7 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { addAndGetUnAckedMsgs(ackOwnerConsumer, -(int) ackedCount); - checkCanRemovePendingAcksAndHandle(position, msgId); + checkCanRemovePendingAcksAndHandle(ackOwnerConsumer, position, msgId); checkAckValidationError(ack, position); @@ -633,14 +643,16 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { } CompletableFuture completableFuture = transactionIndividualAcknowledge(ack.getTxnidMostBits(), - ack.getTxnidLeastBits(), positionsAcked); + ack.getTxnidLeastBits(), positionsAcked.stream().map(Pair::getRight).collect(Collectors.toList())); if (Subscription.isIndividualAckMode(subType)) { completableFuture.whenComplete((v, e) -> - positionsAcked.forEach(positionLongMutablePair -> { + positionsAcked.forEach(positionPair -> { + Consumer ackOwnerConsumer = positionPair.getLeft(); + MutablePair positionLongMutablePair = positionPair.getRight(); if (AckSetStateUtil.hasAckSet(positionLongMutablePair.getLeft())) { if (((PersistentSubscription) subscription) .checkIsCanDeleteConsumerPendingAck(positionLongMutablePair.left)) { - removePendingAcks(positionLongMutablePair.left); + removePendingAcks(ackOwnerConsumer, positionLongMutablePair.left); } } })); @@ -648,24 +660,6 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { return completableFuture.thenApply(__ -> totalAckCount.sum()); } - private long getBatchSize(MessageIdData msgId) { - long batchSize = 1; - if (Subscription.isIndividualAckMode(subType)) { - LongPair longPair = pendingAcks.get(msgId.getLedgerId(), msgId.getEntryId()); - // Consumer may ack the msg that not belongs to it. - if (longPair == null) { - Consumer ackOwnerConsumer = getAckOwnerConsumer(msgId.getLedgerId(), msgId.getEntryId()); - longPair = ackOwnerConsumer.getPendingAcks().get(msgId.getLedgerId(), msgId.getEntryId()); - if (longPair != null) { - batchSize = longPair.first; - } - } else { - batchSize = longPair.first; - } - } - return batchSize; - } - private long getAckedCountForMsgIdNoAckSets(long batchSize, Position position, Consumer consumer) { if (isAcknowledgmentAtBatchIndexLevelEnabled && Subscription.isIndividualAckMode(subType)) { long[] cursorAckSet = getCursorAckSet(position); @@ -725,26 +719,39 @@ private void checkAckValidationError(CommandAck ack, Position position) { } } - private boolean checkCanRemovePendingAcksAndHandle(Position position, MessageIdData msgId) { + private boolean checkCanRemovePendingAcksAndHandle(Consumer ackOwnedConsumer, + Position position, MessageIdData msgId) { if (Subscription.isIndividualAckMode(subType) && msgId.getAckSetsCount() == 0) { - return removePendingAcks(position); + return removePendingAcks(ackOwnedConsumer, position); } return false; } - private Consumer getAckOwnerConsumer(long ledgerId, long entryId) { - Consumer ackOwnerConsumer = this; + /** + * Retrieves the acknowledgment owner consumer and batch size for the specified ledgerId and entryId. + * + * @param ledgerId The ID of the ledger. + * @param entryId The ID of the entry. + * @return Pair + */ + private Pair getAckOwnerConsumerAndBatchSize(long ledgerId, long entryId) { if (Subscription.isIndividualAckMode(subType)) { - if (!getPendingAcks().containsKey(ledgerId, entryId)) { + LongPair longPair = getPendingAcks().get(ledgerId, entryId); + if (longPair != null) { + return Pair.of(this, longPair.first); + } else { + // If there are more consumers, this step will consume more CPU, and it should be optimized later. for (Consumer consumer : subscription.getConsumers()) { - if (consumer != this && consumer.getPendingAcks().containsKey(ledgerId, entryId)) { - ackOwnerConsumer = consumer; - break; + if (consumer != this) { + longPair = consumer.getPendingAcks().get(ledgerId, entryId); + if (longPair != null) { + return Pair.of(consumer, longPair.first); + } } } } } - return ackOwnerConsumer; + return Pair.of(this, 1L); } private long[] getCursorAckSet(Position position) { @@ -1019,44 +1026,24 @@ public int hashCode() { * * @param position */ - private boolean removePendingAcks(Position position) { - Consumer ackOwnedConsumer = null; - if (pendingAcks.get(position.getLedgerId(), position.getEntryId()) == null) { - for (Consumer consumer : subscription.getConsumers()) { - if (!consumer.equals(this) && consumer.getPendingAcks().containsKey(position.getLedgerId(), - position.getEntryId())) { - ackOwnedConsumer = consumer; - break; - } - } - } else { - ackOwnedConsumer = this; + private boolean removePendingAcks(Consumer ackOwnedConsumer, Position position) { + if (!ackOwnedConsumer.getPendingAcks().remove(position.getLedgerId(), position.getEntryId())) { + // Message was already removed by the other consumer + return false; } - - // remove pending message from appropriate consumer and unblock unAckMsg-flow if requires - LongPair ackedPosition = ackOwnedConsumer != null - ? ackOwnedConsumer.getPendingAcks().get(position.getLedgerId(), position.getEntryId()) - : null; - if (ackedPosition != null) { - if (!ackOwnedConsumer.getPendingAcks().remove(position.getLedgerId(), position.getEntryId())) { - // Message was already removed by the other consumer - return false; - } - if (log.isDebugEnabled()) { - log.debug("[{}-{}] consumer {} received ack {}", topicName, subscription, consumerId, position); - } - // unblock consumer-throttling when limit check is disabled or receives half of maxUnackedMessages => - // consumer can start again consuming messages - int unAckedMsgs = UNACKED_MESSAGES_UPDATER.get(ackOwnedConsumer); - if ((((unAckedMsgs <= getMaxUnackedMessages() / 2) && ackOwnedConsumer.blockedConsumerOnUnackedMsgs) - && ackOwnedConsumer.shouldBlockConsumerOnUnackMsgs()) - || !shouldBlockConsumerOnUnackMsgs()) { - ackOwnedConsumer.blockedConsumerOnUnackedMsgs = false; - flowConsumerBlockedPermits(ackOwnedConsumer); - } - return true; + if (log.isDebugEnabled()) { + log.debug("[{}-{}] consumer {} received ack {}", topicName, subscription, consumerId, position); } - return false; + // unblock consumer-throttling when limit check is disabled or receives half of maxUnackedMessages => + // consumer can start again consuming messages + int unAckedMsgs = UNACKED_MESSAGES_UPDATER.get(ackOwnedConsumer); + if ((((unAckedMsgs <= getMaxUnackedMessages() / 2) && ackOwnedConsumer.blockedConsumerOnUnackedMsgs) + && ackOwnedConsumer.shouldBlockConsumerOnUnackMsgs()) + || !shouldBlockConsumerOnUnackMsgs()) { + ackOwnedConsumer.blockedConsumerOnUnackedMsgs = false; + flowConsumerBlockedPermits(ackOwnedConsumer); + } + return true; } public ConcurrentLongLongPairHashMap getPendingAcks() { From 49d3beb4fae7efa60c48ec8dbf1d33f9c033c969 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Mon, 29 Jul 2024 22:37:59 +0800 Subject: [PATCH 400/580] [improve][broker] Support to specify auth-plugin, auth-parameters and tls-enable arguments when init cluster metadata (#23087) ### Motivation When using a global configuration store and geo-replication, support to specify `auth-plugin`, `auth-parameters`, and `tls-enable` arguments when init cluster metadata will be useful, it can reduce one step to create the cluster with auth. ### Modifications Support to specify `auth-plugin`, `auth-parameters` and `tls-enable` arguments when init cluster metadata --- .../pulsar/PulsarClusterMetadataSetup.java | 51 ++++++++++++++++--- .../zookeeper/ClusterMetadataSetupTest.java | 43 ++++++++++++++++ 2 files changed, 86 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java index 04a66ff022e2a..c818dee124a88 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java @@ -97,6 +97,19 @@ private static class Arguments { description = "Broker-service URL for new cluster with TLS encryption", required = false) private String clusterBrokerServiceUrlTls; + @Option(names = {"-te", + "--tls-enable"}, + description = "Enable TLS connection for new cluster") + private Boolean clusterBrokerClientTlsEnabled; + + @Option(names = "--auth-plugin", + description = "The authentication plugin for new cluster") + protected String clusterAuthenticationPlugin; + + @Option(names = "--auth-parameters", + description = "The authentication parameters for new cluster") + protected String clusterAuthenticationParameters; + @Option(names = {"-zk", "--zookeeper"}, description = "Local ZooKeeper quorum connection string", required = false, @@ -317,14 +330,36 @@ private static void initializeCluster(Arguments arguments, int bundleNumberForDe PulsarResources resources = new PulsarResources(localStore, configStore); - ClusterData clusterData = ClusterData.builder() - .serviceUrl(arguments.clusterWebServiceUrl) - .serviceUrlTls(arguments.clusterWebServiceUrlTls) - .brokerServiceUrl(arguments.clusterBrokerServiceUrl) - .brokerServiceUrlTls(arguments.clusterBrokerServiceUrlTls) - .proxyServiceUrl(arguments.clusterProxyUrl) - .proxyProtocol(arguments.clusterProxyProtocol) - .build(); + ClusterData.Builder clusterDataBuilder = ClusterData.builder(); + if (arguments.clusterWebServiceUrl != null) { + clusterDataBuilder.serviceUrl(arguments.clusterWebServiceUrl); + } + if (arguments.clusterWebServiceUrlTls != null) { + clusterDataBuilder.serviceUrlTls(arguments.clusterWebServiceUrlTls); + } + if (arguments.clusterBrokerServiceUrl != null) { + clusterDataBuilder.brokerServiceUrl(arguments.clusterBrokerServiceUrl); + } + if (arguments.clusterBrokerServiceUrlTls != null) { + clusterDataBuilder.brokerServiceUrlTls(arguments.clusterBrokerServiceUrlTls); + } + if (arguments.clusterBrokerClientTlsEnabled != null) { + clusterDataBuilder.brokerClientTlsEnabled(arguments.clusterBrokerClientTlsEnabled); + } + if (arguments.clusterAuthenticationPlugin != null) { + clusterDataBuilder.authenticationPlugin(arguments.clusterAuthenticationPlugin); + } + if (arguments.clusterAuthenticationParameters != null) { + clusterDataBuilder.authenticationParameters(arguments.clusterAuthenticationParameters); + } + if (arguments.clusterProxyUrl != null) { + clusterDataBuilder.proxyServiceUrl(arguments.clusterProxyUrl); + } + if (arguments.clusterProxyProtocol != null) { + clusterDataBuilder.proxyProtocol(arguments.clusterProxyProtocol); + } + + ClusterData clusterData = clusterDataBuilder.build(); if (!resources.getClusterResources().clusterExists(arguments.cluster)) { resources.getClusterResources().createCluster(arguments.cluster, clusterData); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataSetupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataSetupTest.java index da5914f60e2ac..4267c7564fa6f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataSetupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataSetupTest.java @@ -44,6 +44,7 @@ import org.apache.pulsar.PulsarInitialNamespaceSetup; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.resources.TenantResources; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.functions.worker.WorkerUtils; @@ -86,6 +87,48 @@ public void testReSetupClusterMetadata() throws Exception { PulsarClusterMetadataSetup.main(args); SortedMap data3 = localZkS.dumpData(); assertEquals(data1, data3); + String clusterDataJson = data1.get("/admin/clusters/testReSetupClusterMetadata-cluster"); + assertNotNull(clusterDataJson); + ClusterData clusterData = ObjectMapperFactory + .getMapper() + .reader() + .readValue(clusterDataJson, ClusterData.class); + assertEquals(clusterData.getServiceUrl(), "http://127.0.0.1:8080"); + assertEquals(clusterData.getServiceUrlTls(), "https://127.0.0.1:8443"); + assertEquals(clusterData.getBrokerServiceUrl(), "pulsar://127.0.0.1:6650"); + assertEquals(clusterData.getBrokerServiceUrlTls(), "pulsar+ssl://127.0.0.1:6651"); + assertFalse(clusterData.isBrokerClientTlsEnabled()); + } + + public void testSetupClusterMetadataWithAuthEnabled() throws Exception { + String clusterName = "cluster-with-auth"; + String[] args = { + "--cluster", clusterName, + "--zookeeper", "127.0.0.1:" + localZkS.getZookeeperPort(), + "--configuration-store", "127.0.0.1:" + localZkS.getZookeeperPort(), + "--web-service-url", "http://127.0.0.1:8080", + "--web-service-url-tls", "https://127.0.0.1:8443", + "--broker-service-url", "pulsar://127.0.0.1:6650", + "--broker-service-url-tls", "pulsar+ssl://127.0.0.1:6651", + "--tls-enable", + "--auth-plugin", "org.apache.pulsar.client.impl.auth.AuthenticationToken", + "--auth-parameters", "token:my-token" + }; + PulsarClusterMetadataSetup.main(args); + SortedMap data = localZkS.dumpData(); + String clusterDataJson = data.get("/admin/clusters/" + clusterName); + assertNotNull(clusterDataJson); + ClusterData clusterData = ObjectMapperFactory + .getMapper() + .reader() + .readValue(clusterDataJson, ClusterData.class); + assertEquals(clusterData.getServiceUrl(), "http://127.0.0.1:8080"); + assertEquals(clusterData.getServiceUrlTls(), "https://127.0.0.1:8443"); + assertEquals(clusterData.getBrokerServiceUrl(), "pulsar://127.0.0.1:6650"); + assertEquals(clusterData.getBrokerServiceUrlTls(), "pulsar+ssl://127.0.0.1:6651"); + assertTrue(clusterData.isBrokerClientTlsEnabled()); + assertEquals(clusterData.getAuthenticationPlugin(), "org.apache.pulsar.client.impl.auth.AuthenticationToken"); + assertEquals(clusterData.getAuthenticationParameters(), "token:my-token"); } @DataProvider(name = "bundleNumberForDefaultNamespace") From b955c6520d8db948048a1b2dc548a001ee396076 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 29 Jul 2024 23:06:45 +0800 Subject: [PATCH 401/580] [fix] [broker] Internal reader of __change_events can not started after metadata store session rebuilt (#23018) --- .../extensions/ExtensibleLoadManagerImpl.java | 3 +- .../impl/ModularLoadManagerImpl.java | 3 +- .../pulsar/broker/service/Ipv4Proxy.java | 197 ++++++++++++ .../broker/service/NetworkErrorTestBase.java | 297 ++++++++++++++++++ .../broker/service/ZkSessionExpireTest.java | 184 +++++++++++ .../metadata/impl/AbstractMetadataStore.java | 10 + 6 files changed, 692 insertions(+), 2 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/Ipv4Proxy.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index a737a94b998ac..9450c2a9cc467 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -127,7 +127,8 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS private static final Set INTERNAL_TOPICS = Set.of(BROKER_LOAD_DATA_STORE_TOPIC, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TOPIC); - private PulsarService pulsar; + @VisibleForTesting + protected PulsarService pulsar; private ServiceConfiguration conf; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index 8f095b7d84df8..ada1ab665b67f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -158,8 +158,9 @@ public class ModularLoadManagerImpl implements ModularLoadManager { // Policies used to determine which brokers are available for particular namespaces. private SimpleResourceAllocationPolicies policies; + @VisibleForTesting // Pulsar service used to initialize this. - private PulsarService pulsar; + protected PulsarService pulsar; private PulsarResources pulsarResources; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/Ipv4Proxy.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/Ipv4Proxy.java new file mode 100644 index 0000000000000..a84dab4d17dff --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/Ipv4Proxy.java @@ -0,0 +1,197 @@ +/* + * 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.service; + +import io.netty.bootstrap.Bootstrap; +import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.handler.logging.LogLevel; +import io.netty.handler.logging.LoggingHandler; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import lombok.Getter; + +public class Ipv4Proxy { + @Getter + private final int localPort; + private final String backendServerHost; + private final int backendServerPort; + private final EventLoopGroup serverGroup = new NioEventLoopGroup(1); + private final EventLoopGroup workerGroup = new NioEventLoopGroup(); + private ChannelFuture localServerChannel; + private ServerBootstrap serverBootstrap = new ServerBootstrap(); + private List frontChannels = Collections.synchronizedList(new ArrayList<>()); + private AtomicBoolean rejectAllConnections = new AtomicBoolean(); + + public Ipv4Proxy(int localPort, String backendServerHost, int backendServerPort) { + this.localPort = localPort; + this.backendServerHost = backendServerHost; + this.backendServerPort = backendServerPort; + } + + public synchronized void startup() throws InterruptedException { + localServerChannel = serverBootstrap.group(serverGroup, workerGroup) + .channel(NioServerSocketChannel.class) + .handler(new LoggingHandler(LogLevel.INFO)) + .childHandler(new ChannelInitializer() { + @Override + protected void initChannel(SocketChannel ch) { + ch.pipeline().addLast(new FrontendHandler()); + } + }).childOption(ChannelOption.AUTO_READ, false) + .bind(localPort).sync(); + } + + public synchronized void stop() throws InterruptedException{ + localServerChannel.channel().close().sync(); + serverGroup.shutdownGracefully(); + workerGroup.shutdownGracefully(); + } + + private static void closeOnFlush(Channel ch) { + if (ch.isActive()) { + ch.writeAndFlush(Unpooled.EMPTY_BUFFER).addListener(ChannelFutureListener.CLOSE); + } + } + + public void disconnectFrontChannels() throws InterruptedException { + for (Channel channel : frontChannels) { + channel.close(); + } + } + + public void rejectAllConnections() throws InterruptedException { + rejectAllConnections.set(true); + } + + public void unRejectAllConnections() throws InterruptedException { + rejectAllConnections.set(false); + } + + private class FrontendHandler extends ChannelInboundHandlerAdapter { + + private Channel backendChannel; + + @Override + public void channelActive(ChannelHandlerContext ctx) { + if (rejectAllConnections.get()) { + ctx.close(); + return; + } + final Channel frontendChannel = ctx.channel(); + frontChannels.add(frontendChannel); + Bootstrap backendBootstrap = new Bootstrap(); + backendBootstrap.group(frontendChannel.eventLoop()) + .channel(ctx.channel().getClass()) + .handler(new BackendHandler(frontendChannel)) + .option(ChannelOption.AUTO_READ, false); + ChannelFuture backendChannelFuture = + backendBootstrap.connect(Ipv4Proxy.this.backendServerHost, Ipv4Proxy.this.backendServerPort); + backendChannel = backendChannelFuture.channel(); + backendChannelFuture.addListener((ChannelFutureListener) future -> { + if (future.isSuccess()) { + frontendChannel.read(); + } else { + frontChannels.remove(frontendChannel); + frontendChannel.close(); + } + }); + } + + @Override + public void channelRead(final ChannelHandlerContext ctx, Object msg) { + if (backendChannel.isActive()) { + backendChannel.writeAndFlush(msg).addListener((ChannelFutureListener) future -> { + if (future.isSuccess()) { + ctx.channel().read(); + } else { + future.channel().close(); + } + }); + } + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) { + frontChannels.remove(ctx.channel()); + if (backendChannel != null) { + closeOnFlush(backendChannel); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + cause.printStackTrace(); + closeOnFlush(ctx.channel()); + } + } + + private class BackendHandler extends ChannelInboundHandlerAdapter { + + private final Channel frontendChannel; + + public BackendHandler(Channel inboundChannel) { + this.frontendChannel = inboundChannel; + } + + @Override + public void channelActive(ChannelHandlerContext ctx) { + if (!frontendChannel.isActive()) { + closeOnFlush(ctx.channel()); + } else { + ctx.read(); + } + } + + @Override + public void channelRead(final ChannelHandlerContext ctx, Object msg) { + frontendChannel.writeAndFlush(msg).addListener((ChannelFutureListener) future -> { + if (future.isSuccess()) { + ctx.channel().read(); + } else { + future.channel().close(); + } + }); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) { + closeOnFlush(frontendChannel); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + cause.printStackTrace(); + closeOnFlush(ctx.channel()); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java new file mode 100644 index 0000000000000..36f8cb4761248 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java @@ -0,0 +1,297 @@ +/* + * 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.service; + +import com.google.common.collect.Sets; +import java.io.IOException; +import java.net.ServerSocket; +import java.net.URL; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; +import org.apache.pulsar.broker.namespace.LookupOptions; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.ClientBuilder; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.naming.ServiceUnitId; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.tests.TestRetrySupport; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.apache.pulsar.zookeeper.ZookeeperServerTest; +import org.awaitility.reflect.WhiteboxImpl; + +@Slf4j +public abstract class NetworkErrorTestBase extends TestRetrySupport { + + protected final String defaultTenant = "public"; + protected final String defaultNamespace = defaultTenant + "/default"; + protected final String cluster1 = "r1"; + protected URL url1; + protected URL urlTls1; + protected URL url2; + protected URL urlTls2; + protected ServiceConfiguration config1 = new ServiceConfiguration(); + protected ServiceConfiguration config2 = new ServiceConfiguration(); + protected ZookeeperServerTest brokerConfigZk1; + protected Ipv4Proxy metadataZKProxy; + protected LocalBookkeeperEnsemble bkEnsemble1; + protected PulsarService pulsar1; + protected PulsarService pulsar2; + protected BrokerService broker1; + protected BrokerService broker2; + protected PulsarAdmin admin1; + protected PulsarAdmin admin2; + protected PulsarClient client1; + protected PulsarClient client2; + + private final static AtomicReference preferBroker = new AtomicReference<>(); + + protected void startZKAndBK() throws Exception { + // Start ZK & BK. + bkEnsemble1 = new LocalBookkeeperEnsemble(3, 0, () -> 0); + bkEnsemble1.start(); + + metadataZKProxy = new Ipv4Proxy(getOneFreePort(), "127.0.0.1", bkEnsemble1.getZookeeperPort()); + metadataZKProxy.startup(); + } + + protected void startBrokers() throws Exception { + // Start brokers. + setConfigDefaults(config1, cluster1, metadataZKProxy.getLocalPort()); + pulsar1 = new PulsarService(config1); + pulsar1.start(); + broker1 = pulsar1.getBrokerService(); + url1 = new URL(pulsar1.getWebServiceAddress()); + urlTls1 = new URL(pulsar1.getWebServiceAddressTls()); + + setConfigDefaults(config2, cluster1, bkEnsemble1.getZookeeperPort()); + pulsar2 = new PulsarService(config2); + pulsar2.start(); + broker2 = pulsar2.getBrokerService(); + url2 = new URL(pulsar2.getWebServiceAddress()); + urlTls2 = new URL(pulsar2.getWebServiceAddressTls()); + + log.info("broker-1: {}, broker-2: {}", broker1.getListenPort(), broker2.getListenPort()); + } + + protected int getOneFreePort() throws IOException { + ServerSocket serverSocket = new ServerSocket(0); + int port = serverSocket.getLocalPort(); + serverSocket.close(); + return port; + } + + protected void startAdminClient() throws Exception { + admin1 = PulsarAdmin.builder().serviceHttpUrl(url1.toString()).build(); + admin2 = PulsarAdmin.builder().serviceHttpUrl(url2.toString()).build(); + } + + protected void startPulsarClient() throws Exception{ + ClientBuilder clientBuilder1 = PulsarClient.builder().serviceUrl(url1.toString()); + client1 = initClient(clientBuilder1); + ClientBuilder clientBuilder2 = PulsarClient.builder().serviceUrl(url2.toString()); + client2 = initClient(clientBuilder2); + } + + protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { + admin1.clusters().createCluster(cluster1, ClusterData.builder() + .serviceUrl(url1.toString()) + .serviceUrlTls(urlTls1.toString()) + .brokerServiceUrl(pulsar1.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()) + .brokerClientTlsEnabled(false) + .build()); + admin1.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(cluster1))); + admin1.namespaces().createNamespace(defaultNamespace, Sets.newHashSet(cluster1)); + } + + @Override + protected void setup() throws Exception { + incrementSetupNumber(); + + log.info("--- Starting OneWayReplicatorTestBase::setup ---"); + + startZKAndBK(); + + startBrokers(); + + startAdminClient(); + + createDefaultTenantsAndClustersAndNamespace(); + + startPulsarClient(); + + Thread.sleep(100); + log.info("--- OneWayReplicatorTestBase::setup completed ---"); + } + + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, int zkPort) { + config.setClusterName(clusterName); + config.setAdvertisedAddress("localhost"); + config.setWebServicePort(Optional.of(0)); + config.setWebServicePortTls(Optional.of(0)); + config.setMetadataStoreUrl("zk:127.0.0.1:" + zkPort); + config.setConfigurationMetadataStoreUrl("zk:127.0.0.1:" + zkPort + "/config_meta"); + config.setBrokerDeleteInactiveTopicsEnabled(false); + config.setBrokerDeleteInactiveTopicsFrequencySeconds(60); + config.setBrokerShutdownTimeoutMs(0L); + config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); + config.setBrokerServicePort(Optional.of(0)); + config.setBrokerServicePortTls(Optional.of(0)); + config.setBacklogQuotaCheckIntervalInSeconds(5); + config.setDefaultNumberOfNamespaceBundles(1); + config.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); + config.setEnableReplicatedSubscriptions(true); + config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); + config.setLoadBalancerSheddingEnabled(false); + config.setForceDeleteNamespaceAllowed(true); + config.setLoadManagerClassName(PreferBrokerModularLoadManager.class.getName()); + config.setMetadataStoreSessionTimeoutMillis(5000); + } + + @Override + protected void cleanup() throws Exception { + // shutdown. + markCurrentSetupNumberCleaned(); + log.info("--- Shutting down ---"); + + // Stop brokers. + if (client1 != null) { + client1.close(); + client1 = null; + } + if (admin1 != null) { + admin1.close(); + admin1 = null; + } + if (client2 != null) { + client2.close(); + client2 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } + if (pulsar1 != null) { + pulsar1.close(); + pulsar1 = null; + } + if (pulsar2 != null) { + pulsar2.close(); + pulsar2 = null; + } + + // Stop ZK and BK. + if (bkEnsemble1 != null) { + bkEnsemble1.stop(); + bkEnsemble1 = null; + } + if (metadataZKProxy != null) { + metadataZKProxy.stop(); + } + if (brokerConfigZk1 != null) { + brokerConfigZk1.stop(); + brokerConfigZk1 = null; + } + + // Reset configs. + config1 = new ServiceConfiguration(); + preferBroker.set(null); + } + + protected PulsarClient initClient(ClientBuilder clientBuilder) throws Exception { + return clientBuilder.build(); + } + + protected static class PreferBrokerModularLoadManager extends ModularLoadManagerImpl { + + @Override + public String setNamespaceBundleAffinity(String bundle, String broker) { + if (StringUtils.isNotBlank(broker)) { + return broker; + } + Set availableBrokers = NetworkErrorTestBase.getAvailableBrokers(super.pulsar); + String prefer = preferBroker.get(); + if (availableBrokers.contains(prefer)) { + return prefer; + } else { + return null; + } + } + } + + protected static class PreferExtensibleLoadManager extends ExtensibleLoadManagerImpl { + + @Override + public CompletableFuture> selectAsync(ServiceUnitId bundle, + Set excludeBrokerSet, + LookupOptions options) { + Set availableBrokers = NetworkErrorTestBase.getAvailableBrokers(super.pulsar); + String prefer = preferBroker.get(); + if (availableBrokers.contains(prefer)) { + return CompletableFuture.completedFuture(Optional.of(prefer)); + } else { + return super.selectAsync(bundle, excludeBrokerSet, options); + } + } + } + + public void setPreferBroker(PulsarService target) { + for (PulsarService pulsar : Arrays.asList(pulsar1, pulsar2)) { + for (String broker : getAvailableBrokers(pulsar)) { + if (broker.endsWith(target.getBrokerListenPort().orElse(-1) + "") + || broker.endsWith(target.getListenPortHTTPS().orElse(-1) + "") + || broker.endsWith(target.getListenPortHTTP().orElse(-1) + "") + || broker.endsWith(target.getBrokerListenPortTls().orElse(-1) + "")) { + preferBroker.set(broker); + } + } + } + } + + public static Set getAvailableBrokers(PulsarService pulsar) { + Object loadManagerWrapper = pulsar.getLoadManager().get(); + Object loadManager = WhiteboxImpl.getInternalState(loadManagerWrapper, "loadManager"); + if (loadManager instanceof ModularLoadManagerImpl) { + return ((ModularLoadManagerImpl) loadManager).getAvailableBrokers(); + } else if (loadManager instanceof ExtensibleLoadManagerImpl) { + return new HashSet<>(((ExtensibleLoadManagerImpl) loadManager).getBrokerRegistry() + .getAvailableBrokersAsync().join()); + } else { + throw new RuntimeException("Not support for the load manager: " + loadManager.getClass().getName()); + } + } + + public void clearPreferBroker() { + preferBroker.set(null); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java new file mode 100644 index 0000000000000..143557b008b23 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java @@ -0,0 +1,184 @@ +/* + * 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.service; + +import static org.testng.Assert.assertEquals; +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.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.ProducerImpl; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class ZkSessionExpireTest extends NetworkErrorTestBase { + + private java.util.function.Consumer settings; + + @AfterMethod(alwaysRun = true) + @Override + public void cleanup() throws Exception { + super.cleanup(); + } + + private void setupWithSettings(java.util.function.Consumer settings) throws Exception { + this.settings = settings; + super.setup(); + } + + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, int zkPort) { + super.setConfigDefaults(config, clusterName, zkPort); + settings.accept(config); + } + + @DataProvider(name = "settings") + public Object[][] settings() { + return new Object[][]{ + {false, NetworkErrorTestBase.PreferBrokerModularLoadManager.class}, + {true, NetworkErrorTestBase.PreferBrokerModularLoadManager.class} + // Create a separate PR to add this test case. + // {true, NetworkErrorTestBase.PreferExtensibleLoadManager.class}. + }; + } + + @Test(timeOut = 600 * 1000, dataProvider = "settings") + public void testTopicUnloadAfterSessionRebuild(boolean enableSystemTopic, Class loadManager) throws Exception { + // Setup. + setupWithSettings(config -> { + config.setManagedLedgerMaxEntriesPerLedger(1); + config.setSystemTopicEnabled(enableSystemTopic); + config.setTopicLevelPoliciesEnabled(enableSystemTopic); + if (loadManager != null) { + config.setLoadManagerClassName(loadManager.getName()); + } + }); + + // Init topic. + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp"); + admin1.topics().createNonPartitionedTopic(topicName); + admin1.topics().createSubscription(topicName, "s1", MessageId.earliest); + + // Inject a prefer mechanism, so that all topics will be assigned to broker1, which can be injected a ZK + // session expire error. + setPreferBroker(pulsar1); + admin1.namespaces().unload(defaultNamespace); + admin2.namespaces().unload(defaultNamespace); + + // Confirm all brokers registered. + Awaitility.await().untilAsserted(() -> { + assertEquals(getAvailableBrokers(pulsar1).size(), 2); + assertEquals(getAvailableBrokers(pulsar2).size(), 2); + }); + + // Load up a topic, and it will be assigned to broker1. + ProducerImpl p1 = (ProducerImpl) client1.newProducer(Schema.STRING).topic(topicName) + .sendTimeout(10, TimeUnit.SECONDS).create(); + Topic broker1Topic1 = pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + assertNotNull(broker1Topic1); + clearPreferBroker(); + + // Inject a ZK session expire error, and wait for broker1 to offline. + metadataZKProxy.rejectAllConnections(); + metadataZKProxy.disconnectFrontChannels(); + Awaitility.await().untilAsserted(() -> { + assertEquals(getAvailableBrokers(pulsar2).size(), 1); + }); + + // Send messages continuously. + // Verify: the topic was transferred to broker2. + CompletableFuture broker1Send1 = p1.sendAsync("broker1_msg1"); + Producer p2 = client2.newProducer(Schema.STRING).topic(topicName) + .sendTimeout(10, TimeUnit.SECONDS).create(); + CompletableFuture broker2Send1 = p2.sendAsync("broker2_msg1"); + Awaitility.await().untilAsserted(() -> { + CompletableFuture> future = pulsar2.getBrokerService().getTopic(topicName, false); + assertNotNull(future); + assertTrue(future.isDone() && !future.isCompletedExceptionally()); + Optional optional = future.join(); + assertTrue(optional != null && !optional.isEmpty()); + }); + + // Both two brokers assumed they are the owner of the topic. + Topic broker1Topic2 = pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + Topic broker2Topic2 = pulsar2.getBrokerService().getTopic(topicName, false).join().get(); + assertNotNull(broker1Topic2); + assertNotNull(broker2Topic2); + + // Send messages continuously. + // Publishing to broker-1 will fail. + // Publishing to broker-2 will success. + CompletableFuture broker1Send2 = p1.sendAsync("broker1_msg2"); + CompletableFuture broker2Send2 = p2.sendAsync("broker2_msg2"); + try { + broker1Send1.join(); + broker1Send2.join(); + p1.getClientCnx(); + fail("expected a publish error"); + } catch (Exception ex) { + // Expected. + } + broker2Send1.join(); + broker2Send2.join(); + + // Broker rebuild ZK session. + metadataZKProxy.unRejectAllConnections(); + Awaitility.await().untilAsserted(() -> { + assertEquals(getAvailableBrokers(pulsar1).size(), 2); + assertEquals(getAvailableBrokers(pulsar2).size(), 2); + }); + + // Verify: the topic on broker-1 will be unloaded. + // Verify: the topic on broker-2 is fine. + Awaitility.await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> { + CompletableFuture> future = pulsar1.getBrokerService().getTopic(topicName, false); + assertTrue(future == null || future.isCompletedExceptionally()); + }); + Topic broker2Topic3 = pulsar2.getBrokerService().getTopic(topicName, false).join().get(); + assertNotNull(broker2Topic3); + + // Send messages continuously. + // Verify: p1.send will success(it will connect to broker-2). + // Verify: p2.send will success. + CompletableFuture broker1Send3 = p1.sendAsync("broker1_msg3"); + CompletableFuture broker2Send3 = p2.sendAsync("broker2_msg3"); + broker1Send3.join(); + broker2Send3.join(); + + long msgBacklog = admin2.topics().getStats(topicName).getSubscriptions().get("s1").getMsgBacklog(); + log.info("msgBacklog: {}", msgBacklog); + + // cleanup. + p1.close(); + p2.close(); + admin2.topics().delete(topicName, false); + } +} diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java index f35f197463222..c458d0da2146a 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java @@ -496,6 +496,16 @@ public void registerSessionListener(Consumer listener) { protected void receivedSessionEvent(SessionEvent event) { isConnected = event.isConnected(); + + // Clear cache after session expired. + if (event == SessionEvent.SessionReestablished || event == SessionEvent.Reconnected) { + for (MetadataCacheImpl metadataCache : metadataCaches) { + metadataCache.invalidateAll(); + } + invalidateAll(); + } + + // Notice listeners. try { executor.execute(() -> { sessionListeners.forEach(l -> { From 47d35a0af8ef72062288866f0c875312b5684906 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 30 Jul 2024 00:47:34 +0800 Subject: [PATCH 402/580] [improve][build] Move docker-push profile to submodule (#23093) Signed-off-by: Zixuan Liu --- docker/pom.xml | 7 ------- docker/pulsar-all/pom.xml | 7 +++++++ docker/pulsar/pom.xml | 7 +++++++ 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/docker/pom.xml b/docker/pom.xml index f35da13a96e95..90a845400d3e6 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -75,12 +75,5 @@
- - docker-push - - false - linux/amd64,linux/arm64 - - diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 4f1e0425f0997..eb46aa339d61f 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -177,5 +177,12 @@ + + docker-push + + false + linux/amd64,linux/arm64 + + diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 0317effc90815..cd4cbec76372c 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -122,5 +122,12 @@ + + docker-push + + false + linux/amd64,linux/arm64 + + From 19fe2e4039205d20b9d715e5483c4695b7fbe606 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 30 Jul 2024 14:50:24 +0800 Subject: [PATCH 403/580] [cleanup][broker] Remove PersistentSubscription.getStats (#23095) Signed-off-by: Zixuan Liu --- .../persistent/PersistentSubscription.java | 43 +++++-------------- .../service/persistent/PersistentTopic.java | 22 ++++++---- .../broker/service/PersistentTopicTest.java | 34 +++++++++++---- 3 files changed, 49 insertions(+), 50 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 2dd890cfd2942..0a57f98eb7ad6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -31,9 +31,6 @@ import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -334,17 +331,18 @@ public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor // when topic closes: it iterates through concurrent-subscription map to close each subscription. so, // topic.remove again try to access same map which creates deadlock. so, execute it in different thread. topic.getBrokerService().pulsar().getExecutor().execute(() -> { - topic.removeSubscription(subName); - // Also need remove the cursor here, otherwise the data deletion will not work well. - // Because data deletion depends on the mark delete position of all cursors. - if (!isResetCursor) { - try { - topic.getManagedLedger().deleteCursor(cursor.getName()); - topic.getManagedLedger().removeWaitingCursor(cursor); - } catch (InterruptedException | ManagedLedgerException e) { - log.warn("[{}] [{}] Failed to remove non durable cursor", topic.getName(), subName, e); + topic.removeSubscription(subName).thenRunAsync(() -> { + // Also need remove the cursor here, otherwise the data deletion will not work well. + // Because data deletion depends on the mark delete position of all cursors. + if (!isResetCursor) { + try { + topic.getManagedLedger().deleteCursor(cursor.getName()); + topic.getManagedLedger().removeWaitingCursor(cursor); + } catch (InterruptedException | ManagedLedgerException e) { + log.warn("[{}] [{}] Failed to remove non durable cursor", topic.getName(), subName, e); + } } - } + }, topic.getBrokerService().pulsar().getExecutor()); }); } else { topic.getManagedLedger().removeWaitingCursor(cursor); @@ -1202,25 +1200,6 @@ public long estimateBacklogSize() { return cursor.getEstimatedSizeSinceMarkDeletePosition(); } - /** - * @deprecated please call {@link #getStatsAsync(GetStatsOptions)}. - */ - @Deprecated - public SubscriptionStatsImpl getStats(GetStatsOptions getStatsOptions) { - // So far, there is no case hits this check. - if (getStatsOptions.isGetEarliestTimeInBacklog()) { - throw new IllegalArgumentException("Calling the sync method subscription.getStats with" - + " getEarliestTimeInBacklog, it may encountered a deadlock error."); - } - // The method "getStatsAsync" will be a sync method if the param "isGetEarliestTimeInBacklog" is false. - try { - return getStatsAsync(getStatsOptions).get(5, TimeUnit.SECONDS); - } catch (InterruptedException | ExecutionException | TimeoutException e) { - // This error will never occur. - throw new RuntimeException(e); - } - } - public CompletableFuture getStatsAsync(GetStatsOptions getStatsOptions) { SubscriptionStatsImpl subStats = new SubscriptionStatsImpl(); subStats.lastExpireTimestamp = lastExpireTimestamp; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 42487d7239cc6..7926545647e0d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -1402,19 +1402,23 @@ public void deleteCursorFailed(ManagedLedgerException exception, Object ctx) { }, null); } - void removeSubscription(String subscriptionName) { + CompletableFuture removeSubscription(String subscriptionName) { PersistentSubscription sub = subscriptions.remove(subscriptionName); if (sub != null) { // preserve accumulative stats form removed subscription - SubscriptionStatsImpl stats = sub.getStats(new GetStatsOptions(false, false, false, false, false)); - bytesOutFromRemovedSubscriptions.add(stats.bytesOutCounter); - msgOutFromRemovedSubscriptions.add(stats.msgOutCounter); - - if (isSystemCursor(subscriptionName) - || subscriptionName.startsWith(SystemTopicNames.SYSTEM_READER_PREFIX)) { - bytesOutFromRemovedSystemSubscriptions.add(stats.bytesOutCounter); - } + return sub + .getStatsAsync(new GetStatsOptions(false, false, false, false, false)) + .thenAccept(stats -> { + bytesOutFromRemovedSubscriptions.add(stats.bytesOutCounter); + msgOutFromRemovedSubscriptions.add(stats.msgOutCounter); + + if (isSystemCursor(subscriptionName) + || subscriptionName.startsWith(SystemTopicNames.SYSTEM_READER_PREFIX)) { + bytesOutFromRemovedSystemSubscriptions.add(stats.bytesOutCounter); + } + }); } + return CompletableFuture.completedFuture(null); } /** 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 8c21301c15b4c..e83b1bd9b7b79 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 @@ -21,6 +21,7 @@ import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgsRecordingInvocations; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyLong; @@ -53,6 +54,7 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.URL; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -2187,9 +2189,14 @@ public void testKeySharedMetadataExposedToStats() throws Exception { sub1.addConsumer(consumer1); consumer1.close(); - SubscriptionStatsImpl stats1 = sub1.getStats(new GetStatsOptions(false, false, false, false, false)); - assertEquals(stats1.keySharedMode, "AUTO_SPLIT"); - assertFalse(stats1.allowOutOfOrderDelivery); + CompletableFuture stats1Async = + sub1.getStatsAsync(new GetStatsOptions(false, false, false, false, false)); + assertThat(stats1Async).succeedsWithin(Duration.ofSeconds(3)) + .matches(stats1 -> { + assertEquals(stats1.keySharedMode, "AUTO_SPLIT"); + assertFalse(stats1.allowOutOfOrderDelivery); + return true; + }); Consumer consumer2 = new Consumer(sub2, SubType.Key_Shared, topic.getName(), 2, 0, "Cons2", true, serverCnx, "myrole-1", Collections.emptyMap(), false, @@ -2198,9 +2205,14 @@ public void testKeySharedMetadataExposedToStats() throws Exception { sub2.addConsumer(consumer2); consumer2.close(); - SubscriptionStatsImpl stats2 = sub2.getStats(new GetStatsOptions(false, false, false, false, false)); - assertEquals(stats2.keySharedMode, "AUTO_SPLIT"); - assertTrue(stats2.allowOutOfOrderDelivery); + CompletableFuture stats2Async = + sub2.getStatsAsync(new GetStatsOptions(false, false, false, false, false)); + assertThat(stats2Async).succeedsWithin(Duration.ofSeconds(3)) + .matches(stats2 -> { + assertEquals(stats2.keySharedMode, "AUTO_SPLIT"); + assertTrue(stats2.allowOutOfOrderDelivery); + return true; + }); KeySharedMeta ksm = new KeySharedMeta().setKeySharedMode(KeySharedMode.STICKY) .setAllowOutOfOrderDelivery(false); @@ -2210,9 +2222,13 @@ public void testKeySharedMetadataExposedToStats() throws Exception { sub3.addConsumer(consumer3); consumer3.close(); - SubscriptionStatsImpl stats3 = sub3.getStats(new GetStatsOptions(false, false, false, false, false)); - assertEquals(stats3.keySharedMode, "STICKY"); - assertFalse(stats3.allowOutOfOrderDelivery); + CompletableFuture stats3Async = sub3.getStatsAsync(new GetStatsOptions(false, false, false, false, false)); + assertThat(stats3Async).succeedsWithin(Duration.ofSeconds(3)) + .matches(stats3 -> { + assertEquals(stats3.keySharedMode, "STICKY"); + assertFalse(stats3.allowOutOfOrderDelivery); + return true; + }); } private ByteBuf getMessageWithMetadata(byte[] data) { From 6bbaec1f6b1cc09de42f14dccca1afd932c547d5 Mon Sep 17 00:00:00 2001 From: atomchen <492672043@qq.com> Date: Tue, 30 Jul 2024 15:11:05 +0800 Subject: [PATCH 404/580] [fix][client] TransactionCoordinatorClient support retry (#23081) --- .../TransactionCoordinatorClientTest.java | 24 +++++++++++++++++ .../impl/TransactionMetaStoreHandler.java | 26 ++++++++++++++++--- .../TransactionCoordinatorClientImpl.java | 4 +-- 3 files changed, 48 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/coordinator/TransactionCoordinatorClientTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/coordinator/TransactionCoordinatorClientTest.java index c442c3a901464..36bc0e522c210 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/coordinator/TransactionCoordinatorClientTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/coordinator/TransactionCoordinatorClientTest.java @@ -24,14 +24,18 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import lombok.Cleanup; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.TransactionMetadataStoreService; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.transaction.TransactionBufferClient; +import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient.State; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException; import org.apache.pulsar.client.api.transaction.TxnID; +import org.apache.pulsar.client.impl.transaction.TransactionCoordinatorClientImpl; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.Test; @@ -107,4 +111,24 @@ public void testTransactionCoordinatorExceptionUnwrap() { instanceof TransactionCoordinatorClientException.InvalidTxnStatusException); } } + + @Test + public void testClientStartWithRetry() throws Exception{ + String validBrokerServiceUrl = pulsarServices[0].getBrokerServiceUrl(); + String invalidBrokerServiceUrl = "localhost:0"; + String brokerServiceUrl = validBrokerServiceUrl + "," + invalidBrokerServiceUrl; + + @Cleanup + PulsarClient pulsarClient = PulsarClient.builder().serviceUrl(brokerServiceUrl).build(); + @Cleanup + TransactionCoordinatorClient transactionCoordinatorClient = new TransactionCoordinatorClientImpl(pulsarClient); + + try { + transactionCoordinatorClient.start(); + }catch (TransactionCoordinatorClientException e) { + Assert.fail("Shouldn't have exception at here", e); + } + + Assert.assertEquals(transactionCoordinatorClient.getState(), State.READY); + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java index 2a43ca20beb38..e45d53971159e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; @@ -88,6 +89,10 @@ public RequestTime(long creationTime, long requestId) { private Timeout requestTimeout; private final CompletableFuture connectFuture; + private final long lookupDeadline; + private final List previousExceptions = new CopyOnWriteArrayList<>(); + + public TransactionMetaStoreHandler(long transactionCoordinatorId, PulsarClientImpl pulsarClient, String topic, CompletableFuture connectFuture) { @@ -109,6 +114,7 @@ public TransactionMetaStoreHandler(long transactionCoordinatorId, PulsarClientIm this.connectFuture = connectFuture; this.internalPinnedExecutor = pulsarClient.getInternalExecutorService(); this.timer = pulsarClient.timer(); + this.lookupDeadline = System.currentTimeMillis() + client.getConfiguration().getLookupTimeoutMs(); } public void start() { @@ -117,10 +123,22 @@ public void start() { @Override public void connectionFailed(PulsarClientException exception) { - LOG.error("Transaction meta handler with transaction coordinator id {} connection failed.", - transactionCoordinatorId, exception); - if (!this.connectFuture.isDone()) { - this.connectFuture.completeExceptionally(exception); + boolean nonRetriableError = !PulsarClientException.isRetriableError(exception); + boolean timeout = System.currentTimeMillis() > lookupDeadline; + if (nonRetriableError || timeout) { + exception.setPreviousExceptions(previousExceptions); + if (connectFuture.completeExceptionally(exception)) { + if (nonRetriableError) { + LOG.error("Transaction meta handler with transaction coordinator id {} connection failed.", + transactionCoordinatorId, exception); + } else { + LOG.error("Transaction meta handler with transaction coordinator id {} connection failed after " + + "timeout", transactionCoordinatorId, exception); + } + setState(State.Failed); + } + } else { + previousExceptions.add(exception); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionCoordinatorClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionCoordinatorClientImpl.java index 499627f9c73f2..45a3ad4f978b1 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionCoordinatorClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionCoordinatorClientImpl.java @@ -79,8 +79,8 @@ public void start() throws TransactionCoordinatorClientException { @Override public CompletableFuture startAsync() { if (STATE_UPDATER.compareAndSet(this, State.NONE, State.STARTING)) { - return pulsarClient.getLookup() - .getPartitionedTopicMetadata(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, true) + return pulsarClient.getPartitionedTopicMetadata( + SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN.getPartitionedTopicName(), true) .thenCompose(partitionMeta -> { List> connectFutureList = new ArrayList<>(); if (LOG.isDebugEnabled()) { From 5aa5e7d9fbe55d7e22625d67e618aa4934c78ecb Mon Sep 17 00:00:00 2001 From: Anshul Singh Date: Wed, 31 Jul 2024 11:39:41 +0530 Subject: [PATCH 405/580] [fix] [broker] fix replicated namespaces filter in filterAndUnloadMatchedNamespaceAsync (#23100) Co-authored-by: Lari Hotari --- .../broker/admin/impl/ClustersBase.java | 77 +++++++----- ...ApiNamespaceIsolationMultiBrokersTest.java | 114 ++++++++++++++++++ 2 files changed, 158 insertions(+), 33 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiNamespaceIsolationMultiBrokersTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java index 6eb324a63f341..4fe8a01e679da 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java @@ -33,8 +33,8 @@ import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.regex.Pattern; import java.util.stream.Collectors; -import java.util.stream.Stream; import javax.ws.rs.DELETE; import javax.ws.rs.GET; import javax.ws.rs.POST; @@ -723,8 +723,8 @@ public void setNamespaceIsolationPolicy( ).thenCompose(nsIsolationPolicies -> { nsIsolationPolicies.setPolicy(policyName, policyData); return namespaceIsolationPolicies() - .setIsolationDataAsync(cluster, old -> nsIsolationPolicies.getPolicies()); - }).thenCompose(__ -> filterAndUnloadMatchedNamespaceAsync(policyData)) + .setIsolationDataAsync(cluster, old -> nsIsolationPolicies.getPolicies()); + }).thenCompose(__ -> filterAndUnloadMatchedNamespaceAsync(cluster, policyData)) .thenAccept(__ -> { log.info("[{}] Successful to update clusters/{}/namespaceIsolationPolicies/{}.", clientAppId(), cluster, policyName); @@ -758,42 +758,53 @@ public void setNamespaceIsolationPolicy( /** * Get matched namespaces; call unload for each namespaces. */ - private CompletableFuture filterAndUnloadMatchedNamespaceAsync(NamespaceIsolationDataImpl policyData) { + private CompletableFuture filterAndUnloadMatchedNamespaceAsync(String cluster, + NamespaceIsolationDataImpl policyData) { PulsarAdmin adminClient; try { adminClient = pulsar().getAdminClient(); } catch (PulsarServerException e) { return FutureUtil.failedFuture(e); } - return adminClient.tenants().getTenantsAsync() - .thenCompose(tenants -> { - Stream>> completableFutureStream = tenants.stream() - .map(tenant -> adminClient.namespaces().getNamespacesAsync(tenant)); - return FutureUtil.waitForAll(completableFutureStream) - .thenApply(namespaces -> { - // if namespace match any policy regex, add it to ns list to be unload. - return namespaces.stream() - .filter(namespaceName -> - policyData.getNamespaces().stream().anyMatch(namespaceName::matches)) - .collect(Collectors.toList()); - }); - }).thenCompose(shouldUnloadNamespaces -> { - if (CollectionUtils.isEmpty(shouldUnloadNamespaces)) { - return CompletableFuture.completedFuture(null); - } - List> futures = shouldUnloadNamespaces.stream() - .map(namespaceName -> adminClient.namespaces().unloadAsync(namespaceName)) - .collect(Collectors.toList()); - return FutureUtil.waitForAll(futures) - .thenAccept(__ -> { - try { - // write load info to load manager to make the load happens fast - pulsar().getLoadManager().get().writeLoadReportOnZookeeper(true); - } catch (Exception e) { - log.warn("[{}] Failed to writeLoadReportOnZookeeper.", clientAppId(), e); - } - }); - }); + // compile regex patterns once + List namespacePatterns = policyData.getNamespaces().stream().map(Pattern::compile).toList(); + return adminClient.tenants().getTenantsAsync().thenCompose(tenants -> { + List>> filteredNamespacesForEachTenant = tenants.stream() + .map(tenant -> adminClient.namespaces().getNamespacesAsync(tenant).thenCompose(namespaces -> { + List> namespaceNamesInCluster = namespaces.stream() + .filter(namespaceName -> namespacePatterns.stream() + .anyMatch(pattern -> pattern.matcher(namespaceName).matches())) + .map(namespaceName -> adminClient.namespaces().getPoliciesAsync(namespaceName) + .thenApply(policies -> policies.replication_clusters.contains(cluster) + ? namespaceName : null)) + .collect(Collectors.toList()); + return FutureUtil.waitForAll(namespaceNamesInCluster).thenApply( + __ -> namespaceNamesInCluster.stream() + .map(CompletableFuture::join) + .filter(Objects::nonNull) + .collect(Collectors.toList())); + })).toList(); + return FutureUtil.waitForAll(filteredNamespacesForEachTenant) + .thenApply(__ -> filteredNamespacesForEachTenant.stream() + .map(CompletableFuture::join) + .flatMap(List::stream) + .collect(Collectors.toList())); + }).thenCompose(shouldUnloadNamespaces -> { + if (CollectionUtils.isEmpty(shouldUnloadNamespaces)) { + return CompletableFuture.completedFuture(null); + } + List> futures = shouldUnloadNamespaces.stream() + .map(namespaceName -> adminClient.namespaces().unloadAsync(namespaceName)) + .collect(Collectors.toList()); + return FutureUtil.waitForAll(futures).thenAccept(__ -> { + try { + // write load info to load manager to make the load happens fast + pulsar().getLoadManager().get().writeLoadReportOnZookeeper(true); + } catch (Exception e) { + log.warn("[{}] Failed to writeLoadReportOnZookeeper.", clientAppId(), e); + } + }); + }); } @DELETE diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiNamespaceIsolationMultiBrokersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiNamespaceIsolationMultiBrokersTest.java new file mode 100644 index 0000000000000..da7d95d677af8 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiNamespaceIsolationMultiBrokersTest.java @@ -0,0 +1,114 @@ +/* + * 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.admin; + +import static org.testng.Assert.assertEquals; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.MultiBrokerBaseTest; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; +import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.NamespaceIsolationData; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.testng.Assert; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +/** + * Test multi-broker admin api. + */ +@Slf4j +@Test(groups = "broker-admin") +public class AdminApiNamespaceIsolationMultiBrokersTest extends MultiBrokerBaseTest { + + PulsarAdmin localAdmin; + PulsarAdmin remoteAdmin; + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + this.conf.setManagedLedgerMaxEntriesPerLedger(10); + } + + @Override + protected void onCleanup() { + super.onCleanup(); + } + + @BeforeClass + public void setupClusters() throws Exception { + localAdmin = getAllAdmins().get(1); + remoteAdmin = getAllAdmins().get(2); + String localBrokerWebService = additionalPulsarTestContexts.get(0).getPulsarService().getWebServiceAddress(); + String remoteBrokerWebService = additionalPulsarTestContexts.get(1).getPulsarService().getWebServiceAddress(); + localAdmin.clusters() + .createCluster("cluster-1", ClusterData.builder().serviceUrl(localBrokerWebService).build()); + remoteAdmin.clusters() + .createCluster("cluster-2", ClusterData.builder().serviceUrl(remoteBrokerWebService).build()); + TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of(""), Set.of("test", "cluster-1", "cluster-2")); + localAdmin.tenants().createTenant("prop-ig", tenantInfo); + localAdmin.namespaces().createNamespace("prop-ig/ns1", Set.of("test", "cluster-1")); + } + + public void testNamespaceIsolationPolicyForReplNS() throws Exception { + + // Verify that namespace is not present in cluster-2. + Set replicationClusters = localAdmin.namespaces().getPolicies("prop-ig/ns1").replication_clusters; + Assert.assertFalse(replicationClusters.contains("cluster-2")); + + // setup ns-isolation-policy in both the clusters. + String policyName1 = "policy-1"; + Map parameters1 = new HashMap<>(); + parameters1.put("min_limit", "1"); + parameters1.put("usage_threshold", "100"); + List nsRegexList = new ArrayList<>(Arrays.asList("prop-ig/.*")); + + NamespaceIsolationData nsPolicyData1 = NamespaceIsolationData.builder() + // "prop-ig/ns1" is present in test cluster, policy set on test2 should work + .namespaces(nsRegexList) + .primary(Collections.singletonList(".*")) + .secondary(Collections.singletonList("")) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters1) + .build()) + .build(); + + localAdmin.clusters().createNamespaceIsolationPolicy("test", policyName1, nsPolicyData1); + // verify policy is present in local cluster + Map policiesMap = + localAdmin.clusters().getNamespaceIsolationPolicies("test"); + assertEquals(policiesMap.get(policyName1), nsPolicyData1); + + remoteAdmin.clusters().createNamespaceIsolationPolicy("cluster-2", policyName1, nsPolicyData1); + // verify policy is present in remote cluster + policiesMap = remoteAdmin.clusters().getNamespaceIsolationPolicies("cluster-2"); + assertEquals(policiesMap.get(policyName1), nsPolicyData1); + + } + +} From c24953035b88e670541c7f736514cd6b260e6520 Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Wed, 31 Jul 2024 14:10:40 +0800 Subject: [PATCH 406/580] [improve][broker]Reuse method getAvailableBrokersAsync (#23099) --- .../pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index ada1ab665b67f..141e020d7ca45 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -350,8 +350,7 @@ private void reapDeadBrokerPreallocations(List aliveBrokers) { @Override public Set getAvailableBrokers() { try { - return new HashSet<>(brokersData.listLocks(LoadManager.LOADBALANCE_BROKERS_ROOT) - .get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS)); + return getAvailableBrokersAsync().get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); } catch (Exception e) { log.warn("Error when trying to get active brokers", e); return loadData.getBrokerData().keySet(); From 9d0292ebb034a624286a9ffdf992bb00085190e4 Mon Sep 17 00:00:00 2001 From: Marek Czajkowski <76772327+marekczajkowski@users.noreply.github.com> Date: Wed, 31 Jul 2024 14:26:02 +0200 Subject: [PATCH 407/580] [improve][pip] PIP-352: Event time based topic compactor (#22710) --- pip/pip-352.md | 68 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 68 insertions(+) create mode 100644 pip/pip-352.md diff --git a/pip/pip-352.md b/pip/pip-352.md new file mode 100644 index 0000000000000..31641e7e1e1b5 --- /dev/null +++ b/pip/pip-352.md @@ -0,0 +1,68 @@ +# PIP-352: Event time based topic compactor + +# Background knowledge + +Pulsar Topic Compaction provides a key-based data retention mechanism that allows you only to keep the most recent message associated with that key to reduce storage space and improve system efficiency. + +Another Pulsar's internal use case, the Topic Compaction of the new load balancer, changed the strategy of compaction. It only keeps the first value of the key. For more detail, see [PIP-215](https://github.com/apache/pulsar/issues/18099). + +There is also plugable topic compaction service present. For more details, see [PIP-278](https://github.com/apache/pulsar/pull/20624) + +More topic compaction details can be found in [Pulsar Topic Compaction](https://pulsar.apache.org/docs/en/concepts-topic-compaction/). + +# Motivation + +Currently, there are two types of compactors +available: `TwoPhaseCompactor` and `StrategicTwoPhaseCompactor`. The latter +is specifically utilized for internal load balancing purposes and is not +employed for regular compaction of Pulsar topics. On the other hand, the +former can be configured via `CompactionServiceFactory` in the +`broker.conf`. + +I believe it could be advantageous to introduce another type of topic +compactor that operates based on event time. Such a compactor would have +the capability to maintain desired messages within the topic while +preserving the order expected by external applications. Although +applications may send messages with the current event time, variations in +network conditions or redeliveries could result in messages being stored in +the Pulsar topic in a different order than intended. Implementing event +time-based checks could mitigate this inconvenience. + +# Goals +* No impact on current topic compation behavior +* Preserve the order of messages during compaction regardless of network latencies + +## In Scope +* Abstract TwoPhaseCompactor + +* Migrate the current implementation to a new abstraction + +* Introduce new compactor based on event time + +* Makes existing tests compatible with new implementations. + + +# High Level Design + +In order to change the way topic is compacted we need to create `EventTimeCompactionServiceFactory`. This service provides a new +compactor `EventTimeOrderCompactor` which has a logic similar to existing `TwoPhaseCompactor` with a slightly change in algorithm responsible for +deciding which message is outdated. + +New compaction service factory can be enabled via `compactionServiceFactoryClassName` + +# Detailed Design + +## Design & Implementation Details + +* Abstract `TwoPhaseCompactor` and move current logic to new `PublishingOrderCompactor` + +* Implement `EventTimeCompactionServiceFactory` and `EventTimeOrderCompactor` + +* Create `MessageCompactionData` as a holder for compaction related data + +Example implementation can be found [here](https://github.com/apache/pulsar/pull/22517/files) + +# Links + +* Mailing List discussion thread: https://lists.apache.org/thread/nc8r3tm9xv03vl30zrmfhd19q2k308y2 +* Mailing List voting thread: https://lists.apache.org/thread/pp6c0qqw51yjw9szsnl2jbgjsqrx7wkn From 92bcd077c01058597fb910a49cbe01dffea41b36 Mon Sep 17 00:00:00 2001 From: Aurora Twinkle Date: Wed, 31 Jul 2024 21:42:56 +0800 Subject: [PATCH 408/580] [fix][broker]: fix irregular method name (#23108) Co-authored-by: duanlinlin Fixes: fix irregular method name --- .../src/main/java/org/apache/pulsar/broker/PulsarService.java | 2 +- .../apache/pulsar/client/api/OrphanPersistentTopicTest.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index b23851a5ec464..060e905d1f3c5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -2016,7 +2016,7 @@ protected BrokerService newBrokerService(PulsarService pulsar) throws Exception } @VisibleForTesting - public void setTransactionExecutorProvider(TransactionBufferProvider transactionBufferProvider) { + public void setTransactionBufferProvider(TransactionBufferProvider transactionBufferProvider) { this.transactionBufferProvider = transactionBufferProvider; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java index d6473efd788d8..b5af3cc6afd6c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java @@ -152,7 +152,7 @@ public CompletableFuture closeAsync() { } }; TransactionBufferProvider originalTransactionBufferProvider = pulsar.getTransactionBufferProvider(); - pulsar.setTransactionExecutorProvider(mockTransactionBufferProvider); + pulsar.setTransactionBufferProvider(mockTransactionBufferProvider); CompletableFuture> firstLoad = pulsar.getBrokerService().getTopic(tpName, true); Awaitility.await().ignoreExceptions().atMost(5, TimeUnit.SECONDS) .pollInterval(100, TimeUnit.MILLISECONDS) @@ -170,7 +170,7 @@ public CompletableFuture closeAsync() { } // set to back - pulsar.setTransactionExecutorProvider(originalTransactionBufferProvider); + pulsar.setTransactionBufferProvider(originalTransactionBufferProvider); pulsar.getConfig().setTopicLoadTimeoutSeconds(originalTopicLoadTimeoutSeconds); pulsar.getConfig().setBrokerDeduplicationEnabled(false); pulsar.getConfig().setTransactionCoordinatorEnabled(false); From f02ce6c83eef65d71e973ca8e2017d37135e7083 Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Fri, 2 Aug 2024 09:15:52 +0800 Subject: [PATCH 409/580] [fix][broker] fix exception may hidden and result in stuck when topic loading (#23102) Co-authored-by: fanjianye --- .../java/org/apache/pulsar/broker/service/BrokerService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 5dec15fc19b89..8460fe23ac3b7 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 @@ -1800,7 +1800,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { }); return null; }); - } catch (PulsarServerException e) { + } catch (Exception e) { log.warn("Failed to create topic {}: {}", topic, e.getMessage()); pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); topicFuture.completeExceptionally(e); From 12588a8f30f05198e7e82879bb3351290bc15888 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Fri, 2 Aug 2024 14:46:31 +0800 Subject: [PATCH 410/580] [fix][broker] type cast on exceptions in exceptionally can lead to lost calls (#23117) --- .../bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java | 6 ++++-- .../apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java | 8 ++++---- .../pulsar/broker/service/persistent/PersistentTopic.java | 8 +++----- .../org/apache/pulsar/compaction/CompactedTopicUtils.java | 8 +------- 4 files changed, 12 insertions(+), 18 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 398575461d5bf..8ba800ff88130 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -449,7 +449,8 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { }); return future; }).thenAccept(ml -> callback.openLedgerComplete(ml, ctx)).exceptionally(exception -> { - callback.openLedgerFailed((ManagedLedgerException) exception.getCause(), ctx); + callback.openLedgerFailed(ManagedLedgerException + .getManagedLedgerException(FutureUtil.unwrapCompletionException(exception)), ctx); return null; }); } @@ -475,7 +476,8 @@ public void asyncOpenReadOnlyManagedLedger(String managedLedgerName, callback.openReadOnlyManagedLedgerComplete(roManagedLedger, ctx); }).exceptionally(e -> { log.error("[{}] Failed to initialize Read-only managed ledger", managedLedgerName, e); - callback.openReadOnlyManagedLedgerFailed((ManagedLedgerException) e.getCause(), ctx); + callback.openReadOnlyManagedLedgerFailed(ManagedLedgerException + .getManagedLedgerException(FutureUtil.unwrapCompletionException(e)), ctx); return null; }); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 209bf57b24f0f..14d424dc7eacd 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -988,7 +988,8 @@ public synchronized void asyncOpenCursor(final String cursorName, final InitialP if (uninitializedCursors.containsKey(cursorName)) { uninitializedCursors.get(cursorName).thenAccept(cursor -> callback.openCursorComplete(cursor, ctx)) .exceptionally(ex -> { - callback.openCursorFailed((ManagedLedgerException) ex, ctx); + callback.openCursorFailed(ManagedLedgerException + .getManagedLedgerException(FutureUtil.unwrapCompletionException(ex)), ctx); return null; }); return; @@ -2975,9 +2976,8 @@ public void asyncDelete(final DeleteLedgerCallback callback, final Object ctx) { truncateFuture.whenComplete((ignore, exc) -> { if (exc != null) { log.error("[{}] Error truncating ledger for deletion", name, exc); - callback.deleteLedgerFailed(exc instanceof ManagedLedgerException - ? (ManagedLedgerException) exc : new ManagedLedgerException(exc), - ctx); + callback.deleteLedgerFailed(ManagedLedgerException.getManagedLedgerException( + FutureUtil.unwrapCompletionException(exc)), ctx); } else { asyncDeleteInternal(callback, ctx); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 7926545647e0d..7a520d879b782 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -4246,15 +4246,13 @@ public void publishTxnMessage(TxnID txnID, ByteBuf headersAndPayload, PublishCon decrementPendingWriteOpsAndCheck(); }) .exceptionally(throwable -> { - throwable = throwable.getCause(); + throwable = FutureUtil.unwrapCompletionException(throwable); if (throwable instanceof NotAllowedException) { publishContext.completed((NotAllowedException) throwable, -1, -1); decrementPendingWriteOpsAndCheck(); - return null; - } else if (!(throwable instanceof ManagedLedgerException)) { - throwable = new ManagedLedgerException(throwable); + } else { + addFailed(ManagedLedgerException.getManagedLedgerException(throwable), publishContext); } - addFailed((ManagedLedgerException) throwable, publishContext); return null; }); break; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java index aae332acfcbbc..5023180e0b979 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java @@ -102,13 +102,7 @@ public static void asyncReadCompactedEntries(TopicCompactionService topicCompact }); }).exceptionally((exception) -> { exception = FutureUtil.unwrapCompletionException(exception); - ManagedLedgerException managedLedgerException; - if (exception instanceof ManagedLedgerException) { - managedLedgerException = (ManagedLedgerException) exception; - } else { - managedLedgerException = new ManagedLedgerException(exception); - } - callback.readEntriesFailed(managedLedgerException, readEntriesCtx); + callback.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(exception), readEntriesCtx); return null; }); } From f3c177e2243e26a7849feb91dbed9fec4c5723c0 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Sat, 3 Aug 2024 15:30:22 +0800 Subject: [PATCH 411/580] [fix][client] the nullValue in msgMetadata should be true by default (#22372) Co-authored-by: xiangying ### Motivation When a message is not set value, the `nullValue` message metadata should be true and change to false after the value is set. Otherwise, the message data will be set as a [] when the value is not set, that would cause the message data to be encoded and throw a `SchemaSerializationException` when calling `reconsumerLater`. ``` org.apache.pulsar.client.api.PulsarClientException: java.util.concurrent.ExecutionException: org.apache.pulsar.client.api.SchemaSerializationException: Size of data received by IntSchema is not 4 at org.apache.pulsar.client.api.PulsarClientException.unwrap(PulsarClientException.java:1131) at org.apache.pulsar.client.impl.ConsumerBase.reconsumeLater(ConsumerBase.java:467) at org.apache.pulsar.client.impl.ConsumerBase.reconsumeLater(ConsumerBase.java:452) at org.apache.pulsar.client.api.ConsumerRedeliveryTest.testRedeliverMessagesWithoutValue(ConsumerRedeliveryTest.java:445) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:568) at org.testng.internal.invokers.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:139) at org.testng.internal.invokers.TestInvoker.invokeMethod(TestInvoker.java:677) at org.testng.internal.invokers.TestInvoker.invokeTestMethod(TestInvoker.java:221) at org.testng.internal.invokers.MethodRunner.runInSequence(MethodRunner.java:50) at org.testng.internal.invokers.TestInvoker$MethodInvocationAgent.invoke(TestInvoker.java:969) at org.testng.internal.invokers.TestInvoker.invokeTestMethods(TestInvoker.java:194) at org.testng.internal.invokers.TestMethodWorker.invokeTestMethods(TestMethodWorker.java:148) at org.testng.internal.invokers.TestMethodWorker.run(TestMethodWorker.java:128) at java.base/java.util.ArrayList.forEach(ArrayList.java:1511) at org.testng.TestRunner.privateRun(TestRunner.java:829) at org.testng.TestRunner.run(TestRunner.java:602) at org.testng.SuiteRunner.runTest(SuiteRunner.java:437) at org.testng.SuiteRunner.runSequentially(SuiteRunner.java:431) at org.testng.SuiteRunner.privateRun(SuiteRunner.java:391) at org.testng.SuiteRunner.run(SuiteRunner.java:330) at org.testng.SuiteRunnerWorker.runSuite(SuiteRunnerWorker.java:52) at org.testng.SuiteRunnerWorker.run(SuiteRunnerWorker.java:95) at org.testng.TestNG.runSuitesSequentially(TestNG.java:1256) at org.testng.TestNG.runSuitesLocally(TestNG.java:1176) at org.testng.TestNG.runSuites(TestNG.java:1099) at org.testng.TestNG.run(TestNG.java:1067) at com.intellij.rt.testng.IDEARemoteTestNG.run(IDEARemoteTestNG.java:65) at com.intellij.rt.testng.RemoteTestNGStarter.main(RemoteTestNGStarter.java:105) Caused by: java.util.concurrent.ExecutionException: org.apache.pulsar.client.api.SchemaSerializationException: Size of data received by IntSchema is not 4 at java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396) at java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2073) at org.apache.pulsar.client.impl.ConsumerBase.reconsumeLater(ConsumerBase.java:462) ... 29 more Caused by: org.apache.pulsar.client.api.SchemaSerializationException: Size of data received by IntSchema is not 4 at org.apache.pulsar.client.impl.schema.IntSchema.validate(IntSchema.java:49) at org.apache.pulsar.client.impl.schema.AutoProduceBytesSchema.encode(AutoProduceBytesSchema.java:80) at org.apache.pulsar.client.impl.schema.AutoProduceBytesSchema.encode(AutoProduceBytesSchema.java:32) at org.apache.pulsar.client.impl.TypedMessageBuilderImpl.lambda$value$3(TypedMessageBuilderImpl.java:157) at java.base/java.util.Optional.orElseGet(Optional.java:364) at org.apache.pulsar.client.impl.TypedMessageBuilderImpl.value(TypedMessageBuilderImpl.java:156) at org.apache.pulsar.client.impl.ConsumerImpl.doReconsumeLater(ConsumerImpl.java:689) at org.apache.pulsar.client.impl.MultiTopicsConsumerImpl.doReconsumeLater(MultiTopicsConsumerImpl.java:550) at org.apache.pulsar.client.impl.ConsumerBase.reconsumeLaterAsync(ConsumerBase.java:574) ``` ### Modifications When a message is not set value, the `nullValue` message metadata should be true and change to false after the value is set. --- .../client/api/ConsumerRedeliveryTest.java | 24 +++++++++++++ .../client/impl/MessageChecksumTest.java | 5 +++ .../pulsar/compaction/CompactionTest.java | 2 +- .../client/impl/TypedMessageBuilderImpl.java | 35 ++++++++++--------- .../impl/TypedMessageBuilderImplTest.java | 17 +++++++-- 5 files changed, 63 insertions(+), 20 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ConsumerRedeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ConsumerRedeliveryTest.java index 90114add25084..fcf1a638d5884 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ConsumerRedeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ConsumerRedeliveryTest.java @@ -424,4 +424,28 @@ public void testAckNotSent(int numAcked, int batchSize, CommandAck.AckType ackTy assertTrue(values.isEmpty()); } } + + @Test + public void testRedeliverMessagesWithoutValue() throws Exception { + String topic = "persistent://my-property/my-ns/testRedeliverMessagesWithoutValue"; + @Cleanup Consumer consumer = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub") + .enableRetry(true) + .subscribe(); + @Cleanup Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .enableBatching(true) + .create(); + for (int i = 0; i < 10; i++) { + producer.newMessage().key("messages without value").send(); + } + + Message message = consumer.receive(); + consumer.reconsumeLater(message, 2, TimeUnit.SECONDS); + for (int i = 0; i < 9; i++) { + assertNotNull(consumer.receive(5, TimeUnit.SECONDS)); + } + assertTrue(consumer.receive(5, TimeUnit.SECONDS).getTopicName().contains("sub-RETRY")); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java index 0b25e3409563a..94e763847506b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java @@ -24,6 +24,8 @@ import static org.testng.Assert.fail; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; + +import java.lang.reflect.Method; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -225,6 +227,9 @@ public void testTamperingMessageIsDetected() throws Exception { .create(); TypedMessageBuilderImpl msgBuilder = (TypedMessageBuilderImpl) producer.newMessage() .value("a message".getBytes()); + Method method = TypedMessageBuilderImpl.class.getDeclaredMethod("beforeSend"); + method.setAccessible(true); + method.invoke(msgBuilder); MessageMetadata msgMetadata = msgBuilder.getMetadataBuilder() .setProducerName("test") .setSequenceId(1) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java index 081831b0300e0..0cf32859e3dd6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java @@ -1384,7 +1384,7 @@ public void testEmptyPayloadDeletesWhenEncrypted() throws Exception { Message message4 = consumer.receive(); Assert.assertEquals(message4.getKey(), "key2"); - Assert.assertEquals(new String(message4.getData()), ""); + assertNull(message4.getData()); Message message5 = consumer.receive(); Assert.assertEquals(message5.getKey(), "key4"); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java index 026f8a1e69e0b..d90c2e8828364 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java @@ -50,6 +50,7 @@ public class TypedMessageBuilderImpl implements TypedMessageBuilder { private final transient Schema schema; private transient ByteBuffer content; private final transient TransactionImpl txn; + private transient T value; public TypedMessageBuilderImpl(ProducerBase producer, Schema schema) { this(producer, schema, null); @@ -65,6 +66,22 @@ public TypedMessageBuilderImpl(ProducerBase producer, } private long beforeSend() { + if (value == null) { + msgMetadata.setNullValue(true); + } else { + getKeyValueSchema().map(keyValueSchema -> { + if (keyValueSchema.getKeyValueEncodingType() == KeyValueEncodingType.SEPARATED) { + setSeparateKeyValue(value, keyValueSchema); + return this; + } else { + return null; + } + }).orElseGet(() -> { + content = ByteBuffer.wrap(schema.encode(value)); + return this; + }); + } + if (txn == null) { return -1L; } @@ -140,22 +157,8 @@ public TypedMessageBuilder orderingKey(byte[] orderingKey) { @Override public TypedMessageBuilder value(T value) { - if (value == null) { - msgMetadata.setNullValue(true); - return this; - } - - return getKeyValueSchema().map(keyValueSchema -> { - if (keyValueSchema.getKeyValueEncodingType() == KeyValueEncodingType.SEPARATED) { - setSeparateKeyValue(value, keyValueSchema); - return this; - } else { - return null; - } - }).orElseGet(() -> { - content = ByteBuffer.wrap(schema.encode(value)); - return this; - }); + this.value = value; + return this; } @Override diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TypedMessageBuilderImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TypedMessageBuilderImplTest.java index 94c683e527177..05db4402a1586 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TypedMessageBuilderImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TypedMessageBuilderImplTest.java @@ -27,6 +27,8 @@ import org.mockito.Mock; import org.testng.annotations.Test; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.nio.ByteBuffer; import java.util.Base64; @@ -45,7 +47,7 @@ public class TypedMessageBuilderImplTest { protected ProducerBase producerBase; @Test - public void testDefaultValue() { + public void testDefaultValue() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { producerBase = mock(ProducerBase.class); AvroSchema fooSchema = AvroSchema.of(SchemaDefinition.builder().withPojo(SchemaTestUtils.Foo.class).build()); @@ -63,6 +65,9 @@ public void testDefaultValue() { // Check kv.encoding.type default, not set value TypedMessageBuilderImpl typedMessageBuilder = (TypedMessageBuilderImpl)typedMessageBuilderImpl.value(keyValue); + Method method = TypedMessageBuilderImpl.class.getDeclaredMethod("beforeSend"); + method.setAccessible(true); + method.invoke(typedMessageBuilder); ByteBuffer content = typedMessageBuilder.getContent(); byte[] contentByte = new byte[content.remaining()]; content.get(contentByte); @@ -73,7 +78,7 @@ public void testDefaultValue() { } @Test - public void testInlineValue() { + public void testInlineValue() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { producerBase = mock(ProducerBase.class); AvroSchema fooSchema = AvroSchema.of(SchemaDefinition.builder().withPojo(SchemaTestUtils.Foo.class).build()); @@ -91,6 +96,9 @@ public void testInlineValue() { // Check kv.encoding.type INLINE TypedMessageBuilderImpl typedMessageBuilder = (TypedMessageBuilderImpl)typedMessageBuilderImpl.value(keyValue); + Method method = TypedMessageBuilderImpl.class.getDeclaredMethod("beforeSend"); + method.setAccessible(true); + method.invoke(typedMessageBuilder); ByteBuffer content = typedMessageBuilder.getContent(); byte[] contentByte = new byte[content.remaining()]; content.get(contentByte); @@ -101,7 +109,7 @@ public void testInlineValue() { } @Test - public void testSeparatedValue() { + public void testSeparatedValue() throws Exception { producerBase = mock(ProducerBase.class); AvroSchema fooSchema = AvroSchema.of(SchemaDefinition.builder().withPojo(SchemaTestUtils.Foo.class).build()); @@ -119,6 +127,9 @@ public void testSeparatedValue() { // Check kv.encoding.type SEPARATED TypedMessageBuilderImpl typedMessageBuilder = (TypedMessageBuilderImpl)typedMessageBuilderImpl.value(keyValue); + Method method = TypedMessageBuilderImpl.class.getDeclaredMethod("beforeSend"); + method.setAccessible(true); + method.invoke(typedMessageBuilder); ByteBuffer content = typedMessageBuilder.getContent(); byte[] contentByte = new byte[content.remaining()]; content.get(contentByte); From 76f16e811beb4f48fb2ae5c46558b74d333c7d60 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Mon, 5 Aug 2024 10:07:10 +0800 Subject: [PATCH 412/580] [improve][pip] PIP-366: Support to specify different config for Configuration and Local Metadata Store (#23041) --- .../pulsar/broker/ServiceConfiguration.java | 8 ++- .../apache/pulsar/broker/PulsarService.java | 6 +- .../metadata/impl/PulsarZooKeeperClient.java | 53 +++++++++++----- .../pulsar/metadata/impl/ZKMetadataStore.java | 2 + .../metadata/impl/oxia/OxiaMetadataStore.java | 21 ++++--- .../metadata/BaseMetadataStoreTest.java | 2 +- .../pulsar/metadata/MetadataStoreTest.java | 60 ++++++++++++++++++- .../src/test/resources/oxia_client.conf | 20 +++++++ .../resources/zk_client_disabled_sasl.conf | 20 +++++++ 9 files changed, 164 insertions(+), 28 deletions(-) create mode 100644 pulsar-metadata/src/test/resources/oxia_client.conf create mode 100644 pulsar-metadata/src/test/resources/zk_client_disabled_sasl.conf 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 2d2765287c0e0..26b2f99abf545 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 @@ -562,10 +562,16 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece @FieldContext( category = CATEGORY_SERVER, - doc = "Configuration file path for local metadata store. It's supported by RocksdbMetadataStore for now." + doc = "Configuration file path for local metadata store." ) private String metadataStoreConfigPath = null; + @FieldContext( + category = CATEGORY_SERVER, + doc = "Configuration file path for configuration metadata store." + ) + private String configurationStoreConfigPath = null; + @FieldContext( dynamic = true, category = CATEGORY_SERVER, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 060e905d1f3c5..3d57a3bc01042 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -391,11 +391,15 @@ public PulsarService(ServiceConfiguration config, public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchronizer synchronizer, OpenTelemetry openTelemetry) throws MetadataStoreException { + String configFilePath = config.getMetadataStoreConfigPath(); + if (StringUtils.isNotBlank(config.getConfigurationStoreConfigPath())) { + configFilePath = config.getConfigurationStoreConfigPath(); + } return MetadataStoreFactory.create(config.getConfigurationMetadataStoreUrl(), MetadataStoreConfig.builder() .sessionTimeoutMillis((int) config.getMetadataStoreSessionTimeoutMillis()) .allowReadOnlyOperations(config.isMetadataStoreAllowReadOnlyOperations()) - .configFilePath(config.getMetadataStoreConfigPath()) + .configFilePath(configFilePath) .batchingEnabled(config.isMetadataStoreBatchingEnabled()) .batchingMaxDelayMillis(config.getMetadataStoreBatchingMaxDelayMillis()) .batchingMaxOperations(config.getMetadataStoreBatchingMaxOperations()) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/PulsarZooKeeperClient.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/PulsarZooKeeperClient.java index cc29b615c1107..e8bfb39395a0e 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/PulsarZooKeeperClient.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/PulsarZooKeeperClient.java @@ -61,8 +61,10 @@ import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.Watcher.Event.KeeperState; import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.client.ZKClientConfig; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.server.quorum.QuorumPeerConfig; /** * Provide a zookeeper client to handle session expire. @@ -92,6 +94,9 @@ public class PulsarZooKeeperClient extends ZooKeeper implements Watcher, AutoClo private final RetryPolicy connectRetryPolicy; private final RetryPolicy operationRetryPolicy; + // Zookeeper config path + private final String configPath; + // Stats Logger private final OpStatsLogger createStats; private final OpStatsLogger getStats; @@ -120,8 +125,9 @@ public ZooKeeper call() throws KeeperException, InterruptedException { ZooKeeper newZk; try { newZk = createZooKeeper(); - } catch (IOException ie) { - log.error("Failed to create zookeeper instance to " + connectString, ie); + } catch (IOException | QuorumPeerConfig.ConfigException e) { + log.error("Failed to create zookeeper instance to {} with config path {}", + connectString, configPath, e); throw KeeperException.create(KeeperException.Code.CONNECTIONLOSS); } waitForConnection(); @@ -149,7 +155,7 @@ public String toString() { static PulsarZooKeeperClient createConnectedZooKeeperClient( String connectString, int sessionTimeoutMs, Set childWatchers, RetryPolicy operationRetryPolicy) - throws KeeperException, InterruptedException, IOException { + throws KeeperException, InterruptedException, IOException, QuorumPeerConfig.ConfigException { return PulsarZooKeeperClient.newBuilder() .connectString(connectString) .sessionTimeoutMs(sessionTimeoutMs) @@ -171,6 +177,7 @@ public static class Builder { int retryExecThreadCount = DEFAULT_RETRY_EXECUTOR_THREAD_COUNT; double requestRateLimit = 0; boolean allowReadOnlyMode = false; + String configPath = null; private Builder() {} @@ -219,7 +226,15 @@ public Builder allowReadOnlyMode(boolean allowReadOnlyMode) { return this; } - public PulsarZooKeeperClient build() throws IOException, KeeperException, InterruptedException { + public Builder configPath(String configPath) { + this.configPath = configPath; + return this; + } + + public PulsarZooKeeperClient build() throws IOException, + KeeperException, + InterruptedException, + QuorumPeerConfig.ConfigException { requireNonNull(connectString); checkArgument(sessionTimeoutMs > 0); requireNonNull(statsLogger); @@ -251,7 +266,8 @@ public PulsarZooKeeperClient build() throws IOException, KeeperException, Interr statsLogger, retryExecThreadCount, requestRateLimit, - allowReadOnlyMode + allowReadOnlyMode, + configPath ); // Wait for connection to be established. try { @@ -273,16 +289,19 @@ public static Builder newBuilder() { } protected PulsarZooKeeperClient(String connectString, - int sessionTimeoutMs, - ZooKeeperWatcherBase watcherManager, - RetryPolicy connectRetryPolicy, - RetryPolicy operationRetryPolicy, - StatsLogger statsLogger, - int retryExecThreadCount, - double rate, - boolean allowReadOnlyMode) throws IOException { - super(connectString, sessionTimeoutMs, watcherManager, allowReadOnlyMode); + int sessionTimeoutMs, + ZooKeeperWatcherBase watcherManager, + RetryPolicy connectRetryPolicy, + RetryPolicy operationRetryPolicy, + StatsLogger statsLogger, + int retryExecThreadCount, + double rate, + boolean allowReadOnlyMode, + String configPath) throws IOException, QuorumPeerConfig.ConfigException { + super(connectString, sessionTimeoutMs, watcherManager, allowReadOnlyMode, + configPath == null ? null : new ZKClientConfig(configPath)); this.connectString = connectString; + this.configPath = configPath; this.sessionTimeoutMs = sessionTimeoutMs; this.allowReadOnlyMode = allowReadOnlyMode; this.watcherManager = watcherManager; @@ -334,7 +353,11 @@ public void waitForConnection() throws KeeperException, InterruptedException { watcherManager.waitForConnection(); } - protected ZooKeeper createZooKeeper() throws IOException { + protected ZooKeeper createZooKeeper() throws IOException, QuorumPeerConfig.ConfigException { + if (null != configPath) { + return new ZooKeeper(connectString, sessionTimeoutMs, watcherManager, allowReadOnlyMode, + new ZKClientConfig(configPath)); + } return new ZooKeeper(connectString, sessionTimeoutMs, watcherManager, allowReadOnlyMode); } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java index 2e88cb3332467..603a4503dc8bb 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java @@ -100,6 +100,7 @@ public ZKMetadataStore(String metadataURL, MetadataStoreConfig metadataStoreConf .allowReadOnlyMode(metadataStoreConfig.isAllowReadOnlyOperations()) .sessionTimeoutMs(metadataStoreConfig.getSessionTimeoutMillis()) .watchers(Collections.singleton(this::processSessionWatcher)) + .configPath(metadataStoreConfig.getConfigFilePath()) .build(); if (enableSessionWatcher) { sessionWatcher = new ZKSessionWatcher(zkc, this::receivedSessionEvent); @@ -577,6 +578,7 @@ public CompletableFuture initializeCluster() { .connectRetryPolicy( new BoundExponentialBackoffRetryPolicy(metadataStoreConfig.getSessionTimeoutMillis(), metadataStoreConfig.getSessionTimeoutMillis(), 0)) + .configPath(metadataStoreConfig.getConfigFilePath()) .build()) { if (chrootZk.exists(chrootPath, false) == null) { createFullPathOptimistic(chrootZk, chrootPath, new byte[0], CreateMode.PERSISTENT); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java index e9da7ec7c1ab5..9141ad3d29cf7 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/oxia/OxiaMetadataStore.java @@ -41,6 +41,7 @@ import java.util.concurrent.CompletionStage; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataEventSynchronizer; import org.apache.pulsar.metadata.api.MetadataStoreConfig; @@ -80,15 +81,17 @@ public OxiaMetadataStore( } synchronizer = Optional.ofNullable(metadataStoreConfig.getSynchronizer()); identity = UUID.randomUUID().toString(); - client = - OxiaClientBuilder.create(serviceAddress) - .clientIdentifier(identity) - .namespace(namespace) - .sessionTimeout(Duration.ofMillis(metadataStoreConfig.getSessionTimeoutMillis())) - .batchLinger(Duration.ofMillis(linger)) - .maxRequestsPerBatch(metadataStoreConfig.getBatchingMaxOperations()) - .asyncClient() - .get(); + OxiaClientBuilder oxiaClientBuilder = OxiaClientBuilder + .create(serviceAddress) + .clientIdentifier(identity) + .namespace(namespace) + .sessionTimeout(Duration.ofMillis(metadataStoreConfig.getSessionTimeoutMillis())) + .batchLinger(Duration.ofMillis(linger)) + .maxRequestsPerBatch(metadataStoreConfig.getBatchingMaxOperations()); + if (StringUtils.isNotBlank(metadataStoreConfig.getConfigFilePath())) { + oxiaClientBuilder.loadConfig(metadataStoreConfig.getConfigFilePath()); + } + client = oxiaClientBuilder.asyncClient().get(); init(); } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java index 491e3d0b9640c..c77de92ae3c4c 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java @@ -100,7 +100,7 @@ public Object[][] distributedImplementations() { }; } - private synchronized String getOxiaServerConnectString() { + protected synchronized String getOxiaServerConnectString() { if (oxiaServer == null) { oxiaServer = new OxiaContainer(OxiaContainer.DEFAULT_IMAGE_NAME); oxiaServer.start(); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java index b1578188c681d..2c589dfd48222 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTest.java @@ -24,9 +24,11 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.UUID; @@ -36,7 +38,13 @@ import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; + +import io.streamnative.oxia.client.ClientConfig; +import io.streamnative.oxia.client.api.AsyncOxiaClient; +import io.streamnative.oxia.client.session.SessionFactory; +import io.streamnative.oxia.client.session.SessionManager; import lombok.Cleanup; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -51,9 +59,15 @@ import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.Stat; +import org.apache.pulsar.metadata.impl.PulsarZooKeeperClient; import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.apache.pulsar.metadata.impl.oxia.OxiaMetadataStore; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooKeeper; import org.assertj.core.util.Lists; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -457,7 +471,8 @@ public void testThreadSwitchOfZkMetadataStore(boolean hasSynchronizer, boolean e MetadataStoreConfig config = builder.build(); @Cleanup ZKMetadataStore store = (ZKMetadataStore) MetadataStoreFactory.create(zks.getConnectionString(), config); - + ZooKeeper zkClient = store.getZkClient(); + assertTrue(zkClient.getClientConfig().isSaslClientEnabled()); final Runnable verify = () -> { String currentThreadName = Thread.currentThread().getName(); String errorMessage = String.format("Expect to switch to thread %s, but currently it is thread %s", @@ -500,6 +515,49 @@ public void testThreadSwitchOfZkMetadataStore(boolean hasSynchronizer, boolean e }).join(); } + @Test + public void testZkLoadConfigFromFile() throws Exception { + final String metadataStoreName = UUID.randomUUID().toString().replaceAll("-", ""); + MetadataStoreConfig.MetadataStoreConfigBuilder builder = + MetadataStoreConfig.builder().metadataStoreName(metadataStoreName); + builder.fsyncEnable(false); + builder.batchingEnabled(true); + builder.configFilePath("src/test/resources/zk_client_disabled_sasl.conf"); + MetadataStoreConfig config = builder.build(); + @Cleanup + ZKMetadataStore store = (ZKMetadataStore) MetadataStoreFactory.create(zks.getConnectionString(), config); + + PulsarZooKeeperClient zkClient = (PulsarZooKeeperClient) store.getZkClient(); + assertFalse(zkClient.getClientConfig().isSaslClientEnabled()); + + zkClient.process(new WatchedEvent(Watcher.Event.EventType.None, Watcher.Event.KeeperState.Expired, null)); + + var zooKeeperRef = (AtomicReference) WhiteboxImpl.getInternalState(zkClient, "zk"); + var zooKeeper = Awaitility.await().until(zooKeeperRef::get, Objects::nonNull); + assertFalse(zooKeeper.getClientConfig().isSaslClientEnabled()); + } + + @Test + public void testOxiaLoadConfigFromFile() throws Exception { + final String metadataStoreName = UUID.randomUUID().toString().replaceAll("-", ""); + String oxia = "oxia://" + getOxiaServerConnectString(); + MetadataStoreConfig.MetadataStoreConfigBuilder builder = + MetadataStoreConfig.builder().metadataStoreName(metadataStoreName); + builder.fsyncEnable(false); + builder.batchingEnabled(true); + builder.sessionTimeoutMillis(30000); + builder.configFilePath("src/test/resources/oxia_client.conf"); + MetadataStoreConfig config = builder.build(); + + OxiaMetadataStore store = (OxiaMetadataStore) MetadataStoreFactory.create(oxia, config); + var client = (AsyncOxiaClient) WhiteboxImpl.getInternalState(store, "client"); + var sessionManager = (SessionManager) WhiteboxImpl.getInternalState(client, "sessionManager"); + var sessionFactory = (SessionFactory) WhiteboxImpl.getInternalState(sessionManager, "factory"); + var clientConfig = (ClientConfig) WhiteboxImpl.getInternalState(sessionFactory, "config"); + var sessionTimeout = clientConfig.sessionTimeout(); + assertEquals(sessionTimeout, Duration.ofSeconds(60)); + } + @Test(dataProvider = "impl") public void testPersistent(String provider, Supplier urlSupplier) throws Exception { String metadataUrl = urlSupplier.get(); diff --git a/pulsar-metadata/src/test/resources/oxia_client.conf b/pulsar-metadata/src/test/resources/oxia_client.conf new file mode 100644 index 0000000000000..3e92f05a34019 --- /dev/null +++ b/pulsar-metadata/src/test/resources/oxia_client.conf @@ -0,0 +1,20 @@ +# +# 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. +# + +sessionTimeout=60000 diff --git a/pulsar-metadata/src/test/resources/zk_client_disabled_sasl.conf b/pulsar-metadata/src/test/resources/zk_client_disabled_sasl.conf new file mode 100644 index 0000000000000..9e0f6e8fd0fd2 --- /dev/null +++ b/pulsar-metadata/src/test/resources/zk_client_disabled_sasl.conf @@ -0,0 +1,20 @@ +# +# 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. +# + +zookeeper.sasl.client=false From 0e6654788c67c92b05cd7f509ab7c08ab49920cd Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Mon, 5 Aug 2024 15:01:00 +0800 Subject: [PATCH 413/580] [fix][broker] Fix authenticate order in AuthenticationProviderList (#23111) --- .../AuthenticationProviderList.java | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java index 663a6253f4460..211f2ea006bc3 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java @@ -120,7 +120,8 @@ public CompletableFuture authenticateAsync(AuthData authData) { if (log.isDebugEnabled()) { log.debug("Authentication failed for auth provider " + authState.getClass() + ": ", ex); } - authenticateRemainingAuthStates(authChallengeFuture, authData, ex, states.size() - 1); + authenticateRemainingAuthStates(authChallengeFuture, authData, ex, + states.isEmpty() ? -1 : 0); } }); return authChallengeFuture; @@ -130,7 +131,7 @@ private void authenticateRemainingAuthStates(CompletableFuture authCha AuthData clientAuthData, Throwable previousException, int index) { - if (index < 0) { + if (index < 0 || index >= states.size()) { if (previousException == null) { previousException = new AuthenticationException("Authentication required"); } @@ -142,7 +143,7 @@ private void authenticateRemainingAuthStates(CompletableFuture authCha AuthenticationState state = states.get(index); if (state == authState) { // Skip the current auth state - authenticateRemainingAuthStates(authChallengeFuture, clientAuthData, null, index - 1); + authenticateRemainingAuthStates(authChallengeFuture, clientAuthData, null, index + 1); } else { state.authenticateAsync(clientAuthData) .whenComplete((authChallenge, ex) -> { @@ -155,7 +156,7 @@ private void authenticateRemainingAuthStates(CompletableFuture authCha log.debug("Authentication failed for auth provider " + authState.getClass() + ": ", ex); } - authenticateRemainingAuthStates(authChallengeFuture, clientAuthData, ex, index - 1); + authenticateRemainingAuthStates(authChallengeFuture, clientAuthData, ex, index + 1); } }); } @@ -228,7 +229,7 @@ public String getAuthMethodName() { @Override public CompletableFuture authenticateAsync(AuthenticationDataSource authData) { CompletableFuture roleFuture = new CompletableFuture<>(); - authenticateRemainingAuthProviders(roleFuture, authData, null, providers.size() - 1); + authenticateRemainingAuthProviders(roleFuture, authData, null, providers.isEmpty() ? -1 : 0); return roleFuture; } @@ -236,7 +237,7 @@ private void authenticateRemainingAuthProviders(CompletableFuture roleFu AuthenticationDataSource authData, Throwable previousException, int index) { - if (index < 0) { + if (index < 0 || index >= providers.size()) { if (previousException == null) { previousException = new AuthenticationException("Authentication required"); } @@ -254,7 +255,7 @@ private void authenticateRemainingAuthProviders(CompletableFuture roleFu if (log.isDebugEnabled()) { log.debug("Authentication failed for auth provider " + provider.getClass() + ": ", ex); } - authenticateRemainingAuthProviders(roleFuture, authData, ex, index - 1); + authenticateRemainingAuthProviders(roleFuture, authData, ex, index + 1); } }); } From e9deb408eaed2c04e30a27be5fba130f5d4e94b7 Mon Sep 17 00:00:00 2001 From: Okada Haruki Date: Mon, 5 Aug 2024 17:41:54 +0900 Subject: [PATCH 414/580] [improve][misc] Improve AES-GCM cipher performance (#23122) --- .../pulsar/client/impl/crypto/MessageCryptoBc.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/pulsar-client-messagecrypto-bc/src/main/java/org/apache/pulsar/client/impl/crypto/MessageCryptoBc.java b/pulsar-client-messagecrypto-bc/src/main/java/org/apache/pulsar/client/impl/crypto/MessageCryptoBc.java index f31fb1aa8b044..aa97421a42fbb 100644 --- a/pulsar-client-messagecrypto-bc/src/main/java/org/apache/pulsar/client/impl/crypto/MessageCryptoBc.java +++ b/pulsar-client-messagecrypto-bc/src/main/java/org/apache/pulsar/client/impl/crypto/MessageCryptoBc.java @@ -92,6 +92,7 @@ public class MessageCryptoBc implements MessageCrypto Date: Mon, 5 Aug 2024 17:45:10 +0900 Subject: [PATCH 415/580] [fix][test] Fixed many tests of pulsar-proxy are not running (#23118) --- pom.xml | 6 ++++++ pulsar-broker/pom.xml | 6 ------ 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index c2cdcee2ff385..cc4a6b52a031c 100644 --- a/pom.xml +++ b/pom.xml @@ -1626,6 +1626,12 @@ flexible messaging model and an intuitive client API. test + + io.opentelemetry + opentelemetry-sdk-testing + test + + org.projectlombok lombok diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 20117ed21db06..49446e9ca4181 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -148,12 +148,6 @@ ${project.version} - - io.opentelemetry - opentelemetry-sdk-testing - test - - ${project.groupId} pulsar-io-batch-discovery-triggerers From 10f4e0248f0f985b1dc7ad38970c906b7fe629be Mon Sep 17 00:00:00 2001 From: Aurora Twinkle Date: Mon, 5 Aug 2024 19:25:17 +0800 Subject: [PATCH 416/580] [improve][client]PIP-359:Support custom message listener executor for specific subscription (#22861) Co-authored-by: duanlinlin [PIP-359](https://github.com/apache/pulsar/pull/22902) Support custom message listener thread pool for specific subscription, avoid individual subscription listener consuming too much time leading to higher consumption delay in other subscriptions. ### Motivation In our scenario, there is a centralized message proxy service, this service will use the same PulsarClient instance to create a lot of subscription groups to consume many topics and cache messages locally.Then the business will pull messages from the cache of the proxy service. It seems that there is no problem, but during use, we found that when the message processing time of several consumer groups (listener mode) is very high, it almost affects all consumer groups responsible for the proxy service, causing a large number of message delays. By analyzing the source code, we found that by default, all consumer instances created from the same PulsarClient will share a thread pool to process message listeners, and sometimes there are multiple consumer message listeners bound to the same thread. Obviously, when a consumer processes messages and causes long-term blocking, it will cause the messages of other consumers bound to the thread to fail to be processed in time, resulting in message delays. Therefore, for this scenario, it may be necessary to support specific a message listener thread pool with consumer latitudes to avoid mutual influence between different consumers. ### Modifications Support custom message listener thread pool for specific subscription. --- .../api/MessageListenerExecutorTest.java | 193 ++++++++++++++++++ .../pulsar/client/api/ConsumerBuilder.java | 15 ++ .../client/api/MessageListenerExecutor.java | 43 ++++ .../pulsar/client/impl/ConsumerBase.java | 28 ++- .../client/impl/ConsumerBuilderImpl.java | 8 + .../impl/conf/ConsumerConfigurationData.java | 3 + 6 files changed, 280 insertions(+), 10 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageListenerExecutorTest.java create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessageListenerExecutor.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageListenerExecutorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageListenerExecutorTest.java new file mode 100644 index 0000000000000..9e148beb3045d --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageListenerExecutorTest.java @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.api; + +import static org.testng.Assert.assertTrue; +import com.google.common.util.concurrent.Uninterruptibles; +import java.time.Duration; +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.Future; +import java.util.concurrent.atomic.AtomicLong; +import lombok.Cleanup; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.naming.TopicName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Test(groups = "broker-api") +public class MessageListenerExecutorTest extends ProducerConsumerBase { + private static final Logger log = LoggerFactory.getLogger(MessageListenerExecutorTest.class); + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Override + protected void customizeNewPulsarClientBuilder(ClientBuilder clientBuilder) { + // Set listenerThreads to 1 to reproduce the pr more easily in #22861 + clientBuilder.listenerThreads(1); + } + + @Test + public void testConsumerMessageListenerExecutorIsolation() throws Exception { + log.info("-- Starting {} test --", methodName); + + @Cleanup("shutdownNow") + ExecutorService executor = Executors.newCachedThreadPool(); + List> maxConsumeDelayWithDisableIsolationFutures = new ArrayList<>(); + int loops = 5; + long consumeSleepTimeMs = 10000; + for (int i = 0; i < loops; i++) { + // The first consumer will consume messages with sleep block 1s, + // and the others will consume messages without sleep block. + // The maxConsumeDelayWithDisableIsolation of all consumers + // should be greater than sleepTimeMs cause by disable MessageListenerExecutor. + CompletableFuture maxConsumeDelayFuture = startConsumeAndComputeMaxConsumeDelay( + "persistent://my-property/my-ns/testConsumerMessageListenerDisableIsolation-" + i, + "my-sub-testConsumerMessageListenerDisableIsolation-" + i, + i == 0 ? Duration.ofMillis(consumeSleepTimeMs) : Duration.ofMillis(0), + false, + executor); + maxConsumeDelayWithDisableIsolationFutures.add(maxConsumeDelayFuture); + } + + // ensure all consumers consume messages delay more than consumeSleepTimeMs + boolean allDelayMoreThanConsumeSleepTimeMs = maxConsumeDelayWithDisableIsolationFutures.stream() + .map(CompletableFuture::join) + .allMatch(delay -> delay > consumeSleepTimeMs); + assertTrue(allDelayMoreThanConsumeSleepTimeMs); + + List> maxConsumeDelayWhitEnableIsolationFutures = new ArrayList<>(); + for (int i = 0; i < loops; i++) { + // The first consumer will consume messages with sleep block 1s, + // and the others will consume messages without sleep block. + // The maxConsumeDelayWhitEnableIsolation of the first consumer + // should be greater than sleepTimeMs, and the others should be + // less than sleepTimeMs, cause by enable MessageListenerExecutor. + CompletableFuture maxConsumeDelayFuture = startConsumeAndComputeMaxConsumeDelay( + "persistent://my-property/my-ns/testConsumerMessageListenerEnableIsolation-" + i, + "my-sub-testConsumerMessageListenerEnableIsolation-" + i, + i == 0 ? Duration.ofMillis(consumeSleepTimeMs) : Duration.ofMillis(0), + true, + executor); + maxConsumeDelayWhitEnableIsolationFutures.add(maxConsumeDelayFuture); + } + + assertTrue(maxConsumeDelayWhitEnableIsolationFutures.get(0).join() > consumeSleepTimeMs); + boolean remainingAlmostNoDelay = maxConsumeDelayWhitEnableIsolationFutures.stream() + .skip(1) + .map(CompletableFuture::join) + .allMatch(delay -> delay < 1000); + assertTrue(remainingAlmostNoDelay); + + log.info("-- Exiting {} test --", methodName); + } + + private CompletableFuture startConsumeAndComputeMaxConsumeDelay(String topic, String subscriptionName, + Duration consumeSleepTime, + boolean enableMessageListenerExecutorIsolation, + ExecutorService executorService) + throws Exception { + int numMessages = 2; + final CountDownLatch latch = new CountDownLatch(numMessages); + int numPartitions = 50; + TopicName nonIsolationTopicName = TopicName.get(topic); + admin.topics().createPartitionedTopic(nonIsolationTopicName.toString(), numPartitions); + + AtomicLong maxConsumeDelay = new AtomicLong(-1); + ConsumerBuilder consumerBuilder = + pulsarClient.newConsumer(Schema.INT64) + .topic(nonIsolationTopicName.toString()) + .subscriptionName(subscriptionName) + .messageListener((c1, msg) -> { + Assert.assertNotNull(msg, "Message cannot be null"); + log.debug("Received message [{}] in the listener", msg.getValue()); + c1.acknowledgeAsync(msg); + maxConsumeDelay.set(Math.max(maxConsumeDelay.get(), + System.currentTimeMillis() - msg.getValue())); + if (consumeSleepTime.toMillis() > 0) { + Uninterruptibles.sleepUninterruptibly(consumeSleepTime); + } + latch.countDown(); + }); + + ExecutorService executor = Executors.newSingleThreadExecutor( + new ExecutorProvider.ExtendedThreadFactory(subscriptionName + "listener-executor-", true)); + if (enableMessageListenerExecutorIsolation) { + consumerBuilder.messageListenerExecutor((message, runnable) -> executor.execute(runnable)); + } + + Consumer consumer = consumerBuilder.subscribe(); + ProducerBuilder producerBuilder = pulsarClient.newProducer(Schema.INT64) + .topic(nonIsolationTopicName.toString()); + + Producer producer = producerBuilder.create(); + List> futures = new ArrayList<>(); + + // Asynchronously produce messages + for (int i = 0; i < numMessages; i++) { + Future future = producer.sendAsync(System.currentTimeMillis()); + futures.add(future); + } + + log.info("Waiting for async publish to complete"); + for (Future future : futures) { + future.get(); + } + + CompletableFuture maxDelayFuture = new CompletableFuture<>(); + + CompletableFuture.runAsync(() -> { + try { + latch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }, executorService).whenCompleteAsync((v, ex) -> { + maxDelayFuture.complete(maxConsumeDelay.get()); + try { + producer.close(); + consumer.close(); + executor.shutdownNow(); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } + }); + + return maxDelayFuture; + } +} diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java index c7919fa473fd5..1b2e5cc5a5e51 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java @@ -283,6 +283,21 @@ public interface ConsumerBuilder extends Cloneable { */ ConsumerBuilder messageListener(MessageListener messageListener); + /** + * Set the {@link MessageListenerExecutor} to be used for message listeners of current consumer. + * (default: use executor from PulsarClient, + * {@link org.apache.pulsar.client.impl.PulsarClientImpl#externalExecutorProvider}). + * + *

The listener thread pool is exclusively owned by current consumer + * that are using a "listener" model to get messages. For a given internal consumer, + * the listener will always be invoked from the same thread, to ensure ordering. + * + *

The caller need to shut down the thread pool after closing the consumer to avoid leaks. + * @param messageListenerExecutor the executor of the consumer message listener + * @return the consumer builder instance + */ + ConsumerBuilder messageListenerExecutor(MessageListenerExecutor messageListenerExecutor); + /** * Sets a {@link CryptoKeyReader}. * diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessageListenerExecutor.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessageListenerExecutor.java new file mode 100644 index 0000000000000..53bb828c05aa8 --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessageListenerExecutor.java @@ -0,0 +1,43 @@ +/* + * 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; + +/** + * Interface for providing service to execute message listeners. + */ +public interface MessageListenerExecutor { + + /** + * select a thread by message to execute the runnable! + *

+ * Suggestions: + *

+ * 1. The message listener task will be submitted to this executor for execution, + * so the implementations of this interface should carefully consider execution + * order if sequential consumption is required. + *

+ *

+ * 2. The users should release resources(e.g. threads) of the executor after closing + * the consumer to avoid leaks. + *

+ * @param message the message + * @param runnable the runnable to execute, that is, the message listener task + */ + void execute(Message message, Runnable runnable); +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 74abb82bfe809..9748a42f0cb2b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -50,6 +50,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.MessageListener; +import org.apache.pulsar.client.api.MessageListenerExecutor; import org.apache.pulsar.client.api.Messages; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; @@ -82,6 +83,7 @@ public abstract class ConsumerBase extends HandlerState implements Consumer listener; protected final ConsumerEventListener consumerEventListener; protected final ExecutorProvider executorProvider; + protected final MessageListenerExecutor messageListenerExecutor; protected final ExecutorService externalPinnedExecutor; protected final ExecutorService internalPinnedExecutor; protected UnAckedMessageTracker unAckedMessageTracker; @@ -139,6 +141,11 @@ protected ConsumerBase(PulsarClientImpl client, String topic, ConsumerConfigurat this.unAckedChunkedMessageIdSequenceMap = ConcurrentOpenHashMap.newBuilder().build(); this.executorProvider = executorProvider; + this.messageListenerExecutor = conf.getMessageListenerExecutor() == null + ? (conf.getSubscriptionType() == SubscriptionType.Key_Shared + ? this::executeKeySharedMessageListener + : this::executeMessageListener) + : conf.getMessageListenerExecutor(); this.externalPinnedExecutor = executorProvider.getExecutor(); this.internalPinnedExecutor = client.getInternalExecutorService(); this.pendingReceives = Queues.newConcurrentLinkedQueue(); @@ -1127,14 +1134,7 @@ private void triggerListener() { // internal pinned executor thread while the message processing happens final Message finalMsg = msg; MESSAGE_LISTENER_QUEUE_SIZE_UPDATER.incrementAndGet(this); - if (SubscriptionType.Key_Shared == conf.getSubscriptionType()) { - executorProvider.getExecutor(peekMessageKey(msg)).execute(() -> - callMessageListener(finalMsg)); - } else { - getExternalExecutor(msg).execute(() -> { - callMessageListener(finalMsg); - }); - } + messageListenerExecutor.execute(msg, () -> callMessageListener(finalMsg)); } else { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Message has been cleared from the queue", topic, subscription); @@ -1147,6 +1147,14 @@ private void triggerListener() { }); } + private void executeMessageListener(Message message, Runnable runnable) { + getExternalExecutor(message).execute(runnable); + } + + private void executeKeySharedMessageListener(Message message, Runnable runnable) { + executorProvider.getExecutor(peekMessageKey(message)).execute(runnable); + } + protected void callMessageListener(Message msg) { try { if (log.isDebugEnabled()) { @@ -1176,7 +1184,7 @@ protected void callMessageListener(Message msg) { } static final byte[] NONE_KEY = "NONE_KEY".getBytes(StandardCharsets.UTF_8); - protected byte[] peekMessageKey(Message msg) { + protected byte[] peekMessageKey(Message msg) { byte[] key = NONE_KEY; if (msg.hasKey()) { key = msg.getKeyBytes(); @@ -1243,7 +1251,7 @@ public int getCurrentReceiverQueueSize() { protected abstract void completeOpBatchReceive(OpBatchReceive op); - private ExecutorService getExternalExecutor(Message msg) { + private ExecutorService getExternalExecutor(Message msg) { ConsumerImpl receivedConsumer = (msg instanceof TopicMessageImpl) ? ((TopicMessageImpl) msg).receivedByconsumer : null; ExecutorService executor = receivedConsumer != null && receivedConsumer.externalPinnedExecutor != null diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java index 4d6cf96a01068..7197cf6be79d5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java @@ -44,6 +44,7 @@ import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageListener; +import org.apache.pulsar.client.api.MessageListenerExecutor; import org.apache.pulsar.client.api.MessagePayloadProcessor; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.InvalidConfigurationException; @@ -299,6 +300,13 @@ public ConsumerBuilder messageListener(@NonNull MessageListener messageLis return this; } + @Override + public ConsumerBuilder messageListenerExecutor(MessageListenerExecutor messageListenerExecutor) { + checkArgument(messageListenerExecutor != null, "messageListenerExecutor needs to be not null"); + conf.setMessageListenerExecutor(messageListenerExecutor); + return this; + } + @Override public ConsumerBuilder consumerEventListener(@NonNull ConsumerEventListener consumerEventListener) { conf.setConsumerEventListener(consumerEventListener); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java index 18529276c9c04..f9ff5913f62da 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java @@ -43,6 +43,7 @@ import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageListener; +import org.apache.pulsar.client.api.MessageListenerExecutor; import org.apache.pulsar.client.api.MessagePayloadProcessor; import org.apache.pulsar.client.api.RedeliveryBackoff; import org.apache.pulsar.client.api.RegexSubscriptionMode; @@ -90,6 +91,8 @@ public class ConsumerConfigurationData implements Serializable, Cloneable { private SubscriptionMode subscriptionMode = SubscriptionMode.Durable; + @JsonIgnore + private transient MessageListenerExecutor messageListenerExecutor; @JsonIgnore private MessageListener messageListener; From 114880b1428ac1f6bbd97c43a26d4fa313a87b96 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 6 Aug 2024 10:16:33 +0800 Subject: [PATCH 417/580] [fix][broker]A failed consumer/producer future in ServerCnx can never be removed (#23123) --- .../pulsar/broker/service/ServerCnx.java | 24 ++++++- .../service/ServerCnxNonInjectionTest.java | 62 +++++++++++++++++++ .../impl/BrokerClientIntegrationTest.java | 50 +++++++++------ 3 files changed, 115 insertions(+), 21 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxNonInjectionTest.java 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 5df276e8f3dd5..2f9e9b2a1ac2d 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 @@ -3605,8 +3605,14 @@ public String clientSourceAddressAndPort() { @Override public CompletableFuture> checkConnectionLiveness() { + if (!isActive()) { + return CompletableFuture.completedFuture(Optional.of(false)); + } if (connectionLivenessCheckTimeoutMillis > 0) { return NettyFutureUtil.toCompletableFuture(ctx.executor().submit(() -> { + if (!isActive()) { + return CompletableFuture.completedFuture(Optional.of(false)); + } if (connectionCheckInProgress != null) { return connectionCheckInProgress; } else { @@ -3614,10 +3620,24 @@ public CompletableFuture> checkConnectionLiveness() { new CompletableFuture<>(); connectionCheckInProgress = finalConnectionCheckInProgress; ctx.executor().schedule(() -> { - if (finalConnectionCheckInProgress == connectionCheckInProgress - && !finalConnectionCheckInProgress.isDone()) { + if (!isActive()) { + finalConnectionCheckInProgress.complete(Optional.of(false)); + return; + } + if (finalConnectionCheckInProgress.isDone()) { + return; + } + if (finalConnectionCheckInProgress == connectionCheckInProgress) { + /** + * {@link #connectionCheckInProgress} will be completed when + * {@link #channelInactive(ChannelHandlerContext)} event occurs, so skip set it here. + */ log.warn("[{}] Connection check timed out. Closing connection.", this.toString()); ctx.close(); + } else { + log.error("[{}] Reached unexpected code block. Completing connection check.", + this.toString()); + finalConnectionCheckInProgress.complete(Optional.of(true)); } }, connectionLivenessCheckTimeoutMillis, TimeUnit.MILLISECONDS); sendPing(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxNonInjectionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxNonInjectionTest.java new file mode 100644 index 0000000000000..3acc941a2c8c2 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxNonInjectionTest.java @@ -0,0 +1,62 @@ +/* + * 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.service; + +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class ServerCnxNonInjectionTest extends ProducerConsumerBase { + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test(timeOut = 60 * 1000) + public void testCheckConnectionLivenessAfterClosed() throws Exception { + // Create a ServerCnx + final String tp = BrokerTestUtil.newUniqueName("public/default/tp"); + Producer p = pulsarClient.newProducer(Schema.STRING).topic(tp).create(); + ServerCnx serverCnx = (ServerCnx) pulsar.getBrokerService().getTopic(tp, false).join().get() + .getProducers().values().iterator().next().getCnx(); + // Call "CheckConnectionLiveness" after serverCnx is closed. The resulted future should be done eventually. + p.close(); + serverCnx.close(); + Thread.sleep(1000); + serverCnx.checkConnectionLiveness().join(); + } + +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java index c2715de986ad8..06c6069ebae71 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java @@ -67,11 +67,11 @@ import org.apache.bookkeeper.client.PulsarMockBookKeeper; import org.apache.bookkeeper.client.PulsarMockLedgerHandle; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.namespace.OwnershipCache; import org.apache.pulsar.broker.resources.BaseResources; import org.apache.pulsar.broker.service.AbstractDispatcherSingleActiveConsumer; -import org.apache.pulsar.broker.service.ServerCnx; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -1008,28 +1008,36 @@ public void testActiveConsumerCleanup() throws Exception { int numMessages = 100; final CountDownLatch latch = new CountDownLatch(numMessages); - String topic = "persistent://my-property/my-ns/closed-cnx-topic"; + String topic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/closed-cnx-topic"); + admin.topics().createNonPartitionedTopic(topic); String sub = "my-subscriber-name"; @Cleanup PulsarClient pulsarClient = newPulsarClient(lookupUrl.toString(), 0); - pulsarClient.newConsumer().topic(topic).subscriptionName(sub).messageListener((c1, msg) -> { - Assert.assertNotNull(msg, "Message cannot be null"); - String receivedMessage = new String(msg.getData()); - log.debug("Received message [{}] in the listener", receivedMessage); - c1.acknowledgeAsync(msg); - latch.countDown(); - }).subscribe(); - + ConsumerImpl c = + (ConsumerImpl) pulsarClient.newConsumer().topic(topic).subscriptionName(sub).messageListener((c1, msg) -> { + Assert.assertNotNull(msg, "Message cannot be null"); + String receivedMessage = new String(msg.getData()); + log.debug("Received message [{}] in the listener", receivedMessage); + c1.acknowledgeAsync(msg); + latch.countDown(); + }).subscribe(); PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get(); - AbstractDispatcherSingleActiveConsumer dispatcher = (AbstractDispatcherSingleActiveConsumer) topicRef .getSubscription(sub).getDispatcher(); - ServerCnx cnx = (ServerCnx) dispatcher.getActiveConsumer().cnx(); - Field field = ServerCnx.class.getDeclaredField("isActive"); - field.setAccessible(true); - field.set(cnx, false); - assertNotNull(dispatcher.getActiveConsumer()); + + // Inject an blocker to make the "ping & pong" does not work. + CountDownLatch countDownLatch = new CountDownLatch(1); + ConnectionHandler connectionHandler = c.getConnectionHandler(); + ClientCnx clientCnx = connectionHandler.cnx(); + clientCnx.ctx().executor().submit(() -> { + try { + countDownLatch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + @Cleanup PulsarClient pulsarClient2 = newPulsarClient(lookupUrl.toString(), 0); Consumer consumer = null; @@ -1042,15 +1050,19 @@ public void testActiveConsumerCleanup() throws Exception { c1.acknowledgeAsync(msg); latch.countDown(); }).subscribe(); - if (i == 0) { - fail("Should failed with ConsumerBusyException!"); - } } catch (PulsarClientException.ConsumerBusyException ignore) { // It's ok. } } assertNotNull(consumer); log.info("-- Exiting {} test --", methodName); + + // cleanup. + countDownLatch.countDown(); + consumer.close(); + pulsarClient.close(); + pulsarClient2.close(); + admin.topics().delete(topic, false); } @Test From 4a44f45783772780000878cdddbdc2aefd08bcfe Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Tue, 6 Aug 2024 12:05:41 +0800 Subject: [PATCH 418/580] [fix][broker] Handle the case when `getOwnedServiceUnits` fails gracefully (#23119) --- .../channel/ServiceUnitStateChannelImpl.java | 8 +++++-- .../broker/namespace/NamespaceService.java | 6 +++++- .../channel/ServiceUnitStateChannelTest.java | 13 ++++++++++++ .../NamespaceOwnershipListenerTests.java | 21 +++++++++++++++++++ 4 files changed, 45 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index fc4968805f5c1..dbe3b88b61f28 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -127,7 +127,6 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private final String brokerId; private final Map> cleanupJobs; private final StateChangeListeners stateChangeListeners; - private ExtensibleLoadManagerImpl loadManager; private BrokerRegistry brokerRegistry; private LeaderElectionService leaderElectionService; private TableView tableview; @@ -284,7 +283,6 @@ public synchronized void start() throws PulsarServerException { log.warn("Failed to find the channel leader."); } this.channelState = LeaderElectionServiceStarted; - loadManager = getLoadManager(); if (producer != null) { producer.close(); @@ -553,6 +551,9 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) { } private Optional getOwner(String serviceUnit) { + if (!validateChannelState(Started, true)) { + throw new IllegalStateException("Invalid channel state:" + channelState.name()); + } ServiceUnitStateData data = tableview.get(serviceUnit); ServiceUnitState state = state(data); switch (state) { @@ -1763,6 +1764,9 @@ public void listen(StateChangeListener listener) { @Override public Set> getOwnershipEntrySet() { + if (!validateChannelState(Started, true)) { + throw new IllegalStateException("Invalid channel state:" + channelState.name()); + } return tableview.entrySet(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index ec4c907234ab6..800a81a0f7061 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -1342,7 +1342,11 @@ public void addNamespaceBundleOwnershipListener(NamespaceBundleOwnershipListener } } pulsar.runWhenReadyForIncomingRequests(() -> { - getOwnedServiceUnits().forEach(bundle -> notifyNamespaceBundleOwnershipListener(bundle, listeners)); + try { + getOwnedServiceUnits().forEach(bundle -> notifyNamespaceBundleOwnershipListener(bundle, listeners)); + } catch (Exception e) { + LOG.error("Failed to notify namespace bundle ownership listener", e); + } }); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java index aef68aff9a262..e569f0d32d573 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java @@ -40,6 +40,7 @@ import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertThrows; import static org.testng.Assert.expectThrows; +import static org.testng.Assert.fail; import static org.testng.AssertJUnit.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; @@ -1762,6 +1763,18 @@ public void testActiveGetOwner() throws Exception { } + @Test(priority = 20) + public void testGetOwnershipEntrySetBeforeChannelStart() { + var tmpChannel = new ServiceUnitStateChannelImpl(pulsar1); + try { + tmpChannel.getOwnershipEntrySet(); + fail(); + } catch (Exception e) { + assertTrue(e instanceof IllegalStateException); + assertEquals("Invalid channel state:Constructed", e.getMessage()); + } + } + private static ConcurrentHashMap>> getOwnerRequests( ServiceUnitStateChannel channel) throws IllegalAccessException { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceOwnershipListenerTests.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceOwnershipListenerTests.java index 02787aa14358c..8fc19432eb320 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceOwnershipListenerTests.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceOwnershipListenerTests.java @@ -35,6 +35,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; import static org.testng.Assert.assertTrue; @Test(groups = "broker") @@ -102,6 +104,25 @@ public void unLoad(NamespaceBundle bundle) { deleteNamespaceWithRetry(namespace, false); } + @Test + public void testAddNamespaceBundleOwnershipListenerBeforeLBStart() { + NamespaceService namespaceService = spy(new NamespaceService(pulsar)); + doThrow(new IllegalStateException("The LM is not initialized")) + .when(namespaceService).getOwnedServiceUnits(); + namespaceService.addNamespaceBundleOwnershipListener(new NamespaceBundleOwnershipListener() { + @Override + public void onLoad(NamespaceBundle bundle) {} + + @Override + public void unLoad(NamespaceBundle bundle) {} + + @Override + public boolean test(NamespaceBundle namespaceBundle) { + return false; + } + }); + } + @Test public void testGetAllPartitions() throws Exception { final String namespace = "prop/" + UUID.randomUUID().toString(); From 1db3c5fddce45919c6cac3b5a10030183eed3d5c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 6 Aug 2024 16:46:14 +0300 Subject: [PATCH 419/580] [improve][misc] Optimize TLS performance by omitting extra buffer copies (#23115) --- .../service/PulsarChannelInitializer.java | 4 +-- .../client/impl/PulsarChannelInitializer.java | 6 ++-- .../pulsar/common/protocol/ByteBufPair.java | 30 +++++++++++++++++++ 3 files changed, 35 insertions(+), 5 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java index e276ea24fed18..f15f6d67766f1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java @@ -116,10 +116,8 @@ protected void initChannel(SocketChannel ch) throws Exception { } else { ch.pipeline().addLast(TLS_HANDLER, sslCtxRefresher.get().newHandler(ch.alloc())); } - ch.pipeline().addLast("ByteBufPairEncoder", ByteBufPair.COPYING_ENCODER); - } else { - ch.pipeline().addLast("ByteBufPairEncoder", ByteBufPair.ENCODER); } + ch.pipeline().addLast("ByteBufPairEncoder", ByteBufPair.getEncoder(this.enableTls)); if (pulsar.getConfiguration().isHaProxyProtocolEnabled()) { ch.pipeline().addLast(OptionalProxyProtocolDecoder.NAME, new OptionalProxyProtocolDecoder()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java index ed34f7d41c130..dff423d19fbef 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import io.netty.channel.Channel; +import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; import io.netty.handler.codec.LengthFieldBasedFrameDecoder; @@ -147,11 +148,12 @@ public void initChannel(SocketChannel ch) throws Exception { ch.pipeline().addLast("consolidation", new FlushConsolidationHandler(1024, true)); // Setup channel except for the SsHandler for TLS enabled connections - ch.pipeline().addLast("ByteBufPairEncoder", tlsEnabled ? ByteBufPair.COPYING_ENCODER : ByteBufPair.ENCODER); + ch.pipeline().addLast("ByteBufPairEncoder", ByteBufPair.getEncoder(tlsEnabled)); ch.pipeline().addLast("frameDecoder", new LengthFieldBasedFrameDecoder( Commands.DEFAULT_MAX_MESSAGE_SIZE + Commands.MESSAGE_SIZE_FRAME_PADDING, 0, 4, 0, 4)); - ch.pipeline().addLast("handler", clientCnxSupplier.get()); + ChannelHandler clientCnx = clientCnxSupplier.get(); + ch.pipeline().addLast("handler", clientCnx); } /** diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/ByteBufPair.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/ByteBufPair.java index cfd89d3bb28ab..6c4f42fcf88b9 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/ByteBufPair.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/ByteBufPair.java @@ -107,9 +107,39 @@ public ReferenceCounted touch(Object hint) { return this; } + /** + * Encoder that writes a {@link ByteBufPair} to the socket. + * Use {@link #getEncoder(boolean)} to get the appropriate encoder instead of referencing this. + */ + @Deprecated public static final Encoder ENCODER = new Encoder(); + + private static final boolean COPY_ENCODER_REQUIRED_FOR_TLS; + static { + boolean copyEncoderRequiredForTls = false; + try { + // io.netty.handler.ssl.SslHandlerCoalescingBufferQueue is only available in netty 4.1.111 and later + // when the class is available, there's no need to use the CopyingEncoder when TLS is enabled + ByteBuf.class.getClassLoader().loadClass("io.netty.handler.ssl.SslHandlerCoalescingBufferQueue"); + } catch (ClassNotFoundException e) { + copyEncoderRequiredForTls = true; + } + COPY_ENCODER_REQUIRED_FOR_TLS = copyEncoderRequiredForTls; + } + + /** + * Encoder that makes a copy of the ByteBufs before writing them to the socket. + * This is needed with Netty <4.1.111.Final when TLS is enabled, because the SslHandler will modify the input + * ByteBufs. + * Use {@link #getEncoder(boolean)} to get the appropriate encoder instead of referencing this. + */ + @Deprecated public static final CopyingEncoder COPYING_ENCODER = new CopyingEncoder(); + public static ChannelOutboundHandlerAdapter getEncoder(boolean tlsEnabled) { + return tlsEnabled && COPY_ENCODER_REQUIRED_FOR_TLS ? COPYING_ENCODER : ENCODER; + } + @Sharable @SuppressWarnings("checkstyle:JavadocType") public static class Encoder extends ChannelOutboundHandlerAdapter { From b7440e9023fcd497266a848372f61838eff345f5 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 7 Aug 2024 10:20:25 +0800 Subject: [PATCH 420/580] [improve][pip] PIP-368: Support lookup based on the lookup properties (#23075) ### Motivation Currently, the lookup process uses only the topic name as its parameter. However, to enhance this process, it's beneficial for clients to provide additional information. This could be done by introducing the `lookupProperties` field in the client configuration. Clients can then share these properties with the broker during lookup. On the broker side, the broker could also contain some properties that are used for the lookup. We can also support the lookupProperties for the broker. The broker can use these properties to make a better decision on which broker to return. Here is the rack-aware lookup scenario for using the client properties for the lookup: Assuming there are two brokers that broker-0 configures the lookup property "rack" with "A" and broker-1 configures the lookup property "rack" with "B". By using the lookup properties, clients can supply rack information during the lookup, enabling the broker to identify and connect them to the nearest broker within the same rack. If a client that configures the "rack" property with "A" connects to a lookup broker, the customized load manager can determine broker-0 as the owner broker since the broker and the client have the same rack property. ### Modifications Add new configuration `lookupProperties` to the client. While looking up the broker, the client will send the properties to the broker through `CommandLookupTopic` request. The `lookupProperties` will then be added to the `LookupOptions`. The Load Manager implementation can access the `properties` through `LookupOptions` to make a better decision on which broker to return. The properties are used only when the protocol is the binary protocol, starting with `pulsar://` or `pulsar+ssl://`, or if the `loadManagerClassName` in the broker is a class that implements the `ExtensibleLoadManager` interface. To support configuring the `lookupProperties` on the broker side, introduce a new broker configuration `lookupPropertyPrefix`. Any broker configuration properties that start with the `lookupPropertyPrefix` will be included into the `BrokerLookupData` and be persisted in the metadata store. The broker can use these properties during the lookup. In this way, to support the rack-aware lookup scenario mentioned in the "Motivation" part, the client can set the rack information in the client `lookupProperties`. Similarly, the broker can also set the rack information in the broker configuration like `lookup.rack`. The `lookup.rack` will be stored in the `BrokerLookupData`. A customized load manager can then be implemented. For each lookup request, it will go through the `BrokerLookupData` for all brokers and select the broker in the same rack to return. --- pip/pip-368.md | 185 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 185 insertions(+) create mode 100644 pip/pip-368.md diff --git a/pip/pip-368.md b/pip/pip-368.md new file mode 100644 index 0000000000000..06bba2c12761c --- /dev/null +++ b/pip/pip-368.md @@ -0,0 +1,185 @@ +# PIP-368: Support lookup based on the lookup properties + +# Background knowledge + +## How Pulsar Lookup Works + +Before producing or consuming messages, a Pulsar client must first find the broker responsible for the topic. This +happens through the lookup service. The client sends a `CommandLookupTopic` request with the topic name to the broker +lookup service. + +On the broker side, the broker will register itself to the metadata store using a distributed lock with the value +of [`BrokerLookupData`](https://github.com/apache/pulsar/blob/7fe92ac43cfd2f2de5576a023498aac8b46c7ac8/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java#L34-L44) +when starting. The lookup service will first choose the owner broker. And then retrieve the `BrokerLookupData` of the +owner broker and finally return to the client. The client then interacts with this broker to produce or consume +messages. + +Users can customize the lookup process by setting a custom load manager in the `loadManagerClassName` configuration. + +# Motivation + +Currently, the lookup process uses only the topic name as its parameter. However, to enhance this process, it's +beneficial for clients to provide additional information. This could be done by introducing the `lookupProperties` field +in the client configuration. Clients can then share these properties with the broker during lookup. + +On the broker side, the broker could also contain some properties that are used for the lookup. We can also support the +lookupProperties for the broker. The broker can use these properties to make a better decision on which broker to +return. + +Here is the rack-aware lookup scenario for using the client properties for the lookup: +Assuming there are two brokers that broker-0 configures the lookup property "rack" with "A" and broker-1 configures the +lookup property "rack" with "B". By using the lookup properties, clients can supply rack information during the lookup, +enabling the broker to identify and connect them to the nearest broker within the same rack. If a client that configures +the "rack" property with "A" connects to a lookup broker, the customized load manager can determine broker-0 as the +owner broker since the broker and the client have the same rack property. + +# Goals + +## In Scope + +- Enable setting up lookup properties in both client and broker configurations. +- Allow clients to provide extra lookup information to brokers during the lookup process. + +## Out of Scope + +- The implementation of the rack-aware lookup scenario. + +# High Level Design + +Add new configuration `lookupProperties` to the client. While looking up the broker, the client will send the properties +to the broker through `CommandLookupTopic` request. + +The `lookupProperties` will then be added to the `LookupOptions`. The Load Manager implementation can access +the `properties` through `LookupOptions` to make a better decision on which broker to return. + +The properties are used only when the protocol is the binary protocol, starting with `pulsar://` or `pulsar+ssl://`, or +if the `loadManagerClassName` in the broker is a class that implements the `ExtensibleLoadManager` interface. + +To support configuring the `lookupProperties` on the broker side, introduce a new broker +configuration `lookupPropertyPrefix`. Any broker configuration properties that start with the `lookupPropertyPrefix` +will be included into the `BrokerLookupData` and be persisted in the metadata store. The broker can use these properties +during the lookup. + +In this way, to support the rack-aware lookup scenario mentioned in the "Motivation" part, the client can set the rack +information in the client `lookupProperties`. Similarly, the broker can also set the rack information in the broker +configuration like `lookup.rack`. The `lookup.rack` will be stored in the `BrokerLookupData`. A customized load manager +can then be implemented. For each lookup request, it will go through the `BrokerLookupData` for all brokers and select +the broker in the same rack to return. + +# Detailed Design + +## Design & Implementation Details + +## Public-facing Changes + +### Configuration + +Add new configuration `lookupProperties` to the `ClientBuilder`. + +```java +/** + * Set the properties used for topic lookup. + *

+ * When the broker performs topic lookup, these lookup properties will be taken into consideration in a customized load + * manager. + *

+ * Note: The lookup properties are only used in topic lookup when: + * - The protocol is binary protocol, i.e. the service URL starts with "pulsar://" or "pulsar+ssl://" + * - The `loadManagerClassName` config in broker is a class that implements the `ExtensibleLoadManager` interface + */ +ClientBuilder lookupProperties(Map properties); +``` + +Add new broker configuration `lookupPropertyPrefix` to the `ServiceConfiguration`: + +```java + +@FieldContext( + category = CATEGORY_SERVER, + doc = "The properties whose name starts with this prefix will be uploaded to the metadata store for " + + " the topic lookup" +) +private String lookupPropertyPrefix = "lookup."; +``` + +### Binary protocol + +Add `properties` field to the `CommandLookupTopic`. Now the `CommandLookupTopic` will look like: + +```protobuf +message KeyValue { + required string key = 1; + required string value = 2; +} + +message CommandLookupTopic { + required string topic = 1; + required uint64 request_id = 2; + optional bool authoritative = 3 [default = false]; + optional string original_principal = 4; + optional string original_auth_data = 5; + optional string original_auth_method = 6; + optional string advertised_listener_name = 7; + // The properties used for topic lookup + repeated KeyValue properties = 8; +} +``` + +When the client lookups a topic, it will set the client `lookupPorperties` to the `CommandLookupTopic.properties`. + +### Public API + +Currently, there is a public method `assign` in the `ExtensibleLoadManager` interface that will accept +the `LookupOptions` to lookup the topic. + +```java +public interface ExtensibleLoadManager { + CompletableFuture> assign(Optional topic, + ServiceUnitId serviceUnit, + LookupOptions options); +} +``` + +In this proposal, the `properties` will be added to the `LookupOptions`: + +```java +public class LookupOptions { + // Other fields are omitted ... + + // The properties used for topic lookup + private final Map properties; +} +``` + +The `LookupOptions.properties` will be set to the value of `CommandLookupTopic.properties`. +This way, the custom `ExtensibleLoadManager` implementation can retrieve the `properties` from the `LookupOptions` to +make a better decision on which broker to return. + +# Monitoring + +No new metrics are added in this proposal. + +# Security Considerations + +No new security considerations are added in this proposal. + +# Backward & Forward Compatibility + +## Revert + +No changes are needed to revert to the previous version. + +## Upgrade + +No other changes are needed to upgrade to the new version. + +# Alternatives + +None + +# General Notes + +# Links + +* Mailing List discussion thread: https://lists.apache.org/thread/7n2gncxk3c5q8dxj8fw9y5gcwg6jjg6z +* Mailing List voting thread: https://lists.apache.org/thread/z0t3dyqj27ldm8rs6nl5jon152ohghvw From 3b01c96594ae1af215018b1e1df29e5416f240d9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 7 Aug 2024 08:22:25 +0300 Subject: [PATCH 421/580] [fix][client] Fix timeout handling in Pulsar Admin client (#23128) --- .../client/admin/internal/BaseResource.java | 8 +- .../client/admin/internal/BookiesImpl.java | 4 +- .../admin/internal/BrokerStatsImpl.java | 4 +- .../client/admin/internal/BrokersImpl.java | 4 +- .../client/admin/internal/ClustersImpl.java | 4 +- .../admin/internal/ComponentResource.java | 4 +- .../client/admin/internal/FunctionsImpl.java | 4 +- .../client/admin/internal/NamespacesImpl.java | 4 +- .../internal/NonPersistentTopicsImpl.java | 4 +- .../client/admin/internal/PackagesImpl.java | 4 +- .../client/admin/internal/ProxyStatsImpl.java | 4 +- .../admin/internal/PulsarAdminImpl.java | 46 +++--- .../admin/internal/ResourceGroupsImpl.java | 4 +- .../admin/internal/ResourceQuotasImpl.java | 4 +- .../client/admin/internal/SchemasImpl.java | 4 +- .../client/admin/internal/SinksImpl.java | 4 +- .../client/admin/internal/SourcesImpl.java | 4 +- .../client/admin/internal/TenantsImpl.java | 4 +- .../client/admin/internal/TopicsImpl.java | 4 +- .../admin/internal/TransactionsImpl.java | 4 +- .../client/admin/internal/WorkerImpl.java | 4 +- .../internal/http/AsyncHttpConnector.java | 37 +++-- .../internal/http/AsyncHttpConnectorTest.java | 140 ++++++++++++++++++ .../src/test/resources/log4j2.xml | 41 +++++ .../org/apache/pulsar/admin/cli/CmdBase.java | 14 +- 25 files changed, 280 insertions(+), 82 deletions(-) create mode 100644 pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorTest.java create mode 100644 pulsar-client-admin/src/test/resources/log4j2.xml diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BaseResource.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BaseResource.java index 22550666cb698..ea39053c2ceeb 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BaseResource.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BaseResource.java @@ -62,11 +62,11 @@ public abstract class BaseResource { private static final Logger log = LoggerFactory.getLogger(BaseResource.class); protected final Authentication auth; - protected final long readTimeoutMs; + protected final long requestTimeoutMs; - protected BaseResource(Authentication auth, long readTimeoutMs) { + protected BaseResource(Authentication auth, long requestTimeoutMs) { this.auth = auth; - this.readTimeoutMs = readTimeoutMs; + this.requestTimeoutMs = requestTimeoutMs; } public Builder request(final WebTarget target) throws PulsarAdminException { @@ -339,7 +339,7 @@ public static String getReasonFromServer(WebApplicationException e) { protected T sync(Supplier> executor) throws PulsarAdminException { try { - return executor.get().get(this.readTimeoutMs, TimeUnit.MILLISECONDS); + return executor.get().get(this.requestTimeoutMs, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new PulsarAdminException(e); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BookiesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BookiesImpl.java index 2286fb8c8a381..0bf92e0267791 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BookiesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BookiesImpl.java @@ -32,8 +32,8 @@ public class BookiesImpl extends BaseResource implements Bookies { private final WebTarget adminBookies; - public BookiesImpl(WebTarget web, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public BookiesImpl(WebTarget web, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); adminBookies = web.path("/admin/v2/bookies"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokerStatsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokerStatsImpl.java index e409d6f4492de..6ddabe9837ef9 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokerStatsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokerStatsImpl.java @@ -38,8 +38,8 @@ public class BrokerStatsImpl extends BaseResource implements BrokerStats { private final WebTarget adminBrokerStats; private final WebTarget adminV2BrokerStats; - public BrokerStatsImpl(WebTarget target, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public BrokerStatsImpl(WebTarget target, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); adminBrokerStats = target.path("/admin/broker-stats"); adminV2BrokerStats = target.path("/admin/v2/broker-stats"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java index 7b4ebb1778d8e..b82c3fd0f414b 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java @@ -37,8 +37,8 @@ public class BrokersImpl extends BaseResource implements Brokers { private final WebTarget adminBrokers; - public BrokersImpl(WebTarget web, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public BrokersImpl(WebTarget web, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); adminBrokers = web.path("admin/v2/brokers"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ClustersImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ClustersImpl.java index 231d4506d6173..24048ea3c0a41 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ClustersImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ClustersImpl.java @@ -47,8 +47,8 @@ public class ClustersImpl extends BaseResource implements Clusters { private final WebTarget adminClusters; - public ClustersImpl(WebTarget web, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public ClustersImpl(WebTarget web, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); adminClusters = web.path("/admin/v2/clusters"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ComponentResource.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ComponentResource.java index 8beecff38975a..0301f0fc2ee2b 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ComponentResource.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ComponentResource.java @@ -37,8 +37,8 @@ */ public class ComponentResource extends BaseResource { - protected ComponentResource(Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + protected ComponentResource(Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); } public RequestBuilder addAuthHeaders(WebTarget target, RequestBuilder requestBuilder) throws PulsarAdminException { diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/FunctionsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/FunctionsImpl.java index bb4cb0c1ef8ef..97c42e5c1a95a 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/FunctionsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/FunctionsImpl.java @@ -72,8 +72,8 @@ public class FunctionsImpl extends ComponentResource implements Functions { private final WebTarget functions; private final AsyncHttpClient asyncHttpClient; - public FunctionsImpl(WebTarget web, Authentication auth, AsyncHttpClient asyncHttpClient, long readTimeoutMs) { - super(auth, readTimeoutMs); + public FunctionsImpl(WebTarget web, Authentication auth, AsyncHttpClient asyncHttpClient, long requestTimeoutMs) { + super(auth, requestTimeoutMs); this.functions = web.path("/admin/v3/functions"); this.asyncHttpClient = asyncHttpClient; } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java index c7492a26ab324..7d41c7203d2c7 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java @@ -64,8 +64,8 @@ public class NamespacesImpl extends BaseResource implements Namespaces { private final WebTarget adminNamespaces; private final WebTarget adminV2Namespaces; - public NamespacesImpl(WebTarget web, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public NamespacesImpl(WebTarget web, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); adminNamespaces = web.path("/admin/namespaces"); adminV2Namespaces = web.path("/admin/v2/namespaces"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NonPersistentTopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NonPersistentTopicsImpl.java index 76727cd1e0fc4..e98d44fdc4a69 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NonPersistentTopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NonPersistentTopicsImpl.java @@ -38,8 +38,8 @@ public class NonPersistentTopicsImpl extends BaseResource implements NonPersiste private final WebTarget adminNonPersistentTopics; private final WebTarget adminV2NonPersistentTopics; - public NonPersistentTopicsImpl(WebTarget web, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public NonPersistentTopicsImpl(WebTarget web, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); adminNonPersistentTopics = web.path("/admin"); adminV2NonPersistentTopics = web.path("/admin/v2"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PackagesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PackagesImpl.java index 694c2160b0f80..d69bef448c12e 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PackagesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PackagesImpl.java @@ -57,8 +57,8 @@ public class PackagesImpl extends ComponentResource implements Packages { private final WebTarget packages; private final AsyncHttpClient httpClient; - public PackagesImpl(WebTarget webTarget, Authentication auth, AsyncHttpClient client, long readTimeoutMs) { - super(auth, readTimeoutMs); + public PackagesImpl(WebTarget webTarget, Authentication auth, AsyncHttpClient client, long requestTimeoutMs) { + super(auth, requestTimeoutMs); this.httpClient = client; this.packages = webTarget.path("/admin/v3/packages"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ProxyStatsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ProxyStatsImpl.java index e98d9bf57b31e..7ed07a1a6ad54 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ProxyStatsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ProxyStatsImpl.java @@ -32,8 +32,8 @@ public class ProxyStatsImpl extends BaseResource implements ProxyStats { private final WebTarget adminProxyStats; - public ProxyStatsImpl(WebTarget target, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public ProxyStatsImpl(WebTarget target, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); adminProxyStats = target.path("/proxy-stats"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java index 39347850cf69c..e00caa6dbbca1 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java @@ -159,29 +159,29 @@ public PulsarAdminImpl(String serviceUrl, ClientConfigurationData clientConfigDa Math.toIntExact(clientConfigData.getRequestTimeoutMs()), clientConfigData.getAutoCertRefreshSeconds()); - long readTimeoutMs = clientConfigData.getReadTimeoutMs(); - this.clusters = new ClustersImpl(root, auth, readTimeoutMs); - this.brokers = new BrokersImpl(root, auth, readTimeoutMs); - this.brokerStats = new BrokerStatsImpl(root, auth, readTimeoutMs); - this.proxyStats = new ProxyStatsImpl(root, auth, readTimeoutMs); - this.tenants = new TenantsImpl(root, auth, readTimeoutMs); - this.resourcegroups = new ResourceGroupsImpl(root, auth, readTimeoutMs); - this.properties = new TenantsImpl(root, auth, readTimeoutMs); - this.namespaces = new NamespacesImpl(root, auth, readTimeoutMs); - this.topics = new TopicsImpl(root, auth, readTimeoutMs); - this.localTopicPolicies = new TopicPoliciesImpl(root, auth, readTimeoutMs, false); - this.globalTopicPolicies = new TopicPoliciesImpl(root, auth, readTimeoutMs, true); - this.nonPersistentTopics = new NonPersistentTopicsImpl(root, auth, readTimeoutMs); - this.resourceQuotas = new ResourceQuotasImpl(root, auth, readTimeoutMs); - this.lookups = new LookupImpl(root, auth, useTls, readTimeoutMs, topics); - this.functions = new FunctionsImpl(root, auth, asyncHttpConnector.getHttpClient(), readTimeoutMs); - this.sources = new SourcesImpl(root, auth, asyncHttpConnector.getHttpClient(), readTimeoutMs); - this.sinks = new SinksImpl(root, auth, asyncHttpConnector.getHttpClient(), readTimeoutMs); - this.worker = new WorkerImpl(root, auth, readTimeoutMs); - this.schemas = new SchemasImpl(root, auth, readTimeoutMs); - this.bookies = new BookiesImpl(root, auth, readTimeoutMs); - this.packages = new PackagesImpl(root, auth, asyncHttpConnector.getHttpClient(), readTimeoutMs); - this.transactions = new TransactionsImpl(root, auth, readTimeoutMs); + long requestTimeoutMs = clientConfigData.getRequestTimeoutMs(); + this.clusters = new ClustersImpl(root, auth, requestTimeoutMs); + this.brokers = new BrokersImpl(root, auth, requestTimeoutMs); + this.brokerStats = new BrokerStatsImpl(root, auth, requestTimeoutMs); + this.proxyStats = new ProxyStatsImpl(root, auth, requestTimeoutMs); + this.tenants = new TenantsImpl(root, auth, requestTimeoutMs); + this.resourcegroups = new ResourceGroupsImpl(root, auth, requestTimeoutMs); + this.properties = new TenantsImpl(root, auth, requestTimeoutMs); + this.namespaces = new NamespacesImpl(root, auth, requestTimeoutMs); + this.topics = new TopicsImpl(root, auth, requestTimeoutMs); + this.localTopicPolicies = new TopicPoliciesImpl(root, auth, requestTimeoutMs, false); + this.globalTopicPolicies = new TopicPoliciesImpl(root, auth, requestTimeoutMs, true); + this.nonPersistentTopics = new NonPersistentTopicsImpl(root, auth, requestTimeoutMs); + this.resourceQuotas = new ResourceQuotasImpl(root, auth, requestTimeoutMs); + this.lookups = new LookupImpl(root, auth, useTls, requestTimeoutMs, topics); + this.functions = new FunctionsImpl(root, auth, asyncHttpConnector.getHttpClient(), requestTimeoutMs); + this.sources = new SourcesImpl(root, auth, asyncHttpConnector.getHttpClient(), requestTimeoutMs); + this.sinks = new SinksImpl(root, auth, asyncHttpConnector.getHttpClient(), requestTimeoutMs); + this.worker = new WorkerImpl(root, auth, requestTimeoutMs); + this.schemas = new SchemasImpl(root, auth, requestTimeoutMs); + this.bookies = new BookiesImpl(root, auth, requestTimeoutMs); + this.packages = new PackagesImpl(root, auth, asyncHttpConnector.getHttpClient(), requestTimeoutMs); + this.transactions = new TransactionsImpl(root, auth, requestTimeoutMs); if (originalCtxLoader != null) { Thread.currentThread().setContextClassLoader(originalCtxLoader); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ResourceGroupsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ResourceGroupsImpl.java index a8cef60232fc0..4e7230eebd980 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ResourceGroupsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ResourceGroupsImpl.java @@ -32,8 +32,8 @@ public class ResourceGroupsImpl extends BaseResource implements ResourceGroups { private final WebTarget adminResourceGroups; - public ResourceGroupsImpl(WebTarget web, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public ResourceGroupsImpl(WebTarget web, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); adminResourceGroups = web.path("/admin/v2/resourcegroups"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ResourceQuotasImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ResourceQuotasImpl.java index 1e80c9eda94a5..68884d99448dd 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ResourceQuotasImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/ResourceQuotasImpl.java @@ -33,8 +33,8 @@ public class ResourceQuotasImpl extends BaseResource implements ResourceQuotas { private final WebTarget adminQuotas; private final WebTarget adminV2Quotas; - public ResourceQuotasImpl(WebTarget web, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public ResourceQuotasImpl(WebTarget web, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); adminQuotas = web.path("/admin/resource-quotas"); adminV2Quotas = web.path("/admin/v2/resource-quotas"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SchemasImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SchemasImpl.java index 593eb67fc0dc3..28b435ab5676b 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SchemasImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SchemasImpl.java @@ -46,8 +46,8 @@ public class SchemasImpl extends BaseResource implements Schemas { private final WebTarget adminV2; private final WebTarget adminV1; - public SchemasImpl(WebTarget web, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public SchemasImpl(WebTarget web, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); this.adminV1 = web.path("/admin/schemas"); this.adminV2 = web.path("/admin/v2/schemas"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SinksImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SinksImpl.java index c14f75ab36750..a30f51264cc2e 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SinksImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SinksImpl.java @@ -53,8 +53,8 @@ public class SinksImpl extends ComponentResource implements Sinks, Sink { private final WebTarget sink; private final AsyncHttpClient asyncHttpClient; - public SinksImpl(WebTarget web, Authentication auth, AsyncHttpClient asyncHttpClient, long readTimeoutMs) { - super(auth, readTimeoutMs); + public SinksImpl(WebTarget web, Authentication auth, AsyncHttpClient asyncHttpClient, long requestTimeoutMs) { + super(auth, requestTimeoutMs); this.sink = web.path("/admin/v3/sink"); this.asyncHttpClient = asyncHttpClient; } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SourcesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SourcesImpl.java index 6e5b84c7f0412..8821ed61ce5b8 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SourcesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SourcesImpl.java @@ -52,8 +52,8 @@ public class SourcesImpl extends ComponentResource implements Sources, Source { private final WebTarget source; private final AsyncHttpClient asyncHttpClient; - public SourcesImpl(WebTarget web, Authentication auth, AsyncHttpClient asyncHttpClient, long readTimeoutMs) { - super(auth, readTimeoutMs); + public SourcesImpl(WebTarget web, Authentication auth, AsyncHttpClient asyncHttpClient, long requestTimeoutMs) { + super(auth, requestTimeoutMs); this.source = web.path("/admin/v3/source"); this.asyncHttpClient = asyncHttpClient; } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TenantsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TenantsImpl.java index 9b70e39ec4986..c12f3754b4a92 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TenantsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TenantsImpl.java @@ -34,8 +34,8 @@ public class TenantsImpl extends BaseResource implements Tenants, Properties { private final WebTarget adminTenants; - public TenantsImpl(WebTarget web, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public TenantsImpl(WebTarget web, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); adminTenants = web.path("/admin/v2/tenants"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index b7a8b87664075..9c4a6eef753de 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -137,8 +137,8 @@ public class TopicsImpl extends BaseResource implements Topics { public static final String PROPERTY_SHADOW_SOURCE_KEY = "PULSAR.SHADOW_SOURCE"; - public TopicsImpl(WebTarget web, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public TopicsImpl(WebTarget web, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); adminTopics = web.path("/admin"); adminV2Topics = web.path("/admin/v2"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TransactionsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TransactionsImpl.java index 460478787eb10..a0b9dd234d920 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TransactionsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TransactionsImpl.java @@ -46,8 +46,8 @@ public class TransactionsImpl extends BaseResource implements Transactions { private final WebTarget adminV3Transactions; - public TransactionsImpl(WebTarget web, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public TransactionsImpl(WebTarget web, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); adminV3Transactions = web.path("/admin/v3/transactions"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/WorkerImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/WorkerImpl.java index 60b1226d5817e..12a691edb08a2 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/WorkerImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/WorkerImpl.java @@ -40,8 +40,8 @@ public class WorkerImpl extends BaseResource implements Worker { private final WebTarget workerStats; private final WebTarget worker; - public WorkerImpl(WebTarget web, Authentication auth, long readTimeoutMs) { - super(auth, readTimeoutMs); + public WorkerImpl(WebTarget web, Authentication auth, long requestTimeoutMs) { + super(auth, requestTimeoutMs); this.worker = web.path("/admin/v2/worker"); this.workerStats = web.path("/admin/v2/worker-stats"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java index 9ad0ce5029c47..a0569c391ad50 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java @@ -35,7 +35,6 @@ import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeoutException; -import java.util.function.Function; import java.util.function.Supplier; import javax.net.ssl.SSLContext; import javax.ws.rs.client.Client; @@ -59,6 +58,7 @@ import org.asynchttpclient.BoundRequestBuilder; import org.asynchttpclient.DefaultAsyncHttpClient; import org.asynchttpclient.DefaultAsyncHttpClientConfig; +import org.asynchttpclient.ListenableFuture; import org.asynchttpclient.Request; import org.asynchttpclient.Response; import org.asynchttpclient.channel.DefaultKeepAliveStrategy; @@ -74,11 +74,11 @@ */ @Slf4j public class AsyncHttpConnector implements Connector { - private static final TimeoutException READ_TIMEOUT_EXCEPTION = - FutureUtil.createTimeoutException("Read timeout", AsyncHttpConnector.class, "retryOrTimeout(...)"); + private static final TimeoutException REQUEST_TIMEOUT_EXCEPTION = + FutureUtil.createTimeoutException("Request timeout", AsyncHttpConnector.class, "retryOrTimeout(...)"); @Getter private final AsyncHttpClient httpClient; - private final Duration readTimeout; + private final Duration requestTimeout; private final int maxRetries; private final PulsarServiceNameResolver serviceNameResolver; private final ScheduledExecutorService delayer = Executors.newScheduledThreadPool(1, @@ -185,7 +185,7 @@ public boolean keepAlive(InetSocketAddress remoteAddress, Request ahcRequest, confBuilder.setDisableHttpsEndpointIdentificationAlgorithm(!conf.isTlsHostnameVerificationEnable()); } httpClient = new DefaultAsyncHttpClient(confBuilder.build()); - this.readTimeout = Duration.ofMillis(readTimeoutMs); + this.requestTimeout = requestTimeoutMs > 0 ? Duration.ofMillis(requestTimeoutMs) : null; this.maxRetries = httpClient.getConfig().getMaxRequestRetry(); } @@ -264,9 +264,10 @@ public String getReasonPhrase() { private CompletableFuture retryOrTimeOut(ClientRequest request) { final CompletableFuture resultFuture = new CompletableFuture<>(); retryOperation(resultFuture, () -> oneShot(serviceNameResolver.resolveHost(), request), maxRetries); - CompletableFuture timeoutAfter = FutureUtil.createFutureWithTimeout(readTimeout, delayer, - () -> READ_TIMEOUT_EXCEPTION); - return resultFuture.applyToEither(timeoutAfter, Function.identity()); + if (requestTimeout != null) { + FutureUtil.addTimeoutHandling(resultFuture, requestTimeout, delayer, () -> REQUEST_TIMEOUT_EXCEPTION); + } + return resultFuture; } private void retryOperation( @@ -285,11 +286,18 @@ private void retryOperation( new RetryException("Operation future was cancelled.", throwable)); } else { if (retries > 0) { + if (log.isDebugEnabled()) { + log.debug("Retrying operation. Remaining retries: {}", retries); + } retryOperation( resultFuture, operation, retries - 1); } else { + if (log.isDebugEnabled()) { + log.debug("Number of retries has been exhausted. Failing the operation.", + throwable); + } resultFuture.completeExceptionally( new RetryException("Could not complete the operation. Number of retries " + "has been exhausted. Failed reason: " + throwable.getMessage(), @@ -315,7 +323,7 @@ public RetryException(String message, Throwable cause) { } } - private CompletableFuture oneShot(InetSocketAddress host, ClientRequest request) { + protected CompletableFuture oneShot(InetSocketAddress host, ClientRequest request) { ClientRequest currentRequest = new ClientRequest(request); URI newUri = replaceWithNew(host, currentRequest.getUri()); currentRequest.setUri(newUri); @@ -347,7 +355,16 @@ private CompletableFuture oneShot(InetSocketAddress host, ClientReques builder.setHeader(HttpHeaders.ACCEPT_ENCODING, "gzip"); } - return builder.execute().toCompletableFuture(); + ListenableFuture responseFuture = builder.execute(); + CompletableFuture completableFuture = responseFuture.toCompletableFuture(); + completableFuture.whenComplete((response, throwable) -> { + if (throwable != null && (throwable instanceof CancellationException + || throwable instanceof TimeoutException)) { + // abort the request if the future is cancelled or timed out + responseFuture.abort(throwable); + } + }); + return completableFuture; } @Override diff --git a/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorTest.java b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorTest.java new file mode 100644 index 0000000000000..dd3fb40ae9ab0 --- /dev/null +++ b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorTest.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.client.admin.internal.http; + +import static com.github.tomakehurst.wiremock.client.WireMock.aResponse; +import static com.github.tomakehurst.wiremock.client.WireMock.get; +import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import com.github.tomakehurst.wiremock.WireMockServer; +import com.github.tomakehurst.wiremock.core.WireMockConfiguration; +import com.github.tomakehurst.wiremock.stubbing.Scenario; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.URI; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.asynchttpclient.Response; +import org.glassfish.jersey.client.ClientConfig; +import org.glassfish.jersey.client.ClientRequest; +import org.glassfish.jersey.client.ClientResponse; +import org.glassfish.jersey.client.JerseyClient; +import org.glassfish.jersey.client.JerseyClientBuilder; +import org.glassfish.jersey.client.spi.AsyncConnectorCallback; +import org.glassfish.jersey.internal.MapPropertiesDelegate; +import org.glassfish.jersey.internal.PropertiesDelegate; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class AsyncHttpConnectorTest { + WireMockServer server; + + @BeforeClass(alwaysRun = true) + void beforeClass() throws IOException { + server = new WireMockServer(WireMockConfiguration.wireMockConfig() + .port(0)); + server.start(); + } + + @AfterClass(alwaysRun = true) + void afterClass() { + if (server != null) { + server.stop(); + } + } + + static class TestClientRequest extends ClientRequest { + public TestClientRequest(URI uri, ClientConfig clientConfig, PropertiesDelegate propertiesDelegate) { + super(uri, clientConfig, propertiesDelegate); + } + } + + @Test + public void testShouldStopRetriesWhenTimeoutOccurs() throws IOException, ExecutionException, InterruptedException { + server.stubFor(get(urlEqualTo("/admin/v2/clusters")) + .inScenario("once") + .whenScenarioStateIs(Scenario.STARTED) + .willSetStateTo("next") + .willReturn(aResponse() + .withHeader("Content-Type", "application/json") + .withBody("[\"test-cluster\"]"))); + + server.stubFor(get(urlEqualTo("/admin/v2/clusters")) + .inScenario("once") + .whenScenarioStateIs("next") + .willSetStateTo("retried") + .willReturn(aResponse().withStatus(500))); + + ClientConfigurationData conf = new ClientConfigurationData(); + conf.setServiceUrl("http://localhost:" + server.port()); + + int requestTimeout = 500; + + @Cleanup("shutdownNow") + ScheduledExecutorService scheduledExecutor = Executors.newSingleThreadScheduledExecutor(); + Executor delayedExecutor = runnable -> { + scheduledExecutor.schedule(runnable, requestTimeout, TimeUnit.MILLISECONDS); + }; + @Cleanup + AsyncHttpConnector connector = new AsyncHttpConnector(5000, requestTimeout, + requestTimeout, 0, conf, false) { + @Override + protected CompletableFuture oneShot(InetSocketAddress host, ClientRequest request) { + // delay the response to simulate a timeout + return super.oneShot(host, request) + .thenApplyAsync(response -> { + return response; + }, delayedExecutor); + } + }; + + JerseyClient jerseyClient = JerseyClientBuilder.createClient(); + ClientConfig clientConfig = jerseyClient.getConfiguration(); + PropertiesDelegate propertiesDelegate = new MapPropertiesDelegate(); + URI requestUri = URI.create("http://localhost:" + server.port() + "/admin/v2/clusters"); + ClientRequest request = new TestClientRequest(requestUri, clientConfig, propertiesDelegate); + request.setMethod("GET"); + CompletableFuture future = new CompletableFuture<>(); + connector.apply(request, new AsyncConnectorCallback() { + @Override + public void response(ClientResponse response) { + future.complete(response); + } + + @Override + public void failure(Throwable failure) { + future.completeExceptionally(failure); + } + }); + Thread.sleep(2 * requestTimeout); + String scenarioState = + server.getAllScenarios().getScenarios().stream().filter(scenario -> "once".equals(scenario.getName())) + .findFirst().get().getState(); + assertEquals(scenarioState, "next"); + assertTrue(future.isCompletedExceptionally()); + } +} \ No newline at end of file diff --git a/pulsar-client-admin/src/test/resources/log4j2.xml b/pulsar-client-admin/src/test/resources/log4j2.xml new file mode 100644 index 0000000000000..9b57b450ffa43 --- /dev/null +++ b/pulsar-client-admin/src/test/resources/log4j2.xml @@ -0,0 +1,41 @@ + + + + + + + + + + + + + + + + + + + + diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdBase.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdBase.java index 07e8a8b5df63b..8ff7f1c31ce2a 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdBase.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdBase.java @@ -37,10 +37,10 @@ public abstract class CmdBase { private final Supplier adminSupplier; /** - * Default read timeout in milliseconds. - * Used if not found from configuration data in {@link #getReadTimeoutMs()} + * Default request timeout in milliseconds. + * Used if not found from configuration data in {@link #getRequestTimeoutMs()} */ - private static final long DEFAULT_READ_TIMEOUT_MILLIS = 60000; + private static final long DEFAULT_REQUEST_TIMEOUT_MILLIS = 60000; public CmdBase(String cmdName, Supplier adminSupplier) { this.adminSupplier = adminSupplier; @@ -56,17 +56,17 @@ protected PulsarAdmin getAdmin() { return adminSupplier.get(); } - protected long getReadTimeoutMs() { + protected long getRequestTimeoutMs() { PulsarAdmin pulsarAdmin = getAdmin(); if (pulsarAdmin instanceof PulsarAdminImpl) { - return ((PulsarAdminImpl) pulsarAdmin).getClientConfigData().getReadTimeoutMs(); + return ((PulsarAdminImpl) pulsarAdmin).getClientConfigData().getRequestTimeoutMs(); } - return DEFAULT_READ_TIMEOUT_MILLIS; + return DEFAULT_REQUEST_TIMEOUT_MILLIS; } protected T sync(Supplier> executor) throws PulsarAdminException { try { - return executor.get().get(getReadTimeoutMs(), TimeUnit.MILLISECONDS); + return executor.get().get(getRequestTimeoutMs(), TimeUnit.MILLISECONDS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new PulsarAdminException(e); From 175ea005747ea7e229b2f492b039cf51007421ad Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 7 Aug 2024 13:46:38 +0300 Subject: [PATCH 422/580] [fix][build] Remove unnecessary Oracle maven repository from pom.xml (#23132) --- pom.xml | 8 --- pulsar-io/rabbitmq/pom.xml | 14 +++++ .../io/rabbitmq/RabbitMQBrokerManager.java | 18 +----- .../io/rabbitmq/sink/RabbitMQSinkTest.java | 9 ++- .../rabbitmq/source/RabbitMQSourceTest.java | 10 +++- .../rabbitmq/src/test/resources/qpid.json | 59 ++++++++++++++----- 6 files changed, 75 insertions(+), 43 deletions(-) diff --git a/pom.xml b/pom.xml index cc4a6b52a031c..c0659e091d490 100644 --- a/pom.xml +++ b/pom.xml @@ -2804,13 +2804,5 @@ flexible messaging model and an intuitive client API. false - - - oracle.releases - https://download.oracle.com/maven - - false - - diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index f7b9743dccae2..ff5156876a468 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -85,6 +85,20 @@ qpid-broker 9.2.0 test + + + org.apache.qpid + qpid-bdbstore + + + org.apache.qpid + qpid-broker-plugins-amqp-1-0-protocol-bdb-link-store + + + org.apache.qpid + qpid-broker-plugins-derby-store + + org.awaitility diff --git a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java index 83331bf7de810..4ff8c61e4f401 100644 --- a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java +++ b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/RabbitMQBrokerManager.java @@ -18,8 +18,6 @@ */ package org.apache.pulsar.io.rabbitmq; -import java.io.File; -import java.io.FileOutputStream; import java.nio.file.Files; import java.nio.file.Path; import java.util.HashMap; @@ -42,28 +40,14 @@ public void stopBroker() { Map getBrokerOptions(String port) throws Exception { Path tmpFolder = Files.createTempDirectory("qpidWork"); - Path homeFolder = Files.createTempDirectory("qpidHome"); - File etc = new File(homeFolder.toFile(), "etc"); - etc.mkdir(); - FileOutputStream fos = new FileOutputStream(new File(etc, "passwd")); - fos.write("guest:guest\n".getBytes()); - fos.close(); - Map config = new HashMap<>(); config.put("qpid.work_dir", tmpFolder.toAbsolutePath().toString()); config.put("qpid.amqp_port", port); - config.put("qpid.home_dir", homeFolder.toAbsolutePath().toString()); - String configPath = getFile("qpid.json").getAbsolutePath(); Map context = new HashMap<>(); - context.put(SystemConfig.INITIAL_CONFIGURATION_LOCATION, configPath); + context.put(SystemConfig.INITIAL_CONFIGURATION_LOCATION, "classpath:qpid.json"); context.put(SystemConfig.TYPE, "Memory"); context.put(SystemConfig.CONTEXT, config); return context; } - - private File getFile(String name) { - ClassLoader classLoader = getClass().getClassLoader(); - return new File(classLoader.getResource(name).getFile()); - } } diff --git a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/sink/RabbitMQSinkTest.java b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/sink/RabbitMQSinkTest.java index 3b20c61f82636..f03a36ce11485 100644 --- a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/sink/RabbitMQSinkTest.java +++ b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/sink/RabbitMQSinkTest.java @@ -18,12 +18,15 @@ */ package org.apache.pulsar.io.rabbitmq.sink; +import static org.mockito.Mockito.mock; import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.util.HashMap; import java.util.Map; import java.util.Optional; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.instance.SinkRecord; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.rabbitmq.RabbitMQBrokerManager; import org.apache.pulsar.io.rabbitmq.RabbitMQSink; import org.awaitility.Awaitility; @@ -46,7 +49,7 @@ public void tearDown() { } @Test - public void TestOpenAndWriteSink() throws Exception { + public void testOpenAndWriteSink() throws Exception { Map configs = new HashMap<>(); configs.put("host", "localhost"); configs.put("port", "5673"); @@ -66,7 +69,9 @@ public void TestOpenAndWriteSink() throws Exception { // open should success // rabbitmq service may need time to initialize - Awaitility.await().ignoreExceptions().untilAsserted(() -> sink.open(configs, null)); + SinkContext sinkContext = mock(SinkContext.class); + Awaitility.await().ignoreExceptions().pollDelay(Duration.ofSeconds(1)) + .untilAsserted(() -> sink.open(configs, sinkContext)); // write should success Record record = build("test-topic", "fakeKey", "fakeValue", "fakeRoutingKey"); diff --git a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceTest.java b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceTest.java index 2771185b84162..08869e018c625 100644 --- a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceTest.java +++ b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceTest.java @@ -18,6 +18,9 @@ */ package org.apache.pulsar.io.rabbitmq.source; +import static org.mockito.Mockito.mock; +import java.time.Duration; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.rabbitmq.RabbitMQBrokerManager; import org.apache.pulsar.io.rabbitmq.RabbitMQSource; import org.awaitility.Awaitility; @@ -44,7 +47,7 @@ public void tearDown() { } @Test - public void TestOpenAndWriteSink() throws Exception { + public void testOpenAndWriteSink() throws Exception { Map configs = new HashMap<>(); configs.put("host", "localhost"); configs.put("port", "5672"); @@ -66,8 +69,11 @@ public void TestOpenAndWriteSink() throws Exception { // open should success // rabbitmq service may need time to initialize - Awaitility.await().ignoreExceptions().untilAsserted(() -> source.open(configs, null)); + SourceContext sourceContext = mock(SourceContext.class); + Awaitility.await().ignoreExceptions().pollDelay(Duration.ofSeconds(1)) + .untilAsserted(() -> source.open(configs, sourceContext)); source.close(); } + } diff --git a/pulsar-io/rabbitmq/src/test/resources/qpid.json b/pulsar-io/rabbitmq/src/test/resources/qpid.json index 6a0381f6ddc2c..419e9cc1e4a55 100644 --- a/pulsar-io/rabbitmq/src/test/resources/qpid.json +++ b/pulsar-io/rabbitmq/src/test/resources/qpid.json @@ -1,25 +1,57 @@ { - "name": "EmbeddedBroker", + "name": "${broker.name}", "modelVersion": "2.0", - "storeVersion": 1, "authenticationproviders": [ { - "name": "noPassword", - "type": "Anonymous", - "secureOnlyMechanisms": [] - }, + "name": "plain", + "type": "Plain", + "secureOnlyMechanisms": [], + "users": [ + { + "name": "guest", + "password": "guest", + "type": "managed" + } + ] + } + ], + "brokerloggers": [ { - "name": "passwordFile", - "type": "PlainPasswordFile", - "path": "${qpid.home_dir}${file.separator}etc${file.separator}passwd", - "secureOnlyMechanisms": [] + "name": "console", + "type": "Console", + "brokerloginclusionrules": [ + { + "name": "Root", + "type": "NameAndLevel", + "level": "WARN", + "loggerName": "ROOT" + }, + { + "name": "Qpid", + "type": "NameAndLevel", + "level": "INFO", + "loggerName": "org.apache.qpid.*" + }, + { + "name": "Operational", + "type": "NameAndLevel", + "level": "INFO", + "loggerName": "qpid.message.*" + }, + { + "name": "Statistics", + "type": "NameAndLevel", + "level": "INFO", + "loggerName": "qpid.statistics.*" + } + ] } ], "ports": [ { "name": "AMQP", "port": "${qpid.amqp_port}", - "authenticationProvider": "passwordFile", + "authenticationProvider": "plain", "protocols": [ "AMQP_0_9_1" ] @@ -28,10 +60,9 @@ "virtualhostnodes": [ { "name": "default", - "type": "JSON", + "type": "Memory", "defaultVirtualHostNode": "true", - "virtualHostInitialConfiguration": "${qpid.initial_config_virtualhost_config}", - "storeType": "DERBY" + "virtualHostInitialConfiguration": "{\"type\": \"Memory\"}" } ] } \ No newline at end of file From 8707fbe8351fb6ac4337fbd88d86eb32aff55b04 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 7 Aug 2024 13:52:55 +0300 Subject: [PATCH 423/580] [improve][fn] Add support for overriding additionalJavaRuntimeArguments with PF_additionalJavaRuntimeArguments env (#23130) --- docker/pulsar/scripts/gen-yml-from-env.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/pulsar/scripts/gen-yml-from-env.py b/docker/pulsar/scripts/gen-yml-from-env.py index aa40408ed5b1f..916b147f0cbba 100755 --- a/docker/pulsar/scripts/gen-yml-from-env.py +++ b/docker/pulsar/scripts/gen-yml-from-env.py @@ -50,6 +50,9 @@ 'brokerClientTlsProtocols', 'webServiceTlsCiphers', 'webServiceTlsProtocols', + 'additionalJavaRuntimeArguments', + 'additionalEnabledConnectorUrlPatterns', + 'additionalEnabledFunctionsUrlPatterns' ] PF_ENV_PREFIX = 'PF_' From 3560ddb64f44fb2a53d52ef3df0624bb9bda1af6 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Thu, 8 Aug 2024 01:19:38 -0700 Subject: [PATCH 424/580] [fix][broker] Fix the bug that elected leader thinks it's a follower (#23138) --- .../coordination/impl/LeaderElectionImpl.java | 15 ++++- .../pulsar/metadata/LeaderElectionTest.java | 2 + .../apache/pulsar/metadata/ZKSessionTest.java | 55 +++++++++++++++++++ 3 files changed, 69 insertions(+), 3 deletions(-) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java index aa606084173e5..ab35eb7040c10 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java @@ -134,8 +134,11 @@ private synchronized CompletableFuture handleExistingLeader // If the value is the same as our proposed value, it means this instance was the leader at some // point before. The existing value can either be for this same session or for a previous one. if (res.getStat().isCreatedBySelf()) { + log.info("Keeping the existing value {} for {} as it's from the same session stat={}", existingValue, + path, res.getStat()); // The value is still valid because it was created in the same session changeState(LeaderElectionState.Leading); + return CompletableFuture.completedFuture(LeaderElectionState.Leading); } else { log.info("Conditionally deleting existing equals value {} for {} because it's not created in the " + "current session. stat={}", existingValue, path, res.getStat()); @@ -271,7 +274,13 @@ public synchronized CompletableFuture asyncClose() { return CompletableFuture.completedFuture(null); } - return store.delete(path, version); + return store.delete(path, version) + .thenAccept(__ -> { + synchronized (LeaderElectionImpl.this) { + leaderElectionState = LeaderElectionState.NoLeader; + } + } + ); } @Override @@ -292,8 +301,8 @@ public Optional getLeaderValueIfPresent() { private void handleSessionNotification(SessionEvent event) { // Ensure we're only processing one session event at a time. sequencer.sequential(() -> FutureUtil.composeAsync(() -> { - if (event == SessionEvent.SessionReestablished) { - log.info("Revalidating leadership for {}", path); + if (event == SessionEvent.Reconnected || event == SessionEvent.SessionReestablished) { + log.info("Revalidating leadership for {}, event:{}", path, event); return elect().thenAccept(leaderState -> { log.info("Resynced leadership for {} - State: {}", path, leaderState); }).exceptionally(ex -> { diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/LeaderElectionTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/LeaderElectionTest.java index 6b4f74a30b563..4b48f3c20b02b 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/LeaderElectionTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/LeaderElectionTest.java @@ -69,6 +69,8 @@ public void basicTest(String provider, Supplier urlSupplier) throws Exce leaderElection.close(); + assertEquals(leaderElection.getState(), LeaderElectionState.NoLeader); + assertEquals(cache.get("/my/leader-election").join(), Optional.empty()); } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/ZKSessionTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/ZKSessionTest.java index 36cb0f132ba58..02d65fd21ed5c 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/ZKSessionTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/ZKSessionTest.java @@ -27,6 +27,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.coordination.CoordinationService; import org.apache.pulsar.metadata.api.coordination.LeaderElection; @@ -180,4 +181,58 @@ public void testReacquireLeadershipAfterSessionLost() throws Exception { .untilAsserted(()-> assertEquals(le1.getState(),LeaderElectionState.Leading)); assertTrue(store.get(path).join().isPresent()); } + + + @Test + public void testElectAfterReconnected() throws Exception { + // --- init + @Cleanup + MetadataStoreExtended store = MetadataStoreExtended.create(zks.getConnectionString(), + MetadataStoreConfig.builder() + .sessionTimeoutMillis(2_000) + .build()); + + + BlockingQueue sessionEvents = new LinkedBlockingQueue<>(); + store.registerSessionListener(sessionEvents::add); + BlockingQueue leaderElectionEvents = new LinkedBlockingQueue<>(); + String path = newKey(); + + @Cleanup + CoordinationService coordinationService = new CoordinationServiceImpl(store); + @Cleanup + LeaderElection le1 = coordinationService.getLeaderElection(String.class, path, + leaderElectionEvents::add); + + // --- test manual elect + String proposed = "value-1"; + le1.elect(proposed).join(); + assertEquals(le1.getState(), LeaderElectionState.Leading); + LeaderElectionState les = leaderElectionEvents.poll(5, TimeUnit.SECONDS); + assertEquals(les, LeaderElectionState.Leading); + + + // simulate no leader state + FieldUtils.writeDeclaredField(le1, "leaderElectionState", LeaderElectionState.NoLeader, true); + + // reconnect + zks.stop(); + + SessionEvent e = sessionEvents.poll(5, TimeUnit.SECONDS); + assertEquals(e, SessionEvent.ConnectionLost); + + zks.start(); + + + // --- test le1 can be leader + e = sessionEvents.poll(10, TimeUnit.SECONDS); + assertEquals(e, SessionEvent.Reconnected); + Awaitility.await().atMost(Duration.ofSeconds(15)) + .untilAsserted(()-> { + assertEquals(le1.getState(),LeaderElectionState.Leading); + }); // reacquire leadership + + + assertTrue(store.get(path).join().isPresent()); + } } From 3e7dbb4957bf5daae59127cd66e4da3802072853 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 8 Aug 2024 11:55:15 +0300 Subject: [PATCH 425/580] [improve][client] Add maxConnectionsPerHost and connectionMaxIdleSeconds to PulsarAdminBuilder (#22541) --- .../server/src/assemble/LICENSE.bin.txt | 20 +- .../shell/src/assemble/LICENSE.bin.txt | 13 +- pom.xml | 9 +- .../client/admin/PulsarAdminBuilder.java | 26 ++ pulsar-client-admin-shaded/pom.xml | 5 + .../client/admin/internal/FunctionsImpl.java | 70 ++-- .../client/admin/internal/PackagesImpl.java | 68 ++-- .../internal/PulsarAdminBuilderImpl.java | 24 ++ .../admin/internal/PulsarAdminImpl.java | 8 +- .../client/admin/internal/SinksImpl.java | 13 +- .../client/admin/internal/SourcesImpl.java | 13 +- .../internal/http/AsyncHttpConnector.java | 351 +++++++++++++----- .../http/AsyncHttpRequestExecutor.java | 48 +++ .../internal/PulsarAdminBuilderImplTest.java | 2 + .../internal/http/AsyncHttpConnectorTest.java | 200 ++++++++++ pulsar-client-all/pom.xml | 5 + .../pulsar/client/api/ClientBuilder.java | 2 + pulsar-client-shaded/pom.xml | 5 + .../pulsar/client/impl/ConnectionPool.java | 2 +- .../impl/conf/ClientConfigurationData.java | 4 +- .../client/impl/ClientBuilderImplTest.java | 2 +- pulsar-common/pom.xml | 5 + 22 files changed, 675 insertions(+), 220 deletions(-) create mode 100644 pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpRequestExecutor.java diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index af50d818c4e7a..4bbb86653add5 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -536,6 +536,8 @@ The Apache Software License, Version 2.0 - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java17-1.33.3-alpha.jar - io.opentelemetry.instrumentation-opentelemetry-runtime-telemetry-java8-1.33.3-alpha.jar - io.opentelemetry.semconv-opentelemetry-semconv-1.25.0-alpha.jar + * Spotify completable-futures + - com.spotify-completable-futures-0.3.6.jar BSD 3-clause "New" or "Revised" License * Google auth library @@ -580,15 +582,15 @@ CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt - org.glassfish.hk2-osgi-resource-locator-1.0.3.jar - org.glassfish.hk2.external-aopalliance-repackaged-2.6.1.jar * Jersey - - org.glassfish.jersey.containers-jersey-container-servlet-2.41.jar - - org.glassfish.jersey.containers-jersey-container-servlet-core-2.41.jar - - org.glassfish.jersey.core-jersey-client-2.41.jar - - org.glassfish.jersey.core-jersey-common-2.41.jar - - org.glassfish.jersey.core-jersey-server-2.41.jar - - org.glassfish.jersey.ext-jersey-entity-filtering-2.41.jar - - org.glassfish.jersey.media-jersey-media-json-jackson-2.41.jar - - org.glassfish.jersey.media-jersey-media-multipart-2.41.jar - - org.glassfish.jersey.inject-jersey-hk2-2.41.jar + - org.glassfish.jersey.containers-jersey-container-servlet-2.42.jar + - org.glassfish.jersey.containers-jersey-container-servlet-core-2.42.jar + - org.glassfish.jersey.core-jersey-client-2.42.jar + - org.glassfish.jersey.core-jersey-common-2.42.jar + - org.glassfish.jersey.core-jersey-server-2.42.jar + - org.glassfish.jersey.ext-jersey-entity-filtering-2.42.jar + - org.glassfish.jersey.media-jersey-media-json-jackson-2.42.jar + - org.glassfish.jersey.media-jersey-media-multipart-2.42.jar + - org.glassfish.jersey.inject-jersey-hk2-2.42.jar * Mimepull -- org.jvnet.mimepull-mimepull-1.9.15.jar Eclipse Distribution License 1.0 -- ../licenses/LICENSE-EDL-1.0.txt diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 0da56c6afa8fc..31acbd9ac161d 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -417,6 +417,7 @@ The Apache Software License, Version 2.0 - avro-1.11.3.jar - avro-protobuf-1.11.3.jar * RE2j -- re2j-1.7.jar + * Spotify completable-futures -- completable-futures-0.3.6.jar BSD 3-clause "New" or "Revised" License * JSR305 -- jsr305-3.0.2.jar -- ../licenses/LICENSE-JSR305.txt @@ -446,12 +447,12 @@ CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt - aopalliance-repackaged-2.6.1.jar - osgi-resource-locator-1.0.3.jar * Jersey - - jersey-client-2.41.jar - - jersey-common-2.41.jar - - jersey-entity-filtering-2.41.jar - - jersey-media-json-jackson-2.41.jar - - jersey-media-multipart-2.41.jar - - jersey-hk2-2.41.jar + - jersey-client-2.42.jar + - jersey-common-2.42.jar + - jersey-entity-filtering-2.42.jar + - jersey-media-json-jackson-2.42.jar + - jersey-media-multipart-2.42.jar + - jersey-hk2-2.42.jar * Mimepull -- mimepull-1.9.15.jar Eclipse Distribution License 1.0 -- ../licenses/LICENSE-EDL-1.0.txt diff --git a/pom.xml b/pom.xml index c0659e091d490..52843f5079f01 100644 --- a/pom.xml +++ b/pom.xml @@ -152,7 +152,7 @@ flexible messaging model and an intuitive client API. 0.0.24.Final 9.4.54.v20240208 2.5.2 - 2.41 + 2.42 1.10.50 0.16.0 4.5.8 @@ -266,6 +266,7 @@ flexible messaging model and an intuitive client API. 1.25.0-alpha 4.7.5 1.7 + 0.3.6 3.3.2 @@ -665,6 +666,12 @@ flexible messaging model and an intuitive client API. ${re2j.version} + + com.spotify + completable-futures + ${completable-futures.version} + + org.rocksdb rocksdbjni diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java index 1b025a752d9f3..b26e5b2cec802 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java @@ -336,4 +336,30 @@ PulsarAdminBuilder authentication(String authPluginClassName, Map + * By default, the connection pool maintains up to 16 connections to a single host. This method allows you to + * modify this default behavior and limit the number of connections. + *

+ * This setting can be useful in scenarios where you want to limit the resources used by the client library, + * or control the level of parallelism for operations so that a single client does not overwhelm + * the Pulsar cluster with too many concurrent connections. + * + * @param maxConnectionsPerHost the maximum number of connections to establish per host. Set to <= 0 to disable + * the limit. + * @return the PulsarAdminBuilder instance, allowing for method chaining + */ + PulsarAdminBuilder maxConnectionsPerHost(int maxConnectionsPerHost); + + /** + * Sets the maximum idle time for a pooled connection. If a connection is idle for more than the specified + * amount of seconds, it will be released back to the connection pool. + * Defaults to 25 seconds. + * + * @param connectionMaxIdleSeconds the maximum idle time, in seconds, for a pooled connection + * @return the PulsarAdminBuilder instance + */ + PulsarAdminBuilder connectionMaxIdleSeconds(int connectionMaxIdleSeconds); } \ No newline at end of file diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index 1376cefe80368..f2249b4c0ff6a 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -123,6 +123,7 @@ com.google.guava:guava com.google.code.gson:gson com.google.re2j:re2j + com.spotify:completable-futures com.fasterxml.jackson.*:* io.netty:* io.netty.incubator:* @@ -192,6 +193,10 @@ com.google.protobuf.* + + com.spotify.futures + org.apache.pulsar.shade.com.spotify.futures + com.fasterxml.jackson org.apache.pulsar.shade.com.fasterxml.jackson diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/FunctionsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/FunctionsImpl.java index 97c42e5c1a95a..bfcc3fe39a444 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/FunctionsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/FunctionsImpl.java @@ -22,7 +22,6 @@ import static org.asynchttpclient.Dsl.post; import static org.asynchttpclient.Dsl.put; import com.google.gson.Gson; -import io.netty.handler.codec.http.HttpHeaders; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -41,6 +40,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.admin.Functions; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.admin.internal.http.AsyncHttpRequestExecutor; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.FunctionDefinition; @@ -54,10 +54,8 @@ import org.apache.pulsar.common.policies.data.FunctionStats; import org.apache.pulsar.common.policies.data.FunctionStatsImpl; import org.apache.pulsar.common.policies.data.FunctionStatus; -import org.asynchttpclient.AsyncHandler; -import org.asynchttpclient.AsyncHttpClient; +import org.asynchttpclient.AsyncCompletionHandlerBase; import org.asynchttpclient.HttpResponseBodyPart; -import org.asynchttpclient.HttpResponseStatus; import org.asynchttpclient.RequestBuilder; import org.asynchttpclient.request.body.multipart.ByteArrayPart; import org.asynchttpclient.request.body.multipart.FilePart; @@ -70,12 +68,14 @@ public class FunctionsImpl extends ComponentResource implements Functions { private final WebTarget functions; - private final AsyncHttpClient asyncHttpClient; + private final AsyncHttpRequestExecutor asyncHttpRequestExecutor; - public FunctionsImpl(WebTarget web, Authentication auth, AsyncHttpClient asyncHttpClient, long requestTimeoutMs) { + public FunctionsImpl(WebTarget web, Authentication auth, + AsyncHttpRequestExecutor asyncHttpRequestExecutor, + long requestTimeoutMs) { super(auth, requestTimeoutMs); this.functions = web.path("/admin/v3/functions"); - this.asyncHttpClient = asyncHttpClient; + this.asyncHttpRequestExecutor = asyncHttpRequestExecutor; } @Override @@ -171,8 +171,7 @@ public CompletableFuture createFunctionAsync(FunctionConfig functionConfig // If the function code is built in, we don't need to submit here builder.addBodyPart(new FilePart("data", new File(fileName), MediaType.APPLICATION_OCTET_STREAM)); } - asyncHttpClient.executeRequest(addAuthHeaders(functions, builder).build()) - .toCompletableFuture() + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(functions, builder).build()) .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { future.completeExceptionally( @@ -263,8 +262,7 @@ public CompletableFuture updateFunctionAsync( builder.addBodyPart(new FilePart("data", new File(fileName), MediaType.APPLICATION_OCTET_STREAM)); } - asyncHttpClient.executeRequest(addAuthHeaders(functions, builder).build()) - .toCompletableFuture() + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(functions, builder).build()) .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { future.completeExceptionally( @@ -464,7 +462,7 @@ public CompletableFuture uploadFunctionAsync(String sourceFile, String pat .addBodyPart(new FilePart("data", new File(sourceFile), MediaType.APPLICATION_OCTET_STREAM)) .addBodyPart(new StringPart("path", path, MediaType.TEXT_PLAIN)); - asyncHttpClient.executeRequest(addAuthHeaders(functions, builder).build()).toCompletableFuture() + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(functions, builder).build()) .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { future.completeExceptionally( @@ -543,55 +541,31 @@ private CompletableFuture downloadFileAsync(String destinationPath, WebTar RequestBuilder builder = get(target.getUri().toASCIIString()); - CompletableFuture statusFuture = - asyncHttpClient.executeRequest(addAuthHeaders(functions, builder).build(), - new AsyncHandler() { - private HttpResponseStatus status; - - @Override - public State onStatusReceived(HttpResponseStatus responseStatus) throws Exception { - status = responseStatus; - if (status.getStatusCode() != Response.Status.OK.getStatusCode()) { - return State.ABORT; - } - return State.CONTINUE; - } - - @Override - public State onHeadersReceived(HttpHeaders headers) throws Exception { - return State.CONTINUE; - } + CompletableFuture responseFuture = + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(functions, builder).build(), + () -> new AsyncCompletionHandlerBase() { @Override public State onBodyPartReceived(HttpResponseBodyPart bodyPart) throws Exception { os.write(bodyPart.getBodyByteBuffer()); return State.CONTINUE; } + }); - @Override - public HttpResponseStatus onCompleted() throws Exception { - return status; - } - - @Override - public void onThrowable(Throwable t) { - } - }).toCompletableFuture(); - - statusFuture - .whenComplete((status, throwable) -> { + responseFuture + .whenComplete((response, throwable) -> { try { os.close(); } catch (IOException e) { future.completeExceptionally(getApiException(e)); } }) - .thenAccept(status -> { - if (status.getStatusCode() < 200 || status.getStatusCode() >= 300) { + .thenAccept(response -> { + if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { future.completeExceptionally( getApiException(Response - .status(status.getStatusCode()) - .entity(status.getStatusText()) + .status(response.getStatusCode()) + .entity(response.getStatusText()) .build())); } else { future.complete(null); @@ -700,7 +674,7 @@ public CompletableFuture putFunctionStateAsync( .path("state").path(state.getKey()).getUri().toASCIIString()); builder.addBodyPart(new StringPart("state", objectWriter() .writeValueAsString(state), MediaType.APPLICATION_JSON)); - asyncHttpClient.executeRequest(addAuthHeaders(functions, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(functions, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { @@ -740,7 +714,7 @@ public CompletableFuture updateOnWorkerLeaderAsync(String tenant, String n .addBodyPart(new ByteArrayPart("functionMetaData", functionMetaData)) .addBodyPart(new StringPart("delete", Boolean.toString(delete))); - asyncHttpClient.executeRequest(addAuthHeaders(functions, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(functions, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PackagesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PackagesImpl.java index d69bef448c12e..2b8efc3b97c8c 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PackagesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PackagesImpl.java @@ -20,7 +20,6 @@ import static org.asynchttpclient.Dsl.get; import com.google.gson.Gson; -import io.netty.handler.codec.http.HttpHeaders; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -36,15 +35,14 @@ import javax.ws.rs.core.Response; import org.apache.pulsar.client.admin.Packages; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.admin.internal.http.AsyncHttpRequestExecutor; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.packages.management.core.common.PackageMetadata; import org.apache.pulsar.packages.management.core.common.PackageName; -import org.asynchttpclient.AsyncHandler; -import org.asynchttpclient.AsyncHttpClient; +import org.asynchttpclient.AsyncCompletionHandlerBase; import org.asynchttpclient.Dsl; import org.asynchttpclient.HttpResponseBodyPart; -import org.asynchttpclient.HttpResponseStatus; import org.asynchttpclient.RequestBuilder; import org.asynchttpclient.request.body.multipart.FilePart; import org.asynchttpclient.request.body.multipart.StringPart; @@ -55,11 +53,12 @@ public class PackagesImpl extends ComponentResource implements Packages { private final WebTarget packages; - private final AsyncHttpClient httpClient; + private final AsyncHttpRequestExecutor asyncHttpRequestExecutor; - public PackagesImpl(WebTarget webTarget, Authentication auth, AsyncHttpClient client, long requestTimeoutMs) { + public PackagesImpl(WebTarget webTarget, Authentication auth, AsyncHttpRequestExecutor asyncHttpRequestExecutor, + long requestTimeoutMs) { super(auth, requestTimeoutMs); - this.httpClient = client; + this.asyncHttpRequestExecutor = asyncHttpRequestExecutor; this.packages = webTarget.path("/admin/v3/packages"); } @@ -98,7 +97,7 @@ public CompletableFuture uploadAsync(PackageMetadata metadata, String pack .post(packages.path(PackageName.get(packageName).toRestPath()).getUri().toASCIIString()) .addBodyPart(new FilePart("file", new File(path), MediaType.APPLICATION_OCTET_STREAM)) .addBodyPart(new StringPart("metadata", new Gson().toJson(metadata), MediaType.APPLICATION_JSON)); - httpClient.executeRequest(addAuthHeaders(packages, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(packages, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { @@ -138,55 +137,30 @@ public CompletableFuture downloadAsync(String packageName, String path) { FileChannel os = new FileOutputStream(destinyPath.toFile()).getChannel(); RequestBuilder builder = get(webTarget.getUri().toASCIIString()); - CompletableFuture statusFuture = - httpClient.executeRequest(addAuthHeaders(webTarget, builder).build(), - new AsyncHandler() { - private HttpResponseStatus status; + CompletableFuture responseFuture = + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(webTarget, builder).build(), + () -> new AsyncCompletionHandlerBase() { - @Override - public State onStatusReceived(HttpResponseStatus httpResponseStatus) throws Exception { - status = httpResponseStatus; - if (status.getStatusCode() != Response.Status.OK.getStatusCode()) { - return State.ABORT; + @Override + public State onBodyPartReceived(HttpResponseBodyPart bodyPart) throws Exception { + os.write(bodyPart.getBodyByteBuffer()); + return State.CONTINUE; } - return State.CONTINUE; - } - - @Override - public State onHeadersReceived(HttpHeaders httpHeaders) throws Exception { - return State.CONTINUE; - } - - @Override - public State onBodyPartReceived(HttpResponseBodyPart httpResponseBodyPart) throws Exception { - os.write(httpResponseBodyPart.getBodyByteBuffer()); - return State.CONTINUE; - } - - @Override - public void onThrowable(Throwable throwable) { - // we don't need to handle that throwable and use the returned future to handle it. - } - - @Override - public HttpResponseStatus onCompleted() throws Exception { - return status; - } - }).toCompletableFuture(); - statusFuture - .whenComplete((status, throwable) -> { + }); + responseFuture + .whenComplete((response, throwable) -> { try { os.close(); } catch (IOException e) { future.completeExceptionally(getApiException(throwable)); } }) - .thenAccept(status -> { - if (status.getStatusCode() < 200 || status.getStatusCode() >= 300) { + .thenAccept(response -> { + if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { future.completeExceptionally( getApiException(Response - .status(status.getStatusCode()) - .entity(status.getStatusText()) + .status(response.getStatusCode()) + .entity(response.getStatusText()) .build())); } else { future.complete(null); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java index f7b1695f5f37b..9bfb4fc45f3b7 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java @@ -47,6 +47,7 @@ public PulsarAdmin build() throws PulsarClientException { public PulsarAdminBuilderImpl() { this.conf = new ClientConfigurationData(); + this.conf.setConnectionsPerBroker(16); } private PulsarAdminBuilderImpl(ClientConfigurationData conf) { @@ -73,6 +74,15 @@ public PulsarAdminBuilder loadConf(Map config) { acceptGzipCompression = Boolean.parseBoolean(acceptGzipCompressionObj.toString()); } } + // in ClientConfigurationData, the maxConnectionsPerHost maps to connectionsPerBroker + if (config.containsKey("maxConnectionsPerHost")) { + Object maxConnectionsPerHostObj = config.get("maxConnectionsPerHost"); + if (maxConnectionsPerHostObj instanceof Integer) { + maxConnectionsPerHost((Integer) maxConnectionsPerHostObj); + } else { + maxConnectionsPerHost(Integer.parseInt(maxConnectionsPerHostObj.toString())); + } + } return this; } @@ -245,4 +255,18 @@ public PulsarAdminBuilder acceptGzipCompression(boolean acceptGzipCompression) { this.acceptGzipCompression = acceptGzipCompression; return this; } + + @Override + public PulsarAdminBuilder maxConnectionsPerHost(int maxConnectionsPerHost) { + // reuse the same configuration as the client, however for the admin client, the connection + // is usually established to a cluster address and not to a broker address + this.conf.setConnectionsPerBroker(maxConnectionsPerHost); + return this; + } + + @Override + public PulsarAdminBuilder connectionMaxIdleSeconds(int connectionMaxIdleSeconds) { + this.conf.setConnectionMaxIdleSeconds(connectionMaxIdleSeconds); + return this; + } } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java index e00caa6dbbca1..aaea8a89f8db5 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java @@ -174,13 +174,13 @@ public PulsarAdminImpl(String serviceUrl, ClientConfigurationData clientConfigDa this.nonPersistentTopics = new NonPersistentTopicsImpl(root, auth, requestTimeoutMs); this.resourceQuotas = new ResourceQuotasImpl(root, auth, requestTimeoutMs); this.lookups = new LookupImpl(root, auth, useTls, requestTimeoutMs, topics); - this.functions = new FunctionsImpl(root, auth, asyncHttpConnector.getHttpClient(), requestTimeoutMs); - this.sources = new SourcesImpl(root, auth, asyncHttpConnector.getHttpClient(), requestTimeoutMs); - this.sinks = new SinksImpl(root, auth, asyncHttpConnector.getHttpClient(), requestTimeoutMs); + this.functions = new FunctionsImpl(root, auth, asyncHttpConnector, requestTimeoutMs); + this.sources = new SourcesImpl(root, auth, asyncHttpConnector, requestTimeoutMs); + this.sinks = new SinksImpl(root, auth, asyncHttpConnector, requestTimeoutMs); this.worker = new WorkerImpl(root, auth, requestTimeoutMs); this.schemas = new SchemasImpl(root, auth, requestTimeoutMs); this.bookies = new BookiesImpl(root, auth, requestTimeoutMs); - this.packages = new PackagesImpl(root, auth, asyncHttpConnector.getHttpClient(), requestTimeoutMs); + this.packages = new PackagesImpl(root, auth, asyncHttpConnector, requestTimeoutMs); this.transactions = new TransactionsImpl(root, auth, requestTimeoutMs); if (originalCtxLoader != null) { diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SinksImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SinksImpl.java index a30f51264cc2e..bba0289d81254 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SinksImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SinksImpl.java @@ -34,13 +34,13 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.Sink; import org.apache.pulsar.client.admin.Sinks; +import org.apache.pulsar.client.admin.internal.http.AsyncHttpRequestExecutor; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.common.functions.UpdateOptions; import org.apache.pulsar.common.functions.UpdateOptionsImpl; import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.policies.data.SinkStatus; -import org.asynchttpclient.AsyncHttpClient; import org.asynchttpclient.RequestBuilder; import org.asynchttpclient.request.body.multipart.FilePart; import org.asynchttpclient.request.body.multipart.StringPart; @@ -51,12 +51,13 @@ public class SinksImpl extends ComponentResource implements Sinks, Sink { private final WebTarget sink; - private final AsyncHttpClient asyncHttpClient; + private final AsyncHttpRequestExecutor asyncHttpRequestExecutor; - public SinksImpl(WebTarget web, Authentication auth, AsyncHttpClient asyncHttpClient, long requestTimeoutMs) { + public SinksImpl(WebTarget web, Authentication auth, AsyncHttpRequestExecutor asyncHttpRequestExecutor, + long requestTimeoutMs) { super(auth, requestTimeoutMs); this.sink = web.path("/admin/v3/sink"); - this.asyncHttpClient = asyncHttpClient; + this.asyncHttpRequestExecutor = asyncHttpRequestExecutor; } @Override @@ -145,7 +146,7 @@ public CompletableFuture createSinkAsync(SinkConfig sinkConfig, String fil // If the function code is built in, we don't need to submit here builder.addBodyPart(new FilePart("data", new File(fileName), MediaType.APPLICATION_OCTET_STREAM)); } - asyncHttpClient.executeRequest(addAuthHeaders(sink, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(sink, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { @@ -233,7 +234,7 @@ public CompletableFuture updateSinkAsync( // If the function code is built in, we don't need to submit here builder.addBodyPart(new FilePart("data", new File(fileName), MediaType.APPLICATION_OCTET_STREAM)); } - asyncHttpClient.executeRequest(addAuthHeaders(sink, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(sink, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SourcesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SourcesImpl.java index 8821ed61ce5b8..56cf7db229b78 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SourcesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SourcesImpl.java @@ -33,13 +33,13 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.Source; import org.apache.pulsar.client.admin.Sources; +import org.apache.pulsar.client.admin.internal.http.AsyncHttpRequestExecutor; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.common.functions.UpdateOptions; import org.apache.pulsar.common.functions.UpdateOptionsImpl; import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.policies.data.SourceStatus; -import org.asynchttpclient.AsyncHttpClient; import org.asynchttpclient.RequestBuilder; import org.asynchttpclient.request.body.multipart.FilePart; import org.asynchttpclient.request.body.multipart.StringPart; @@ -50,12 +50,13 @@ public class SourcesImpl extends ComponentResource implements Sources, Source { private final WebTarget source; - private final AsyncHttpClient asyncHttpClient; + private final AsyncHttpRequestExecutor asyncHttpRequestExecutor; - public SourcesImpl(WebTarget web, Authentication auth, AsyncHttpClient asyncHttpClient, long requestTimeoutMs) { + public SourcesImpl(WebTarget web, Authentication auth, AsyncHttpRequestExecutor asyncHttpRequestExecutor, + long requestTimeoutMs) { super(auth, requestTimeoutMs); this.source = web.path("/admin/v3/source"); - this.asyncHttpClient = asyncHttpClient; + this.asyncHttpRequestExecutor = asyncHttpRequestExecutor; } @Override @@ -124,7 +125,7 @@ public CompletableFuture createSourceAsync(SourceConfig sourceConfig, Stri // If the function code is built in, we don't need to submit here builder.addBodyPart(new FilePart("data", new File(fileName), MediaType.APPLICATION_OCTET_STREAM)); } - asyncHttpClient.executeRequest(addAuthHeaders(source, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(source, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { @@ -202,7 +203,7 @@ public CompletableFuture updateSourceAsync( // If the function code is built in, we don't need to submit here builder.addBodyPart(new FilePart("data", new File(fileName), MediaType.APPLICATION_OCTET_STREAM)); } - asyncHttpClient.executeRequest(addAuthHeaders(source, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(source, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java index a0569c391ad50..1423d52642027 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java @@ -18,6 +18,17 @@ */ package org.apache.pulsar.client.admin.internal.http; +import static org.asynchttpclient.util.HttpConstants.Methods.GET; +import static org.asynchttpclient.util.HttpConstants.Methods.HEAD; +import static org.asynchttpclient.util.HttpConstants.Methods.OPTIONS; +import static org.asynchttpclient.util.HttpConstants.ResponseStatusCodes.FOUND_302; +import static org.asynchttpclient.util.HttpConstants.ResponseStatusCodes.MOVED_PERMANENTLY_301; +import static org.asynchttpclient.util.HttpConstants.ResponseStatusCodes.PERMANENT_REDIRECT_308; +import static org.asynchttpclient.util.HttpConstants.ResponseStatusCodes.SEE_OTHER_303; +import static org.asynchttpclient.util.HttpConstants.ResponseStatusCodes.TEMPORARY_REDIRECT_307; +import static org.asynchttpclient.util.MiscUtils.isNonEmpty; +import com.spotify.futures.ConcurrencyReducer; +import io.netty.handler.codec.http.DefaultHttpHeaders; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.ssl.SslContext; @@ -27,9 +38,12 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.URI; +import java.security.GeneralSecurityException; import java.time.Duration; +import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -37,32 +51,39 @@ import java.util.concurrent.TimeoutException; import java.util.function.Supplier; import javax.net.ssl.SSLContext; +import javax.ws.rs.ProcessingException; import javax.ws.rs.client.Client; import javax.ws.rs.core.HttpHeaders; import javax.ws.rs.core.Response.Status; import lombok.Getter; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.Validate; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; import org.apache.pulsar.client.api.AuthenticationDataProvider; import org.apache.pulsar.client.api.KeyStoreParams; import org.apache.pulsar.client.impl.PulsarServiceNameResolver; +import org.apache.pulsar.client.impl.ServiceNameResolver; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.util.WithSNISslEngineFactory; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.SecurityUtility; import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext; +import org.asynchttpclient.AsyncCompletionHandlerBase; +import org.asynchttpclient.AsyncHandler; import org.asynchttpclient.AsyncHttpClient; +import org.asynchttpclient.AsyncHttpClientConfig; import org.asynchttpclient.BoundRequestBuilder; import org.asynchttpclient.DefaultAsyncHttpClient; import org.asynchttpclient.DefaultAsyncHttpClientConfig; import org.asynchttpclient.ListenableFuture; import org.asynchttpclient.Request; import org.asynchttpclient.Response; +import org.asynchttpclient.SslEngineFactory; import org.asynchttpclient.channel.DefaultKeepAliveStrategy; import org.asynchttpclient.netty.ssl.JsseSslEngineFactory; +import org.asynchttpclient.uri.Uri; import org.glassfish.jersey.client.ClientProperties; import org.glassfish.jersey.client.ClientRequest; import org.glassfish.jersey.client.ClientResponse; @@ -73,17 +94,19 @@ * Customized Jersey client connector with multi-host support. */ @Slf4j -public class AsyncHttpConnector implements Connector { +public class AsyncHttpConnector implements Connector, AsyncHttpRequestExecutor { private static final TimeoutException REQUEST_TIMEOUT_EXCEPTION = FutureUtil.createTimeoutException("Request timeout", AsyncHttpConnector.class, "retryOrTimeout(...)"); + private static final int DEFAULT_MAX_QUEUE_SIZE_PER_HOST = 10000; @Getter private final AsyncHttpClient httpClient; private final Duration requestTimeout; private final int maxRetries; - private final PulsarServiceNameResolver serviceNameResolver; + private final ServiceNameResolver serviceNameResolver; private final ScheduledExecutorService delayer = Executors.newScheduledThreadPool(1, new DefaultThreadFactory("delayer")); private final boolean acceptGzipCompression; + private final Map> concurrencyReducers = new ConcurrentHashMap<>(); public AsyncHttpConnector(Client client, ClientConfigurationData conf, int autoCertRefreshTimeSeconds, boolean acceptGzipCompression) { @@ -99,10 +122,47 @@ public AsyncHttpConnector(int connectTimeoutMs, int readTimeoutMs, int requestTimeoutMs, int autoCertRefreshTimeSeconds, ClientConfigurationData conf, boolean acceptGzipCompression) { + Validate.notEmpty(conf.getServiceUrl(), "Service URL is not provided"); + serviceNameResolver = new PulsarServiceNameResolver(); + String serviceUrl = conf.getServiceUrl(); + serviceNameResolver.updateServiceUrl(serviceUrl); this.acceptGzipCompression = acceptGzipCompression; + AsyncHttpClientConfig asyncHttpClientConfig = + createAsyncHttpClientConfig(conf, connectTimeoutMs, readTimeoutMs, requestTimeoutMs, + autoCertRefreshTimeSeconds); + httpClient = createAsyncHttpClient(asyncHttpClientConfig); + this.requestTimeout = requestTimeoutMs > 0 ? Duration.ofMillis(requestTimeoutMs) : null; + this.maxRetries = httpClient.getConfig().getMaxRequestRetry(); + } + + private AsyncHttpClientConfig createAsyncHttpClientConfig(ClientConfigurationData conf, int connectTimeoutMs, + int readTimeoutMs, + int requestTimeoutMs, int autoCertRefreshTimeSeconds) + throws GeneralSecurityException, IOException { DefaultAsyncHttpClientConfig.Builder confBuilder = new DefaultAsyncHttpClientConfig.Builder(); + configureAsyncHttpClientConfig(conf, connectTimeoutMs, readTimeoutMs, requestTimeoutMs, confBuilder); + if (conf.getServiceUrl().startsWith("https://")) { + configureAsyncHttpClientSslEngineFactory(conf, autoCertRefreshTimeSeconds, confBuilder); + } + AsyncHttpClientConfig asyncHttpClientConfig = confBuilder.build(); + return asyncHttpClientConfig; + } + + private void configureAsyncHttpClientConfig(ClientConfigurationData conf, int connectTimeoutMs, int readTimeoutMs, + int requestTimeoutMs, + DefaultAsyncHttpClientConfig.Builder confBuilder) { + if (conf.getConnectionsPerBroker() > 0) { + confBuilder.setMaxConnectionsPerHost(conf.getConnectionsPerBroker()); + // Use the request timeout value for acquireFreeChannelTimeout so that we don't need to add + // yet another configuration property. When the ConcurrencyReducer is in use, it shouldn't be necessary to + // wait for a free channel since the ConcurrencyReducer will queue the requests. + confBuilder.setAcquireFreeChannelTimeout(conf.getRequestTimeoutMs()); + } + if (conf.getConnectionMaxIdleSeconds() > 0) { + confBuilder.setPooledConnectionIdleTimeout(conf.getConnectionMaxIdleSeconds() * 1000); + } confBuilder.setUseProxyProperties(true); - confBuilder.setFollowRedirect(true); + confBuilder.setFollowRedirect(false); confBuilder.setRequestTimeout(conf.getRequestTimeoutMs()); confBuilder.setConnectTimeout(connectTimeoutMs); confBuilder.setReadTimeout(readTimeoutMs); @@ -118,75 +178,75 @@ public boolean keepAlive(InetSocketAddress remoteAddress, Request ahcRequest, && super.keepAlive(remoteAddress, ahcRequest, request, response); } }); + confBuilder.setDisableHttpsEndpointIdentificationAlgorithm(!conf.isTlsHostnameVerificationEnable()); + } - serviceNameResolver = new PulsarServiceNameResolver(); - if (conf != null && StringUtils.isNotBlank(conf.getServiceUrl())) { - serviceNameResolver.updateServiceUrl(conf.getServiceUrl()); - if (conf.getServiceUrl().startsWith("https://")) { - // Set client key and certificate if available - AuthenticationDataProvider authData = conf.getAuthentication().getAuthData(); - - if (conf.isUseKeyStoreTls()) { - KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : - new KeyStoreParams(conf.getTlsKeyStoreType(), conf.getTlsKeyStorePath(), - conf.getTlsKeyStorePassword()); - - final SSLContext sslCtx = KeyStoreSSLContext.createClientSslContext( - conf.getSslProvider(), - params.getKeyStoreType(), - params.getKeyStorePath(), - params.getKeyStorePassword(), - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustStoreType(), - conf.getTlsTrustStorePath(), - conf.getTlsTrustStorePassword(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - - JsseSslEngineFactory sslEngineFactory = new JsseSslEngineFactory(sslCtx); - confBuilder.setSslEngineFactory(sslEngineFactory); - } else { - SslProvider sslProvider = null; - if (conf.getSslProvider() != null) { - sslProvider = SslProvider.valueOf(conf.getSslProvider()); - } - SslContext sslCtx = null; - if (authData.hasDataForTls()) { - sslCtx = authData.getTlsTrustStoreStream() == null - ? SecurityUtility.createAutoRefreshSslContextForClient( - sslProvider, - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustCertsFilePath(), authData.getTlsCertificateFilePath(), - authData.getTlsPrivateKeyFilePath(), null, autoCertRefreshTimeSeconds, delayer) - : SecurityUtility.createNettySslContextForClient( - sslProvider, - conf.isTlsAllowInsecureConnection(), - authData.getTlsTrustStoreStream(), authData.getTlsCertificates(), - authData.getTlsPrivateKey(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - } else { - sslCtx = SecurityUtility.createNettySslContextForClient( - sslProvider, - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustCertsFilePath(), - conf.getTlsCertificateFilePath(), - conf.getTlsKeyFilePath(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - } - confBuilder.setSslContext(sslCtx); - if (!conf.isTlsHostnameVerificationEnable()) { - confBuilder.setSslEngineFactory(new WithSNISslEngineFactory(serviceNameResolver - .resolveHostUri().getHost())); - } - } + protected AsyncHttpClient createAsyncHttpClient(AsyncHttpClientConfig asyncHttpClientConfig) { + return new DefaultAsyncHttpClient(asyncHttpClientConfig); + } + + private void configureAsyncHttpClientSslEngineFactory(ClientConfigurationData conf, int autoCertRefreshTimeSeconds, + DefaultAsyncHttpClientConfig.Builder confBuilder) + throws GeneralSecurityException, IOException { + // Set client key and certificate if available + AuthenticationDataProvider authData = conf.getAuthentication().getAuthData(); + + SslEngineFactory sslEngineFactory = null; + if (conf.isUseKeyStoreTls()) { + KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : + new KeyStoreParams(conf.getTlsKeyStoreType(), conf.getTlsKeyStorePath(), + conf.getTlsKeyStorePassword()); + + final SSLContext sslCtx = KeyStoreSSLContext.createClientSslContext( + conf.getSslProvider(), + params.getKeyStoreType(), + params.getKeyStorePath(), + params.getKeyStorePassword(), + conf.isTlsAllowInsecureConnection(), + conf.getTlsTrustStoreType(), + conf.getTlsTrustStorePath(), + conf.getTlsTrustStorePassword(), + conf.getTlsCiphers(), + conf.getTlsProtocols()); + + sslEngineFactory = new JsseSslEngineFactory(sslCtx); + confBuilder.setSslEngineFactory(sslEngineFactory); + } else { + SslProvider sslProvider = null; + if (conf.getSslProvider() != null) { + sslProvider = SslProvider.valueOf(conf.getSslProvider()); + } + SslContext sslCtx = null; + if (authData.hasDataForTls()) { + sslCtx = authData.getTlsTrustStoreStream() == null + ? SecurityUtility.createAutoRefreshSslContextForClient( + sslProvider, + conf.isTlsAllowInsecureConnection(), + conf.getTlsTrustCertsFilePath(), authData.getTlsCertificateFilePath(), + authData.getTlsPrivateKeyFilePath(), null, autoCertRefreshTimeSeconds, delayer) + : SecurityUtility.createNettySslContextForClient( + sslProvider, + conf.isTlsAllowInsecureConnection(), + authData.getTlsTrustStoreStream(), authData.getTlsCertificates(), + authData.getTlsPrivateKey(), + conf.getTlsCiphers(), + conf.getTlsProtocols()); + } else { + sslCtx = SecurityUtility.createNettySslContextForClient( + sslProvider, + conf.isTlsAllowInsecureConnection(), + conf.getTlsTrustCertsFilePath(), + conf.getTlsCertificateFilePath(), + conf.getTlsKeyFilePath(), + conf.getTlsCiphers(), + conf.getTlsProtocols()); + } + confBuilder.setSslContext(sslCtx); + if (!conf.isTlsHostnameVerificationEnable()) { + confBuilder.setSslEngineFactory(new WithSNISslEngineFactory(serviceNameResolver + .resolveHostUri().getHost())); } - confBuilder.setDisableHttpsEndpointIdentificationAlgorithm(!conf.isTlsHostnameVerificationEnable()); } - httpClient = new DefaultAsyncHttpClient(confBuilder.build()); - this.requestTimeout = requestTimeoutMs > 0 ? Duration.ofMillis(requestTimeoutMs) : null; - this.maxRetries = httpClient.getConfig().getMaxRequestRetry(); } @Override @@ -206,9 +266,8 @@ public void failure(Throwable failure) { try { return future.get(); } catch (InterruptedException | ExecutionException e) { - log.error(e.getMessage()); + throw new ProcessingException(e.getCause()); } - return null; } private URI replaceWithNew(InetSocketAddress address, URI uri) { @@ -270,6 +329,8 @@ private CompletableFuture retryOrTimeOut(ClientRequest request) { return resultFuture; } + // TODO: There are problems with this solution since AsyncHttpClient already contains logic to retry requests. + // This solution doesn't contain backoff handling. private void retryOperation( final CompletableFuture resultFuture, final Supplier> operation, @@ -281,9 +342,13 @@ private void retryOperation( operationFuture.whenComplete( (t, throwable) -> { if (throwable != null) { + throwable = FutureUtil.unwrapCompletionException(throwable); if (throwable instanceof CancellationException) { resultFuture.completeExceptionally( new RetryException("Operation future was cancelled.", throwable)); + } else if (throwable instanceof MaxRedirectException) { + // don't retry on max redirect + resultFuture.completeExceptionally(throwable); } else { if (retries > 0) { if (log.isDebugEnabled()) { @@ -323,7 +388,129 @@ public RetryException(String message, Throwable cause) { } } + public static class MaxRedirectException extends Exception { + public MaxRedirectException(String msg) { + super(msg, null, true, false); + } + } + protected CompletableFuture oneShot(InetSocketAddress host, ClientRequest request) { + Request preparedRequest; + try { + preparedRequest = prepareRequest(host, request); + } catch (IOException e) { + return FutureUtil.failedFuture(e); + } + return executeRequest(preparedRequest); + } + + public CompletableFuture executeRequest(Request request) { + return executeRequest(request, () -> new AsyncCompletionHandlerBase()); + } + + public CompletableFuture executeRequest(Request request, + Supplier> handlerSupplier) { + return executeRequest(request, handlerSupplier, 0); + } + + private CompletableFuture executeRequest(Request request, + Supplier> handlerSupplier, + int redirectCount) { + int maxRedirects = httpClient.getConfig().getMaxRedirects(); + if (redirectCount > maxRedirects) { + return FutureUtil.failedFuture( + new MaxRedirectException("Maximum redirect reached: " + maxRedirects + " uri:" + request.getUri())); + } + CompletableFuture responseFuture; + if (httpClient.getConfig().getMaxConnectionsPerHost() > 0) { + String hostAndPort = request.getUri().getHost() + ":" + request.getUri().getPort(); + ConcurrencyReducer responseConcurrencyReducer = concurrencyReducers.computeIfAbsent(hostAndPort, + h -> ConcurrencyReducer.create(httpClient.getConfig().getMaxConnectionsPerHost(), + DEFAULT_MAX_QUEUE_SIZE_PER_HOST)); + responseFuture = responseConcurrencyReducer.add(() -> doExecuteRequest(request, handlerSupplier)); + } else { + responseFuture = doExecuteRequest(request, handlerSupplier); + } + CompletableFuture futureWithRedirect = responseFuture.thenCompose(response -> { + if (isRedirectStatusCode(response.getStatusCode())) { + return executeRedirect(request, response, handlerSupplier, redirectCount); + } + return CompletableFuture.completedFuture(response); + }); + futureWithRedirect.whenComplete((response, throwable) -> { + // propagate cancellation or timeout to the original response future + responseFuture.cancel(false); + }); + return futureWithRedirect; + } + + private CompletableFuture executeRedirect(Request request, Response response, + Supplier> handlerSupplier, + int redirectCount) { + String originalMethod = request.getMethod(); + int statusCode = response.getStatusCode(); + boolean switchToGet = !originalMethod.equals(GET) + && !originalMethod.equals(OPTIONS) && !originalMethod.equals(HEAD) && ( + statusCode == MOVED_PERMANENTLY_301 || statusCode == SEE_OTHER_303 || statusCode == FOUND_302); + boolean keepBody = statusCode == TEMPORARY_REDIRECT_307 || statusCode == PERMANENT_REDIRECT_308; + String location = response.getHeader(HttpHeaders.LOCATION); + Uri newUri = Uri.create(request.getUri(), location); + BoundRequestBuilder builder = httpClient.prepareRequest(request); + if (switchToGet) { + builder.setMethod(GET); + } + builder.setUri(newUri); + if (keepBody) { + builder.setCharset(request.getCharset()); + if (isNonEmpty(request.getFormParams())) { + builder.setFormParams(request.getFormParams()); + } else if (request.getStringData() != null) { + builder.setBody(request.getStringData()); + } else if (request.getByteData() != null){ + builder.setBody(request.getByteData()); + } else if (request.getByteBufferData() != null) { + builder.setBody(request.getByteBufferData()); + } else if (request.getBodyGenerator() != null) { + builder.setBody(request.getBodyGenerator()); + } else if (isNonEmpty(request.getBodyParts())) { + builder.setBodyParts(request.getBodyParts()); + } + } else { + builder.resetFormParams(); + builder.resetNonMultipartData(); + builder.resetMultipartData(); + io.netty.handler.codec.http.HttpHeaders headers = new DefaultHttpHeaders(); + headers.add(request.getHeaders()); + headers.remove(HttpHeaders.CONTENT_LENGTH); + headers.remove(HttpHeaders.CONTENT_TYPE); + headers.remove(HttpHeaders.CONTENT_ENCODING); + builder.setHeaders(headers); + } + return executeRequest(builder.build(), handlerSupplier, redirectCount + 1); + } + + private static boolean isRedirectStatusCode(int statusCode) { + return statusCode == MOVED_PERMANENTLY_301 || statusCode == FOUND_302 || statusCode == SEE_OTHER_303 + || statusCode == TEMPORARY_REDIRECT_307 || statusCode == PERMANENT_REDIRECT_308; + } + + private CompletableFuture doExecuteRequest(Request request, + Supplier> handlerSupplier) { + ListenableFuture responseFuture = + httpClient.executeRequest(request, handlerSupplier.get()); + CompletableFuture completableFuture = responseFuture.toCompletableFuture(); + completableFuture.whenComplete((response, throwable) -> { + throwable = FutureUtil.unwrapCompletionException(throwable); + if (throwable != null && (throwable instanceof CancellationException + || throwable instanceof TimeoutException)) { + // abort the request if the future is cancelled or timed out + responseFuture.abort(throwable); + } + }); + return completableFuture; + } + + private Request prepareRequest(InetSocketAddress host, ClientRequest request) throws IOException { ClientRequest currentRequest = new ClientRequest(request); URI newUri = replaceWithNew(host, currentRequest.getUri()); currentRequest.setUri(newUri); @@ -334,14 +521,7 @@ protected CompletableFuture oneShot(InetSocketAddress host, ClientRequ if (currentRequest.hasEntity()) { ByteArrayOutputStream outStream = new ByteArrayOutputStream(); currentRequest.setStreamProvider(contentLength -> outStream); - try { - currentRequest.writeEntity(); - } catch (IOException e) { - CompletableFuture r = new CompletableFuture<>(); - r.completeExceptionally(e); - return r; - } - + currentRequest.writeEntity(); builder.setBody(outStream.toByteArray()); } @@ -355,16 +535,7 @@ protected CompletableFuture oneShot(InetSocketAddress host, ClientRequ builder.setHeader(HttpHeaders.ACCEPT_ENCODING, "gzip"); } - ListenableFuture responseFuture = builder.execute(); - CompletableFuture completableFuture = responseFuture.toCompletableFuture(); - completableFuture.whenComplete((response, throwable) -> { - if (throwable != null && (throwable instanceof CancellationException - || throwable instanceof TimeoutException)) { - // abort the request if the future is cancelled or timed out - responseFuture.abort(throwable); - } - }); - return completableFuture; + return builder.build(); } @Override diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpRequestExecutor.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpRequestExecutor.java new file mode 100644 index 0000000000000..d3c7a653b36b4 --- /dev/null +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpRequestExecutor.java @@ -0,0 +1,48 @@ +/* + * 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.admin.internal.http; + +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; +import org.asynchttpclient.AsyncHandler; +import org.asynchttpclient.Request; +import org.asynchttpclient.Response; + +/** + * Interface for executing HTTP requests asynchronously. + * This is used internally in the Pulsar Admin client for executing HTTP requests that by-pass the Jersey client + * and use the AsyncHttpClient API directly. + */ +public interface AsyncHttpRequestExecutor { + /** + * Execute the given HTTP request asynchronously. + * + * @param request the HTTP request to execute + * @return a future that will be completed with the HTTP response + */ + CompletableFuture executeRequest(Request request); + /** + * Execute the given HTTP request asynchronously. + * + * @param request the HTTP request to execute + * @param handlerSupplier a supplier for the async handler to use for the request + * @return a future that will be completed with the HTTP response + */ + CompletableFuture executeRequest(Request request, Supplier> handlerSupplier); +} diff --git a/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImplTest.java b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImplTest.java index 8f4162ca74b32..b61b8774b6e2e 100644 --- a/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImplTest.java +++ b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImplTest.java @@ -66,6 +66,7 @@ public void testGetPropertiesFromConf() throws Exception { config.put("autoCertRefreshSeconds", 20); config.put("connectionTimeoutMs", 30); config.put("readTimeoutMs", 40); + config.put("maxConnectionsPerHost", 50); PulsarAdminBuilder adminBuilder = PulsarAdmin.builder().loadConf(config); @Cleanup PulsarAdminImpl admin = (PulsarAdminImpl) adminBuilder.build(); @@ -74,6 +75,7 @@ public void testGetPropertiesFromConf() throws Exception { Assert.assertEquals(clientConfigData.getAutoCertRefreshSeconds(), 20); Assert.assertEquals(clientConfigData.getConnectionTimeoutMs(), 30); Assert.assertEquals(clientConfigData.getReadTimeoutMs(), 40); + Assert.assertEquals(clientConfigData.getConnectionsPerBroker(), 50); } @Test diff --git a/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorTest.java b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorTest.java index dd3fb40ae9ab0..f8518b5931034 100644 --- a/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorTest.java +++ b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorTest.java @@ -20,23 +20,34 @@ import static com.github.tomakehurst.wiremock.client.WireMock.aResponse; import static com.github.tomakehurst.wiremock.client.WireMock.get; +import static com.github.tomakehurst.wiremock.client.WireMock.post; import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import com.github.tomakehurst.wiremock.WireMockServer; +import com.github.tomakehurst.wiremock.common.FileSource; import com.github.tomakehurst.wiremock.core.WireMockConfiguration; +import com.github.tomakehurst.wiremock.extension.Parameters; +import com.github.tomakehurst.wiremock.extension.ResponseTransformer; import com.github.tomakehurst.wiremock.stubbing.Scenario; import java.io.IOException; import java.net.InetSocketAddress; import java.net.URI; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.common.util.FutureUtil; +import org.asynchttpclient.Request; +import org.asynchttpclient.RequestBuilder; import org.asynchttpclient.Response; import org.glassfish.jersey.client.ClientConfig; import org.glassfish.jersey.client.ClientRequest; @@ -52,10 +63,74 @@ public class AsyncHttpConnectorTest { WireMockServer server; + ConcurrencyTestTransformer concurrencyTestTransformer = new ConcurrencyTestTransformer(); + + private static class CopyRequestBodyToResponseBodyTransformer extends ResponseTransformer { + @Override + public com.github.tomakehurst.wiremock.http.Response transform( + com.github.tomakehurst.wiremock.http.Request request, + com.github.tomakehurst.wiremock.http.Response response, FileSource fileSource, Parameters parameters) { + return com.github.tomakehurst.wiremock.http.Response.Builder.like(response) + .body(request.getBodyAsString()) + .build(); + } + + @Override + public String getName() { + return "copy-body"; + } + + @Override + public boolean applyGlobally() { + return false; + } + } + + private static class ConcurrencyTestTransformer extends ResponseTransformer { + private static final long DELAY_MS = 100; + private final AtomicInteger concurrencyCounter = new AtomicInteger(0); + private final AtomicInteger maxConcurrency = new AtomicInteger(0); + + @Override + public com.github.tomakehurst.wiremock.http.Response transform( + com.github.tomakehurst.wiremock.http.Request request, + com.github.tomakehurst.wiremock.http.Response response, FileSource fileSource, Parameters parameters) { + int currentCounter = concurrencyCounter.incrementAndGet(); + maxConcurrency.updateAndGet(v -> Math.max(v, currentCounter)); + try { + try { + Thread.sleep(DELAY_MS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + return com.github.tomakehurst.wiremock.http.Response.Builder.like(response) + .body(String.valueOf(currentCounter)) + .build(); + } finally { + concurrencyCounter.decrementAndGet(); + } + } + + public int getMaxConcurrency() { + return maxConcurrency.get(); + } + + @Override + public String getName() { + return "concurrency-test"; + } + + @Override + public boolean applyGlobally() { + return false; + } + } @BeforeClass(alwaysRun = true) void beforeClass() throws IOException { server = new WireMockServer(WireMockConfiguration.wireMockConfig() + .extensions(new CopyRequestBodyToResponseBodyTransformer(), concurrencyTestTransformer) + .containerThreads(100) .port(0)); server.start(); } @@ -137,4 +212,129 @@ public void failure(Throwable failure) { assertEquals(scenarioState, "next"); assertTrue(future.isCompletedExceptionally()); } + + @Test + void testMaxRedirects() { + // Redirect to itself to test max redirects + server.stubFor(get(urlEqualTo("/admin/v2/clusters")) + .willReturn(aResponse() + .withStatus(301) + .withHeader("Location", "http://localhost:" + server.port() + "/admin/v2/clusters"))); + + ClientConfigurationData conf = new ClientConfigurationData(); + conf.setServiceUrl("http://localhost:" + server.port()); + + @Cleanup + AsyncHttpConnector connector = new AsyncHttpConnector(5000, 5000, + 5000, 0, conf, false); + + Request request = new RequestBuilder("GET") + .setUrl("http://localhost:" + server.port() + "/admin/v2/clusters") + .build(); + + try { + connector.executeRequest(request).get(); + fail(); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof AsyncHttpConnector.MaxRedirectException); + } catch (InterruptedException e) { + fail(); + } + } + + @Test + void testRelativeRedirect() throws ExecutionException, InterruptedException { + doTestRedirect("path2"); + } + + @Test + void testAbsoluteRedirect() throws ExecutionException, InterruptedException { + doTestRedirect("/path2"); + } + + @Test + void testUrlRedirect() throws ExecutionException, InterruptedException { + doTestRedirect("http://localhost:" + server.port() + "/path2"); + } + + private void doTestRedirect(String location) throws InterruptedException, ExecutionException { + server.stubFor(get(urlEqualTo("/path1")) + .willReturn(aResponse() + .withStatus(301) + .withHeader("Location", location))); + + server.stubFor(get(urlEqualTo("/path2")) + .willReturn(aResponse() + .withBody("OK"))); + + ClientConfigurationData conf = new ClientConfigurationData(); + conf.setServiceUrl("http://localhost:" + server.port()); + + @Cleanup + AsyncHttpConnector connector = new AsyncHttpConnector(5000, 5000, + 5000, 0, conf, false); + + Request request = new RequestBuilder("GET") + .setUrl("http://localhost:" + server.port() + "/path1") + .build(); + + Response response = connector.executeRequest(request).get(); + assertEquals(response.getResponseBody(), "OK"); + } + + @Test + void testRedirectWithBody() throws ExecutionException, InterruptedException { + server.stubFor(post(urlEqualTo("/path1")) + .willReturn(aResponse() + .withStatus(307) + .withHeader("Location", "/path2"))); + + server.stubFor(post(urlEqualTo("/path2")) + .willReturn(aResponse() + .withTransformers("copy-body"))); + + ClientConfigurationData conf = new ClientConfigurationData(); + conf.setServiceUrl("http://localhost:" + server.port()); + + @Cleanup + AsyncHttpConnector connector = new AsyncHttpConnector(5000, 5000, + 5000, 0, conf, false); + + Request request = new RequestBuilder("POST") + .setUrl("http://localhost:" + server.port() + "/path1") + .setBody("Hello world!") + .build(); + + Response response = connector.executeRequest(request).get(); + assertEquals(response.getResponseBody(), "Hello world!"); + } + + @Test + void testMaxConnections() throws ExecutionException, InterruptedException { + server.stubFor(post(urlEqualTo("/concurrency-test")) + .willReturn(aResponse() + .withTransformers("concurrency-test"))); + + ClientConfigurationData conf = new ClientConfigurationData(); + int maxConnections = 10; + conf.setConnectionsPerBroker(maxConnections); + conf.setServiceUrl("http://localhost:" + server.port()); + + @Cleanup + AsyncHttpConnector connector = new AsyncHttpConnector(5000, 5000, + 5000, 0, conf, false); + + Request request = new RequestBuilder("POST") + .setUrl("http://localhost:" + server.port() + "/concurrency-test") + .build(); + + List> futures = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + futures.add(connector.executeRequest(request)); + } + FutureUtil.waitForAll(futures).get(); + int maxConcurrency = concurrencyTestTransformer.getMaxConcurrency(); + assertTrue(maxConcurrency > maxConnections / 2 && maxConcurrency <= maxConnections, + "concurrency didn't get limited as expected (max: " + maxConcurrency + ")"); + } } \ No newline at end of file diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 65d24e3394d10..484869e35b604 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -167,6 +167,7 @@ com.google.j2objc:* com.google.code.gson:gson com.google.re2j:re2j + com.spotify:completable-futures com.fasterxml.jackson.*:* io.netty:netty io.netty:netty-all @@ -243,6 +244,10 @@ com.google.protobuf.* + + com.spotify.futures + org.apache.pulsar.shade.com.spotify.futures + com.fasterxml.jackson org.apache.pulsar.shade.com.fasterxml.jackson diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java index 735aeeed55916..7b98fa57bf0de 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java @@ -130,6 +130,8 @@ public interface ClientBuilder extends Serializable, Cloneable { /** * Release the connection if it is not used for more than {@param connectionMaxIdleSeconds} seconds. + * Defaults to 25 seconds. + * * @return the client builder instance */ ClientBuilder connectionMaxIdleSeconds(int connectionMaxIdleSeconds); diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index c18d3123e66be..13f3d237d6e82 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -145,6 +145,7 @@ com.google.j2objc:* com.google.code.gson:gson com.google.re2j:re2j + com.spotify:completable-futures com.fasterxml.jackson.*:* io.netty:* io.netty.incubator:* @@ -204,6 +205,10 @@ com.google.protobuf.* + + com.spotify.futures + org.apache.pulsar.shade.com.spotify.futures + com.fasterxml.jackson org.apache.pulsar.shade.com.fasterxml.jackson diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java index d5adbdd7098ed..21575578e76f2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java @@ -67,7 +67,7 @@ public class ConnectionPool implements AutoCloseable { - public static final int IDLE_DETECTION_INTERVAL_SECONDS_MIN = 60; + public static final int IDLE_DETECTION_INTERVAL_SECONDS_MIN = 15; protected final ConcurrentMap> pool; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java index 6dcea7dc46672..237c6b5aebc3c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl.conf; import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.opentelemetry.api.OpenTelemetry; import io.swagger.annotations.ApiModelProperty; @@ -49,6 +50,7 @@ @Data @NoArgsConstructor @AllArgsConstructor +@JsonIgnoreProperties(ignoreUnknown = true) public class ClientConfigurationData implements Serializable, Cloneable { private static final long serialVersionUID = 1L; @@ -134,7 +136,7 @@ public class ClientConfigurationData implements Serializable, Cloneable { value = "Release the connection if it is not used for more than [connectionMaxIdleSeconds] seconds. " + "If [connectionMaxIdleSeconds] < 0, disabled the feature that auto release the idle connections" ) - private int connectionMaxIdleSeconds = 180; + private int connectionMaxIdleSeconds = 25; @ApiModelProperty( name = "useTcpNoDelay", diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientBuilderImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientBuilderImplTest.java index f56427b12038a..2f9c7536d753d 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientBuilderImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientBuilderImplTest.java @@ -123,7 +123,7 @@ public void testConnectionMaxIdleSeconds() throws Exception { PulsarClient.builder().connectionMaxIdleSeconds(60); // test config not correct. try { - PulsarClient.builder().connectionMaxIdleSeconds(30); + PulsarClient.builder().connectionMaxIdleSeconds(14); fail(); } catch (IllegalArgumentException e){ } diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index aa7e4998e5c3e..4b53954aab1df 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -210,6 +210,11 @@ re2j + + com.spotify + completable-futures + + org.bouncycastle From f4a8094f8c71cf9060566dd627e5c00003ca3833 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Fri, 9 Aug 2024 01:08:41 +0800 Subject: [PATCH 426/580] [fix][metadata] Upgrade Oxia to 0.3.2 (#23140) --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 4bbb86653add5..505c4b30093e7 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -480,8 +480,8 @@ The Apache Software License, Version 2.0 * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia - - io.streamnative.oxia-oxia-client-api-0.3.1.jar - - io.streamnative.oxia-oxia-client-0.3.1.jar + - io.streamnative.oxia-oxia-client-api-0.3.2.jar + - io.streamnative.oxia-oxia-client-0.3.2.jar * OpenHFT - net.openhft-zero-allocation-hashing-0.16.jar * Java JSON WebTokens diff --git a/pom.xml b/pom.xml index 52843f5079f01..b71508366da53 100644 --- a/pom.xml +++ b/pom.xml @@ -252,7 +252,7 @@ flexible messaging model and an intuitive client API. 4.5.13 4.4.15 0.7.7 - 0.3.1 + 0.3.2 2.0 1.10.12 5.5.0 From 4824df5f1495c69fe6214963315d0540577affb1 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 9 Aug 2024 10:36:12 +0300 Subject: [PATCH 427/580] [improve][doc] Add Pulsar Geo-Replication considerations to the PIP template (#23145) --- pip/TEMPLATE.md | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/pip/TEMPLATE.md b/pip/TEMPLATE.md index 94f7b47732cd4..0eaed8f2b6810 100644 --- a/pip/TEMPLATE.md +++ b/pip/TEMPLATE.md @@ -139,16 +139,22 @@ If there is uncertainty for this section, please submit the PIP and request for # Backward & Forward Compatibility -## Revert +## Upgrade -## Upgrade +## Downgrade / Rollback + +## Pulsar Geo-Replication Upgrade & Downgrade/Rollback Considerations + + # Alternatives From 6f5c6568ea4c6be527b420bde413c7376db245c5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 9 Aug 2024 14:28:52 +0300 Subject: [PATCH 428/580] [improve][build] Bump version to 4.0.0-SNAPSHOT (#23146) --- bouncy-castle/bc/pom.xml | 2 +- bouncy-castle/bcfips-include-test/pom.xml | 2 +- bouncy-castle/bcfips/pom.xml | 2 +- bouncy-castle/pom.xml | 2 +- buildtools/pom.xml | 4 ++-- distribution/io/pom.xml | 2 +- distribution/offloaders/pom.xml | 2 +- distribution/pom.xml | 2 +- distribution/server/pom.xml | 2 +- distribution/shell/pom.xml | 2 +- docker/pom.xml | 2 +- docker/pulsar-all/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- jetcd-core-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- microbench/pom.xml | 2 +- pom.xml | 4 ++-- pulsar-bom/pom.xml | 4 ++-- pulsar-broker-auth-athenz/pom.xml | 2 +- pulsar-broker-auth-oidc/pom.xml | 2 +- pulsar-broker-auth-sasl/pom.xml | 2 +- pulsar-broker-common/pom.xml | 2 +- pulsar-broker/pom.xml | 2 +- pulsar-cli-utils/pom.xml | 2 +- pulsar-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml | 2 +- pulsar-client-admin-api/pom.xml | 2 +- pulsar-client-admin-shaded/pom.xml | 2 +- pulsar-client-admin/pom.xml | 2 +- pulsar-client-all/pom.xml | 2 +- pulsar-client-api/pom.xml | 2 +- pulsar-client-auth-athenz/pom.xml | 2 +- pulsar-client-auth-sasl/pom.xml | 2 +- pulsar-client-messagecrypto-bc/pom.xml | 2 +- pulsar-client-shaded/pom.xml | 2 +- pulsar-client-tools-api/pom.xml | 2 +- pulsar-client-tools-customcommand-example/pom.xml | 2 +- pulsar-client-tools-test/pom.xml | 2 +- pulsar-client-tools/pom.xml | 2 +- pulsar-client/pom.xml | 2 +- pulsar-common/pom.xml | 2 +- pulsar-config-validation/pom.xml | 2 +- pulsar-docs-tools/pom.xml | 2 +- pulsar-functions/api-java/pom.xml | 2 +- pulsar-functions/instance/pom.xml | 2 +- pulsar-functions/java-examples-builtin/pom.xml | 2 +- pulsar-functions/java-examples/pom.xml | 2 +- pulsar-functions/localrun-shaded/pom.xml | 2 +- pulsar-functions/localrun/pom.xml | 2 +- pulsar-functions/pom.xml | 2 +- pulsar-functions/proto/pom.xml | 2 +- pulsar-functions/runtime-all/pom.xml | 2 +- pulsar-functions/runtime/pom.xml | 2 +- pulsar-functions/secrets/pom.xml | 2 +- pulsar-functions/utils/pom.xml | 2 +- pulsar-functions/worker/pom.xml | 2 +- pulsar-io/aerospike/pom.xml | 2 +- pulsar-io/alluxio/pom.xml | 2 +- pulsar-io/aws/pom.xml | 2 +- pulsar-io/azure-data-explorer/pom.xml | 2 +- pulsar-io/batch-data-generator/pom.xml | 2 +- pulsar-io/batch-discovery-triggerers/pom.xml | 2 +- pulsar-io/canal/pom.xml | 2 +- pulsar-io/cassandra/pom.xml | 2 +- pulsar-io/common/pom.xml | 2 +- pulsar-io/core/pom.xml | 2 +- pulsar-io/data-generator/pom.xml | 2 +- pulsar-io/debezium/core/pom.xml | 2 +- pulsar-io/debezium/mongodb/pom.xml | 2 +- pulsar-io/debezium/mssql/pom.xml | 2 +- pulsar-io/debezium/mysql/pom.xml | 2 +- pulsar-io/debezium/oracle/pom.xml | 2 +- pulsar-io/debezium/pom.xml | 2 +- pulsar-io/debezium/postgres/pom.xml | 2 +- pulsar-io/docs/pom.xml | 2 +- pulsar-io/dynamodb/pom.xml | 2 +- pulsar-io/elastic-search/pom.xml | 2 +- pulsar-io/file/pom.xml | 2 +- pulsar-io/flume/pom.xml | 2 +- pulsar-io/hbase/pom.xml | 2 +- pulsar-io/hdfs2/pom.xml | 2 +- pulsar-io/hdfs3/pom.xml | 2 +- pulsar-io/http/pom.xml | 2 +- pulsar-io/influxdb/pom.xml | 2 +- pulsar-io/jdbc/clickhouse/pom.xml | 2 +- pulsar-io/jdbc/core/pom.xml | 2 +- pulsar-io/jdbc/mariadb/pom.xml | 2 +- pulsar-io/jdbc/openmldb/pom.xml | 2 +- pulsar-io/jdbc/pom.xml | 2 +- pulsar-io/jdbc/postgres/pom.xml | 2 +- pulsar-io/jdbc/sqlite/pom.xml | 2 +- pulsar-io/kafka-connect-adaptor-nar/pom.xml | 2 +- pulsar-io/kafka-connect-adaptor/pom.xml | 2 +- pulsar-io/kafka/pom.xml | 2 +- pulsar-io/kinesis/pom.xml | 2 +- pulsar-io/mongo/pom.xml | 2 +- pulsar-io/netty/pom.xml | 2 +- pulsar-io/nsq/pom.xml | 2 +- pulsar-io/pom.xml | 2 +- pulsar-io/rabbitmq/pom.xml | 2 +- pulsar-io/redis/pom.xml | 2 +- pulsar-io/solr/pom.xml | 2 +- pulsar-io/twitter/pom.xml | 2 +- pulsar-metadata/pom.xml | 2 +- pulsar-opentelemetry/pom.xml | 2 +- pulsar-package-management/bookkeeper-storage/pom.xml | 2 +- pulsar-package-management/core/pom.xml | 2 +- pulsar-package-management/filesystem-storage/pom.xml | 2 +- pulsar-package-management/pom.xml | 2 +- pulsar-proxy/pom.xml | 2 +- pulsar-testclient/pom.xml | 2 +- pulsar-transaction/common/pom.xml | 2 +- pulsar-transaction/coordinator/pom.xml | 2 +- pulsar-transaction/pom.xml | 2 +- pulsar-websocket/pom.xml | 2 +- structured-event-log/pom.xml | 2 +- testmocks/pom.xml | 2 +- tests/bc_2_0_0/pom.xml | 2 +- tests/bc_2_0_1/pom.xml | 2 +- tests/bc_2_6_0/pom.xml | 2 +- tests/docker-images/java-test-functions/pom.xml | 2 +- tests/docker-images/java-test-image/pom.xml | 2 +- tests/docker-images/java-test-plugins/pom.xml | 2 +- tests/docker-images/latest-version-image/pom.xml | 2 +- tests/docker-images/pom.xml | 2 +- tests/integration/pom.xml | 2 +- tests/pom.xml | 2 +- tests/pulsar-client-admin-shade-test/pom.xml | 2 +- tests/pulsar-client-all-shade-test/pom.xml | 2 +- tests/pulsar-client-shade-test/pom.xml | 2 +- tiered-storage/file-system/pom.xml | 2 +- tiered-storage/jcloud/pom.xml | 2 +- tiered-storage/pom.xml | 2 +- 135 files changed, 138 insertions(+), 138 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index afe3f5e4a6312..e440923af6de5 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT bouncy-castle-bc diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 775a82861ccfb..f4478174b86dd 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ org.apache.pulsar bouncy-castle-parent - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT bcfips-include-test diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index a5cab68961ed2..250b3db6b9b08 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT bouncy-castle-bcfips diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index 7641e79a48942..4d85a163104a2 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 4a44b35066583..b1ae0cd9b73f3 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ org.apache.pulsar buildtools - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT jar Pulsar Build Tools - 2024-05-13T09:56:11Z + 2024-08-09T08:42:01Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 96dd8b071106b..813c4d26d9391 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT pulsar-io-distribution diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 131eacf986af9..38beeacde8ba4 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT pulsar-offloader-distribution diff --git a/distribution/pom.xml b/distribution/pom.xml index 0ed2219ec3aef..67604e145dd73 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT distribution diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index c42b0a137850c..36641dea20f0c 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT pulsar-server-distribution diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 905bcc747450a..45108aba68f87 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT pulsar-shell-distribution diff --git a/docker/pom.xml b/docker/pom.xml index 90a845400d3e6..a5ea238241c6a 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index eb46aa339d61f..b43121dd0f613 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index cd4cbec76372c..68d82ae552825 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index f0e456b5c00f8..dd19faad904bc 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT jclouds-shaded diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml index de5d654851a52..a0885f8509547 100644 --- a/jetcd-core-shaded/pom.xml +++ b/jetcd-core-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT jetcd-core-shaded diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index fac39103c49fb..22b093f7aafd7 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT managed-ledger diff --git a/microbench/pom.xml b/microbench/pom.xml index 62561339e8879..bef02794adbd6 100644 --- a/microbench/pom.xml +++ b/microbench/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT microbench diff --git a/pom.xml b/pom.xml index b71508366da53..13bac3f639d1d 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ org.apache.pulsar pulsar - 3.4.0-SNAPSHOT + 4.0.0-SNAPSHOT Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -96,7 +96,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2024-05-13T09:56:12Z + 2024-08-09T08:42:01Z true 3. + pulsar1.close(); + Awaitility.await().atMost(60, TimeUnit.SECONDS).untilAsserted(() -> { + CompletableFuture checkStatesFuture2 = new CompletableFuture<>(); + executor.submit(() -> { + boolean res = stateArray[0] == PulsarServiceState.Failed; + res = res & stateArray[1] == PulsarServiceState.Failed; + res = res & stateArray[2] == PulsarServiceState.Healthy; + checkStatesFuture2.complete(res); + }); + Assert.assertTrue(checkStatesFuture2.join()); + producer.send("0->2"); + Assert.assertEquals(failover.getCurrentPulsarServiceIndex(), 2); + }); + + // Test recover 2 --> 1. + executor.execute(() -> { + urlArray[1] = url2; + }); + Awaitility.await().atMost(60, TimeUnit.SECONDS).untilAsserted(() -> { + CompletableFuture checkStatesFuture3 = new CompletableFuture<>(); + executor.submit(() -> { + boolean res = stateArray[0] == PulsarServiceState.Failed; + res = res & stateArray[1] == PulsarServiceState.Healthy; + res = res & stateArray[2] == PulsarServiceState.Healthy; + checkStatesFuture3.complete(res); + }); + Assert.assertTrue(checkStatesFuture3.join()); + producer.send("2->1"); + Assert.assertEquals(failover.getCurrentPulsarServiceIndex(), 1); + }); + + // Test recover 1 --> 0. + executor.execute(() -> { + urlArray[0] = url2; + }); + Awaitility.await().atMost(60, TimeUnit.SECONDS).untilAsserted(() -> { + CompletableFuture checkStatesFuture4 = new CompletableFuture<>(); + executor.submit(() -> { + boolean res = stateArray[0] == PulsarServiceState.Healthy; + res = res & stateArray[1] == PulsarServiceState.Healthy; + res = res & stateArray[2] == PulsarServiceState.Healthy; + checkStatesFuture4.complete(res); + }); + Assert.assertTrue(checkStatesFuture4.join()); + producer.send("1->0"); + Assert.assertEquals(failover.getCurrentPulsarServiceIndex(), 0); + }); + + // cleanup. + producer.close(); + client.close(); + dummyServer.close(); + } + + @Override + protected void cleanupPulsarResources() { + // Nothing to do. + } + +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index eef4469aa95fa..e155e399e2437 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -84,7 +84,7 @@ public abstract class MockedPulsarServiceBaseTest extends TestRetrySupport { // All certificate-authority files are copied from the tests/certificate-authority directory and all share the same // root CA. - protected static String getTlsFileForClient(String name) { + public static String getTlsFileForClient(String name) { return ResourceUtils.getAbsolutePath(String.format("certificate-authority/client-keys/%s.pem", name)); } public final static String CA_CERT_FILE_PATH = diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java index 36f8cb4761248..742194d9b12a1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java @@ -102,7 +102,7 @@ protected void startBrokers() throws Exception { log.info("broker-1: {}, broker-2: {}", broker1.getListenPort(), broker2.getListenPort()); } - protected int getOneFreePort() throws IOException { + public static int getOneFreePort() throws IOException { ServerSocket serverSocket = new ServerSocket(0); int port = serverSocket.getLocalPort(); serverSocket.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index d66e666e3a055..f3076ebdec6c9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -18,6 +18,9 @@ */ package org.apache.pulsar.broker.service; +import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.BROKER_CERT_FILE_PATH; +import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.BROKER_KEY_FILE_PATH; +import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.CA_CERT_FILE_PATH; import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; @@ -267,10 +270,18 @@ protected void setConfigDefaults(ServiceConfiguration config, String clusterName config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); config.setLoadBalancerSheddingEnabled(false); config.setForceDeleteNamespaceAllowed(true); + config.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + config.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); + config.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + config.setClusterName(clusterName); + config.setTlsRequireTrustedClientCertOnConnect(false); + Set tlsProtocols = Sets.newConcurrentHashSet(); + tlsProtocols.add("TLSv1.3"); + tlsProtocols.add("TLSv1.2"); + config.setTlsProtocols(tlsProtocols); } - @Override - protected void cleanup() throws Exception { + protected void cleanupPulsarResources() throws Exception { // delete namespaces. waitChangeEventsInit(replicatedNamespace); admin1.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster1)); @@ -283,6 +294,12 @@ protected void cleanup() throws Exception { admin2.namespaces().deleteNamespace(replicatedNamespace, true); admin2.namespaces().deleteNamespace(nonReplicatedNamespace, true); } + } + + @Override + protected void cleanup() throws Exception { + // cleanup pulsar resources. + cleanupPulsarResources(); // shutdown. markCurrentSetupNumberCleaned(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java index bbac688d9224c..80adc79e6fee8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java @@ -34,6 +34,7 @@ import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.ImmutablePositionImpl; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.BrokerTestUtil; @@ -46,10 +47,12 @@ import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.api.proto.CommandFlow; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; import org.testng.Assert; +import org.testng.AssertJUnit; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; @@ -542,18 +545,34 @@ public void testReaderInitAtDeletedPosition() throws Exception { .getStats(topicName, true, true, true).getSubscriptions().get("s1"); log.info("backlog size: {}", subscriptionStats.getMsgBacklog()); assertEquals(subscriptionStats.getMsgBacklog(), 0); - ManagedLedgerInternalStats.CursorStats cursorStats = - admin.topics().getInternalStats(topicName).cursors.get("s1"); + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName); + ManagedLedgerInternalStats.CursorStats cursorStats = internalStats.cursors.get("s1"); String[] ledgerIdAndEntryId = cursorStats.markDeletePosition.split(":"); - Position actMarkDeletedPos = - PositionFactory.create(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); - Position expectedMarkDeletedPos = - PositionFactory.create(msgIdInDeletedLedger5.getLedgerId(), msgIdInDeletedLedger5.getEntryId()); + ImmutablePositionImpl actMarkDeletedPos = + new ImmutablePositionImpl(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); + ImmutablePositionImpl expectedMarkDeletedPos = + new ImmutablePositionImpl(msgIdInDeletedLedger5.getLedgerId(), msgIdInDeletedLedger5.getEntryId()); + log.info("LAC: {}", internalStats.lastConfirmedEntry); log.info("Expected mark deleted position: {}", expectedMarkDeletedPos); log.info("Actual mark deleted position: {}", cursorStats.markDeletePosition); - assertTrue(actMarkDeletedPos.compareTo(expectedMarkDeletedPos) >= 0); + AssertJUnit.assertTrue(actMarkDeletedPos.compareTo(expectedMarkDeletedPos) >= 0); }); + admin.topics().createSubscription(topicName, "s2", MessageId.earliest); + admin.topics().createSubscription(topicName, "s3", MessageId.latest); + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName); + ManagedLedgerInternalStats.CursorStats cursorStats2 = internalStats.cursors.get("s2"); + String[] ledgerIdAndEntryId2 = cursorStats2.markDeletePosition.split(":"); + ImmutablePositionImpl actMarkDeletedPos2 = + new ImmutablePositionImpl(Long.valueOf(ledgerIdAndEntryId2[0]), Long.valueOf(ledgerIdAndEntryId2[1])); + ManagedLedgerInternalStats.CursorStats cursorStats3 = internalStats.cursors.get("s3"); + String[] ledgerIdAndEntryId3 = cursorStats3.markDeletePosition.split(":"); + ImmutablePositionImpl actMarkDeletedPos3 = + new ImmutablePositionImpl(Long.valueOf(ledgerIdAndEntryId3[0]), Long.valueOf(ledgerIdAndEntryId3[1])); + log.info("LAC: {}", internalStats.lastConfirmedEntry); + log.info("Actual mark deleted position 2: {}", actMarkDeletedPos2); + log.info("Actual mark deleted position 3: {}", actMarkDeletedPos3); + pulsar.getBrokerService().getTopic(topicName, false).join().get(); // cleanup. reader.close(); producer.close(); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java index 5cb22276553ab..e8b513b103f65 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java @@ -56,7 +56,7 @@ public interface ServiceUrlProvider extends AutoCloseable { * */ @Override - default void close() { + default void close() throws Exception { // do nothing } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java new file mode 100644 index 0000000000000..4beff4719c895 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java @@ -0,0 +1,341 @@ +/* + * 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.impl; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.netty.channel.EventLoopGroup; +import io.netty.util.concurrent.ScheduledFuture; +import java.util.Arrays; +import java.util.HashSet; +import java.util.concurrent.TimeUnit; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.netty.EventLoopUtil; + +@Slf4j +@SuppressFBWarnings(value = {"EI_EXPOSE_REP2"}) +public class SameAuthParamsLookupAutoClusterFailover implements ServiceUrlProvider { + + private PulsarClientImpl pulsarClient; + private EventLoopGroup executor; + private volatile boolean closed; + private ScheduledFuture scheduledCheckTask; + @Getter + private int failoverThreshold = 5; + @Getter + private int recoverThreshold = 5; + @Getter + private long checkHealthyIntervalMs = 1000; + @Getter + private boolean markTopicNotFoundAsAvailable = true; + @Getter + private String testTopic = "public/default/tp_test"; + + private String[] pulsarServiceUrlArray; + private PulsarServiceState[] pulsarServiceStateArray; + private MutableInt[] checkCounterArray; + @Getter + private volatile int currentPulsarServiceIndex; + + private SameAuthParamsLookupAutoClusterFailover() {} + + @Override + public void initialize(PulsarClient client) { + this.currentPulsarServiceIndex = 0; + this.pulsarClient = (PulsarClientImpl) client; + this.executor = EventLoopUtil.newEventLoopGroup(1, false, + new ExecutorProvider.ExtendedThreadFactory("broker-service-url-check")); + scheduledCheckTask = executor.scheduleAtFixedRate(() -> { + if (closed) { + return; + } + checkPulsarServices(); + int firstHealthyPulsarService = firstHealthyPulsarService(); + if (firstHealthyPulsarService == currentPulsarServiceIndex) { + return; + } + if (firstHealthyPulsarService < 0) { + int failoverTo = findFailoverTo(); + if (failoverTo < 0) { + // No healthy pulsar service to connect. + log.error("Failed to choose a pulsar service to connect, no one pulsar service is healthy. Current" + + " pulsar service: [{}] {}. States: {}, Counters: {}", currentPulsarServiceIndex, + pulsarServiceUrlArray[currentPulsarServiceIndex], Arrays.toString(pulsarServiceStateArray), + Arrays.toString(checkCounterArray)); + } else { + // Failover to low priority pulsar service. + updateServiceUrl(failoverTo); + } + } else { + // Back to high priority pulsar service. + updateServiceUrl(firstHealthyPulsarService); + } + }, checkHealthyIntervalMs, checkHealthyIntervalMs, TimeUnit.MILLISECONDS); + } + + @Override + public String getServiceUrl() { + return pulsarServiceUrlArray[currentPulsarServiceIndex]; + } + + @Override + public void close() throws Exception { + log.info("Closing service url provider. Current pulsar service: [{}] {}", currentPulsarServiceIndex, + pulsarServiceUrlArray[currentPulsarServiceIndex]); + closed = true; + scheduledCheckTask.cancel(false); + executor.shutdownNow(); + } + + private int firstHealthyPulsarService() { + for (int i = 0; i <= currentPulsarServiceIndex; i++) { + if (pulsarServiceStateArray[i] == PulsarServiceState.Healthy + || pulsarServiceStateArray[i] == PulsarServiceState.PreFail) { + return i; + } + } + return -1; + } + + private int findFailoverTo() { + for (int i = currentPulsarServiceIndex + 1; i <= pulsarServiceUrlArray.length; i++) { + if (probeAvailable(i)) { + return i; + } + } + return -1; + } + + private void checkPulsarServices() { + for (int i = 0; i <= currentPulsarServiceIndex; i++) { + if (probeAvailable(i)) { + switch (pulsarServiceStateArray[i]) { + case Healthy: { + break; + } + case PreFail: { + pulsarServiceStateArray[i] = PulsarServiceState.Healthy; + checkCounterArray[i].setValue(0); + break; + } + case Failed: { + pulsarServiceStateArray[i] = PulsarServiceState.PreRecover; + checkCounterArray[i].setValue(1); + break; + } + case PreRecover: { + checkCounterArray[i].setValue(checkCounterArray[i].getValue() + 1); + if (checkCounterArray[i].getValue() >= recoverThreshold) { + pulsarServiceStateArray[i] = PulsarServiceState.Healthy; + checkCounterArray[i].setValue(0); + } + break; + } + } + } else { + switch (pulsarServiceStateArray[i]) { + case Healthy: { + pulsarServiceStateArray[i] = PulsarServiceState.PreFail; + checkCounterArray[i].setValue(1); + break; + } + case PreFail: { + checkCounterArray[i].setValue(checkCounterArray[i].getValue() + 1); + if (checkCounterArray[i].getValue() >= failoverThreshold) { + pulsarServiceStateArray[i] = PulsarServiceState.Failed; + checkCounterArray[i].setValue(0); + } + break; + } + case Failed: { + break; + } + case PreRecover: { + pulsarServiceStateArray[i] = PulsarServiceState.Failed; + checkCounterArray[i].setValue(0); + break; + } + } + } + } + } + + private boolean probeAvailable(int brokerServiceIndex) { + String url = pulsarServiceUrlArray[brokerServiceIndex]; + try { + LookupTopicResult res = pulsarClient.getLookup(url).getBroker(TopicName.get(testTopic)) + .get(3, TimeUnit.SECONDS); + if (log.isDebugEnabled()) { + log.debug("Success to probe available(lookup res: {}), [{}] {}}. States: {}, Counters: {}", + res.toString(), brokerServiceIndex, url, Arrays.toString(pulsarServiceStateArray), + Arrays.toString(checkCounterArray)); + } + return true; + } catch (Exception e) { + Throwable actEx = FutureUtil.unwrapCompletionException(e); + if (actEx instanceof PulsarAdminException.NotFoundException + || actEx instanceof PulsarClientException.NotFoundException + || actEx instanceof PulsarClientException.TopicDoesNotExistException + || actEx instanceof PulsarClientException.LookupException) { + if (markTopicNotFoundAsAvailable) { + if (log.isDebugEnabled()) { + log.debug("Success to probe available(case tenant/namespace/topic not found), [{}] {}." + + " States: {}, Counters: {}", brokerServiceIndex, url, + Arrays.toString(pulsarServiceStateArray), Arrays.toString(checkCounterArray)); + } + return true; + } else { + log.warn("Failed to probe available(error tenant/namespace/topic not found), [{}] {}. States: {}," + + " Counters: {}", brokerServiceIndex, url, Arrays.toString(pulsarServiceStateArray), + Arrays.toString(checkCounterArray)); + return false; + } + } + log.warn("Failed to probe available, [{}] {}. States: {}, Counters: {}", brokerServiceIndex, url, + Arrays.toString(pulsarServiceStateArray), Arrays.toString(checkCounterArray)); + return false; + } + } + + private void updateServiceUrl(int targetIndex) { + String currentUrl = pulsarServiceUrlArray[currentPulsarServiceIndex]; + String targetUrl = pulsarServiceUrlArray[targetIndex]; + String logMsg; + if (targetIndex < currentPulsarServiceIndex) { + logMsg = String.format("Recover to high priority pulsar service [%s] %s --> [%s] %s. States: %s," + + " Counters: %s", currentPulsarServiceIndex, currentUrl, targetIndex, targetUrl, + Arrays.toString(pulsarServiceStateArray), Arrays.toString(checkCounterArray)); + } else { + logMsg = String.format("Failover to low priority pulsar service [%s] %s --> [%s] %s. States: %s," + + " Counters: %s", currentPulsarServiceIndex, currentUrl, targetIndex, targetUrl, + Arrays.toString(pulsarServiceStateArray), Arrays.toString(checkCounterArray)); + } + log.info(logMsg); + try { + pulsarClient.updateServiceUrl(targetUrl); + pulsarClient.reloadLookUp(); + currentPulsarServiceIndex = targetIndex; + } catch (Exception e) { + log.error("Failed to {}", logMsg, e); + } + } + + public enum PulsarServiceState { + Healthy, + PreFail, + Failed, + PreRecover; + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + + private SameAuthParamsLookupAutoClusterFailover + sameAuthParamsLookupAutoClusterFailover = new SameAuthParamsLookupAutoClusterFailover(); + + public Builder failoverThreshold(int failoverThreshold) { + if (failoverThreshold < 1) { + throw new IllegalArgumentException("failoverThreshold must be larger than 0"); + } + sameAuthParamsLookupAutoClusterFailover.failoverThreshold = failoverThreshold; + return this; + } + + public Builder recoverThreshold(int recoverThreshold) { + if (recoverThreshold < 1) { + throw new IllegalArgumentException("recoverThreshold must be larger than 0"); + } + sameAuthParamsLookupAutoClusterFailover.recoverThreshold = recoverThreshold; + return this; + } + + public Builder checkHealthyIntervalMs(int checkHealthyIntervalMs) { + if (checkHealthyIntervalMs < 1) { + throw new IllegalArgumentException("checkHealthyIntervalMs must be larger than 0"); + } + sameAuthParamsLookupAutoClusterFailover.checkHealthyIntervalMs = checkHealthyIntervalMs; + return this; + } + + public Builder testTopic(String testTopic) { + if (StringUtils.isBlank(testTopic) && TopicName.get(testTopic) != null) { + throw new IllegalArgumentException("testTopic can not be blank"); + } + sameAuthParamsLookupAutoClusterFailover.testTopic = testTopic; + return this; + } + + public Builder markTopicNotFoundAsAvailable(boolean markTopicNotFoundAsAvailable) { + sameAuthParamsLookupAutoClusterFailover.markTopicNotFoundAsAvailable = markTopicNotFoundAsAvailable; + return this; + } + + public Builder pulsarServiceUrlArray(String[] pulsarServiceUrlArray) { + if (pulsarServiceUrlArray == null || pulsarServiceUrlArray.length == 0) { + throw new IllegalArgumentException("pulsarServiceUrlArray can not be empty"); + } + sameAuthParamsLookupAutoClusterFailover.pulsarServiceUrlArray = pulsarServiceUrlArray; + int pulsarServiceLen = pulsarServiceUrlArray.length; + HashSet uniqueChecker = new HashSet<>(); + for (int i = 0; i < pulsarServiceLen; i++) { + String pulsarService = pulsarServiceUrlArray[i]; + if (StringUtils.isBlank(pulsarService)) { + throw new IllegalArgumentException("pulsarServiceUrlArray contains a blank value at index " + i); + } + if (pulsarService.startsWith("http") || pulsarService.startsWith("HTTP")) { + throw new IllegalArgumentException("SameAuthParamsLookupAutoClusterFailover does not support HTTP" + + " protocol pulsar service url so far."); + } + if (!uniqueChecker.add(pulsarService)) { + throw new IllegalArgumentException("pulsarServiceUrlArray contains duplicated value " + + pulsarServiceUrlArray[i]); + } + } + return this; + } + + public SameAuthParamsLookupAutoClusterFailover build() { + String[] pulsarServiceUrlArray = sameAuthParamsLookupAutoClusterFailover.pulsarServiceUrlArray; + if (pulsarServiceUrlArray == null) { + throw new IllegalArgumentException("pulsarServiceUrlArray can not be empty"); + } + int pulsarServiceLen = pulsarServiceUrlArray.length; + sameAuthParamsLookupAutoClusterFailover.pulsarServiceStateArray = new PulsarServiceState[pulsarServiceLen]; + sameAuthParamsLookupAutoClusterFailover.checkCounterArray = new MutableInt[pulsarServiceLen]; + for (int i = 0; i < pulsarServiceLen; i++) { + sameAuthParamsLookupAutoClusterFailover.pulsarServiceStateArray[i] = PulsarServiceState.Healthy; + sameAuthParamsLookupAutoClusterFailover.checkCounterArray[i] = new MutableInt(0); + } + return sameAuthParamsLookupAutoClusterFailover; + } + } +} + diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java index 545cf7483e4e3..b275ffb6012ca 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java @@ -23,7 +23,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; -import java.io.IOException; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -32,7 +31,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.awaitility.Awaitility; @@ -43,7 +41,7 @@ @Slf4j public class AutoClusterFailoverTest { @Test - public void testBuildAutoClusterFailoverInstance() throws PulsarClientException { + public void testBuildAutoClusterFailoverInstance() throws Exception { String primary = "pulsar://localhost:6650"; String secondary = "pulsar://localhost:6651"; long failoverDelay = 30; @@ -106,7 +104,7 @@ public void testBuildAutoClusterFailoverInstance() throws PulsarClientException } @Test - public void testInitialize() { + public void testInitialize() throws Exception { String primary = "pulsar://localhost:6650"; String secondary = "pulsar://localhost:6651"; long failoverDelay = 10; @@ -151,7 +149,7 @@ public void testInitialize() { } @Test - public void testAutoClusterFailoverSwitchWithoutAuthentication() { + public void testAutoClusterFailoverSwitchWithoutAuthentication() throws Exception { String primary = "pulsar://localhost:6650"; String secondary = "pulsar://localhost:6651"; long failoverDelay = 1; @@ -187,7 +185,7 @@ public void testAutoClusterFailoverSwitchWithoutAuthentication() { } @Test - public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException { + public void testAutoClusterFailoverSwitchWithAuthentication() throws Exception { String primary = "pulsar+ssl://localhost:6651"; String secondary = "pulsar+ssl://localhost:6661"; long failoverDelay = 1; @@ -251,7 +249,7 @@ public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException } @Test - public void testAutoClusterFailoverSwitchTlsTrustStore() throws IOException { + public void testAutoClusterFailoverSwitchTlsTrustStore() throws Exception { String primary = "pulsar+ssl://localhost:6651"; String secondary = "pulsar+ssl://localhost:6661"; long failoverDelay = 1; diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java index 36160d40d540a..fa7145794e1e2 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.client.impl; -import java.io.IOException; import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -37,7 +36,7 @@ @Test(groups = "broker-impl") public class ControlledClusterFailoverTest { @Test - public void testBuildControlledClusterFailoverInstance() throws IOException { + public void testBuildControlledClusterFailoverInstance() throws Exception { String defaultServiceUrl = "pulsar://localhost:6650"; String urlProvider = "http://localhost:8080/test"; String keyA = "key-a"; @@ -67,7 +66,7 @@ public void testBuildControlledClusterFailoverInstance() throws IOException { } @Test - public void testControlledClusterFailoverSwitch() throws IOException { + public void testControlledClusterFailoverSwitch() throws Exception { String defaultServiceUrl = "pulsar+ssl://localhost:6651"; String backupServiceUrl = "pulsar+ssl://localhost:6661"; String urlProvider = "http://localhost:8080"; From 66cc754006ce95b8a7a90af31024462478831d7e Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Tue, 13 Aug 2024 10:02:58 +0800 Subject: [PATCH 434/580] [fix][broker] Fix AvgShedder strategy check (#23156) --- .../broker/loadbalance/impl/ModularLoadManagerImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index 141e020d7ca45..05c984d0349b7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -275,11 +275,11 @@ public void initialize(final PulsarService pulsar) { // if the placement strategy is also a load shedding strategy // we need to check two strategies are the same if (!conf.getLoadBalancerLoadSheddingStrategy().equals( - conf.getLoadBalancerPlacementStrategy())) { + conf.getLoadBalancerLoadPlacementStrategy())) { throw new IllegalArgumentException("The load shedding strategy: " + conf.getLoadBalancerLoadSheddingStrategy() + " can't work with the placement strategy: " - + conf.getLoadBalancerPlacementStrategy()); + + conf.getLoadBalancerLoadPlacementStrategy()); } // bind the load shedding strategy and the placement strategy loadSheddingStrategy = (LoadSheddingStrategy) placementStrategy; From 9bf714ff756b0729094e9a8611137d3dcfaed5b0 Mon Sep 17 00:00:00 2001 From: hanmz Date: Tue, 13 Aug 2024 11:06:01 +0800 Subject: [PATCH 435/580] [fix][broker] Fix 'Disabled replicated subscriptions controller' logic and logging (#23142) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 7a520d879b782..e890bac620e7b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -4103,8 +4103,8 @@ private synchronized void checkReplicatedSubscriptionControllerState(boolean sho log.info("[{}] Enabling replicated subscriptions controller", topic); replicatedSubscriptionsController = Optional.of(new ReplicatedSubscriptionsController(this, brokerService.pulsar().getConfiguration().getClusterName())); - } else if (isCurrentlyEnabled && !shouldBeEnabled || !isEnableReplicatedSubscriptions - || !replicationEnabled) { + } else if (isCurrentlyEnabled && (!shouldBeEnabled || !isEnableReplicatedSubscriptions + || !replicationEnabled)) { log.info("[{}] Disabled replicated subscriptions controller", topic); replicatedSubscriptionsController.ifPresent(ReplicatedSubscriptionsController::close); replicatedSubscriptionsController = Optional.empty(); From fe21441f101f3d6d47a243b81157e5c8bf3ad573 Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Wed, 14 Aug 2024 01:28:48 +0800 Subject: [PATCH 436/580] [improve] [pip] PIP-363: Add callback parameters to the method: org.apache.pulsar.client.impl.SendCallback.sendComplete. (#22940) --- pip/pip-363.md | 111 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 111 insertions(+) create mode 100644 pip/pip-363.md diff --git a/pip/pip-363.md b/pip/pip-363.md new file mode 100644 index 0000000000000..2b250e69871e1 --- /dev/null +++ b/pip/pip-363.md @@ -0,0 +1,111 @@ +# PIP-363: Add callback parameters to the method: `org.apache.pulsar.client.impl.SendCallback.sendComplete`. + +# Background knowledge + + +As introduced in [PIP-264](https://github.com/apache/pulsar/blob/master/pip/pip-264.md), Pulsar has been fully integrated into the `OpenTelemetry` system, which defines some metric specifications for [messaging systems](https://opentelemetry.io/docs/specs/semconv/messaging/messaging-metrics/#metric-messagingpublishduration). + +In the current Pulsar client code, it is not possible to obtain the number of messages sent in batches(as well as some other sending data), making it impossible to implement `messaging.publish.messages` metric. + +In the `opentelemetry-java-instrumentation` code, the `org.apache.pulsar.client.impl.SendCallback` interface is used to instrument data points. For specific implementation details, we can refer to [this](https://github.com/open-telemetry/opentelemetry-java-instrumentation/blob/main/instrumentation/pulsar/pulsar-2.8/javaagent/src/main/java/io/opentelemetry/javaagent/instrumentation/pulsar/v2_8/ProducerImplInstrumentation.java#L89-L135). + +# Motivation + + +In the current situation, `org.apache.pulsar.client.impl.ProducerImpl` does not provide a public method to obtain the `numMessagesInBatch`. + +So, we can add some of `org.apache.pulsar.client.impl.ProducerImpl.OpSendMsg`'s key data into the `org.apache.pulsar.client.impl.SendCallback.sendComplete` method. + +# Detailed Design + +Add callback parameters to the method: `org.apache.pulsar.client.impl.SendCallback.sendComplete`: + +```java +public interface SendCallback { + + /** + * invoked when send operation completes. + * + * @param e + */ + void sendComplete(Throwable e, OpSendMsgStats stats); +} + +public interface OpSendMsgStats { + long getUncompressedSize(); + + long getSequenceId(); + + int getRetryCount(); + + long getBatchSizeByte(); + + int getNumMessagesInBatch(); + + long getHighestSequenceId(); + + int getTotalChunks(); + + int getChunkId(); +} + +@Builder +public class OpSendMsgStatsImpl implements OpSendMsgStats { + private long uncompressedSize; + private long sequenceId; + private int retryCount; + private long batchSizeByte; + private int numMessagesInBatch; + private long highestSequenceId; + private int totalChunks; + private int chunkId; + + @Override + public long getUncompressedSize() { + return uncompressedSize; + } + + @Override + public long getSequenceId() { + return sequenceId; + } + + @Override + public int getRetryCount() { + return retryCount; + } + + @Override + public long getBatchSizeByte() { + return batchSizeByte; + } + + @Override + public int getNumMessagesInBatch() { + return numMessagesInBatch; + } + + @Override + public long getHighestSequenceId() { + return highestSequenceId; + } + + @Override + public int getTotalChunks() { + return totalChunks; + } + + @Override + public int getChunkId() { + return chunkId; + } +} +``` + +# Links + + +* Mailing List discussion thread: https://lists.apache.org/thread/8pgmsvx1bxz4z1w8prpvpnfpt1kb57c9 +* Mailing List voting thread: https://lists.apache.org/thread/t0olt3722j17gjtdxqqsl3cpy104ogpr From 3e461c004ea229ef9b526a51fd0ed91e8157e873 Mon Sep 17 00:00:00 2001 From: Yuri Mizushima Date: Wed, 14 Aug 2024 11:09:50 +0900 Subject: [PATCH 437/580] [improve][proxy] Reuse authentication instance in pulsar-proxy (#23113) --- .../ProxySaslAuthenticationTest.java | 6 +- .../proxy/server/AdminProxyHandler.java | 23 +---- .../proxy/server/DirectProxyHandler.java | 4 +- .../pulsar/proxy/server/ProxyService.java | 12 +-- .../proxy/server/ProxyServiceStarter.java | 47 ++++++++-- .../SimpleProxyExtensionTestBase.java | 12 ++- .../AdminProxyHandlerKeystoreTLSTest.java | 13 ++- .../proxy/server/AdminProxyHandlerTest.java | 3 +- .../server/AuthedAdminProxyHandlerTest.java | 12 ++- .../server/FunctionWorkerRoutingTest.java | 10 +- ...nvalidProxyConfigForAuthorizationTest.java | 3 +- .../server/ProxyAdditionalServletTest.java | 15 ++- ...roxyAuthenticatedProducerConsumerTest.java | 11 ++- .../proxy/server/ProxyAuthenticationTest.java | 7 +- .../server/ProxyConnectionThrottlingTest.java | 11 ++- .../server/ProxyDisableZeroCopyTest.java | 2 +- .../ProxyEnableHAProxyProtocolTest.java | 12 ++- .../server/ProxyForwardAuthDataTest.java | 10 +- .../proxy/server/ProxyIsAHttpProxyTest.java | 59 ++++++++++-- .../server/ProxyKeyStoreTlsTransportTest.java | 12 ++- .../server/ProxyKeyStoreTlsWithAuthTest.java | 12 ++- .../ProxyKeyStoreTlsWithoutAuthTest.java | 12 ++- .../server/ProxyLookupThrottlingTest.java | 11 ++- .../proxy/server/ProxyMutualTlsTest.java | 12 ++- .../pulsar/proxy/server/ProxyParserTest.java | 11 ++- .../server/ProxyPrometheusMetricsTest.java | 15 ++- .../proxy/server/ProxyRefreshAuthTest.java | 12 ++- .../server/ProxyRolesEnforcementTest.java | 9 +- .../proxy/server/ProxyServiceStarterTest.java | 91 +++++++++++++++++++ .../pulsar/proxy/server/ProxyStatsTest.java | 14 ++- .../server/ProxyStuckConnectionTest.java | 12 ++- .../apache/pulsar/proxy/server/ProxyTest.java | 14 ++- .../pulsar/proxy/server/ProxyTlsTest.java | 12 ++- .../proxy/server/ProxyTlsWithAuthTest.java | 12 ++- .../server/ProxyWithAuthorizationNegTest.java | 10 +- .../server/ProxyWithAuthorizationTest.java | 19 +++- .../ProxyWithExtensibleLoadManagerTest.java | 11 ++- .../server/ProxyWithJwtAuthorizationTest.java | 18 +++- .../ProxyWithoutServiceDiscoveryTest.java | 11 ++- .../SuperUserAuthedAdminProxyHandlerTest.java | 12 ++- .../server/UnauthedAdminProxyHandlerTest.java | 16 +++- 41 files changed, 536 insertions(+), 94 deletions(-) diff --git a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java index a27384c989000..ca28befabc145 100644 --- a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java +++ b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java @@ -260,7 +260,11 @@ void testAuthentication() throws Exception { proxyConfig.setForwardAuthorizationCredentials(true); AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); - ProxyService proxyService = new ProxyService(proxyConfig, authenticationService); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + ProxyService proxyService = new ProxyService(proxyConfig, authenticationService, proxyClientAuthentication); proxyService.start(); final String proxyServiceUrl = "pulsar://localhost:" + proxyService.getListenPort().get(); diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java index caaa99c5d40cc..0108b770249a0 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java @@ -29,7 +29,6 @@ import java.util.Collections; import java.util.HashSet; import java.util.Iterator; -import java.util.Objects; import java.util.Set; import java.util.concurrent.Executor; import javax.net.ssl.SSLContext; @@ -40,7 +39,6 @@ import org.apache.pulsar.broker.web.AuthenticationFilter; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; -import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.KeyStoreParams; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.util.SecurityUtility; @@ -87,12 +85,15 @@ class AdminProxyHandler extends ProxyServlet { private final ProxyConfiguration config; private final BrokerDiscoveryProvider discoveryProvider; + private final Authentication proxyClientAuthentication; private final String brokerWebServiceUrl; private final String functionWorkerWebServiceUrl; - AdminProxyHandler(ProxyConfiguration config, BrokerDiscoveryProvider discoveryProvider) { + AdminProxyHandler(ProxyConfiguration config, BrokerDiscoveryProvider discoveryProvider, + Authentication proxyClientAuthentication) { this.config = config; this.discoveryProvider = discoveryProvider; + this.proxyClientAuthentication = proxyClientAuthentication; this.brokerWebServiceUrl = config.isTlsEnabledWithBroker() ? config.getBrokerWebServiceURLTLS() : config.getBrokerWebServiceURL(); this.functionWorkerWebServiceUrl = config.isTlsEnabledWithBroker() ? config.getFunctionWorkerWebServiceURLTLS() @@ -256,22 +257,13 @@ protected ContentProvider proxyRequestContent(HttpServletRequest request, @Override protected HttpClient newHttpClient() { try { - Authentication auth = AuthenticationFactory.create( - config.getBrokerClientAuthenticationPlugin(), - config.getBrokerClientAuthenticationParameters() - ); - - Objects.requireNonNull(auth, "No supported auth found for proxy"); - - auth.start(); - if (config.isTlsEnabledWithBroker()) { try { X509Certificate[] trustCertificates = SecurityUtility .loadCertificatesFromPemFile(config.getBrokerClientTrustCertsFilePath()); SSLContext sslCtx; - AuthenticationDataProvider authData = auth.getAuthData(); + AuthenticationDataProvider authData = proxyClientAuthentication.getAuthData(); if (config.isBrokerClientTlsEnabledWithKeyStore()) { KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : null; sslCtx = KeyStoreSSLContext.createClientSslContext( @@ -311,11 +303,6 @@ protected HttpClient newHttpClient() { return new JettyHttpClient(contextFactory); } catch (Exception e) { LOG.error("new jetty http client exception ", e); - try { - auth.close(); - } catch (IOException ioe) { - LOG.error("Failed to close the authentication service", ioe); - } throw new PulsarClientException.InvalidConfigurationException(e.getMessage()); } } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java index d63b04b6734de..4678db82c6e55 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java @@ -52,7 +52,6 @@ import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; -import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.AuthData; @@ -114,8 +113,7 @@ public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection) if (!isEmpty(config.getBrokerClientAuthenticationPlugin())) { try { - authData = AuthenticationFactory.create(config.getBrokerClientAuthenticationPlugin(), - config.getBrokerClientAuthenticationParameters()).getAuthData(); + authData = authentication.getAuthData(); } catch (PulsarClientException e) { throw new RuntimeException(e); } 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 ea9e4ebfaa9b8..5cf01d6668b9b 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,8 +64,6 @@ import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlets; import org.apache.pulsar.client.api.Authentication; -import org.apache.pulsar.client.api.AuthenticationFactory; -import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.util.netty.DnsResolverUtil; @@ -158,7 +156,8 @@ public class ProxyService implements Closeable { private boolean gracefulShutdown = true; public ProxyService(ProxyConfiguration proxyConfig, - AuthenticationService authenticationService) throws Exception { + AuthenticationService authenticationService, + Authentication proxyClientAuthentication) throws Exception { requireNonNull(proxyConfig); this.proxyConfig = proxyConfig; this.clientCnxs = Sets.newConcurrentHashSet(); @@ -207,12 +206,7 @@ public ProxyService(ProxyConfiguration proxyConfig, }); }, 60, TimeUnit.SECONDS); this.proxyAdditionalServlets = AdditionalServlets.load(proxyConfig); - if (proxyConfig.getBrokerClientAuthenticationPlugin() != null) { - proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), - proxyConfig.getBrokerClientAuthenticationParameters()); - } else { - proxyClientAuthentication = AuthenticationDisabled.INSTANCE; - } + this.proxyClientAuthentication = proxyClientAuthentication; this.connectionController = new ConnectionController.DefaultConnectionController( proxyConfig.getMaxConcurrentInboundConnections(), proxyConfig.getMaxConcurrentInboundConnectionsPerIp()); diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index 10121e7f5d61d..a5504cac100a4 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -29,11 +29,13 @@ import io.prometheus.client.Gauge; import io.prometheus.client.Gauge.Child; import io.prometheus.client.hotspot.DefaultExports; +import java.io.IOException; import java.text.DateFormat; import java.text.SimpleDateFormat; import java.util.Collection; import java.util.Collections; import java.util.Date; +import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import lombok.Getter; @@ -44,6 +46,10 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServletWithClassLoader; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.configuration.VipStatus; import org.apache.pulsar.common.policies.data.ClusterData; @@ -104,6 +110,9 @@ public class ProxyServiceStarter { private ProxyConfiguration config; + @Getter + private Authentication proxyClientAuthentication; + @Getter private ProxyService proxyService; @@ -244,8 +253,27 @@ public static void main(String[] args) throws Exception { public void start() throws Exception { AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(config)); + + if (config.getBrokerClientAuthenticationPlugin() != null) { + proxyClientAuthentication = AuthenticationFactory.create(config.getBrokerClientAuthenticationPlugin(), + config.getBrokerClientAuthenticationParameters()); + Objects.requireNonNull(proxyClientAuthentication, "No supported auth found for proxy"); + try { + proxyClientAuthentication.start(); + } catch (Exception e) { + try { + proxyClientAuthentication.close(); + } catch (IOException ioe) { + log.error("Failed to close the authentication service", ioe); + } + throw new PulsarClientException.InvalidConfigurationException(e.getMessage()); + } + } else { + proxyClientAuthentication = AuthenticationDisabled.INSTANCE; + } + // create proxy service - proxyService = new ProxyService(config, authenticationService); + proxyService = new ProxyService(config, authenticationService, proxyClientAuthentication); // create a web-service server = new WebServer(config, authenticationService); @@ -293,7 +321,8 @@ public double get() { } AtomicReference webSocketServiceRef = new AtomicReference<>(); - addWebServerHandlers(server, config, proxyService, proxyService.getDiscoveryProvider(), webSocketServiceRef); + addWebServerHandlers(server, config, proxyService, proxyService.getDiscoveryProvider(), webSocketServiceRef, + proxyClientAuthentication); webSocketService = webSocketServiceRef.get(); // start web-service @@ -311,6 +340,9 @@ public void close() { if (webSocketService != null) { webSocketService.close(); } + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } catch (Exception e) { log.warn("server couldn't stop gracefully {}", e.getMessage(), e); } finally { @@ -323,15 +355,17 @@ public void close() { public static void addWebServerHandlers(WebServer server, ProxyConfiguration config, ProxyService service, - BrokerDiscoveryProvider discoveryProvider) throws Exception { - addWebServerHandlers(server, config, service, discoveryProvider, null); + BrokerDiscoveryProvider discoveryProvider, + Authentication proxyClientAuthentication) throws Exception { + addWebServerHandlers(server, config, service, discoveryProvider, null, proxyClientAuthentication); } public static void addWebServerHandlers(WebServer server, ProxyConfiguration config, ProxyService service, BrokerDiscoveryProvider discoveryProvider, - AtomicReference webSocketServiceRef) throws Exception { + AtomicReference webSocketServiceRef, + Authentication proxyClientAuthentication) throws Exception { // We can make 'status.html' publicly accessible without authentication since // it does not contain any sensitive data. server.addRestResource("/", VipStatus.ATTRIBUTE_STATUS_FILE_PATH, config.getStatusFilePath(), @@ -348,7 +382,8 @@ public static void addWebServerHandlers(WebServer server, } } - AdminProxyHandler adminProxyHandler = new AdminProxyHandler(config, discoveryProvider); + AdminProxyHandler adminProxyHandler = new AdminProxyHandler(config, discoveryProvider, + proxyClientAuthentication); ServletHolder servletHolder = new ServletHolder(adminProxyHandler); server.addServlet("/admin", servletHolder); server.addServlet("/lookup", servletHolder); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/extensions/SimpleProxyExtensionTestBase.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/extensions/SimpleProxyExtensionTestBase.java index f9ace716ecd06..050199acc496d 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/extensions/SimpleProxyExtensionTestBase.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/extensions/SimpleProxyExtensionTestBase.java @@ -26,6 +26,8 @@ import org.apache.commons.io.IOUtils; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.util.PortManager; import org.apache.pulsar.metadata.impl.ZKMetadataStore; @@ -121,6 +123,7 @@ public void close() { private ProxyService proxyService; private boolean useSeparateThreadPoolForProxyExtensions; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; public SimpleProxyExtensionTestBase(boolean useSeparateThreadPoolForProxyExtensions) { this.useSeparateThreadPoolForProxyExtensions = useSeparateThreadPoolForProxyExtensions; @@ -142,8 +145,12 @@ protected void setup() throws Exception { proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); proxyConfig.setClusterName(configClusterName); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -174,6 +181,9 @@ public void testBootstrapProtocolHandler() throws Exception { protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } if (tempDirectory != null) { FileUtils.deleteDirectory(tempDirectory); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java index 92c644b470dcd..5995d11b33b21 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java @@ -24,6 +24,8 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.ClusterData; @@ -47,6 +49,8 @@ public class AdminProxyHandlerKeystoreTLSTest extends MockedPulsarServiceBaseTes private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; + private WebServer webServer; private BrokerDiscoveryProvider discoveryProvider; @@ -103,6 +107,10 @@ protected void setup() throws Exception { KEYSTORE_TYPE, BROKER_KEYSTORE_FILE_PATH, BROKER_KEYSTORE_PW)); proxyConfig.setClusterName(configClusterName); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + resource = new PulsarResources(registerCloseable(new ZKMetadataStore(mockZooKeeper)), registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))); webServer = new WebServer(proxyConfig, new AuthenticationService( @@ -110,7 +118,7 @@ protected void setup() throws Exception { discoveryProvider = spy(registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); LoadManagerReport report = new LoadReport(brokerUrl.toString(), brokerUrlTls.toString(), null, null); doReturn(report).when(discoveryProvider).nextBroker(); - ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider)); + ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider, proxyClientAuthentication)); webServer.addServlet("/admin", servletHolder); webServer.addServlet("/lookup", servletHolder); webServer.start(); @@ -120,6 +128,9 @@ protected void setup() throws Exception { @Override protected void cleanup() throws Exception { webServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } super.internalCleanup(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java index becebe0059e56..4f925618e8a79 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java @@ -32,6 +32,7 @@ import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import org.apache.pulsar.client.api.Authentication; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.client.api.Request; import org.testng.Assert; @@ -46,7 +47,7 @@ public void setupMocks() throws ServletException { // given HttpClient httpClient = mock(HttpClient.class); adminProxyHandler = new AdminProxyHandler(mock(ProxyConfiguration.class), - mock(BrokerDiscoveryProvider.class)) { + mock(BrokerDiscoveryProvider.class), mock(Authentication.class)) { @Override protected HttpClient createHttpClient() throws ServletException { return httpClient; diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java index ef58648e35a25..97bb91d924cf8 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java @@ -32,6 +32,8 @@ import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.ClusterData; @@ -51,6 +53,7 @@ public class AuthedAdminProxyHandlerTest extends MockedPulsarServiceBaseTest { private static final Logger LOG = LoggerFactory.getLogger(AuthedAdminProxyHandlerTest.class); private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; private WebServer webServer; private BrokerDiscoveryProvider discoveryProvider; private PulsarResources resource; @@ -99,6 +102,10 @@ protected void setup() throws Exception { proxyConfig.setBrokerClientTrustCertsFilePath(CA_CERT_FILE_PATH); proxyConfig.setAuthenticationProviders(ImmutableSet.of(AuthenticationProviderTls.class.getName())); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + resource = new PulsarResources(registerCloseable(new ZKMetadataStore(mockZooKeeper)), registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))); webServer = new WebServer(proxyConfig, new AuthenticationService( @@ -107,7 +114,7 @@ protected void setup() throws Exception { LoadManagerReport report = new LoadReport(brokerUrl.toString(), brokerUrlTls.toString(), null, null); doReturn(report).when(discoveryProvider).nextBroker(); - ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider)); + ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider, proxyClientAuthentication)); webServer.addServlet("/admin", servletHolder); webServer.addServlet("/lookup", servletHolder); @@ -119,6 +126,9 @@ protected void setup() throws Exception { @Override protected void cleanup() throws Exception { webServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } super.internalCleanup(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/FunctionWorkerRoutingTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/FunctionWorkerRoutingTest.java index db5e9e12bd2db..a07a0f082d39a 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/FunctionWorkerRoutingTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/FunctionWorkerRoutingTest.java @@ -18,6 +18,9 @@ */ package org.apache.pulsar.proxy.server; +import lombok.Cleanup; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.testng.Assert; import org.testng.annotations.Test; @@ -37,8 +40,13 @@ public void testFunctionWorkerRedirect() throws Exception { proxyConfig.setBrokerWebServiceURL(brokerUrl); proxyConfig.setFunctionWorkerWebServiceURL(functionWorkerUrl); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + BrokerDiscoveryProvider discoveryProvider = mock(BrokerDiscoveryProvider.class); - AdminProxyHandler handler = new AdminProxyHandler(proxyConfig, discoveryProvider); + AdminProxyHandler handler = new AdminProxyHandler(proxyConfig, discoveryProvider, proxyClientAuthentication); String funcUrl = handler.rewriteTarget(buildRequest("/admin/v3/functions/test/test")); Assert.assertEquals(funcUrl, String.format("%s/admin/v3/functions/%s/%s", 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 c29bfaa964812..b7ef0855e383c 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 @@ -22,6 +22,7 @@ import static org.testng.Assert.fail; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; import org.mockito.Mockito; import org.testng.annotations.Test; @@ -33,7 +34,7 @@ void startupShouldFailWhenAuthorizationIsEnabledWithoutAuthentication() throws E proxyConfiguration.setAuthorizationEnabled(true); proxyConfiguration.setAuthenticationEnabled(false); try (ProxyService proxyService = new ProxyService(proxyConfiguration, - Mockito.mock(AuthenticationService.class))) { + Mockito.mock(AuthenticationService.class), Mockito.mock(Authentication.class))) { proxyService.start(); fail("An exception should have been thrown"); } catch (Exception e) { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java index f61a73bbf9177..e12224da37199 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAdditionalServletTest.java @@ -25,6 +25,8 @@ import org.apache.commons.lang3.RandomUtils; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServletWithClassLoader; @@ -65,6 +67,7 @@ public class ProxyAdditionalServletTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private WebServer proxyWebServer; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -83,8 +86,13 @@ protected void setup() throws Exception { // this is for nar package test // addServletNar(); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, - new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)))); + new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)), + proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -99,7 +107,7 @@ protected void setup() throws Exception { mockAdditionalServlet(); proxyWebServer = new WebServer(proxyConfig, authService); - ProxyServiceStarter.addWebServerHandlers(proxyWebServer, proxyConfig, proxyService, null); + ProxyServiceStarter.addWebServerHandlers(proxyWebServer, proxyConfig, proxyService, null, proxyClientAuthentication); proxyWebServer.start(); } @@ -180,6 +188,9 @@ protected void cleanup() throws Exception { proxyService.close(); proxyWebServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java index 4083c984d9874..2a9a9f15b4568 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java @@ -35,6 +35,7 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; @@ -74,6 +75,7 @@ public class ProxyAuthenticatedProducerConsumerTest extends ProducerConsumerBase private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; private final String configClusterName = "test"; @BeforeMethod @@ -139,8 +141,12 @@ protected void setup() throws Exception { proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); proxyConfig.setClusterName(configClusterName); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -152,6 +158,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } /** diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java index 662b8305c0e26..7d3cf57d594df 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java @@ -43,6 +43,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -235,7 +236,11 @@ void testAuthentication() throws Exception { AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); @Cleanup - ProxyService proxyService = new ProxyService(proxyConfig, authenticationService); + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + @Cleanup + ProxyService proxyService = new ProxyService(proxyConfig, authenticationService, proxyClientAuthentication); proxyService.start(); final String proxyServiceUrl = proxyService.getServiceUrl(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java index 78ab9bd0d9581..671e68e5c3fb7 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java @@ -27,6 +27,8 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.limiter.ConnectionController; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; @@ -46,6 +48,7 @@ public class ProxyConnectionThrottlingTest extends MockedPulsarServiceBaseTest { private final int NUM_CONCURRENT_INBOUND_CONNECTION = 4; private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -60,8 +63,11 @@ protected void setup() throws Exception { proxyConfig.setMaxConcurrentInboundConnections(NUM_CONCURRENT_INBOUND_CONNECTION); proxyConfig.setMaxConcurrentInboundConnectionsPerIp(NUM_CONCURRENT_INBOUND_CONNECTION); proxyConfig.setClusterName(configClusterName); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -74,6 +80,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyDisableZeroCopyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyDisableZeroCopyTest.java index 5ddb084e3c77f..6a3992c550fd3 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyDisableZeroCopyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyDisableZeroCopyTest.java @@ -21,7 +21,7 @@ public class ProxyDisableZeroCopyTest extends ProxyTest { @Override - protected void initializeProxyConfig() { + protected void initializeProxyConfig() throws Exception { super.initializeProxyConfig(); proxyConfig.setProxyZeroCopyModeEnabled(false); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java index 413774daf2cd1..40aa8f5040556 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java @@ -22,6 +22,8 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.ConsumerImpl; @@ -48,6 +50,7 @@ public class ProxyEnableHAProxyProtocolTest extends MockedPulsarServiceBaseTest private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -62,8 +65,12 @@ protected void setup() throws Exception { proxyConfig.setHaProxyProtocolEnabled(true); proxyConfig.setClusterName(configClusterName); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -77,6 +84,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java index 5e969ca26e4fd..9c3a69b5f4451 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java @@ -30,6 +30,8 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -118,7 +120,11 @@ public void testForwardAuthData() throws Exception { AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); - try (ProxyService proxyService = new ProxyService(proxyConfig, authenticationService)) { + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + try (ProxyService proxyService = new ProxyService(proxyConfig, authenticationService, proxyClientAuthentication)) { proxyService.start(); try (PulsarClient proxyClient = createPulsarClient(proxyService.getServiceUrl(), clientAuthParams)) { proxyClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe(); @@ -134,7 +140,7 @@ public void testForwardAuthData() throws Exception { PulsarConfigurationLoader.convertFrom(proxyConfig)); @Cleanup - ProxyService proxyService = new ProxyService(proxyConfig, authenticationService); + ProxyService proxyService = new ProxyService(proxyConfig, authenticationService, proxyClientAuthentication); proxyService.start(); @Cleanup diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyIsAHttpProxyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyIsAHttpProxyTest.java index 90e15ede2f436..cf587015544b7 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyIsAHttpProxyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyIsAHttpProxyTest.java @@ -33,9 +33,12 @@ import javax.ws.rs.client.Client; import javax.ws.rs.client.ClientBuilder; import javax.ws.rs.core.Response; +import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.resources.PulsarResources; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.eclipse.jetty.client.HttpClient; @@ -197,10 +200,14 @@ public void testSingleRedirect() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); webServer.start(); try { Response r = client.target(webServer.getServiceUri()).path("/ui/foobar").request().get(); @@ -226,10 +233,14 @@ public void testMultipleRedirect() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); webServer.start(); try { Response r1 = client.target(webServer.getServiceUri()).path("/server1/foobar").request().get(); @@ -257,10 +268,14 @@ public void testTryingToUseExistingPath() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); } @@ -276,10 +291,14 @@ public void testLongPathInProxyTo() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); webServer.start(); try { Response r = client.target(webServer.getServiceUri()).path("/ui/foobar").request().get(); @@ -303,10 +322,14 @@ public void testProxyToEndsInSlash() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); webServer.start(); try { Response r = client.target(webServer.getServiceUri()).path("/ui/foobar").request().get(); @@ -329,10 +352,14 @@ public void testLongPath() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); webServer.start(); try { Response r = client.target(webServer.getServiceUri()).path("/foo/bar/blah/foobar").request().get(); @@ -354,6 +381,10 @@ public void testLongUri() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); StringBuilder longUri = new StringBuilder("/service3/tp"); for (int i = 10 * 1024; i > 0; i = i - 11){ @@ -362,7 +393,7 @@ public void testLongUri() throws Exception { WebServer webServerMaxUriLen8k = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServerMaxUriLen8k, proxyConfig, null, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); webServerMaxUriLen8k.start(); try { Response r = client.target(webServerMaxUriLen8k.getServiceUri()).path(longUri.toString()).request().get(); @@ -374,7 +405,7 @@ public void testLongUri() throws Exception { proxyConfig.setHttpMaxRequestHeaderSize(12 * 1024); WebServer webServerMaxUriLen12k = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServerMaxUriLen12k, proxyConfig, null, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); webServerMaxUriLen12k.start(); try { Response r = client.target(webServerMaxUriLen12k.getServiceUri()).path(longUri.toString()).request().get(); @@ -395,10 +426,14 @@ public void testPathEndsInSlash() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); webServer.start(); try { Response r = client.target(webServer.getServiceUri()).path("/ui/foobar").request().get(); @@ -427,10 +462,14 @@ public void testStreaming() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); webServer.start(); HttpClient httpClient = new HttpClient(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java index 5671c527f68f9..8aa5581a0fe46 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java @@ -24,6 +24,8 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.ClientBuilder; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; @@ -40,6 +42,7 @@ public class ProxyKeyStoreTlsTransportTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeMethod @@ -87,9 +90,13 @@ protected void setup() throws Exception { proxyConfig.setBrokerClientTlsTrustStore(BROKER_TRUSTSTORE_FILE_PATH); proxyConfig.setBrokerClientTlsTrustStorePassword(BROKER_TRUSTSTORE_PW); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -103,6 +110,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } protected PulsarClient newClient() throws Exception { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithAuthTest.java index 99fb8c03a819f..2c6d080bf2c0f 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithAuthTest.java @@ -33,6 +33,8 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationProviderTls; 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.ClientBuilder; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -54,6 +56,7 @@ public class ProxyKeyStoreTlsWithAuthTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeMethod @@ -88,9 +91,13 @@ protected void setup() throws Exception { providers.add(AuthenticationProviderTls.class.getName()); proxyConfig.setAuthenticationProviders(providers); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -104,6 +111,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } protected PulsarClient internalSetUpForClient(boolean addCertificates, String lookupUrl) throws Exception { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithoutAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithoutAuthTest.java index 1dcebda7935d7..3a20273b8c067 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithoutAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsWithoutAuthTest.java @@ -29,6 +29,8 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.ClientBuilder; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -50,6 +52,7 @@ public class ProxyKeyStoreTlsWithoutAuthTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeMethod @@ -76,8 +79,12 @@ protected void setup() throws Exception { proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); proxyConfig.setClusterName(configClusterName); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -109,6 +116,9 @@ protected PulsarClient internalSetUpForClient(boolean addCertificates, String lo protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java index a9017404d0e9f..4d12fdd77e763 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java @@ -31,6 +31,8 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; @@ -53,6 +55,7 @@ public class ProxyLookupThrottlingTest extends MockedPulsarServiceBaseTest { private final int NUM_CONCURRENT_INBOUND_CONNECTION = 5; private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeMethod(alwaysRun = true) @@ -69,7 +72,10 @@ protected void setup() throws Exception { AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); - proxyService = Mockito.spy(new ProxyService(proxyConfig, authenticationService)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, authenticationService, proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -84,6 +90,9 @@ protected void cleanup() throws Exception { if (proxyService != null) { proxyService.close(); } + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test(groups = "quarantine") diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java index fae44c00ada42..ab428c31b7fd9 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java @@ -26,6 +26,8 @@ import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -48,6 +50,7 @@ public class ProxyMutualTlsTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -68,8 +71,12 @@ protected void setup() throws Exception { proxyConfig.setTlsAllowInsecureConnection(false); proxyConfig.setClusterName(configClusterName); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -83,6 +90,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java index 1a9459619ebe9..583ab7000e54f 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java @@ -31,6 +31,8 @@ import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -62,6 +64,7 @@ public class ProxyParserTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -75,9 +78,12 @@ protected void setup() throws Exception { proxyConfig.setClusterName(configClusterName); //enable full parsing feature proxyConfig.setProxyLogLevel(Optional.ofNullable(2)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -93,6 +99,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPrometheusMetricsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPrometheusMetricsTest.java index b692987d17af6..4dd7bc981e59b 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPrometheusMetricsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPrometheusMetricsTest.java @@ -42,6 +42,8 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.awaitility.Awaitility; @@ -59,6 +61,7 @@ public class ProxyPrometheusMetricsTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private WebServer proxyWebServer; private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -72,8 +75,13 @@ protected void setup() throws Exception { proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); proxyConfig.setClusterName(TEST_CLUSTER); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, - new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)))); + new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)), + proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -86,7 +94,7 @@ protected void setup() throws Exception { PulsarConfigurationLoader.convertFrom(proxyConfig)); proxyWebServer = new WebServer(proxyConfig, authService); - ProxyServiceStarter.addWebServerHandlers(proxyWebServer, proxyConfig, proxyService, null); + ProxyServiceStarter.addWebServerHandlers(proxyWebServer, proxyConfig, proxyService, null, proxyClientAuthentication); proxyWebServer.start(); } @@ -109,6 +117,9 @@ protected void cleanup() throws Exception { if (proxyService != null) { proxyService.close(); } + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } /** diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java index d06cf4201ff6f..bdabfecaa439d 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java @@ -35,6 +35,8 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.ClientCnx; @@ -57,6 +59,7 @@ public class ProxyRefreshAuthTest extends ProducerConsumerBase { private ProxyService proxyService; private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override protected void doInitConf() throws Exception { @@ -127,9 +130,13 @@ protected void setup() throws Exception { properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY)); proxyConfig.setProperties(properties); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); } @AfterClass(alwaysRun = true) @@ -137,6 +144,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } private void startProxy(boolean forwardAuthData) throws Exception { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java index a1ffc13ee9350..883b725e15dd2 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java @@ -28,6 +28,7 @@ import java.util.Optional; import java.util.Set; import javax.naming.AuthenticationException; +import lombok.Cleanup; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.AuthenticationProvider; @@ -35,6 +36,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -219,9 +221,14 @@ public void testIncorrectRoles() throws Exception { providers.add(BasicAuthenticationProvider.class.getName()); proxyConfig.setAuthenticationProviders(providers); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + try (ProxyService proxyService = new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))) { + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)) { proxyService.start(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java index 0b9b6f17d1254..d96d2cd1f6e9c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java @@ -20,16 +20,22 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.io.IOException; import java.net.URI; import java.nio.ByteBuffer; import java.util.Base64; +import java.util.Map; import java.util.Optional; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.Future; +import java.util.function.Consumer; import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.websocket.data.ProducerMessage; import org.eclipse.jetty.client.HttpClient; @@ -160,4 +166,89 @@ public String getResponse() throws InterruptedException { } } + @Test + public void testProxyClientAuthentication() throws Exception { + final Consumer initConfig = (proxyConfig) -> { + proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); + proxyConfig.setBrokerWebServiceURL(pulsar.getWebServiceAddress()); + proxyConfig.setWebServicePort(Optional.of(0)); + proxyConfig.setServicePort(Optional.of(0)); + proxyConfig.setWebSocketServiceEnabled(true); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setClusterName(configClusterName); + }; + + + + ProxyServiceStarter serviceStarter = new ProxyServiceStarter(ARGS, null, true); + initConfig.accept(serviceStarter.getConfig()); + // ProxyServiceStarter will throw an exception when Authentication#start is failed + serviceStarter.getConfig().setBrokerClientAuthenticationPlugin(ExceptionAuthentication1.class.getName()); + try { + serviceStarter.start(); + fail("ProxyServiceStarter should throw an exception when Authentication#start is failed"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("ExceptionAuthentication1#start")); + assertTrue(serviceStarter.getProxyClientAuthentication() instanceof ExceptionAuthentication1); + } + + serviceStarter = new ProxyServiceStarter(ARGS, null, true); + initConfig.accept(serviceStarter.getConfig()); + // ProxyServiceStarter will throw an exception when Authentication#start and Authentication#close are failed + serviceStarter.getConfig().setBrokerClientAuthenticationPlugin(ExceptionAuthentication2.class.getName()); + try { + serviceStarter.start(); + fail("ProxyServiceStarter should throw an exception when Authentication#start and Authentication#close are failed"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("ExceptionAuthentication2#start")); + assertTrue(serviceStarter.getProxyClientAuthentication() instanceof ExceptionAuthentication2); + } + } + + public static class ExceptionAuthentication1 implements Authentication { + + @Override + public String getAuthMethodName() { + return "org.apache.pulsar.proxy.server.ProxyConfigurationTest.ExceptionAuthentication1"; + } + + @Override + public void configure(Map authParams) { + // no-op + } + + @Override + public void start() throws PulsarClientException { + throw new PulsarClientException("ExceptionAuthentication1#start"); + } + + @Override + public void close() throws IOException { + // no-op + } + } + + public static class ExceptionAuthentication2 implements Authentication { + + @Override + public String getAuthMethodName() { + return "org.apache.pulsar.proxy.server.ProxyConfigurationTest.ExceptionAuthentication2"; + } + + @Override + public void configure(Map authParams) { + // no-op + } + + @Override + public void start() throws PulsarClientException { + throw new PulsarClientException("ExceptionAuthentication2#start"); + } + + @Override + public void close() throws IOException { + throw new IOException("ExceptionAuthentication2#close"); + } + } + } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java index 2866c6c26907c..86d572702f3b1 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java @@ -38,6 +38,8 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -61,6 +63,7 @@ public class ProxyStatsTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private WebServer proxyWebServer; private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -76,8 +79,12 @@ protected void setup() throws Exception { // enable full parsing feature proxyConfig.setProxyLogLevel(Optional.of(2)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, - new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)))); + new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -90,7 +97,7 @@ protected void setup() throws Exception { PulsarConfigurationLoader.convertFrom(proxyConfig)); proxyWebServer = new WebServer(proxyConfig, authService); - ProxyServiceStarter.addWebServerHandlers(proxyWebServer, proxyConfig, proxyService, null); + ProxyServiceStarter.addWebServerHandlers(proxyWebServer, proxyConfig, proxyService, null, proxyClientAuthentication); proxyWebServer.start(); } @@ -109,6 +116,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); proxyWebServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } /** diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java index 6e66008c15aef..30c6e45654ba0 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java @@ -28,6 +28,8 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.Consumer; import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.Message; @@ -56,6 +58,7 @@ public class ProxyStuckConnectionTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig; + private Authentication proxyClientAuthentication; private SocatContainer socatContainer; private String brokerServiceUriSocat; @@ -81,6 +84,10 @@ protected void setup() throws Exception { proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); proxyConfig.setClusterName(configClusterName); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + startProxyService(); // use the same port for subsequent restarts proxyConfig.setServicePort(proxyService.getListenPort()); @@ -88,7 +95,7 @@ protected void setup() throws Exception { private void startProxyService() throws Exception { proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig))) { + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication) { @Override protected LookupProxyHandler newLookupProxyHandler(ProxyConnection proxyConnection) { return new TestLookupProxyHandler(this, proxyConnection); @@ -107,6 +114,9 @@ protected void cleanup() throws Exception { if (proxyService != null) { proxyService.close(); } + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } if (socatContainer != null) { socatContainer.close(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java index e1e49f9e8c5f2..e101eb4ff7a2b 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java @@ -38,6 +38,8 @@ import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -74,6 +76,7 @@ public class ProxyTest extends MockedPulsarServiceBaseTest { protected ProxyService proxyService; protected ProxyConfiguration proxyConfig = new ProxyConfiguration(); + protected Authentication proxyClientAuthentication; @Data @ToString @@ -94,7 +97,7 @@ protected void setup() throws Exception { initializeProxyConfig(); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -102,12 +105,16 @@ protected void setup() throws Exception { proxyService.start(); } - protected void initializeProxyConfig() { + 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 @@ -116,6 +123,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java index 4e300d39741c3..0f0dc30b62096 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java @@ -27,6 +27,8 @@ import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -45,6 +47,7 @@ public class ProxyTlsTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -63,8 +66,12 @@ protected void setup() throws Exception { proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); proxyConfig.setClusterName(configClusterName); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -78,6 +85,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsWithAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsWithAuthTest.java index 16f610d6d0a3a..42b5ae178d3b0 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsWithAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsWithAuthTest.java @@ -27,6 +27,8 @@ import org.apache.pulsar.broker.auth.MockOIDCIdentityProvider; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.mockito.Mockito; @@ -38,6 +40,7 @@ public class ProxyTlsWithAuthTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; private MockOIDCIdentityProvider server; @@ -75,8 +78,12 @@ protected void setup() throws Exception { " \"privateKey\":\"file://" + tempFile.getAbsolutePath() + "\"}"); proxyConfig.setClusterName(configClusterName); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -89,6 +96,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } server.stop(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java index cf9ad5831ec0a..92a54aa12fda2 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java @@ -34,6 +34,7 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; @@ -73,6 +74,7 @@ public class ProxyWithAuthorizationNegTest extends ProducerConsumerBase { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @BeforeMethod @Override @@ -138,7 +140,10 @@ protected void setup() throws Exception { AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); - proxyService = Mockito.spy(new ProxyService(proxyConfig, authenticationService)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, authenticationService, proxyClientAuthentication)); proxyService.start(); } @@ -148,6 +153,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } /** diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java index bc96c7ea51041..51f42ea077165 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java @@ -38,6 +38,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -87,6 +88,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase { private ProxyService proxyService; private WebServer webServer; private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @DataProvider(name = "hostnameVerification") public Object[][] hostnameVerificationCodecProvider() { @@ -230,7 +232,10 @@ protected void setup() throws Exception { AuthenticationService authService = new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)); - proxyService = Mockito.spy(new ProxyService(proxyConfig, authService)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, authService, proxyClientAuthentication)); proxyService.setGracefulShutdown(false); webServer = new WebServer(proxyConfig, authService); } @@ -241,11 +246,14 @@ protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); webServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } private void startProxy() throws Exception { proxyService.start(); - ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, null); + ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, null, proxyClientAuthentication); webServer.start(); } @@ -459,10 +467,15 @@ public void tlsCiphersAndProtocols(Set tlsCiphers, Set tlsProtoc proxyConfig.setTlsProtocols(tlsProtocols); proxyConfig.setTlsCiphers(tlsCiphers); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + @Cleanup ProxyService proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); proxyService.setGracefulShutdown(false); try { proxyService.start(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java index d3c05fec721b0..3567c8264f1a3 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithExtensibleLoadManagerTest.java @@ -49,6 +49,8 @@ import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; @@ -75,6 +77,7 @@ public class ProxyWithExtensibleLoadManagerTest extends MultiBrokerBaseTest { private static final int TEST_TIMEOUT_MS = 30_000; + private Authentication proxyClientAuthentication; private ProxyService proxyService; @Override @@ -150,8 +153,11 @@ private String getDstBrokerLookupUrl(TopicName topicName) throws Exception { @BeforeMethod(alwaysRun = true) public void proxySetup() throws Exception { var proxyConfig = initializeProxyConfig(); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeper))).when(proxyService).createLocalMetadataStore(); doReturn(registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))).when(proxyService) .createConfigurationMetadataStore(); @@ -163,6 +169,9 @@ public void proxyCleanup() throws Exception { if (proxyService != null) { proxyService.close(); } + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test(timeOut = TEST_TIMEOUT_MS) diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java index 5fb3e04682421..63929ee72e446 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java @@ -39,6 +39,7 @@ import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.Consumer; @@ -83,6 +84,7 @@ public class ProxyWithJwtAuthorizationTest extends ProducerConsumerBase { private ProxyService proxyService; private WebServer webServer; private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @BeforeMethod @Override @@ -130,7 +132,10 @@ protected void setup() throws Exception { AuthenticationService authService = new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)); - proxyService = Mockito.spy(new ProxyService(proxyConfig, authService)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, authService, proxyClientAuthentication)); webServer = new WebServer(proxyConfig, authService); } @@ -140,11 +145,14 @@ protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); webServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } private void startProxy() throws Exception { proxyService.start(); - ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, null); + ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, null, proxyClientAuthentication); webServer.start(); } @@ -425,7 +433,7 @@ void testGetStatus() throws Exception { PulsarConfigurationLoader.convertFrom(proxyConfig)); final WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); webServer.start(); @Cleanup final Client client = javax.ws.rs.client.ClientBuilder @@ -450,7 +458,7 @@ void testGetMetrics() throws Exception { proxyConfig.setAuthenticateMetricsEndpoint(false); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); webServer.start(); @Cleanup Client client = javax.ws.rs.client.ClientBuilder.newClient(new ClientConfig().register(LoggingFeature.class)); @@ -463,7 +471,7 @@ void testGetMetrics() throws Exception { proxyConfig.setAuthenticateMetricsEndpoint(true); webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, - registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); + registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource)), proxyClientAuthentication); webServer.start(); try { Response r = client.target(webServer.getServiceUri()).path("/metrics").request().get(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java index 9d9490e74b5ad..885064b8e7404 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java @@ -33,6 +33,7 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; @@ -57,6 +58,7 @@ public class ProxyWithoutServiceDiscoveryTest extends ProducerConsumerBase { private static final String CLUSTER_NAME = "without-service-discovery"; private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @BeforeMethod @@ -122,9 +124,13 @@ protected void setup() throws Exception { proxyConfig.setAuthenticationProviders(providers); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); proxyService.start(); } @@ -134,6 +140,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } /** diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java index 57522186c8f16..71025ed484f7c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java @@ -32,6 +32,8 @@ import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.ClusterData; @@ -47,6 +49,7 @@ public class SuperUserAuthedAdminProxyHandlerTest extends MockedPulsarServiceBaseTest { private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; private WebServer webServer; private BrokerDiscoveryProvider discoveryProvider; private PulsarResources resource; @@ -94,6 +97,10 @@ protected void setup() throws Exception { proxyConfig.setBrokerClientTrustCertsFilePath(CA_CERT_FILE_PATH); proxyConfig.setAuthenticationProviders(ImmutableSet.of(AuthenticationProviderTls.class.getName())); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + resource = new PulsarResources(registerCloseable(new ZKMetadataStore(mockZooKeeper)), registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))); webServer = new WebServer(proxyConfig, new AuthenticationService( @@ -102,7 +109,7 @@ protected void setup() throws Exception { LoadManagerReport report = new LoadReport(brokerUrl.toString(), brokerUrlTls.toString(), null, null); doReturn(report).when(discoveryProvider).nextBroker(); - ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider)); + ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider, proxyClientAuthentication)); webServer.addServlet("/admin", servletHolder); webServer.addServlet("/lookup", servletHolder); @@ -114,6 +121,9 @@ protected void setup() throws Exception { @Override protected void cleanup() throws Exception { webServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } super.internalCleanup(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java index fe8b1f45385e4..0b597b933544a 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java @@ -35,6 +35,8 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.configuration.VipStatus; import org.apache.pulsar.metadata.impl.ZKMetadataStore; @@ -49,6 +51,7 @@ public class UnauthedAdminProxyHandlerTest extends MockedPulsarServiceBaseTest { private final String STATUS_FILE_PATH = "./src/test/resources/vip_status.html"; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; private WebServer webServer; private BrokerDiscoveryProvider discoveryProvider; private AdminProxyWrapper adminProxyHandler; @@ -77,13 +80,17 @@ protected void setup() throws Exception { proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); proxyConfig.setClusterName(configClusterName); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + webServer = new WebServer(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig))); resource = new PulsarResources(registerCloseable(new ZKMetadataStore(mockZooKeeper)), registerCloseable(new ZKMetadataStore(mockZooKeeperGlobal))); discoveryProvider = spy(registerCloseable(new BrokerDiscoveryProvider(proxyConfig, resource))); - adminProxyHandler = new AdminProxyWrapper(proxyConfig, discoveryProvider); + adminProxyHandler = new AdminProxyWrapper(proxyConfig, discoveryProvider, proxyClientAuthentication); ServletHolder servletHolder = new ServletHolder(adminProxyHandler); webServer.addServlet("/admin", servletHolder); webServer.addServlet("/lookup", servletHolder); @@ -101,6 +108,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { internalCleanup(); webServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test @@ -128,8 +138,8 @@ public void testVipStatus() throws Exception { static class AdminProxyWrapper extends AdminProxyHandler { String rewrittenUrl; - AdminProxyWrapper(ProxyConfiguration config, BrokerDiscoveryProvider discoveryProvider) { - super(config, discoveryProvider); + AdminProxyWrapper(ProxyConfiguration config, BrokerDiscoveryProvider discoveryProvider, Authentication proxyClientAuthentication) { + super(config, discoveryProvider, proxyClientAuthentication); } @Override From c07b158f003c5a5623296189f0932d7058d2e75a Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Wed, 14 Aug 2024 10:26:47 +0800 Subject: [PATCH 438/580] [fix][client] Fix for early hit `beforeConsume` for MultiTopicConsumer (#23141) --- .../pulsar/client/api/InterceptorsTest.java | 44 +++++++++----- .../client/impl/MultiTopicsConsumerImpl.java | 58 +++++++++++++++++-- 2 files changed, 83 insertions(+), 19 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java index f23d82b32cd43..afb17a186477c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import com.google.common.collect.Sets; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -29,8 +30,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; - -import com.google.common.collect.Sets; import lombok.Cleanup; import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.RandomUtils; @@ -79,6 +78,12 @@ public Object[][] getTopicPartition() { return new Object[][] {{ 0 }, { 3 }}; } + @DataProvider(name = "topics") + public Object[][] getTopics() { + return new Object[][] {{ List.of("persistent://my-property/my-ns/my-topic") }, + { List.of("persistent://my-property/my-ns/my-topic", "persistent://my-property/my-ns/my-topic1") }}; + } + @Test public void testProducerInterceptor() throws Exception { Map> ackCallback = new HashMap<>(); @@ -403,9 +408,9 @@ public void close() { @Override public Message beforeConsume(Consumer consumer, Message message) { - MessageImpl msg = (MessageImpl) message; + MessageImpl msg = ((MessageImpl) ((TopicMessageImpl) message).getMessage()); msg.getMessageBuilder().addProperty().setKey("beforeConsumer").setValue("1"); - return msg; + return message; } @Override @@ -449,13 +454,19 @@ public void onAckTimeoutSend(Consumer consumer, Set messageId int keyCount = 0; for (int i = 0; i < 2; i++) { - Message received = consumer.receive(); + Message received; + if (i % 2 == 0) { + received = consumer.receive(); + } else { + received = consumer.receiveAsync().join(); + } MessageImpl msg = (MessageImpl) ((TopicMessageImpl) received).getMessage(); for (KeyValue keyValue : msg.getMessageBuilder().getPropertiesList()) { if ("beforeConsumer".equals(keyValue.getKey())) { keyCount++; } } + Assert.assertEquals(keyCount, i + 1); consumer.acknowledge(received); } Assert.assertEquals(2, keyCount); @@ -475,9 +486,9 @@ public void close() { @Override public Message beforeConsume(Consumer consumer, Message message) { - MessageImpl msg = (MessageImpl) message; + MessageImpl msg = ((MessageImpl) ((TopicMessageImpl) message).getMessage()); msg.getMessageBuilder().addProperty().setKey("beforeConsumer").setValue("1"); - return msg; + return message; } @Override @@ -612,8 +623,8 @@ public void onAckTimeoutSend(Consumer consumer, Set messageId consumer.close(); } - @Test - public void testConsumerInterceptorForNegativeAcksSend() throws PulsarClientException, InterruptedException { + @Test(dataProvider = "topics") + public void testConsumerInterceptorForNegativeAcksSend(List topics) throws PulsarClientException, InterruptedException { final int totalNumOfMessages = 100; CountDownLatch latch = new CountDownLatch(totalNumOfMessages / 2); @@ -640,6 +651,7 @@ public void onAcknowledgeCumulative(Consumer consumer, MessageId message @Override public void onNegativeAcksSend(Consumer consumer, Set messageIds) { + Assert.assertTrue(latch.getCount() > 0); messageIds.forEach(messageId -> latch.countDown()); } @@ -650,7 +662,7 @@ public void onAckTimeoutSend(Consumer consumer, Set messageId }; Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic("persistent://my-property/my-ns/my-topic") + .topics(topics) .subscriptionType(SubscriptionType.Failover) .intercept(interceptor) .negativeAckRedeliveryDelay(100, TimeUnit.MILLISECONDS) @@ -658,7 +670,7 @@ public void onAckTimeoutSend(Consumer consumer, Set messageId .subscribe(); Producer producer = pulsarClient.newProducer(Schema.STRING) - .topic("persistent://my-property/my-ns/my-topic") + .topic(topics.get(0)) .create(); for (int i = 0; i < totalNumOfMessages; i++) { @@ -682,8 +694,9 @@ public void onAckTimeoutSend(Consumer consumer, Set messageId consumer.close(); } - @Test - public void testConsumerInterceptorForAckTimeoutSend() throws PulsarClientException, InterruptedException { + @Test(dataProvider = "topics") + public void testConsumerInterceptorForAckTimeoutSend(List topics) throws PulsarClientException, + InterruptedException { final int totalNumOfMessages = 100; CountDownLatch latch = new CountDownLatch(totalNumOfMessages / 2); @@ -714,16 +727,17 @@ public void onNegativeAcksSend(Consumer consumer, Set message @Override public void onAckTimeoutSend(Consumer consumer, Set messageIds) { + Assert.assertTrue(latch.getCount() > 0); messageIds.forEach(messageId -> latch.countDown()); } }; Producer producer = pulsarClient.newProducer(Schema.STRING) - .topic("persistent://my-property/my-ns/my-topic") + .topic(topics.get(0)) .create(); Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic("persistent://my-property/my-ns/my-topic") + .topics(topics) .subscriptionName("foo") .intercept(interceptor) .ackTimeout(2, TimeUnit.SECONDS) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 3f5e501b28130..bf8bd6cc95117 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -26,6 +26,7 @@ import com.google.common.collect.Lists; import io.netty.util.Timeout; import io.netty.util.TimerTask; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -108,6 +109,7 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { private final MessageIdAdv startMessageId; private volatile boolean duringSeek = false; private final long startMessageRollbackDurationInSec; + private final ConsumerInterceptors internalConsumerInterceptors; MultiTopicsConsumerImpl(PulsarClientImpl client, ConsumerConfigurationData conf, ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist) { @@ -137,6 +139,11 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { long startMessageRollbackDurationInSec) { super(client, singleTopic, conf, Math.max(2, conf.getReceiverQueueSize()), executorProvider, subscribeFuture, schema, interceptors); + if (interceptors != null) { + this.internalConsumerInterceptors = getInternalConsumerInterceptors(interceptors); + } else { + this.internalConsumerInterceptors = null; + } checkArgument(conf.getReceiverQueueSize() > 0, "Receiver queue size needs to be greater than 0 for Topics Consumer"); @@ -316,7 +323,8 @@ private void messageReceived(ConsumerImpl consumer, Message message) { CompletableFuture> receivedFuture = nextPendingReceive(); if (receivedFuture != null) { unAckedMessageTracker.add(topicMessage.getMessageId(), topicMessage.getRedeliveryCount()); - completePendingReceive(receivedFuture, topicMessage); + final Message interceptMessage = beforeConsume(topicMessage); + completePendingReceive(receivedFuture, interceptMessage); } else if (enqueueMessageAndCheckBatchReceive(topicMessage) && hasPendingBatchReceive()) { notifyPendingBatchReceivedCallBack(); } @@ -369,7 +377,7 @@ protected Message internalReceive() throws PulsarClientException { checkState(message instanceof TopicMessageImpl); unAckedMessageTracker.add(message.getMessageId(), message.getRedeliveryCount()); resumeReceivingFromPausedConsumersIfNeeded(); - return message; + return beforeConsume(message); } catch (Exception e) { ExceptionHandler.handleInterruptedException(e); throw PulsarClientException.unwrap(e); @@ -388,6 +396,7 @@ protected Message internalReceive(long timeout, TimeUnit unit) throws PulsarC decreaseIncomingMessageSize(message); checkArgument(message instanceof TopicMessageImpl); trackUnAckedMsgIfNoListener(message.getMessageId(), message.getRedeliveryCount()); + message = beforeConsume(message); } resumeReceivingFromPausedConsumersIfNeeded(); return message; @@ -447,7 +456,7 @@ protected CompletableFuture> internalReceiveAsync() { checkState(message instanceof TopicMessageImpl); unAckedMessageTracker.add(message.getMessageId(), message.getRedeliveryCount()); resumeReceivingFromPausedConsumersIfNeeded(); - result.complete(message); + result.complete(beforeConsume(message)); } }); return result; @@ -1185,7 +1194,7 @@ private ConsumerImpl createInternalConsumer(ConsumerConfigurationData conf return ConsumerImpl.newConsumerImpl(client, partitionName, configurationData, client.externalExecutorProvider(), partitionIndex, true, listener != null, subFuture, - startMessageId, schema, interceptors, + startMessageId, schema, this.internalConsumerInterceptors, createIfDoesNotExist, startMessageRollbackDurationInSec); } @@ -1595,4 +1604,45 @@ private CompletableFuture> getExistsPartitions(String topic) { return list; }); } + + private ConsumerInterceptors getInternalConsumerInterceptors(ConsumerInterceptors multiTopicInterceptors) { + return new ConsumerInterceptors(new ArrayList<>()) { + + @Override + public Message beforeConsume(Consumer consumer, Message message) { + return message; + } + + @Override + public void onAcknowledge(Consumer consumer, MessageId messageId, Throwable exception) { + multiTopicInterceptors.onAcknowledge(consumer, messageId, exception); + } + + @Override + public void onAcknowledgeCumulative(Consumer consumer, + MessageId messageId, Throwable exception) { + multiTopicInterceptors.onAcknowledgeCumulative(consumer, messageId, exception); + } + + @Override + public void onNegativeAcksSend(Consumer consumer, Set set) { + multiTopicInterceptors.onNegativeAcksSend(consumer, set); + } + + @Override + public void onAckTimeoutSend(Consumer consumer, Set set) { + multiTopicInterceptors.onAckTimeoutSend(consumer, set); + } + + @Override + public void onPartitionsChange(String topicName, int partitions) { + multiTopicInterceptors.onPartitionsChange(topicName, partitions); + } + + @Override + public void close() throws IOException { + multiTopicInterceptors.close(); + } + }; + } } From a0259380e1eb86dbe4e80d27c585188671b25135 Mon Sep 17 00:00:00 2001 From: Omar Yasin Date: Wed, 14 Aug 2024 01:00:23 -0700 Subject: [PATCH 439/580] [fix][client] Create the retry producer async (#23157) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Ómar Yasin --- .../pulsar/client/impl/ConsumerImpl.java | 96 ++++++++++--------- 1 file changed, 52 insertions(+), 44 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 1806d13493b2f..3acf55afaed51 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -203,7 +203,7 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private volatile CompletableFuture> deadLetterProducer; - private volatile Producer retryLetterProducer; + private volatile CompletableFuture> retryLetterProducer; private final ReadWriteLock createProducerLock = new ReentrantReadWriteLock(); protected volatile boolean paused; @@ -643,6 +643,7 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a Map customProperties, long delayTime, TimeUnit unit) { + MessageId messageId = message.getMessageId(); if (messageId == null) { return FutureUtil.failedFuture(new PulsarClientException @@ -659,29 +660,8 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a } return FutureUtil.failedFuture(exception); } - if (delayTime < 0) { - delayTime = 0; - } - if (retryLetterProducer == null) { - createProducerLock.writeLock().lock(); - try { - if (retryLetterProducer == null) { - retryLetterProducer = client.newProducer(Schema.AUTO_PRODUCE_BYTES(schema)) - .topic(this.deadLetterPolicy.getRetryLetterTopic()) - .enableBatching(false) - .enableChunking(true) - .blockIfQueueFull(false) - .create(); - stats.setRetryLetterProducerStats(retryLetterProducer.getStats()); - } - } catch (Exception e) { - log.error("Create retry letter producer exception with topic: {}", - deadLetterPolicy.getRetryLetterTopic(), e); - return FutureUtil.failedFuture(e); - } finally { - createProducerLock.writeLock().unlock(); - } - } + + initRetryLetterProducerIfNeeded(); CompletableFuture result = new CompletableFuture<>(); if (retryLetterProducer != null) { try { @@ -701,7 +681,7 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a } propertiesMap.put(RetryMessageUtil.SYSTEM_PROPERTY_RECONSUMETIMES, String.valueOf(reconsumeTimes)); propertiesMap.put(RetryMessageUtil.SYSTEM_PROPERTY_DELAY_TIME, - String.valueOf(unit.toMillis(delayTime))); + String.valueOf(unit.toMillis(delayTime < 0 ? 0 : delayTime))); MessageId finalMessageId = messageId; if (reconsumeTimes > this.deadLetterPolicy.getMaxRedeliverCount() @@ -732,23 +712,29 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a }); } else { assert retryMessage != null; - TypedMessageBuilder typedMessageBuilderNew = retryLetterProducer - .newMessage(Schema.AUTO_PRODUCE_BYTES(message.getReaderSchema().get())) - .value(retryMessage.getData()) - .properties(propertiesMap); - if (delayTime > 0) { - typedMessageBuilderNew.deliverAfter(delayTime, unit); - } - if (message.hasKey()) { - typedMessageBuilderNew.key(message.getKey()); - } - typedMessageBuilderNew.sendAsync() - .thenCompose(__ -> doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null)) - .thenAccept(v -> result.complete(null)) - .exceptionally(ex -> { - result.completeExceptionally(ex); - return null; - }); + retryLetterProducer.thenAcceptAsync(rtlProducer -> { + TypedMessageBuilder typedMessageBuilderNew = rtlProducer + .newMessage(Schema.AUTO_PRODUCE_BYTES(message.getReaderSchema().get())) + .value(retryMessage.getData()) + .properties(propertiesMap); + if (delayTime > 0) { + typedMessageBuilderNew.deliverAfter(delayTime, unit); + } + if (message.hasKey()) { + typedMessageBuilderNew.key(message.getKey()); + } + typedMessageBuilderNew.sendAsync() + .thenCompose(__ -> doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null)) + .thenAccept(v -> result.complete(null)) + .exceptionally(ex -> { + result.completeExceptionally(ex); + return null; + }); + }, internalPinnedExecutor).exceptionally(ex -> { + result.completeExceptionally(ex); + retryLetterProducer = null; + return null; + }); } } catch (Exception e) { result.completeExceptionally(e); @@ -757,7 +743,7 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a MessageId finalMessageId = messageId; result.exceptionally(ex -> { log.error("Send to retry letter topic exception with topic: {}, messageId: {}", - retryLetterProducer.getTopic(), finalMessageId, ex); + this.deadLetterPolicy.getRetryLetterTopic(), finalMessageId, ex); Set messageIds = Collections.singleton(finalMessageId); unAckedMessageTracker.remove(finalMessageId); redeliverUnacknowledgedMessages(messageIds); @@ -1136,7 +1122,7 @@ public synchronized CompletableFuture closeAsync() { ArrayList> closeFutures = new ArrayList<>(4); closeFutures.add(closeFuture); if (retryLetterProducer != null) { - closeFutures.add(retryLetterProducer.closeAsync().whenComplete((ignore, ex) -> { + closeFutures.add(retryLetterProducer.thenCompose(p -> p.closeAsync()).whenComplete((ignore, ex) -> { if (ex != null) { log.warn("Exception ignored in closing retryLetterProducer of consumer", ex); } @@ -2267,6 +2253,28 @@ private void initDeadLetterProducerIfNeeded() { } } + private void initRetryLetterProducerIfNeeded() { + if (retryLetterProducer == null) { + createProducerLock.writeLock().lock(); + try { + if (retryLetterProducer == null) { + retryLetterProducer = client + .newProducer(Schema.AUTO_PRODUCE_BYTES(schema)) + .topic(this.deadLetterPolicy.getRetryLetterTopic()) + .enableBatching(false) + .enableChunking(true) + .blockIfQueueFull(false) + .createAsync(); + retryLetterProducer.thenAccept(rtlProducer -> { + stats.setRetryLetterProducerStats(rtlProducer.getStats()); + }); + } + } finally { + createProducerLock.writeLock().unlock(); + } + } + } + @Override public void seek(MessageId messageId) throws PulsarClientException { try { From d5ce1cee35363ba2372375c2e8740be6d87488d8 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 14 Aug 2024 16:39:55 +0800 Subject: [PATCH 440/580] [improve] [broker] Avoid subscription fenced error with consumer.seek whenever possible (#23163) --- .../persistent/PersistentSubscription.java | 32 ++++++--- .../broker/service/SubscriptionSeekTest.java | 65 +++++++++++++++++++ 2 files changed, 87 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 0a57f98eb7ad6..f59ea18ce8ea7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -132,6 +132,7 @@ public class PersistentSubscription extends AbstractSubscription { private final PendingAckHandle pendingAckHandle; private volatile Map subscriptionProperties; private volatile CompletableFuture fenceFuture; + private volatile CompletableFuture inProgressResetCursorFuture; static Map getBaseCursorProperties(boolean isReplicated) { return isReplicated ? REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES : NON_REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES; @@ -220,6 +221,16 @@ public boolean setReplicated(boolean replicated) { @Override public CompletableFuture addConsumer(Consumer consumer) { + CompletableFuture inProgressResetCursorFuture = this.inProgressResetCursorFuture; + if (inProgressResetCursorFuture != null) { + return inProgressResetCursorFuture.handle((ignore, ignoreEx) -> null) + .thenCompose(ignore -> addConsumerInternal(consumer)); + } else { + return addConsumerInternal(consumer); + } + } + + private CompletableFuture addConsumerInternal(Consumer consumer) { return pendingAckHandle.pendingAckHandleFuture().thenCompose(future -> { synchronized (PersistentSubscription.this) { cursor.updateLastActive(); @@ -775,7 +786,8 @@ public void findEntryComplete(Position position, Object ctx) { } else { finalPosition = position.getNext(); } - resetCursor(finalPosition, future); + CompletableFuture resetCursorFuture = resetCursor(finalPosition); + FutureUtil.completeAfter(future, resetCursorFuture); } @Override @@ -794,18 +806,13 @@ public void findEntryFailed(ManagedLedgerException exception, } @Override - public CompletableFuture resetCursor(Position position) { - CompletableFuture future = new CompletableFuture<>(); - resetCursor(position, future); - return future; - } - - private void resetCursor(Position finalPosition, CompletableFuture future) { + public CompletableFuture resetCursor(Position finalPosition) { if (!IS_FENCED_UPDATER.compareAndSet(PersistentSubscription.this, FALSE, TRUE)) { - future.completeExceptionally(new SubscriptionBusyException("Failed to fence subscription")); - return; + return CompletableFuture.failedFuture(new SubscriptionBusyException("Failed to fence subscription")); } + final CompletableFuture future = new CompletableFuture<>(); + inProgressResetCursorFuture = future; final CompletableFuture disconnectFuture; // Lock the Subscription object before locking the Dispatcher object to avoid deadlocks @@ -825,6 +832,7 @@ private void resetCursor(Position finalPosition, CompletableFuture future) if (throwable != null) { log.error("[{}][{}] Failed to disconnect consumer from subscription", topicName, subName, throwable); IS_FENCED_UPDATER.set(PersistentSubscription.this, FALSE); + inProgressResetCursorFuture = null; future.completeExceptionally( new SubscriptionBusyException("Failed to disconnect consumers from subscription")); return; @@ -864,6 +872,7 @@ public void resetComplete(Object ctx) { dispatcher.afterAckMessages(null, finalPosition); } IS_FENCED_UPDATER.set(PersistentSubscription.this, FALSE); + inProgressResetCursorFuture = null; future.complete(null); } @@ -872,6 +881,7 @@ public void resetFailed(ManagedLedgerException exception, Object ctx) { log.error("[{}][{}] Failed to reset subscription to position {}", topicName, subName, finalPosition, exception); IS_FENCED_UPDATER.set(PersistentSubscription.this, FALSE); + inProgressResetCursorFuture = null; // todo - retry on InvalidCursorPositionException // or should we just ask user to retry one more time? if (exception instanceof InvalidCursorPositionException) { @@ -886,10 +896,12 @@ public void resetFailed(ManagedLedgerException exception, Object ctx) { }).exceptionally((e) -> { log.error("[{}][{}] Error while resetting cursor", topicName, subName, e); IS_FENCED_UPDATER.set(PersistentSubscription.this, FALSE); + inProgressResetCursorFuture = null; future.completeExceptionally(new BrokerServiceException(e)); return null; }); }); + return future; } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java index fd08f284bbf99..3fc795a8c3e2a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java @@ -34,12 +34,14 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.InjectedClientCnxClientBuilder; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -50,8 +52,13 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.BatchMessageIdImpl; +import org.apache.pulsar.client.impl.ClientBuilderImpl; +import org.apache.pulsar.client.impl.ClientCnx; import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.impl.metrics.InstrumentProvider; +import org.apache.pulsar.common.api.proto.CommandError; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.util.RelativeTimeUtil; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; @@ -781,6 +788,64 @@ public void testSeekByFunctionAndMultiTopic() throws Exception { assertEquals(count, (msgInTopic1Partition0 + msgInTopic1Partition1 + msgInTopic1Partition2) * 2); } + @Test + public void testSeekWillNotEncounteredFencedError() throws Exception { + String topicName = "persistent://prop/ns-abc/my-topic2"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topicPolicies().setRetention(topicName, new RetentionPolicies(3600, 0)); + // Create a pulsar client with a subscription fenced counter. + ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); + AtomicInteger receivedFencedErrorCounter = new AtomicInteger(); + PulsarClient client = InjectedClientCnxClientBuilder.create(clientBuilder, (conf, eventLoopGroup) -> + new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup) { + protected void handleError(CommandError error) { + if (error.getMessage() != null && error.getMessage().contains("Subscription is fenced")) { + receivedFencedErrorCounter.incrementAndGet(); + } + super.handleError(error); + } + }); + + // publish some messages. + org.apache.pulsar.client.api.Consumer consumer = client.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName("s1") + .subscribe(); + Producer producer = client.newProducer(Schema.STRING) + .topic(topicName).create(); + MessageIdImpl msgId1 = (MessageIdImpl) producer.send("0"); + for (int i = 1; i < 11; i++) { + admin.topics().unload(topicName); + producer.send(i + ""); + } + + // Inject a delay for reset-cursor. + mockZooKeeper.delay(3000, (op, path) -> { + if (path.equals("/managed-ledgers/prop/ns-abc/persistent/my-topic2/s1")) { + return op.toString().equalsIgnoreCase("SET"); + } + return false; + }); + + // Verify: consumer will not receive "subscription fenced" error after a seek. + for (int i = 1; i < 11; i++) { + Message msg = consumer.receive(2, TimeUnit.SECONDS); + assertNotNull(msg); + consumer.acknowledge(msg); + } + consumer.seek(msgId1); + Awaitility.await().untilAsserted(() -> { + assertTrue(consumer.isConnected()); + }); + assertEquals(receivedFencedErrorCounter.get(), 0); + + // cleanup. + producer.close(); + consumer.close(); + client.close(); + admin.topics().delete(topicName); + } + @Test public void testExceptionBySeekFunction() throws Exception { final String topicName = "persistent://prop/use/ns-abc/test" + UUID.randomUUID(); From 606b6a71efd76d7c695414aa61701c01a645f0f4 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 14 Aug 2024 20:57:49 +0800 Subject: [PATCH 441/580] [fix] [test] Revert the modification to NonDurableSubscriptionTest caused by a mistake in the PR#23129 (#23168) --- .../api/NonDurableSubscriptionTest.java | 33 ++++--------------- 1 file changed, 7 insertions(+), 26 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java index 80adc79e6fee8..bbac688d9224c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java @@ -34,7 +34,6 @@ import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.impl.ImmutablePositionImpl; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.BrokerTestUtil; @@ -47,12 +46,10 @@ import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.api.proto.CommandFlow; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; -import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; import org.testng.Assert; -import org.testng.AssertJUnit; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; @@ -545,34 +542,18 @@ public void testReaderInitAtDeletedPosition() throws Exception { .getStats(topicName, true, true, true).getSubscriptions().get("s1"); log.info("backlog size: {}", subscriptionStats.getMsgBacklog()); assertEquals(subscriptionStats.getMsgBacklog(), 0); - PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName); - ManagedLedgerInternalStats.CursorStats cursorStats = internalStats.cursors.get("s1"); + ManagedLedgerInternalStats.CursorStats cursorStats = + admin.topics().getInternalStats(topicName).cursors.get("s1"); String[] ledgerIdAndEntryId = cursorStats.markDeletePosition.split(":"); - ImmutablePositionImpl actMarkDeletedPos = - new ImmutablePositionImpl(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); - ImmutablePositionImpl expectedMarkDeletedPos = - new ImmutablePositionImpl(msgIdInDeletedLedger5.getLedgerId(), msgIdInDeletedLedger5.getEntryId()); - log.info("LAC: {}", internalStats.lastConfirmedEntry); + Position actMarkDeletedPos = + PositionFactory.create(Long.valueOf(ledgerIdAndEntryId[0]), Long.valueOf(ledgerIdAndEntryId[1])); + Position expectedMarkDeletedPos = + PositionFactory.create(msgIdInDeletedLedger5.getLedgerId(), msgIdInDeletedLedger5.getEntryId()); log.info("Expected mark deleted position: {}", expectedMarkDeletedPos); log.info("Actual mark deleted position: {}", cursorStats.markDeletePosition); - AssertJUnit.assertTrue(actMarkDeletedPos.compareTo(expectedMarkDeletedPos) >= 0); + assertTrue(actMarkDeletedPos.compareTo(expectedMarkDeletedPos) >= 0); }); - admin.topics().createSubscription(topicName, "s2", MessageId.earliest); - admin.topics().createSubscription(topicName, "s3", MessageId.latest); - PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName); - ManagedLedgerInternalStats.CursorStats cursorStats2 = internalStats.cursors.get("s2"); - String[] ledgerIdAndEntryId2 = cursorStats2.markDeletePosition.split(":"); - ImmutablePositionImpl actMarkDeletedPos2 = - new ImmutablePositionImpl(Long.valueOf(ledgerIdAndEntryId2[0]), Long.valueOf(ledgerIdAndEntryId2[1])); - ManagedLedgerInternalStats.CursorStats cursorStats3 = internalStats.cursors.get("s3"); - String[] ledgerIdAndEntryId3 = cursorStats3.markDeletePosition.split(":"); - ImmutablePositionImpl actMarkDeletedPos3 = - new ImmutablePositionImpl(Long.valueOf(ledgerIdAndEntryId3[0]), Long.valueOf(ledgerIdAndEntryId3[1])); - log.info("LAC: {}", internalStats.lastConfirmedEntry); - log.info("Actual mark deleted position 2: {}", actMarkDeletedPos2); - log.info("Actual mark deleted position 3: {}", actMarkDeletedPos3); - pulsar.getBrokerService().getTopic(topicName, false).join().get(); // cleanup. reader.close(); producer.close(); From ce38ee2bccba1f9e1687b53722e175b45b296e76 Mon Sep 17 00:00:00 2001 From: Anshul Singh Date: Wed, 14 Aug 2024 19:34:58 +0530 Subject: [PATCH 442/580] [improve][pip] PIP-369: Flag based selective unload on changing ns-isolation-policy (#23116) --- pip/pip-369.md | 124 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 124 insertions(+) create mode 100644 pip/pip-369.md diff --git a/pip/pip-369.md b/pip/pip-369.md new file mode 100644 index 0000000000000..9aeb598110d72 --- /dev/null +++ b/pip/pip-369.md @@ -0,0 +1,124 @@ +# PIP-369: Flag based selective unload on changing ns-isolation-policy + +# Background knowledge + +In Apache Pulsar, namespace isolation policies are used to limit the ownership of certain subsets of namespaces to specific broker groups. +These policies are defined using regular expressions to match namespaces and specify primary and secondary broker groups along with failover policy configurations. +This ensures that the ownership of the specified namespaces is restricted to the designated broker groups. + +For more information, refer to the [Pulsar documentation on namespace isolation](https://pulsar.apache.org/docs/next/administration-isolation/#isolation-levels). + +# History/Context +In Apache Pulsar 2.7.1+, there was a flag introduced (`enableNamespaceIsolationUpdateOnTime`) that controlled whether to unload namespaces or not when a namespace isolation policy is applied. https://github.com/apache/pulsar/pull/8976 + +Later on, in 2.11, rework was done as part of [PIP-149](https://github.com/apache/pulsar/issues/14365) to make get/set isolationData calls async, +which resulted in namespaces to always get unloaded irrespective of `enableNamespaceIsolationUpdateOnTime` config, not adhering to this config at all. + +And now in 3.3, `enableNamespaceIsolationUpdateOnTime` broker config was deprecated as it no longer serves any purpose. https://github.com/apache/pulsar/pull/22449 + +# Motivation + +In Apache Pulsar 3.x, changing a namespace isolation policy results in unloading all namespace bundles that match the namespace's regular expression provided in the isolation policy. +This can be problematic for cases where the regex matches a large subset of namespaces, such as `tenant-x/.*`. +One of such case is mentioned on this issue [#23092](https://github.com/apache/pulsar/issues/23092) where policy change resulted in 100+ namespace bundles to get unloaded. +And broker exhausted all the available connections due to too many unload calls happening at once resulting in 5xx response. +Other issues that happens with this approach are huge latency spikes as topics are unavailable until bundles are loaded back, increasing the pending produce calls. +The only benefit this approach serves is ensuring that all the namespaces matching the policy regex will come to correct broker group. +But when namespace bundles are already on the correct broker group (according to the policy), unloading those namespaces doesn't serve any purpose. + +This PIP aims to address the need to either prevent unnecessary unloading or provide a more granular approach to determine what should be unloaded. + +Some of the cases covered by this PIP are discussed in [#23094](https://github.com/apache/pulsar/issues/23094) by @grssam. +> - unload nothing as part of the set policy call +> - unload every matching namespace as part of the policy set call +> - unload only the changed namespaces (newly added + removed) + +# Goals + +## In Scope +This PIP proposes a flag-based approach to control what should be unloaded when an isolation policy is applied. +The possible values for this flag are: +- **all_matching**: Unload all the namespaces that matches either old or new policy change. +- **changed**: Only unload namespaces that are either added or removed due to the policy change. +- **none**: Do not unload anything. Unloading can occur naturally due to load balancing or can be done manually using the unload admin call. + +This flag will be a part of isolation policy data with defaults. Objective is to keep the default behavior unchanged on applying the new policy. + +## Out of Scope + +Applying concurrency reducer to limit how many async calls will happen in parallel is out of the scope for this PIP. +This should be addressed in a separate PIP, as solving the issue of infinite asynchronous calls probably requires changes to broker configurations and is a problem present in multiple areas. + +# Detailed Design + +## Design & Implementation Details + +A new flag will be introduced in `NamespaceIsolationData`. + +```java +enum UnloadScope { + all_matching, // unloads everything, OLD ⋃ NEW + changed, // unload namespaces delta, (new ⋃ old) - (new ∩ old) + none, // skip unloading anything, ϕ +}; +``` +Filters will be added based on the above when namespaces are selected for unload in set policy call. +`UnloadScope.all_matching` will be the default in current version. + +> **_NOTE:_** +> For 3.x unchanged behaviour, the `all_matching` UnloadScope option should only unload namespaces matching new policy (NEW). This matches the current behavior and maintains backward compatibility from implementation POV. +> +> For 4.x, +> 1. The behaviour for the `all_matching` flag should change to unload everything matching either the old or new policy (union of both). +> 2. The default flag value should be `changed`, so accidentally missing this flag while applying the policy shouldn't impact workloads already on the correct broker group. + +### Public API + +A new flag will be added in the NamespaceIsolationData. This changes the request body when set policy API is called. +To keep things backwards compatible, `unload_scope` will be optional. API params will remain unchanged. + +Path: `/{cluster}/namespaceIsolationPolicies/{policyName}` +```json +{ + "policy-name": { + "namespaces": [...], + "primary": [...], + "secondary": [...], + "auto_failover_policy": { + ... + }, + "unload_scope": "all_matching|changed|none" + } +} +``` + +### CLI + +```shell +# set call will have an optional flag. Sample command as shown below: +# +pulsar-admin ns-isolation-policy set cluster-name policy-name --unload-scope none +# Possible values for unload-scope: [all_matching, changed, none] +``` + +# Backward & Forward Compatibility + +Added flag is optional, that doesn't require any changes to pre-existing policy data. If the flag is not present then default value shall be considered. + +# Alternatives + +Boolean flag passed during set policy call to either unload the delta namespaces (removed and added) without affecting unchanged namespaces or unload nothing. PR: https://github.com/apache/pulsar/pull/23094 + +Limitation: This approach does not consider cases where unloading is needed for every matching namespace as part of the policy set call. +Manual unloading would be required for unchanged namespaces not on the correct broker group. + +# Links + + +* Mailing List discussion thread: https://lists.apache.org/thread/6f8k1typ48817w65pjh6orhks1smpbqg +* Mailing List voting thread: https://lists.apache.org/thread/0pj3llwpcy73mrs5s3l5t8kctn2mzyf7 + + +PS: This PIP should get cherry-picked to 3.0.x as it provides a way to resolve the bug mentioned at [#23092](https://github.com/apache/pulsar/issues/23092) which exist in all the production system today. \ No newline at end of file From 15b88d250818bada5c1a94f5c54ef7806f88a500 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Thu, 15 Aug 2024 11:57:52 +0800 Subject: [PATCH 443/580] [fix][broker] Fix shadow topics cannot be consumed when the entry is not cached (#23147) ### Motivation For shadow topics, a `ReadOnlyLedgerHandle` is created to read messages from the source topic when the entry is not cached. However, it leverages the `readAsync` API that validates the `lastAddConfirmed` field (LAC). In `ReadOnlyLedgerHandle`, this field could never be updated, so `readAsync` could fail immediately. See `LedgerHandle#readAsync`: ```java if (lastEntry > lastAddConfirmed) { LOG.error("ReadAsync exception on ledgerId:{} firstEntry:{} lastEntry:{} lastAddConfirmed:{}", ledgerId, firstEntry, lastEntry, lastAddConfirmed); return FutureUtils.exception(new BKReadException()); } ``` This bug is not exposed because: 1. `PulsarMockReadHandle` does not maintain a LAC field. 2. The case for cache miss is never tested. ### Modifications Replace `readAsync` with `readUnconfirmedAsync` and compare the entry range with the `ManagedLedger#getLastConfirmedEntry`. The managed ledger already maintains a `lastConfirmedEntry` to limit the last entry. See `ManagedLedgerImpl#internalReadFromLedger`: ```java Position lastPosition = lastConfirmedEntry; if (ledger.getId() == lastPosition.getLedgerId()) { lastEntryInLedger = lastPosition.getEntryId(); ``` Add `ShadowTopicRealBkTest` to cover two code changes `RangeEntryCacheImpl#readFromStorage` and `EntryCache#asyncReadEntry`. Exceptionally, compare the entry range with the LAC of a ledger handle when it does not exist in the managed ledger. It's because `ReadOnlyManagedLedgerImpl` could read a ledger in another managed ledger. ### Documentation - [ ] `doc` - [ ] `doc-required` - [x] `doc-not-needed` - [ ] `doc-complete` ### Matching PR in forked repository PR in forked repository: https://github.com/BewareMyPower/pulsar/pull/33 --- .../mledger/impl/ManagedLedgerImpl.java | 2 + .../impl/cache/EntryCacheDisabled.java | 4 +- .../impl/cache/RangeEntryCacheImpl.java | 4 +- .../mledger/impl/cache/ReadEntryUtils.java | 54 +++++ .../mledger/impl/EntryCacheManagerTest.java | 7 +- .../mledger/impl/EntryCacheTest.java | 187 ++++++++---------- .../mledger/impl/OffloadPrefixReadTest.java | 2 +- .../persistent/ShadowTopicRealBkTest.java | 109 ++++++++++ 8 files changed, 253 insertions(+), 116 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/ReadEntryUtils.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicRealBkTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 14d424dc7eacd..2f60eeff2fbd3 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -4050,6 +4050,8 @@ public static ManagedLedgerException createManagedLedgerException(int bkErrorCod public static ManagedLedgerException createManagedLedgerException(Throwable t) { if (t instanceof org.apache.bookkeeper.client.api.BKException) { return createManagedLedgerException(((org.apache.bookkeeper.client.api.BKException) t).getCode()); + } else if (t instanceof ManagedLedgerException) { + return (ManagedLedgerException) t; } else if (t instanceof CompletionException && !(t.getCause() instanceof CompletionException) /* check to avoid stackoverlflow */) { return createManagedLedgerException(t.getCause()); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java index 4f8f70bc81bab..92541a7a72578 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java @@ -79,7 +79,7 @@ public void invalidateEntriesBeforeTimestamp(long timestamp) { @Override public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boolean isSlowestReader, final AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { - lh.readAsync(firstEntry, lastEntry).thenAcceptAsync( + ReadEntryUtils.readAsync(ml, lh, firstEntry, lastEntry).thenAcceptAsync( ledgerEntries -> { List entries = new ArrayList<>(); long totalSize = 0; @@ -107,7 +107,7 @@ public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boole @Override public void asyncReadEntry(ReadHandle lh, Position position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { - lh.readAsync(position.getEntryId(), position.getEntryId()).whenCompleteAsync( + ReadEntryUtils.readAsync(ml, lh, position.getEntryId(), position.getEntryId()).whenCompleteAsync( (ledgerEntries, exception) -> { if (exception != null) { ml.invalidateLedgerHandle(lh); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java index 254a517786a55..cb006a5f0cea9 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java @@ -249,7 +249,7 @@ private void asyncReadEntry0(ReadHandle lh, Position position, final ReadEntryCa manager.mlFactoryMBean.recordCacheHit(cachedEntry.getLength()); callback.readEntryComplete(cachedEntry, ctx); } else { - lh.readAsync(position.getEntryId(), position.getEntryId()).thenAcceptAsync( + ReadEntryUtils.readAsync(ml, lh, position.getEntryId(), position.getEntryId()).thenAcceptAsync( ledgerEntries -> { try { Iterator iterator = ledgerEntries.iterator(); @@ -429,7 +429,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { CompletableFuture> readFromStorage(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry) { final int entriesToRead = (int) (lastEntry - firstEntry) + 1; - CompletableFuture> readResult = lh.readAsync(firstEntry, lastEntry) + CompletableFuture> readResult = ReadEntryUtils.readAsync(ml, lh, firstEntry, lastEntry) .thenApply( ledgerEntries -> { requireNonNull(ml.getName()); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/ReadEntryUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/ReadEntryUtils.java new file mode 100644 index 0000000000000..5cf5f053f0ce7 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/ReadEntryUtils.java @@ -0,0 +1,54 @@ +/* + * 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.bookkeeper.mledger.impl.cache; + +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.client.api.LedgerEntries; +import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerException; + +class ReadEntryUtils { + + static CompletableFuture readAsync(ManagedLedger ml, ReadHandle handle, long firstEntry, + long lastEntry) { + if (ml.getOptionalLedgerInfo(handle.getId()).isEmpty()) { + // The read handle comes from another managed ledger, in this case, we can only compare the entry range with + // the LAC of that read handle. Specifically, it happens when this method is called by a + // ReadOnlyManagedLedgerImpl object. + return handle.readAsync(firstEntry, lastEntry); + } + // Compare the entry range with the lastConfirmedEntry maintained by the managed ledger because the entry cache + // of `ShadowManagedLedgerImpl` reads entries via `ReadOnlyLedgerHandle`, which never updates `lastAddConfirmed` + final var lastConfirmedEntry = ml.getLastConfirmedEntry(); + if (lastConfirmedEntry == null) { + return CompletableFuture.failedFuture(new ManagedLedgerException( + "LastConfirmedEntry is null when reading ledger " + handle.getId())); + } + if (handle.getId() > lastConfirmedEntry.getLedgerId()) { + return CompletableFuture.failedFuture(new ManagedLedgerException("LastConfirmedEntry is " + + lastConfirmedEntry + " when reading ledger " + handle.getId())); + } + if (handle.getId() == lastConfirmedEntry.getLedgerId() && lastEntry > lastConfirmedEntry.getEntryId()) { + return CompletableFuture.failedFuture(new ManagedLedgerException("LastConfirmedEntry is " + + lastConfirmedEntry + " when reading entry " + lastEntry)); + } + return handle.readUnconfirmedAsync(firstEntry, lastEntry); + } +} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java index ece75a2de80d8..f00efb27ca5ab 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java @@ -28,6 +28,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -45,6 +46,7 @@ import org.apache.bookkeeper.mledger.impl.cache.EntryCache; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheDisabled; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.testng.Assert; import org.testng.annotations.Test; @@ -392,6 +394,9 @@ void entryCacheDisabledAsyncReadEntry() throws Exception { EntryCache entryCache = cacheManager.getEntryCache(ml1); final CountDownLatch counter = new CountDownLatch(1); + when(ml1.getLastConfirmedEntry()).thenReturn(PositionFactory.create(1L, 1L)); + when(ml1.getOptionalLedgerInfo(lh.getId())).thenReturn(Optional.of(mock( + MLDataFormats.ManagedLedgerInfo.LedgerInfo.class))); entryCache.asyncReadEntry(lh, PositionFactory.create(1L,1L), new AsyncCallbacks.ReadEntryCallback() { public void readEntryComplete(Entry entry, Object ctx) { Assert.assertNotEquals(entry, null); @@ -406,7 +411,7 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { }, null); counter.await(); - verify(lh).readAsync(anyLong(), anyLong()); + verify(lh).readUnconfirmedAsync(anyLong(), anyLong()); } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java index c8338798f271b..551aa80bc07dc 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java @@ -25,14 +25,16 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import io.netty.buffer.Unpooled; - import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; - +import java.util.concurrent.ExecutionException; +import java.util.function.Consumer; import lombok.Cleanup; import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException; import org.apache.bookkeeper.client.api.LedgerEntries; @@ -43,10 +45,11 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.cache.EntryCache; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; -import org.testng.Assert; import org.testng.annotations.Test; public class EntryCacheTest extends MockedBookKeeperTestCase { @@ -60,6 +63,8 @@ protected void setUpTestCase() throws Exception { when(ml.getExecutor()).thenReturn(executor); when(ml.getMbean()).thenReturn(new ManagedLedgerMBeanImpl(ml)); when(ml.getConfig()).thenReturn(new ManagedLedgerConfig()); + when(ml.getOptionalLedgerInfo(0L)).thenReturn(Optional.of(mock( + MLDataFormats.ManagedLedgerInfo.LedgerInfo.class))); } @Test(timeOut = 5000) @@ -76,22 +81,13 @@ public void testRead() throws Exception { entryCache.insert(EntryImpl.create(0, i, data)); } - final CountDownLatch counter = new CountDownLatch(1); - - entryCache.asyncReadEntry(lh, 0, 9, false, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - assertEquals(entries.size(), 10); - entries.forEach(Entry::release); - counter.countDown(); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - Assert.fail("should not have failed"); - } - }, null); - counter.await(); + when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 9)); + final var entries = readEntry(entryCache, lh, 0, 9, false, null); + assertEquals(entries.size(), 10); + entries.forEach(Entry::release); // Verify no entries were read from bookkeeper + verify(lh, never()).readUnconfirmedAsync(anyLong(), anyLong()); verify(lh, never()).readAsync(anyLong(), anyLong()); } @@ -109,19 +105,9 @@ public void testReadMissingBefore() throws Exception { entryCache.insert(EntryImpl.create(0, i, data)); } - final CountDownLatch counter = new CountDownLatch(1); - - entryCache.asyncReadEntry(lh, 0, 9, false, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - assertEquals(entries.size(), 10); - counter.countDown(); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - Assert.fail("should not have failed"); - } - }, null); - counter.await(); + when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 9)); + final var entries = readEntry(entryCache, lh, 0, 9, false, null); + assertEquals(entries.size(), 10); } @Test(timeOut = 5000) @@ -138,19 +124,9 @@ public void testReadMissingAfter() throws Exception { entryCache.insert(EntryImpl.create(0, i, data)); } - final CountDownLatch counter = new CountDownLatch(1); - - entryCache.asyncReadEntry(lh, 0, 9, false, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - assertEquals(entries.size(), 10); - counter.countDown(); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - Assert.fail("should not have failed"); - } - }, null); - counter.await(); + when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 9)); + final var entries = readEntry(entryCache, lh, 0, 9, false, null); + assertEquals(entries.size(), 10); } @Test(timeOut = 5000) @@ -168,19 +144,9 @@ public void testReadMissingMiddle() throws Exception { entryCache.insert(EntryImpl.create(0, 8, data)); entryCache.insert(EntryImpl.create(0, 9, data)); - final CountDownLatch counter = new CountDownLatch(1); - - entryCache.asyncReadEntry(lh, 0, 9, false, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - assertEquals(entries.size(), 10); - counter.countDown(); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - Assert.fail("should not have failed"); - } - }, null); - counter.await(); + when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 9)); + final var entries = readEntry(entryCache, lh, 0, 9, false, null); + assertEquals(entries.size(), 10); } @Test(timeOut = 5000) @@ -198,19 +164,9 @@ public void testReadMissingMultiple() throws Exception { entryCache.insert(EntryImpl.create(0, 5, data)); entryCache.insert(EntryImpl.create(0, 8, data)); - final CountDownLatch counter = new CountDownLatch(1); - - entryCache.asyncReadEntry(lh, 0, 9, false, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - assertEquals(entries.size(), 10); - counter.countDown(); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - Assert.fail("should not have failed"); - } - }, null); - counter.await(); + when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 9)); + final var entries = readEntry(entryCache, lh, 0, 9, false, null); + assertEquals(entries.size(), 10); } @Test @@ -222,19 +178,25 @@ public void testCachedReadReturnsDifferentByteBuffer() throws Exception { @Cleanup(value = "clear") EntryCache entryCache = cacheManager.getEntryCache(ml); - CompletableFuture> cacheMissFutureEntries = new CompletableFuture<>(); - - entryCache.asyncReadEntry(lh, 0, 1, true, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - cacheMissFutureEntries.complete(entries); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - cacheMissFutureEntries.completeExceptionally(exception); - } - }, null); - - List cacheMissEntries = cacheMissFutureEntries.get(); + readEntry(entryCache, lh, 0, 1, true, e -> { + assertTrue(e instanceof ManagedLedgerException); + assertTrue(e.getMessage().contains("LastConfirmedEntry is null when reading ledger 0")); + }); + + when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(-1, -1)); + readEntry(entryCache, lh, 0, 1, true, e -> { + assertTrue(e instanceof ManagedLedgerException); + assertTrue(e.getMessage().contains("LastConfirmedEntry is -1:-1 when reading ledger 0")); + }); + + when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 0)); + readEntry(entryCache, lh, 0, 1, true, e -> { + assertTrue(e instanceof ManagedLedgerException); + assertTrue(e.getMessage().contains("LastConfirmedEntry is 0:0 when reading entry 1")); + }); + + when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 1)); + List cacheMissEntries = readEntry(entryCache, lh, 0, 1, true, null); // Ensure first entry is 0 and assertEquals(cacheMissEntries.size(), 2); assertEquals(cacheMissEntries.get(0).getEntryId(), 0); @@ -243,19 +205,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { // Move the reader index to simulate consumption cacheMissEntries.get(0).getDataBuffer().readerIndex(10); - CompletableFuture> cacheHitFutureEntries = new CompletableFuture<>(); - - entryCache.asyncReadEntry(lh, 0, 1, true, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - cacheHitFutureEntries.complete(entries); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - cacheHitFutureEntries.completeExceptionally(exception); - } - }, null); - - List cacheHitEntries = cacheHitFutureEntries.get(); + List cacheHitEntries = readEntry(entryCache, lh, 0, 1, true, null); assertEquals(cacheHitEntries.get(0).getEntryId(), 0); assertEquals(cacheHitEntries.get(0).getDataBuffer().readerIndex(), 0); } @@ -269,7 +219,7 @@ public void testReadWithError() throws Exception { CompletableFuture future = new CompletableFuture<>(); future.completeExceptionally(new BKNoSuchLedgerExistsException()); return future; - }).when(lh).readAsync(anyLong(), anyLong()); + }).when(lh).readUnconfirmedAsync(anyLong(), anyLong()); EntryCacheManager cacheManager = factory.getEntryCacheManager(); @Cleanup(value = "clear") @@ -278,18 +228,9 @@ public void testReadWithError() throws Exception { byte[] data = new byte[10]; entryCache.insert(EntryImpl.create(0, 2, data)); - final CountDownLatch counter = new CountDownLatch(1); - - entryCache.asyncReadEntry(lh, 0, 9, false, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - Assert.fail("should not complete"); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - counter.countDown(); - } - }, null); - counter.await(); + when(ml.getLastConfirmedEntry()).thenReturn(PositionFactory.create(0, 9)); + readEntry(entryCache, lh, 0, 9, false, e -> + assertTrue(e instanceof ManagedLedgerException.LedgerNotExistException)); } static ReadHandle getLedgerHandle() { @@ -306,9 +247,35 @@ static ReadHandle getLedgerHandle() { LedgerEntries ledgerEntries = mock(LedgerEntries.class); doAnswer((invocation2) -> entries.iterator()).when(ledgerEntries).iterator(); return CompletableFuture.completedFuture(ledgerEntries); - }).when(lh).readAsync(anyLong(), anyLong()); + }).when(lh).readUnconfirmedAsync(anyLong(), anyLong()); return lh; } + private List readEntry(EntryCache entryCache, ReadHandle lh, long firstEntry, long lastEntry, + boolean shouldCacheEntry, Consumer assertion) + throws InterruptedException { + final var future = new CompletableFuture>(); + entryCache.asyncReadEntry(lh, firstEntry, lastEntry, shouldCacheEntry, new ReadEntriesCallback() { + @Override + public void readEntriesComplete(List entries, Object ctx) { + future.complete(entries); + } + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + }, null); + try { + final var entries = future.get(); + assertNull(assertion); + return entries; + } catch (ExecutionException e) { + if (assertion != null) { + assertion.accept(e.getCause()); + } + return List.of(); + } + } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java index 29138145d1505..6d8ecba868847 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java @@ -314,7 +314,7 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr @Override public CompletableFuture readUnconfirmedAsync(long firstEntry, long lastEntry) { - return unsupported(); + return readAsync(firstEntry, lastEntry); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicRealBkTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicRealBkTest.java new file mode 100644 index 0000000000000..9d810b06a7c7b --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicRealBkTest.java @@ -0,0 +1,109 @@ +/* + * 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.service.persistent; + +import com.google.common.collect.Lists; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.mledger.impl.ShadowManagedLedgerImpl; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.util.PortManager; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.awaitility.Awaitility; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class ShadowTopicRealBkTest { + + private static final String cluster = "test"; + private final int zkPort = PortManager.nextLockedFreePort(); + private final LocalBookkeeperEnsemble bk = new LocalBookkeeperEnsemble(2, zkPort, PortManager::nextLockedFreePort); + private PulsarService pulsar; + private PulsarAdmin admin; + + @BeforeClass + public void setup() throws Exception { + bk.start(); + final var config = new ServiceConfiguration(); + config.setClusterName(cluster); + config.setAdvertisedAddress("localhost"); + config.setBrokerServicePort(Optional.of(0)); + config.setWebServicePort(Optional.of(0)); + config.setMetadataStoreUrl("zk:localhost:" + zkPort); + pulsar = new PulsarService(config); + pulsar.start(); + admin = pulsar.getAdminClient(); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()) + .brokerServiceUrl(pulsar.getBrokerServiceUrl()).build()); + admin.tenants().createTenant("public", TenantInfo.builder().allowedClusters(Set.of(cluster)).build()); + admin.namespaces().createNamespace("public/default"); + } + + @AfterClass(alwaysRun = true) + public void cleanup() throws Exception { + if (pulsar != null) { + pulsar.close(); + } + bk.stop(); + } + + @Test + public void testReadFromStorage() throws Exception { + final var sourceTopic = TopicName.get("test-read-from-source").toString(); + final var shadowTopic = sourceTopic + "-shadow"; + + admin.topics().createNonPartitionedTopic(sourceTopic); + admin.topics().createShadowTopic(shadowTopic, sourceTopic); + admin.topics().setShadowTopics(sourceTopic, Lists.newArrayList(shadowTopic)); + + Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(()->{ + final var sourcePersistentTopic = (PersistentTopic) pulsar.getBrokerService() + .getTopicIfExists(sourceTopic).get().orElseThrow(); + final var replicator = (ShadowReplicator) sourcePersistentTopic.getShadowReplicators().get(shadowTopic); + Assert.assertNotNull(replicator); + Assert.assertEquals(String.valueOf(replicator.getState()), "Started"); + }); + + final var client = pulsar.getClient(); + // When the message was sent, there is no cursor, so it will read from the cache + final var producer = client.newProducer().topic(sourceTopic).create(); + producer.send("message".getBytes()); + // 1. Verify RangeEntryCacheImpl#readFromStorage + final var consumer = client.newConsumer().topic(shadowTopic).subscriptionName("sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe(); + final var msg = consumer.receive(5, TimeUnit.SECONDS); + Assert.assertNotNull(msg); + Assert.assertEquals(msg.getValue(), "message".getBytes()); + + // 2. Verify EntryCache#asyncReadEntry + final var shadowManagedLedger = ((PersistentTopic) pulsar.getBrokerService().getTopicIfExists(shadowTopic).get() + .orElseThrow()).getManagedLedger(); + Assert.assertTrue(shadowManagedLedger instanceof ShadowManagedLedgerImpl); + shadowManagedLedger.getEarliestMessagePublishTimeInBacklog().get(3, TimeUnit.SECONDS); + } +} From 46c25ac73427312db7f38e150cd797a8cee23f28 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov <8622884+dlg99@users.noreply.github.com> Date: Thu, 15 Aug 2024 10:33:56 -0700 Subject: [PATCH 444/580] [fix] DLQ to handle bytes key properly (#23172) --- .../client/api/DeadLetterTopicTest.java | 60 +++++++++++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 56 +++++++++-------- 2 files changed, 92 insertions(+), 24 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java index 4433670c7a595..83320fffa1a7f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java @@ -137,6 +137,66 @@ public void testDeadLetterTopicWithMessageKey() throws Exception { consumer.close(); } + @Test + public void testDeadLetterTopicWithBinaryMessageKey() throws Exception { + final String topic = "persistent://my-property/my-ns/dead-letter-topic"; + + final int maxRedeliveryCount = 1; + + final int sendMessages = 100; + + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) + .topic(topic) + .subscriptionName("my-subscription") + .subscriptionType(SubscriptionType.Shared) + .ackTimeout(1, TimeUnit.SECONDS) + .deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build()) + .receiverQueueSize(100) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + @Cleanup + PulsarClient newPulsarClient = newPulsarClient(lookupUrl.toString(), 0);// Creates new client connection + Consumer deadLetterConsumer = newPulsarClient.newConsumer(Schema.BYTES) + .topic("persistent://my-property/my-ns/dead-letter-topic-my-subscription-DLQ") + .subscriptionName("my-subscription") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .create(); + + byte[] key = new byte[]{1, 2, 3, 4}; + for (int i = 0; i < sendMessages; i++) { + producer.newMessage() + .keyBytes(key) + .value(String.format("Hello Pulsar [%d]", i).getBytes()) + .send(); + } + + producer.close(); + + int totalReceived = 0; + do { + Message message = consumer.receive(); + log.info("consumer received message : {} {}", message.getMessageId(), new String(message.getData())); + totalReceived++; + } while (totalReceived < sendMessages * (maxRedeliveryCount + 1)); + + int totalInDeadLetter = 0; + do { + Message message = deadLetterConsumer.receive(); + assertEquals(message.getKeyBytes(), key); + log.info("dead letter consumer received message : {} {}", message.getMessageId(), new String(message.getData())); + deadLetterConsumer.acknowledge(message); + totalInDeadLetter++; + } while (totalInDeadLetter < sendMessages); + + deadLetterConsumer.close(); + consumer.close(); + } + public void testDeadLetterTopicWithProducerName() throws Exception { final String topic = "persistent://my-property/my-ns/dead-letter-topic"; final String subscription = "my-subscription"; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 3acf55afaed51..36cd52f955409 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -237,6 +237,7 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private final AtomicReference clientCnxUsedForConsumerRegistration = new AtomicReference<>(); private final List previousExceptions = new CopyOnWriteArrayList(); private volatile boolean hasSoughtByTimestamp = false; + static ConsumerImpl newConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurationData conf, @@ -280,10 +281,12 @@ static ConsumerImpl newConsumerImpl(PulsarClientImpl client, } protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurationData conf, - ExecutorProvider executorProvider, int partitionIndex, boolean hasParentConsumer, - boolean parentConsumerHasListener, CompletableFuture> subscribeFuture, MessageId startMessageId, - long startMessageRollbackDurationInSec, Schema schema, ConsumerInterceptors interceptors, - boolean createTopicIfDoesNotExist) { + ExecutorProvider executorProvider, int partitionIndex, boolean hasParentConsumer, + boolean parentConsumerHasListener, CompletableFuture> subscribeFuture, + MessageId startMessageId, + long startMessageRollbackDurationInSec, Schema schema, + ConsumerInterceptors interceptors, + boolean createTopicIfDoesNotExist) { super(client, topic, conf, conf.getReceiverQueueSize(), executorProvider, subscribeFuture, schema, interceptors); this.consumerId = client.newConsumerId(); @@ -355,21 +358,21 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat } this.connectionHandler = new ConnectionHandler(this, - new BackoffBuilder() - .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), - TimeUnit.NANOSECONDS) - .setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) - .setMandatoryStop(0, TimeUnit.MILLISECONDS) - .create(), + new BackoffBuilder() + .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), + TimeUnit.NANOSECONDS) + .setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) + .setMandatoryStop(0, TimeUnit.MILLISECONDS) + .create(), this); this.topicName = TopicName.get(topic); if (this.topicName.isPersistent()) { this.acknowledgmentsGroupingTracker = - new PersistentAcknowledgmentsGroupingTracker(this, conf, client.eventLoopGroup()); + new PersistentAcknowledgmentsGroupingTracker(this, conf, client.eventLoopGroup()); } else { this.acknowledgmentsGroupingTracker = - NonPersistentAcknowledgmentGroupingTracker.of(); + NonPersistentAcknowledgmentGroupingTracker.of(); } if (conf.getDeadLetterPolicy() != null) { @@ -470,16 +473,16 @@ public CompletableFuture unsubscribeAsync(boolean force) { log.error("[{}][{}] Failed to unsubscribe: {}", topic, subscription, e.getCause().getMessage()); setState(State.Ready); unsubscribeFuture.completeExceptionally( - PulsarClientException.wrap(e.getCause(), - String.format("Failed to unsubscribe the subscription %s of topic %s", - subscription, topicName.toString()))); + PulsarClientException.wrap(e.getCause(), + String.format("Failed to unsubscribe the subscription %s of topic %s", + subscription, topicName.toString()))); return null; }); } else { unsubscribeFuture.completeExceptionally( - new PulsarClientException.NotConnectedException( - String.format("The client is not connected to the broker when unsubscribing the " - + "subscription %s of the topic %s", subscription, topicName.toString()))); + new PulsarClientException.NotConnectedException( + String.format("The client is not connected to the broker when unsubscribing the " + + "subscription %s of the topic %s", subscription, topicName.toString()))); } return unsubscribeFuture; } @@ -636,6 +639,15 @@ protected CompletableFuture doAcknowledge(List messageIdList, A } } + private static void copyMessageKeyIfNeeded(Message message, TypedMessageBuilder typedMessageBuilderNew) { + if (message.hasKey()) { + if (message.hasBase64EncodedKey()) { + typedMessageBuilderNew.keyBytes(message.getKeyBytes()); + } else { + typedMessageBuilderNew.key(message.getKey()); + } + } + } @SuppressWarnings("unchecked") @Override @@ -720,9 +732,7 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a if (delayTime > 0) { typedMessageBuilderNew.deliverAfter(delayTime, unit); } - if (message.hasKey()) { - typedMessageBuilderNew.key(message.getKey()); - } + copyMessageKeyIfNeeded(message, typedMessageBuilderNew); typedMessageBuilderNew.sendAsync() .thenCompose(__ -> doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null)) .thenAccept(v -> result.complete(null)) @@ -2186,9 +2196,7 @@ private CompletableFuture processPossibleToDLQ(MessageIdAdv messageId) producerDLQ.newMessage(Schema.AUTO_PRODUCE_BYTES(message.getReaderSchema().get())) .value(message.getData()) .properties(getPropertiesMap(message, originMessageIdStr, originTopicNameStr)); - if (message.hasKey()) { - typedMessageBuilderNew.key(message.getKey()); - } + copyMessageKeyIfNeeded(message, typedMessageBuilderNew); typedMessageBuilderNew.sendAsync() .thenAccept(messageIdInDLQ -> { possibleSendToDeadLetterTopicMessages.remove(messageId); From 1f90897c890a3b41153b332264624916b926f3a7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 16 Aug 2024 10:43:45 +0800 Subject: [PATCH 445/580] [improve] [pip] PIP-370: configurable remote topic creation in geo-replication (#23124) --- pip/pip-370.md | 109 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 109 insertions(+) create mode 100644 pip/pip-370.md diff --git a/pip/pip-370.md b/pip/pip-370.md new file mode 100644 index 0000000000000..6699846cee105 --- /dev/null +++ b/pip/pip-370.md @@ -0,0 +1,109 @@ +# PIP-370: configurable remote topic creation in geo-replication + +# Background knowledge + +**The current topic creation behavior when enabling Geo-Replication** +Users using Geo-Replication backup data across multiple clusters, as well as Admin APIs related to Geo-Replication and internal replicators of brokers, will trigger topics of auto-creation between clusters. +- For partitioned topics. + - After enabling namespace-level Geo-Replication: the broker will create topics on the remote cluster automatically when calling `pulsar-admin topics create-partitioned-topic`. It does not depend on enabling `allowAutoTopicCreation`. + - When enabling topic-level Geo-Replication on a partitioned topic: the broker will create topics on the remote cluster automatically. It does not depend on enabling `allowAutoTopicCreation`. + - When calling `pulsar-admin topics update-partitioned-topic -p {partitions}`, the broker will also update partitions on the remote cluster automatically. +- For non-partitioned topics and partitions of partitioned topics. + - The internal Geo-Replicator will trigger topics auto-creation for remote clusters. **(Highlight)** It depends on enabling `allowAutoTopicCreation`. In fact, this behavior is not related to Geo-Replication, it is the behavior of the internal producer of Geo-Replicator, + +# Motivation + +In the following scenarios, automatic topic creation across clusters is problematic due to race conditions during deployments, and there is no choice that prevents pulsar resource creation affects each other between clusters. + +- Users want to maintain pulsar resources manually. +- Users pulsar resources using `GitOps CD` automated deployment, for which + - Clusters are deployed simultaneously without user intervention. + - Each cluster is precisely configured from git repo config variables - including the list of all tenants/namespaces/topics to be created in each cluster. + - Clusters are configured to be exact clones of each other in terms of pulsar resources. + +**Passed solution**: disable `allowAutoTopicCreation`, the APIs `pulsar-admin topics create-partitioned-topic` still create topics on the remote cluster when enabled namespace level replication, the API `enable topic-level replication` still create topics, And the internal replicator will keep printing error logs due to a not found error. + +# Goals + +- **Phase 1**: Introduce a flag to disable the replicators to automatically trigger topic creation. +- **Phase 2**: Move all topic creation/expand-partitions behaviors related to Replication to the internal Replicator, pulsar admin API that relates to pulsar topics management does not care about replication anymore. + - Move the topic creation operations from `pulsar-admin topics create-partitioned-topic` and `pulsar-admin topics set-replication-clusters` to the component Replicator in the broker internal. + - (The same as before)When calling `pulsar-admin topics update-partitioned-topic -p {partitions}`, the broker will also update partitions on the remote cluster automatically. + +Note: the proposal will only focus on phase one, and the detailed design for phase two with come up with another proposal. + +# Detailed Design + +## Configuration + +**broker.conf** +```properties +# Whether the internal replication of the local cluster will trigger topic auto-creation on the remote cluster. +# 1. After enabling namespace-level Geo-Replication: whether the local broker will create topics on the remote cluster automatically when calling `pulsar-admin topics create-partitioned-topic`. +# 2. When enabling topic-level Geo-Replication on a partitioned topic: whether the local broker will create topics on the remote cluster. +# 3. Whether the internal Geo-Replicator in the local cluster will trigger non-persistent topic auto-creation for remote clusters. +# It is not a dynamic config, the default value is "true" to preserve backward-compatible behavior. +createTopicToRemoteClusterForReplication=true +``` + +## Design & Implementation Details + +### Phase 1: Introduce a flag to disable the replicators to automatically trigger topic creation. +- If `createTopicToRemoteClusterForReplication` is set to `false`. + 1. After enabling namespace-level Geo-Replication: the broker will not create topics on the remote cluster automatically when calling `pulsar-admin topics create-partitioned-topic`. + 2. When enabling topic-level Geo-Replication on a partitioned topic: broker will not create topics on the remote cluster automatically. + 3. The internal Geo-Replicator will not trigger topic auto-creation for remote clusters, it just keeps retrying to check if the topic exists on the remote cluster, once the topic is created, the replicator starts. + 4. It does not change the behavior of creating subscriptions after enabling `enableReplicatedSubscriptions`, the subscription will also be created on the remote cluster after users enable. `enableReplicatedSubscriptions`. + 5. The config `allowAutoTopicCreation` still works for the local cluster as before, it will not be affected by the new config `createTopicToRemoteClusterForReplication`. +- If `createTopicToRemoteClusterForReplication` is set to `true`. + a. All components work as before, see details: `Motivation -> The current topic creation behavior when enabling Geo-Replication` + +### Phase 2: The replicator will check remote topics' partitioned metadata and update partitions in the remote cluster to the same as the current cluster if needed. +- If `createTopicToRemoteClusterForReplication` is set to `false`. + - The behavior is the same as Phase 1. +- If `createTopicToRemoteClusterForReplication` is set to `true`. + - Pulsar admin API that relates to pulsar topics management does not care about replication anymore. + - When a replicator for a topic partition starts, it checks the partitioned metadata in the remote cluster first and updates partitions in the remote cluster to the same as the current cluster if needed. Seem the example as follows: + +| `partitions` of local cluster | `partitions` of remote cluster | After `PIP-370 Phase 2` | Before `PIP-370 Phase 2` | +| --- | --- | --- | --- | +| `2` | no topic exists | create a partitioned topic with `2` partitions in the remote cluster | the replicator will only trigger partition creation (`{topic}-partition-0` and `{topic}-partition-1`), and will not care about partitioned metadata. | +| `2` | `1`| **In dispute:** The replicator copies messages from `partition-0` to the remote cluster, does not copy any data for `partition-1` and just prints error logs in the background. | the replicator will only trigger partition creation (`{topic}-partition-0` and `{topic}-partition-1`), and the partitioned metadata in the remote cluster is still `1` | +| `2` | `2` | modifies nothing. | The same as "After `PIP-370 Phase 2`" | +| `2` | `>2` | **In dispute:** modifies nothing, the messages will be copied to the same partition in the remote cluster, and no message will be copied to the partition who is larger than `2` in the remote cluster | The same as "After `PIP-370 Phase 2`" | +| `2` | `0`(non-partitioned topic) | **In dispute:** The replicator does not copy any data and just prints error logs in the background. | the replicator will only trigger partition creation (`{topic}-partition-0` and `{topic}-partition-1`), then users will get `3` non-partitioned topics: `[{tp}, {topic}-partition-0, {topic}-partition-1`. | +| `0`(non-partitioned topic) | `0`(non-partitioned topic) | Copy data normally | It is the same as before `PIP-370`. | +| `0`(non-partitioned topic) | no topic exists | create a non-partitioned topic in the remote cluster. | It is the same as before `PIP-370`. | +| `0`(non-partitioned topic) | `>=1` | **In dispute:** The replicator does not copy any data and just prints error logs in the background. | The replicator will only trigger a non-partitioned topic's creation, then users will get `1` non-partitioned topic and `1` partitioned topic. | + +## Metrics + + +| Name | Description | Attributes | Units| +| --- | --- | --- | --- | +| `pulsar_broker_replication_count` | Counter. The number of topics enabled replication. | cluster | - | +| `pulsar_broker_replication_disconnected_count` | Counter. The number of topics that enabled replication and its replicator failed to connect | cluster | - | + + +# Monitoring +- If `pulsar_broker_replication_disconnected_count` keeps larger than `0` for a period of time, it means some replicators do not work, we should push an alert out. + +# Backward & Forward Compatibility + +## Regarding to Phase-1 +This PIP guarantees full compatibility with default settings(the default value of `createTopicToRemoteClusterForReplication` is `true`). If you want to cherry-pick PIP-370 for another branch in the future, you need to cherry-pick PIP-344 as well. Because the behavior of disables `createTopicToRemoteClusterForReplication` depends on the API `PulsarClient.getPartitionsForTopic(String topic, boolean metadataAutoCreationEnabled)`, which was introduced by [PIP-344](https://github.com/apache/pulsar/blob/master/pip/pip-344.md). + +## Regarding to Phase-2 +The two scenarios are as follows, the replication will not work as before, which will lead backlog increase, please take care of checking your clusters before upgrading. +- `local_cluster.topic.partitions = 2` and `remote_cluster.topic.partitions = 0(non-partitioned topic)`: see detail in the section `Design & Implementation Details -> Phase-2`. +- `local_cluster.topic.partitions = 0(non-partitioned topic)` and and `remote_cluster.topic.partitions >= 1`: see detail in the section `Design & Implementation Details -> Phase-2`. + +# Links +* Mailing List discussion thread: https://lists.apache.org/thread/9fx354cqcy3412w1nx8kwdf9h141omdg +* Mailing List voting thread: https://lists.apache.org/thread/vph22st5td1rdh1gd68gkrnp9doo6ct2 From 67fc5b9f5342bd35d3fdacf37cf172a629ee15f9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 16 Aug 2024 06:33:18 +0300 Subject: [PATCH 446/580] [fix][client] Copy orderingKey to retry letter topic and DLQ messages and fix bug in copying (#23182) Fixes #23173 Fixes #23181 ### Motivation See #23173 and #23181 ### Modifications - copy ordering key to messages sent to retry letter topic and DLQ topic --- .../client/api/DeadLetterTopicTest.java | 60 +++++++++++++++++++ .../pulsar/client/api/RetryTopicTest.java | 17 +++++- .../pulsar/client/impl/ConsumerImpl.java | 10 +++- 3 files changed, 83 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java index 83320fffa1a7f..f5a74dcd1661b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DeadLetterTopicTest.java @@ -197,6 +197,66 @@ public void testDeadLetterTopicWithBinaryMessageKey() throws Exception { consumer.close(); } + @Test + public void testDeadLetterTopicMessagesWithOrderingKey() throws Exception { + final String topic = "persistent://my-property/my-ns/dead-letter-topic"; + + final int maxRedeliveryCount = 1; + + final int sendMessages = 100; + + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) + .topic(topic) + .subscriptionName("my-subscription") + .subscriptionType(SubscriptionType.Shared) + .ackTimeout(1, TimeUnit.SECONDS) + .deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build()) + .receiverQueueSize(100) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + @Cleanup + PulsarClient newPulsarClient = newPulsarClient(lookupUrl.toString(), 0);// Creates new client connection + Consumer deadLetterConsumer = newPulsarClient.newConsumer(Schema.BYTES) + .topic("persistent://my-property/my-ns/dead-letter-topic-my-subscription-DLQ") + .subscriptionName("my-subscription") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .create(); + + byte[] key = new byte[]{1, 2, 3, 4}; + for (int i = 0; i < sendMessages; i++) { + producer.newMessage() + .orderingKey(key) + .value(String.format("Hello Pulsar [%d]", i).getBytes()) + .send(); + } + + producer.close(); + + int totalReceived = 0; + do { + Message message = consumer.receive(); + log.info("consumer received message : {} {}", message.getMessageId(), new String(message.getData())); + totalReceived++; + } while (totalReceived < sendMessages * (maxRedeliveryCount + 1)); + + int totalInDeadLetter = 0; + do { + Message message = deadLetterConsumer.receive(); + assertEquals(message.getOrderingKey(), key); + log.info("dead letter consumer received message : {} {}", message.getMessageId(), new String(message.getData())); + deadLetterConsumer.acknowledge(message); + totalInDeadLetter++; + } while (totalInDeadLetter < sendMessages); + + deadLetterConsumer.close(); + consumer.close(); + } + public void testDeadLetterTopicWithProducerName() throws Exception { final String topic = "persistent://my-property/my-ns/dead-letter-topic"; final String subscription = "my-subscription"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java index 2ccae72143443..9cb82fde04118 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java @@ -257,6 +257,9 @@ public void testAutoConsumeSchemaRetryLetter() throws Exception { public void testRetryTopicProperties() throws Exception { final String topic = "persistent://my-property/my-ns/retry-topic"; + byte[] key = "key".getBytes(); + byte[] orderingKey = "orderingKey".getBytes(); + final int maxRedeliveryCount = 3; final int sendMessages = 10; @@ -285,7 +288,11 @@ public void testRetryTopicProperties() throws Exception { Set originMessageIds = new HashSet<>(); for (int i = 0; i < sendMessages; i++) { - MessageId msgId = producer.send(String.format("Hello Pulsar [%d]", i).getBytes()); + MessageId msgId = producer.newMessage() + .value(String.format("Hello Pulsar [%d]", i).getBytes()) + .keyBytes(key) + .orderingKey(orderingKey) + .send(); originMessageIds.add(msgId.toString()); } @@ -298,6 +305,10 @@ public void testRetryTopicProperties() throws Exception { if (message.hasProperty(RetryMessageUtil.SYSTEM_PROPERTY_RECONSUMETIMES)) { // check the REAL_TOPIC property assertEquals(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_REAL_TOPIC), topic); + assertTrue(message.hasKey()); + assertEquals(message.getKeyBytes(), key); + assertTrue(message.hasOrderingKey()); + assertEquals(message.getOrderingKey(), orderingKey); retryMessageIds.add(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID)); } consumer.reconsumeLater(message, 1, TimeUnit.SECONDS); @@ -317,6 +328,10 @@ public void testRetryTopicProperties() throws Exception { if (message.hasProperty(RetryMessageUtil.SYSTEM_PROPERTY_RECONSUMETIMES)) { // check the REAL_TOPIC property assertEquals(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_REAL_TOPIC), topic); + assertTrue(message.hasKey()); + assertEquals(message.getKeyBytes(), key); + assertTrue(message.hasOrderingKey()); + assertEquals(message.getOrderingKey(), orderingKey); deadLetterMessageIds.add(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID)); } deadLetterConsumer.acknowledge(message); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 36cd52f955409..596e65484d1b2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -639,7 +639,7 @@ protected CompletableFuture doAcknowledge(List messageIdList, A } } - private static void copyMessageKeyIfNeeded(Message message, TypedMessageBuilder typedMessageBuilderNew) { + private static void copyMessageKeysIfNeeded(Message message, TypedMessageBuilder typedMessageBuilderNew) { if (message.hasKey()) { if (message.hasBase64EncodedKey()) { typedMessageBuilderNew.keyBytes(message.getKeyBytes()); @@ -647,6 +647,9 @@ private static void copyMessageKeyIfNeeded(Message message, TypedMessageBuild typedMessageBuilderNew.key(message.getKey()); } } + if (message.hasOrderingKey()) { + typedMessageBuilderNew.orderingKey(message.getOrderingKey()); + } } @SuppressWarnings("unchecked") @@ -704,6 +707,7 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a dlqProducer.newMessage(Schema.AUTO_PRODUCE_BYTES(retryMessage.getReaderSchema().get())) .value(retryMessage.getData()) .properties(propertiesMap); + copyMessageKeysIfNeeded(message, typedMessageBuilderNew); typedMessageBuilderNew.sendAsync().thenAccept(msgId -> { consumerDlqMessagesCounter.increment(); @@ -732,7 +736,7 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a if (delayTime > 0) { typedMessageBuilderNew.deliverAfter(delayTime, unit); } - copyMessageKeyIfNeeded(message, typedMessageBuilderNew); + copyMessageKeysIfNeeded(message, typedMessageBuilderNew); typedMessageBuilderNew.sendAsync() .thenCompose(__ -> doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null)) .thenAccept(v -> result.complete(null)) @@ -2196,7 +2200,7 @@ private CompletableFuture processPossibleToDLQ(MessageIdAdv messageId) producerDLQ.newMessage(Schema.AUTO_PRODUCE_BYTES(message.getReaderSchema().get())) .value(message.getData()) .properties(getPropertiesMap(message, originMessageIdStr, originTopicNameStr)); - copyMessageKeyIfNeeded(message, typedMessageBuilderNew); + copyMessageKeysIfNeeded(message, typedMessageBuilderNew); typedMessageBuilderNew.sendAsync() .thenAccept(messageIdInDLQ -> { possibleSendToDeadLetterTopicMessages.remove(messageId); From 3ada56635a6bf54eccdbaa572b6a023baa6f9bfa Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 16 Aug 2024 07:48:18 +0300 Subject: [PATCH 447/580] [feat] Add scripts for updating BK RocksDB ini files (#23178) --- docker/pulsar/scripts/update-ini-from-env.py | 70 +++++++++++++++ .../scripts/update-rocksdb-conf-from-env.py | 86 +++++++++++++++++++ 2 files changed, 156 insertions(+) create mode 100755 docker/pulsar/scripts/update-ini-from-env.py create mode 100755 docker/pulsar/scripts/update-rocksdb-conf-from-env.py diff --git a/docker/pulsar/scripts/update-ini-from-env.py b/docker/pulsar/scripts/update-ini-from-env.py new file mode 100755 index 0000000000000..6b0d7a795c3f8 --- /dev/null +++ b/docker/pulsar/scripts/update-ini-from-env.py @@ -0,0 +1,70 @@ +#!/usr/bin/env python3 +# +# 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. +# + +import os +import sys +import configparser +import re + +def get_first_word(section_name): + # Split the section name by any non-word character and return the first word + return re.split(r'\W+', section_name)[0] + +def update_ini_file(ini_file_path, env_prefix): + # Read the existing INI file + config = configparser.ConfigParser() + config.read(ini_file_path) + + # Flag to track if any updates were made + updated = False + + # Iterate over environment variables + for key, value in os.environ.items(): + if env_prefix and not key.startswith(env_prefix): + continue + + stripped_key = key[len(env_prefix):] if env_prefix else key + + # Iterate through sections + for section in config.sections(): + first_word = get_first_word(section) + prefix = first_word + '_' + if stripped_key.startswith(prefix): + config.set(section, stripped_key[len(prefix):], value) + updated = True + break + elif config.has_option(section, stripped_key): + config.set(section, stripped_key, value) + updated = True + break + + # Write the updated INI file only if there were updates + if updated: + with open(ini_file_path, 'w') as configfile: + config.write(configfile) + +if __name__ == "__main__": + if len(sys.argv) != 3: + print("Usage: python3 update-ini-from-env.py ") + sys.exit(1) + + ini_file_path = sys.argv[1] + env_prefix = sys.argv[2] + update_ini_file(ini_file_path, env_prefix) \ No newline at end of file diff --git a/docker/pulsar/scripts/update-rocksdb-conf-from-env.py b/docker/pulsar/scripts/update-rocksdb-conf-from-env.py new file mode 100755 index 0000000000000..2e55b455de3b7 --- /dev/null +++ b/docker/pulsar/scripts/update-rocksdb-conf-from-env.py @@ -0,0 +1,86 @@ +#!/usr/bin/env python3 +# +# 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. +# + +# allows tuning of RocksDB configuration via environment variables which were effective +# before Pulsar 2.11 / BookKeeper 4.15 / https://github.com/apache/bookkeeper/pull/3056 +# the script should be applied to the `conf/entry_location_rocksdb.conf` file + +import os +import sys +import configparser + +# Constants for section keys +DB_OPTIONS = "DBOptions" +CF_OPTIONS = "CFOptions \"default\"" +TABLE_OPTIONS = "TableOptions/BlockBasedTable \"default\"" + +def update_ini_file(ini_file_path): + config = configparser.ConfigParser() + config.read(ini_file_path) + updated = False + + # Mapping of environment variables to INI sections and keys + env_to_ini_mapping = { + "dbStorage_rocksDB_logPath": (DB_OPTIONS, "log_path"), + "dbStorage_rocksDB_logLevel": (DB_OPTIONS, "info_log_level"), + "dbStorage_rocksDB_lz4CompressionEnabled": (CF_OPTIONS, "compression"), + "dbStorage_rocksDB_writeBufferSizeMB": (CF_OPTIONS, "write_buffer_size"), + "dbStorage_rocksDB_sstSizeInMB": (CF_OPTIONS, "target_file_size_base"), + "dbStorage_rocksDB_blockSize": (TABLE_OPTIONS, "block_size"), + "dbStorage_rocksDB_bloomFilterBitsPerKey": (TABLE_OPTIONS, "filter_policy"), + "dbStorage_rocksDB_blockCacheSize": (TABLE_OPTIONS, "block_cache"), + "dbStorage_rocksDB_numLevels": (CF_OPTIONS, "num_levels"), + "dbStorage_rocksDB_numFilesInLevel0": (CF_OPTIONS, "level0_file_num_compaction_trigger"), + "dbStorage_rocksDB_maxSizeInLevel1MB": (CF_OPTIONS, "max_bytes_for_level_base"), + "dbStorage_rocksDB_format_version": (TABLE_OPTIONS, "format_version") + } + + # Type conversion functions + def mb_to_bytes(mb): + return str(int(mb) * 1024 * 1024) + + def str_to_bool(value): + return True if value.lower() in ["true", "1", "yes"] else False + + # Iterate over environment variables + for key, value in os.environ.items(): + if key.startswith("PULSAR_PREFIX_"): + key = key[len("PULSAR_PREFIX_"):] + + if key in env_to_ini_mapping: + section, option = env_to_ini_mapping[key] + if key in ["dbStorage_rocksDB_writeBufferSizeMB", "dbStorage_rocksDB_sstSizeInMB", "dbStorage_rocksDB_maxSizeInLevel1MB"]: + value = mb_to_bytes(value) + elif key == "dbStorage_rocksDB_lz4CompressionEnabled": + value = "kLZ4Compression" if str_to_bool(value) else "kNoCompression" + elif key == "dbStorage_rocksDB_bloomFilterBitsPerKey": + value = "rocksdb.BloomFilter:{}:false".format(value) + if config.get(section, option, fallback=None) != value: + config.set(section, option, value) + updated = True + + # Write the updated INI file only if there were updates + if updated: + with open(ini_file_path, 'w') as configfile: + config.write(configfile) + +if __name__ == "__main__": + ini_file_path = sys.argv[1] if len(sys.argv) > 1 else "conf/entry_location_rocksdb.conf" + update_ini_file(ini_file_path) \ No newline at end of file From b6815d2163b4632eea17a473ecb2fcbde394b1f7 Mon Sep 17 00:00:00 2001 From: hrsakai Date: Fri, 16 Aug 2024 13:54:56 +0900 Subject: [PATCH 448/580] [fix][sec]Upgrade jackson to 2.17.2 (#23174) --- .../server/src/assemble/LICENSE.bin.txt | 22 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 22 +++++++++---------- pom.xml | 4 ++-- .../pulsar/common/util/FieldParser.java | 7 ++---- 4 files changed, 26 insertions(+), 29 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 505c4b30093e7..d738b4a5027dc 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -249,17 +249,17 @@ The Apache Software License, Version 2.0 - info.picocli-picocli-shell-jline3-4.7.5.jar * High Performance Primitive Collections for Java -- com.carrotsearch-hppc-0.9.1.jar * Jackson - - com.fasterxml.jackson.core-jackson-annotations-2.14.2.jar - - com.fasterxml.jackson.core-jackson-core-2.14.2.jar - - com.fasterxml.jackson.core-jackson-databind-2.14.2.jar - - com.fasterxml.jackson.dataformat-jackson-dataformat-yaml-2.14.2.jar - - com.fasterxml.jackson.jaxrs-jackson-jaxrs-base-2.14.2.jar - - com.fasterxml.jackson.jaxrs-jackson-jaxrs-json-provider-2.14.2.jar - - com.fasterxml.jackson.module-jackson-module-jaxb-annotations-2.14.2.jar - - com.fasterxml.jackson.module-jackson-module-jsonSchema-2.14.2.jar - - com.fasterxml.jackson.datatype-jackson-datatype-jdk8-2.14.2.jar - - com.fasterxml.jackson.datatype-jackson-datatype-jsr310-2.14.2.jar - - com.fasterxml.jackson.module-jackson-module-parameter-names-2.14.2.jar + - com.fasterxml.jackson.core-jackson-annotations-2.17.2.jar + - com.fasterxml.jackson.core-jackson-core-2.17.2.jar + - com.fasterxml.jackson.core-jackson-databind-2.17.2.jar + - com.fasterxml.jackson.dataformat-jackson-dataformat-yaml-2.17.2.jar + - com.fasterxml.jackson.jaxrs-jackson-jaxrs-base-2.17.2.jar + - com.fasterxml.jackson.jaxrs-jackson-jaxrs-json-provider-2.17.2.jar + - com.fasterxml.jackson.module-jackson-module-jaxb-annotations-2.17.2.jar + - com.fasterxml.jackson.module-jackson-module-jsonSchema-2.17.2.jar + - com.fasterxml.jackson.datatype-jackson-datatype-jdk8-2.17.2.jar + - com.fasterxml.jackson.datatype-jackson-datatype-jsr310-2.17.2.jar + - com.fasterxml.jackson.module-jackson-module-parameter-names-2.17.2.jar * Caffeine -- com.github.ben-manes.caffeine-caffeine-2.9.1.jar * Conscrypt -- org.conscrypt-conscrypt-openjdk-uber-2.5.2.jar * Proto Google Common Protos -- com.google.api.grpc-proto-google-common-protos-2.17.0.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 31acbd9ac161d..944c4901cf1b9 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -313,17 +313,17 @@ The Apache Software License, Version 2.0 - picocli-4.7.5.jar - picocli-shell-jline3-4.7.5.jar * Jackson - - jackson-annotations-2.14.2.jar - - jackson-core-2.14.2.jar - - jackson-databind-2.14.2.jar - - jackson-dataformat-yaml-2.14.2.jar - - jackson-jaxrs-base-2.14.2.jar - - jackson-jaxrs-json-provider-2.14.2.jar - - jackson-module-jaxb-annotations-2.14.2.jar - - jackson-module-jsonSchema-2.14.2.jar - - jackson-datatype-jdk8-2.14.2.jar - - jackson-datatype-jsr310-2.14.2.jar - - jackson-module-parameter-names-2.14.2.jar + - jackson-annotations-2.17.2.jar + - jackson-core-2.17.2.jar + - jackson-databind-2.17.2.jar + - jackson-dataformat-yaml-2.17.2.jar + - jackson-jaxrs-base-2.17.2.jar + - jackson-jaxrs-json-provider-2.17.2.jar + - jackson-module-jaxb-annotations-2.17.2.jar + - jackson-module-jsonSchema-2.17.2.jar + - jackson-datatype-jdk8-2.17.2.jar + - jackson-datatype-jsr310-2.17.2.jar + - jackson-module-parameter-names-2.17.2.jar * Conscrypt -- conscrypt-openjdk-uber-2.5.2.jar * Gson - gson-2.8.9.jar diff --git a/pom.xml b/pom.xml index 13bac3f639d1d..4fa0df897689d 100644 --- a/pom.xml +++ b/pom.xml @@ -163,7 +163,7 @@ flexible messaging model and an intuitive client API. 1.78.1 1.0.7 1.0.2.5 - 2.14.2 + 2.17.2 0.10.2 1.6.2 10.14.2 @@ -301,7 +301,7 @@ flexible messaging model and an intuitive client API. 3.11.0 3.5.0 2.3.0 - 3.4.1 + 3.6.0 3.1.0 3.6.0 1.1.0 diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/FieldParser.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/FieldParser.java index 8d1ae5294ff7b..10c1951ab208b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/FieldParser.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/FieldParser.java @@ -21,8 +21,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.lang.String.format; import static java.util.Objects.requireNonNull; -import com.fasterxml.jackson.databind.AnnotationIntrospector; -import com.fasterxml.jackson.databind.introspect.JacksonAnnotationIntrospector; import com.fasterxml.jackson.databind.util.EnumResolver; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -58,8 +56,6 @@ public final class FieldParser { private static final Map CONVERTERS = new HashMap<>(); private static final Map, Class> WRAPPER_TYPES = new HashMap<>(); - private static final AnnotationIntrospector ANNOTATION_INTROSPECTOR = new JacksonAnnotationIntrospector(); - static { // Preload converters and wrapperTypes. initConverters(); @@ -100,7 +96,8 @@ public static T convert(Object from, Class to) { if (to.isEnum()) { // Converting string to enum - EnumResolver r = EnumResolver.constructUsingToString((Class>) to, ANNOTATION_INTROSPECTOR); + EnumResolver r = EnumResolver.constructUsingToString( + ObjectMapperFactory.getMapper().getObjectMapper().getDeserializationConfig(), to); T value = (T) r.findEnum((String) from); if (value == null) { throw new RuntimeException("Invalid value '" + from + "' for enum " + to); From a1f3322ed358ab6841f0d3e43f2afcc54788b887 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 16 Aug 2024 10:40:28 +0300 Subject: [PATCH 449/580] [fix][test] Fix flaky SubscriptionSeekTest.testSeekIsByReceive (#23170) --- .../broker/service/SubscriptionSeekTest.java | 37 +++++++++++++++---- 1 file changed, 30 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java index 3fc795a8c3e2a..582d10294a5a4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java @@ -86,9 +86,11 @@ protected void cleanup() throws Exception { public void testSeek() throws Exception { final String topicName = "persistent://prop/use/ns-abc/testSeek"; + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); // Disable pre-fetch in consumer to track the messages received + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) .subscriptionName("my-subscription").receiverQueueSize(0).subscribe(); @@ -138,11 +140,13 @@ public void testSeek() throws Exception { @Test public void testSeekIsByReceive() throws PulsarClientException { - final String topicName = "persistent://prop/use/ns-abc/testSeek"; + final String topicName = "persistent://prop/use/ns-abc/testSeekIsByReceive"; + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); String subscriptionName = "my-subscription"; + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) .subscriptionName(subscriptionName) .subscribe(); @@ -164,6 +168,7 @@ public void testSeekForBatch() throws Exception { final String topicName = "persistent://prop/use/ns-abcd/testSeekForBatch"; String subscriptionName = "my-subscription-batch"; + @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING) .enableBatching(true) .batchingMaxMessages(3) @@ -190,6 +195,7 @@ public void testSeekForBatch() throws Exception { producer.close(); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer(Schema.STRING) .topic(topicName) .subscriptionName(subscriptionName) @@ -220,6 +226,7 @@ public void testSeekForBatchMessageAndSpecifiedBatchIndex() throws Exception { final String topicName = "persistent://prop/use/ns-abcd/testSeekForBatchMessageAndSpecifiedBatchIndex"; String subscriptionName = "my-subscription-batch"; + @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING) .enableBatching(true) .batchingMaxMessages(3) @@ -264,6 +271,7 @@ public void testSeekForBatchMessageAndSpecifiedBatchIndex() throws Exception { .serviceUrl(lookupUrl.toString()) .build(); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = newPulsarClient.newConsumer(Schema.STRING) .topic(topicName) .subscriptionName(subscriptionName) @@ -300,6 +308,7 @@ public void testSeekForBatchByAdmin() throws PulsarClientException, ExecutionExc final String topicName = "persistent://prop/use/ns-abcd/testSeekForBatchByAdmin-" + UUID.randomUUID().toString(); String subscriptionName = "my-subscription-batch"; + @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING) .enableBatching(true) .batchingMaxMessages(3) @@ -325,7 +334,7 @@ public void testSeekForBatchByAdmin() throws PulsarClientException, ExecutionExc producer.close(); - + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer(Schema.STRING) .topic(topicName) .subscriptionName(subscriptionName) @@ -381,6 +390,7 @@ public void testConcurrentResetCursor() throws Exception { final String topicName = "persistent://prop/use/ns-abc/testConcurrentReset_" + System.currentTimeMillis(); final String subscriptionName = "test-sub-name"; + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); admin.topics().createSubscription(topicName, subscriptionName, MessageId.earliest); @@ -430,6 +440,7 @@ public void testSeekOnPartitionedTopic() throws Exception { final String topicName = "persistent://prop/use/ns-abc/testSeekPartitions"; admin.topics().createPartitionedTopic(topicName, 2); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) .subscriptionName("my-subscription").subscribe(); @@ -447,9 +458,11 @@ public void testSeekTime() throws Exception { long resetTimeInMillis = TimeUnit.SECONDS .toMillis(RelativeTimeUtil.parseRelativeTimeInSeconds(resetTimeStr)); + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); // Disable pre-fetch in consumer to track the messages received + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) .subscriptionName("my-subscription").receiverQueueSize(0).subscribe(); @@ -483,6 +496,7 @@ public void testSeekTimeByFunction() throws Exception { int msgNum = 20; admin.topics().createPartitionedTopic(topicName, partitionNum); creatProducerAndSendMsg(topicName, msgNum); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient .newConsumer(Schema.STRING).startMessageIdInclusive() .topic(topicName).subscriptionName("my-sub").subscribe(); @@ -530,6 +544,7 @@ public void testSeekTimeOnPartitionedTopic() throws Exception { long resetTimeInMillis = TimeUnit.SECONDS .toMillis(RelativeTimeUtil.parseRelativeTimeInSeconds(resetTimeStr)); admin.topics().createPartitionedTopic(topicName, partitions); + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); // Disable pre-fetch in consumer to track the messages received org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) @@ -583,12 +598,14 @@ public void testSeekTimeOnPartitionedTopic() throws Exception { public void testShouldCloseAllConsumersForMultipleConsumerDispatcherWhenSeek() throws Exception { final String topicName = "persistent://prop/use/ns-abc/testShouldCloseAllConsumersForMultipleConsumerDispatcherWhenSeek"; // Disable pre-fetch in consumer to track the messages received + @Cleanup org.apache.pulsar.client.api.Consumer consumer1 = pulsarClient.newConsumer() .topic(topicName) .subscriptionType(SubscriptionType.Shared) .subscriptionName("my-subscription") .subscribe(); + @Cleanup org.apache.pulsar.client.api.Consumer consumer2 = pulsarClient.newConsumer() .topic(topicName) .subscriptionType(SubscriptionType.Shared) @@ -615,20 +632,20 @@ public void testShouldCloseAllConsumersForMultipleConsumerDispatcherWhenSeek() t for (Consumer consumer : consumers) { assertFalse(connectedSinceSet.contains(consumer.getStats().getConnectedSince())); } - consumer1.close(); - consumer2.close(); } @Test public void testOnlyCloseActiveConsumerForSingleActiveConsumerDispatcherWhenSeek() throws Exception { final String topicName = "persistent://prop/use/ns-abc/testOnlyCloseActiveConsumerForSingleActiveConsumerDispatcherWhenSeek"; // Disable pre-fetch in consumer to track the messages received + @Cleanup org.apache.pulsar.client.api.Consumer consumer1 = pulsarClient.newConsumer() .topic(topicName) .subscriptionType(SubscriptionType.Failover) .subscriptionName("my-subscription") .subscribe(); + @Cleanup org.apache.pulsar.client.api.Consumer consumer2 = pulsarClient.newConsumer() .topic(topicName) .subscriptionType(SubscriptionType.Failover) @@ -668,11 +685,13 @@ public void testSeekByFunction() throws Exception { int msgNum = 160; admin.topics().createPartitionedTopic(topicName, partitionNum); creatProducerAndSendMsg(topicName, msgNum); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient .newConsumer(Schema.STRING).startMessageIdInclusive() .topic(topicName).subscriptionName("my-sub").subscribe(); TopicName partitionedTopic = TopicName.get(topicName); + @Cleanup Reader reader = pulsarClient.newReader(Schema.STRING) .startMessageId(MessageId.earliest) .topic(partitionedTopic.getPartition(0).toString()).create(); @@ -721,12 +740,11 @@ public void testSeekByFunction() throws Exception { for (MessageId messageId : msgNotIn) { assertFalse(received.contains(messageId)); } - reader.close(); - consumer.close(); } private List creatProducerAndSendMsg(String topic, int msgNum) throws Exception { List messageIds = new ArrayList<>(); + @Cleanup Producer producer = pulsarClient .newProducer(Schema.STRING) .enableBatching(false) @@ -735,7 +753,6 @@ private List creatProducerAndSendMsg(String topic, int msgNum) throws for (int i = 0; i < msgNum; i++) { messageIds.add(producer.send("msg" + i)); } - producer.close(); return messageIds; } @@ -756,6 +773,7 @@ public void testSeekByFunctionAndMultiTopic() throws Exception { MessageId msgIdInTopic2Partition0 = admin.topics().getLastMessageId(topic2.getPartition(0).toString()); MessageId msgIdInTopic2Partition2 = admin.topics().getLastMessageId(topic2.getPartition(2).toString()); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient .newConsumer(Schema.STRING).startMessageIdInclusive() .topics(Arrays.asList(topicName, topicName2)).subscriptionName("my-sub").subscribe(); @@ -796,6 +814,7 @@ public void testSeekWillNotEncounteredFencedError() throws Exception { // Create a pulsar client with a subscription fenced counter. ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); AtomicInteger receivedFencedErrorCounter = new AtomicInteger(); + @Cleanup PulsarClient client = InjectedClientCnxClientBuilder.create(clientBuilder, (conf, eventLoopGroup) -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoopGroup) { protected void handleError(CommandError error) { @@ -807,10 +826,13 @@ protected void handleError(CommandError error) { }); // publish some messages. + @Cleanup org.apache.pulsar.client.api.Consumer consumer = client.newConsumer(Schema.STRING) .topic(topicName) .subscriptionName("s1") .subscribe(); + + @Cleanup Producer producer = client.newProducer(Schema.STRING) .topic(topicName).create(); MessageIdImpl msgId1 = (MessageIdImpl) producer.send("0"); @@ -850,6 +872,7 @@ protected void handleError(CommandError error) { public void testExceptionBySeekFunction() throws Exception { final String topicName = "persistent://prop/use/ns-abc/test" + UUID.randomUUID(); creatProducerAndSendMsg(topicName,10); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient .newConsumer() .topic(topicName).subscriptionName("my-sub").subscribe(); From 7f04364f13330b56cabeab48b9b5055a70a88119 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 16 Aug 2024 23:30:34 +0800 Subject: [PATCH 450/580] [improve][broker] Support customized shadow managed ledger implementation (#23179) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 3 +-- .../pulsar/broker/namespace/NamespaceServiceTest.java | 2 ++ .../pulsar/broker/service/MessageCumulativeAckTest.java | 5 ++++- .../service/PersistentDispatcherFailoverConsumerTest.java | 1 + .../apache/pulsar/broker/service/PersistentTopicTest.java | 6 ++++++ .../org/apache/pulsar/broker/service/ServerCnxTest.java | 1 + .../systopic/NamespaceEventsSystemTopicServiceTest.java | 7 ++++++- .../apache/pulsar/broker/transaction/TransactionTest.java | 2 ++ .../mledger/offload/jcloud/impl/MockManagedLedger.java | 2 +- 9 files changed, 24 insertions(+), 5 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index e890bac620e7b..c26725deaeab5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -84,7 +84,6 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer.CursorInfo; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.ShadowManagedLedgerImpl; import org.apache.bookkeeper.mledger.util.Futures; import org.apache.bookkeeper.mledger.util.ManagedLedgerImplUtils; import org.apache.bookkeeper.net.BookieId; @@ -426,7 +425,7 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS this.transactionBuffer = new TransactionBufferDisable(this); } transactionBuffer.syncMaxReadPositionForNormalPublish(ledger.getLastConfirmedEntry(), true); - if (ledger instanceof ShadowManagedLedgerImpl) { + if (ledger.getConfig().getShadowSource() != null) { shadowSourceTopic = TopicName.get(ledger.getConfig().getShadowSource()); } else { shadowSourceTopic = null; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java index 422e9b80aeffa..6b2669275dfdb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java @@ -54,6 +54,7 @@ import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.tuple.Pair; @@ -198,6 +199,7 @@ public void testSplitMapWithRefreshedStatMap() throws Exception { ManagedLedger ledger = mock(ManagedLedger.class); when(ledger.getCursors()).thenReturn(new ArrayList<>()); + when(ledger.getConfig()).thenReturn(new ManagedLedgerConfig()); doReturn(CompletableFuture.completedFuture(null)).when(MockOwnershipCache).disableOwnership(any(NamespaceBundle.class)); Field ownership = NamespaceService.class.getDeclaredField("ownershipCache"); 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 f3fe26af4b968..cc4fe22962484 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 @@ -37,6 +37,7 @@ import io.netty.channel.ChannelHandlerContext; import java.net.InetSocketAddress; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -74,7 +75,9 @@ public void setup() throws Exception { .when(serverCnx).getCommandSender(); String topicName = TopicName.get("MessageCumulativeAckTest").toString(); - PersistentTopic persistentTopic = new PersistentTopic(topicName, mock(ManagedLedger.class), pulsarTestContext.getBrokerService()); + var mockManagedLedger = mock(ManagedLedger.class); + when(mockManagedLedger.getConfig()).thenReturn(new ManagedLedgerConfig()); + var persistentTopic = new PersistentTopic(topicName, mockManagedLedger, pulsarTestContext.getBrokerService()); sub = spy(new PersistentSubscription(persistentTopic, "sub-1", mock(ManagedCursorImpl.class), false)); doNothing().when(sub).acknowledgeMessage(any(), any(), any()); 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 f30ee62b64659..000ea7af91525 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 @@ -180,6 +180,7 @@ void setupMLAsyncCallbackMocks() { cursorMock = mock(ManagedCursorImpl.class); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); doReturn("mockCursor").when(cursorMock).getName(); // call openLedgerComplete with ledgerMock on ML factory asyncOpen 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 e83b1bd9b7b79..f2ed015bd1e67 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 @@ -239,6 +239,7 @@ public void teardown() throws Exception { @Test public void testCreateTopic() { final ManagedLedger ledgerMock = mock(ManagedLedger.class); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); final String topicName = "persistent://prop/use/ns-abc/topic1"; @@ -366,6 +367,7 @@ public void testPublishMessageMLFailure() throws Exception { final String successTopicName = "persistent://prop/use/ns-abc/successTopic"; final ManagedLedger ledgerMock = mock(ManagedLedger.class); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); @@ -1374,6 +1376,7 @@ void setupMLAsyncCallbackMocks() { final CompletableFuture closeFuture = new CompletableFuture<>(); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); doReturn("mockCursor").when(cursorMock).getName(); doReturn(true).when(cursorMock).isDurable(); // doNothing().when(cursorMock).asyncClose(new CloseCallback() { @@ -1671,6 +1674,7 @@ public void testAtomicReplicationRemoval() throws Exception { String remoteCluster = "remote"; final ManagedLedger ledgerMock = mock(ManagedLedger.class); doNothing().when(ledgerMock).asyncDeleteCursor(any(), any(), any()); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); PersistentTopic topic = new PersistentTopic(globalTopicName, ledgerMock, brokerService); @@ -1730,6 +1734,7 @@ public void testClosingReplicationProducerTwice() throws Exception { final ManagedLedger ledgerMock = mock(ManagedLedger.class); doNothing().when(ledgerMock).asyncDeleteCursor(any(), any(), any()); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); PersistentTopic topic = new PersistentTopic(globalTopicName, ledgerMock, brokerService); @@ -2120,6 +2125,7 @@ public void testTopicCloseFencingTimeout() throws Exception { @Test public void testGetDurableSubscription() throws Exception { ManagedLedger mockLedger = mock(ManagedLedger.class); + doReturn(new ManagedLedgerConfig()).when(mockLedger).getConfig(); ManagedCursor mockCursor = mock(ManagedCursorImpl.class); Position mockPosition = mock(Position.class); doReturn("test").when(mockCursor).getName(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index 58c6b96a0f346..03115d79af0a0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -2919,6 +2919,7 @@ private void setupMLAsyncCallbackMocks() { ledgerMock = mock(ManagedLedger.class); cursorMock = mock(ManagedCursor.class); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); // call openLedgerComplete with ledgerMock on ML factory asyncOpen doAnswer((Answer) invocationOnMock -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java index 44a4de5e8a923..e66140efb32bb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/NamespaceEventsSystemTopicServiceTest.java @@ -20,10 +20,13 @@ import static org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService.getEventKey; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + import com.google.common.collect.Sets; import java.util.HashSet; import lombok.Cleanup; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -99,7 +102,9 @@ public void testSystemTopicSchemaCompatibility() throws Exception { TopicPoliciesSystemTopicClient systemTopicClientForNamespace1 = systemTopicFactory .createTopicPoliciesSystemTopicClient(NamespaceName.get(NAMESPACE1)); String topicName = systemTopicClientForNamespace1.getTopicName().toString(); - SystemTopic topic = new SystemTopic(topicName, mock(ManagedLedger.class), pulsar.getBrokerService()); + final var mockManagedLedger = mock(ManagedLedger.class); + when(mockManagedLedger.getConfig()).thenReturn(new ManagedLedgerConfig()); + SystemTopic topic = new SystemTopic(topicName, mockManagedLedger, pulsar.getBrokerService()); Assert.assertEquals(SchemaCompatibilityStrategy.ALWAYS_COMPATIBLE, topic.getSchemaCompatibilityStrategy()); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index 2a928084e648a..246ab5ef26a8f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -71,6 +71,7 @@ import org.apache.bookkeeper.common.util.Bytes; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.Position; @@ -1648,6 +1649,7 @@ public void testTBRecoverChangeStateError() throws InterruptedException, Timeout // Mock managedLedger. ManagedLedgerImpl managedLedger = mock(ManagedLedgerImpl.class); ManagedCursorContainer managedCursors = new ManagedCursorContainer(); + when(managedLedger.getConfig()).thenReturn(new ManagedLedgerConfig()); when(managedLedger.getCursors()).thenReturn(managedCursors); Position position = PositionFactory.EARLIEST; when(managedLedger.getLastConfirmedEntry()).thenReturn(position); diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java index 66ace69d7cda2..8f52d20c5ee83 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java @@ -274,7 +274,7 @@ public boolean isTerminated() { @Override public ManagedLedgerConfig getConfig() { - return null; + return new ManagedLedgerConfig(); } @Override From 3053b647e0ca646b2df9f03815947104cd2e705f Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Sat, 17 Aug 2024 00:17:55 +0800 Subject: [PATCH 451/580] [improve][broker] Should notify bundle ownership listener onLoad event when ServiceUnitState start (ExtensibleLoadManagerImpl only) (#23152) --- .../channel/ServiceUnitStateChannelImpl.java | 16 +++++- .../ExtensibleLoadManagerImplTest.java | 50 +++++++++++++++++++ 2 files changed, 64 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index dbe3b88b61f28..1063f8124ece8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -324,7 +324,8 @@ public synchronized void start() throws PulsarServerException { "topicCompactionStrategyClassName", ServiceUnitStateCompactionStrategy.class.getName())) .create(); - tableview.listen((key, value) -> handle(key, value)); + tableview.listen(this::handleEvent); + tableview.forEach(this::handleExisting); var strategy = (ServiceUnitStateCompactionStrategy) TopicCompactionStrategy.getInstance(TABLE_VIEW_TAG); if (strategy == null) { String err = TABLE_VIEW_TAG + "tag TopicCompactionStrategy is null."; @@ -690,7 +691,7 @@ public CompletableFuture publishSplitEventAsync(Split split) { }).thenApply(__ -> null); } - private void handle(String serviceUnit, ServiceUnitStateData data) { + private void handleEvent(String serviceUnit, ServiceUnitStateData data) { long totalHandledRequests = getHandlerTotalCounter(data).incrementAndGet(); if (debug()) { log.info("{} received a handle request for serviceUnit:{}, data:{}. totalHandledRequests:{}", @@ -716,6 +717,17 @@ private void handle(String serviceUnit, ServiceUnitStateData data) { } } + private void handleExisting(String serviceUnit, ServiceUnitStateData data) { + if (debug()) { + log.info("Loaded the service unit state data. serviceUnit: {}, data: {}", serviceUnit, data); + } + ServiceUnitState state = state(data); + if (state.equals(Owned) && isTargetBroker(data.dstBroker())) { + pulsar.getNamespaceService() + .onNamespaceBundleOwned(LoadManagerShared.getNamespaceBundle(pulsar, serviceUnit)); + } + } + private static boolean isTransferCommand(ServiceUnitStateData data) { if (data == null) { return false; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index 69a65caf2943c..51966f420bf25 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -417,6 +417,56 @@ public boolean test(NamespaceBundle namespaceBundle) { } } + @Test(timeOut = 30 * 1000) + public void testNamespaceOwnershipListener() throws Exception { + Pair topicAndBundle = + getBundleIsNotOwnByChangeEventTopic("test-namespace-ownership-listener"); + TopicName topicName = topicAndBundle.getLeft(); + NamespaceBundle bundle = topicAndBundle.getRight(); + + String broker = admin.lookups().lookupTopic(topicName.toString()); + log.info("Assign the bundle {} to {}", bundle, broker); + + checkOwnershipState(broker, bundle); + + AtomicInteger onloadCount = new AtomicInteger(0); + AtomicInteger unloadCount = new AtomicInteger(0); + + NamespaceBundleOwnershipListener listener = new NamespaceBundleOwnershipListener() { + @Override + public void onLoad(NamespaceBundle bundle) { + onloadCount.incrementAndGet(); + } + + @Override + public void unLoad(NamespaceBundle bundle) { + unloadCount.incrementAndGet(); + } + + @Override + public boolean test(NamespaceBundle namespaceBundle) { + return namespaceBundle.equals(bundle); + } + }; + pulsar1.getNamespaceService().addNamespaceBundleOwnershipListener(listener); + pulsar2.getNamespaceService().addNamespaceBundleOwnershipListener(listener); + + // There are a service unit state channel already started, when add listener, it will trigger the onload event. + Awaitility.await().untilAsserted(() -> { + assertEquals(onloadCount.get(), 1); + assertEquals(unloadCount.get(), 0); + }); + + ServiceUnitStateChannelImpl channel = new ServiceUnitStateChannelImpl(pulsar1); + channel.start(); + Awaitility.await().untilAsserted(() -> { + assertEquals(onloadCount.get(), 2); + assertEquals(unloadCount.get(), 0); + }); + + channel.close(); + } + @DataProvider(name = "isPersistentTopicSubscriptionTypeTest") public Object[][] isPersistentTopicSubscriptionTypeTest() { return new Object[][]{ From 576666de4fdb6cacc66e3f73831312a25559de45 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 16 Aug 2024 20:12:38 +0300 Subject: [PATCH 452/580] [improve][misc] Set format_version=5, checksum=kxxHash in Bookkeeper RocksDB configs (#23175) --- conf/bookkeeper.conf | 9 ++++++++- conf/default_rocksdb.conf | 11 +++++++++-- conf/entry_location_rocksdb.conf | 6 +++--- conf/ledger_metadata_rocksdb.conf | 11 +++++++++-- 4 files changed, 29 insertions(+), 8 deletions(-) diff --git a/conf/bookkeeper.conf b/conf/bookkeeper.conf index 548ece01b842d..4058d787e2c00 100644 --- a/conf/bookkeeper.conf +++ b/conf/bookkeeper.conf @@ -658,7 +658,7 @@ diskCheckInterval=10000 ############################################################################# # Metadata service uri that bookkeeper uses for loading the corresponding metadata driver and resolving its metadata service location -# Examples: +# Examples: # - metadataServiceUri=zk+hierarchical://my-zk-1:2181/ledgers # - metadataServiceUri=etcd+hierarchical:http://my-etcd:2379 # - metadataServiceUri=metadata-store:zk:my-zk-1:2281 @@ -761,13 +761,20 @@ dbStorage_readAheadCacheBatchSize=1000 ## DbLedgerStorage uses RocksDB to store the indexes from ## (ledgerId, entryId) -> (entryLog, offset) +# These settings are ignored since Pulsar 2.11 / Bookkeeper 4.15 +# NOTICE: The settings in conf/default_rocksdb.conf, conf/entry_location_rocksdb.conf and +# conf/ledger_metadata_rocksdb.conf files are primarily used to configure RocksDB +# settings. dbStorage_rocksDB_* config keys are ignored. + # Size of RocksDB block-cache. For best performance, this cache # should be big enough to hold a significant portion of the index # database which can reach ~2GB in some cases # Default is to use 10% of the direct memory size +# These settings are ignored since Pulsar 2.11 / Bookkeeper 4.15 dbStorage_rocksDB_blockCacheSize= # Other RocksDB specific tunables +# These settings are ignored since Pulsar 2.11 / Bookkeeper 4.15 dbStorage_rocksDB_writeBufferSizeMB=64 dbStorage_rocksDB_sstSizeInMB=64 dbStorage_rocksDB_blockSize=65536 diff --git a/conf/default_rocksdb.conf b/conf/default_rocksdb.conf index e1a21bb845222..74e3005ba6687 100644 --- a/conf/default_rocksdb.conf +++ b/conf/default_rocksdb.conf @@ -24,7 +24,14 @@ info_log_level=INFO_LEVEL # set by jni: options.setKeepLogFileNum keep_log_file_num=30 + # set by jni: options.setLogFileTimeToRoll + log_file_time_to_roll=86400 [CFOptions "default"] - # set by jni: options.setLogFileTimeToRoll - log_file_time_to_roll=86400 \ No newline at end of file + #no default setting in CFOptions + +[TableOptions/BlockBasedTable "default"] + # set by jni: tableOptions.setFormatVersion + format_version=5 + # set by jni: tableOptions.setChecksumType + checksum=kxxHash \ No newline at end of file diff --git a/conf/entry_location_rocksdb.conf b/conf/entry_location_rocksdb.conf index 42d916ded378f..9c675554b24ae 100644 --- a/conf/entry_location_rocksdb.conf +++ b/conf/entry_location_rocksdb.conf @@ -27,7 +27,7 @@ # set by jni: options.setLogFileTimeToRoll log_file_time_to_roll=86400 # set by jni: options.setMaxBackgroundJobs or options.setIncreaseParallelism - max_background_jobs=2 + max_background_jobs=32 # set by jni: options.setMaxSubcompactions max_subcompactions=1 # set by jni: options.setMaxTotalWalSize @@ -61,10 +61,10 @@ # set by jni: tableOptions.setBlockCache block_cache=206150041 # set by jni: tableOptions.setFormatVersion - format_version=2 + format_version=5 # set by jni: tableOptions.setChecksumType checksum=kxxHash # set by jni: tableOptions.setFilterPolicy, bloomfilter:[bits_per_key]:[use_block_based_builder] filter_policy=rocksdb.BloomFilter:10:false # set by jni: tableOptions.setCacheIndexAndFilterBlocks - cache_index_and_filter_blocks=true + cache_index_and_filter_blocks=true \ No newline at end of file diff --git a/conf/ledger_metadata_rocksdb.conf b/conf/ledger_metadata_rocksdb.conf index e1a21bb845222..74e3005ba6687 100644 --- a/conf/ledger_metadata_rocksdb.conf +++ b/conf/ledger_metadata_rocksdb.conf @@ -24,7 +24,14 @@ info_log_level=INFO_LEVEL # set by jni: options.setKeepLogFileNum keep_log_file_num=30 + # set by jni: options.setLogFileTimeToRoll + log_file_time_to_roll=86400 [CFOptions "default"] - # set by jni: options.setLogFileTimeToRoll - log_file_time_to_roll=86400 \ No newline at end of file + #no default setting in CFOptions + +[TableOptions/BlockBasedTable "default"] + # set by jni: tableOptions.setFormatVersion + format_version=5 + # set by jni: tableOptions.setChecksumType + checksum=kxxHash \ No newline at end of file From 73433cd06e65ce5e194372a657c5a414e820138b Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sat, 17 Aug 2024 17:55:37 +0800 Subject: [PATCH 453/580] [improve] [broker] Optimize performance for checking max topics when the topic is a system topic (#23185) --- .../java/org/apache/pulsar/broker/service/BrokerService.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 8460fe23ac3b7..d13d3b3174a7a 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 @@ -3705,6 +3705,9 @@ public CompletableFuture deleteSchema(TopicName topicName) { } private CompletableFuture checkMaxTopicsPerNamespace(TopicName topicName, int numPartitions) { + if (isSystemTopic(topicName)) { + return CompletableFuture.completedFuture(null); + } return pulsar.getPulsarResources().getNamespaceResources() .getPoliciesAsync(topicName.getNamespaceObject()) .thenCompose(optPolicies -> { From aa8226f45e3b28a14377f9f949d5a34f61b27e9a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Sat, 17 Aug 2024 19:50:29 +0800 Subject: [PATCH 454/580] [fix][broker] Skip reading entries from closed cursor. (#22751) --- ...PersistentDispatcherMultipleConsumers.java | 26 +++- ...sistentDispatcherSingleActiveConsumer.java | 26 +++- ...istentDispatcherMultipleConsumersTest.java | 71 ++++++++++ ...entDispatcherSingleActiveConsumerTest.java | 127 ++++++++++++++++++ 4 files changed, 244 insertions(+), 6 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumerTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 6eca58d070777..274bdd9947a07 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service.persistent; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.collect.Range; import java.util.ArrayList; @@ -299,6 +300,12 @@ public void readMoreEntriesAsync() { } public synchronized void readMoreEntries() { + if (cursor.isClosed()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor is already closed, skipping read more entries.", cursor.getName()); + } + return; + } if (isSendInProgress()) { // we cannot read more entries while sending the previous batch // otherwise we could re-read the same entries and send duplicates @@ -895,7 +902,14 @@ public synchronized void readEntriesFailed(ManagedLedgerException exception, Obj ReadType readType = (ReadType) ctx; long waitTimeMillis = readFailureBackoff.next(); - if (exception instanceof NoMoreEntriesToReadException) { + // Do not keep reading more entries if the cursor is already closed. + if (exception instanceof ManagedLedgerException.CursorAlreadyClosedException) { + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor is already closed, skipping read more entries", cursor.getName()); + } + // Set the wait time to -1 to avoid rescheduling the read. + waitTimeMillis = -1; + } else if (exception instanceof NoMoreEntriesToReadException) { if (cursor.getNumberOfEntriesInBacklog(false) == 0) { // Topic has been terminated and there are no more entries to read // Notify the consumer only if all the messages were already acknowledged @@ -934,7 +948,14 @@ public synchronized void readEntriesFailed(ManagedLedgerException exception, Obj } readBatchSize = serviceConfig.getDispatcherMinReadBatchSize(); + // Skip read if the waitTimeMillis is a nagetive value. + if (waitTimeMillis >= 0) { + scheduleReadEntriesWithDelay(exception, readType, waitTimeMillis); + } + } + @VisibleForTesting + void scheduleReadEntriesWithDelay(Exception e, ReadType readType, long waitTimeMillis) { topic.getBrokerService().executor().schedule(() -> { synchronized (PersistentDispatcherMultipleConsumers.this) { // If it's a replay read we need to retry even if there's already @@ -944,11 +965,10 @@ public synchronized void readEntriesFailed(ManagedLedgerException exception, Obj log.info("[{}] Retrying read operation", name); readMoreEntries(); } else { - log.info("[{}] Skipping read retry: havePendingRead {}", name, havePendingRead, exception); + log.info("[{}] Skipping read retry: havePendingRead {}", name, havePendingRead, e); } } }, waitTimeMillis, TimeUnit.MILLISECONDS); - } private boolean needTrimAckedMessages() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index 600fbb26eb511..b451a8ad5dc0d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -20,6 +20,7 @@ import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; +import com.google.common.annotations.VisibleForTesting; import io.netty.util.Recycler; import java.util.Iterator; import java.util.List; @@ -313,7 +314,14 @@ public void redeliverUnacknowledgedMessages(Consumer consumer, List po redeliverUnacknowledgedMessages(consumer, DEFAULT_CONSUMER_EPOCH); } - private void readMoreEntries(Consumer consumer) { + @VisibleForTesting + void readMoreEntries(Consumer consumer) { + if (cursor.isClosed()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor is already closed, skipping read more entries", cursor.getName()); + } + return; + } // consumer can be null when all consumers are disconnected from broker. // so skip reading more entries if currently there is no active consumer. if (null == consumer) { @@ -499,6 +507,14 @@ private synchronized void internalReadEntriesFailed(ManagedLedgerException excep Consumer c = readEntriesCtx.getConsumer(); readEntriesCtx.recycle(); + // Do not keep reading messages from a closed cursor. + if (exception instanceof ManagedLedgerException.CursorAlreadyClosedException) { + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor was already closed, skipping read more entries", cursor.getName()); + } + return; + } + if (exception instanceof ConcurrentWaitCallbackException) { // At most one pending read request is allowed when there are no more entries, we should not trigger more // read operations in this case and just wait the existing read operation completes. @@ -535,6 +551,11 @@ private synchronized void internalReadEntriesFailed(ManagedLedgerException excep // Reduce read batch size to avoid flooding bookies with retries readBatchSize = serviceConfig.getDispatcherMinReadBatchSize(); + scheduleReadEntriesWithDelay(c, waitTimeMillis); + } + + @VisibleForTesting + void scheduleReadEntriesWithDelay(Consumer c, long delay) { topic.getBrokerService().executor().schedule(() -> { // Jump again into dispatcher dedicated thread @@ -556,8 +577,7 @@ private synchronized void internalReadEntriesFailed(ManagedLedgerException excep } } }); - }, waitTimeMillis, TimeUnit.MILLISECONDS); - + }, delay, TimeUnit.MILLISECONDS); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java index f24c5c5933e5b..a03ed92b81590 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java @@ -20,15 +20,24 @@ import com.carrotsearch.hppc.ObjectSet; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.awaitility.reflect.WhiteboxImpl; +import org.mockito.Mockito; +import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -98,4 +107,66 @@ public void testTopicDeleteIfConsumerSetMismatchConsumerList2() throws Exception consumer.close(); admin.topics().delete(topicName, false); } + + @Test + public void testSkipReadEntriesFromCloseCursor() throws Exception { + final String topicName = + BrokerTestUtil.newUniqueName("persistent://public/default/testSkipReadEntriesFromCloseCursor"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + for (int i = 0; i < 10; i++) { + producer.send("message-" + i); + } + producer.close(); + + // Get the dispatcher of the topic. + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService() + .getTopic(topicName, false).join().get(); + + ManagedCursor cursor = Mockito.mock(ManagedCursorImpl.class); + Mockito.doReturn(subscription).when(cursor).getName(); + Subscription sub = Mockito.mock(PersistentSubscription.class); + Mockito.doReturn(topic).when(sub).getTopic(); + // Mock the dispatcher. + PersistentDispatcherMultipleConsumers dispatcher = + Mockito.spy(new PersistentDispatcherMultipleConsumers(topic, cursor, sub)); + // Return 10 permits to make the dispatcher can read more entries. + Mockito.doReturn(10).when(dispatcher).getFirstAvailableConsumerPermits(); + + // Make the count + 1 when call the scheduleReadEntriesWithDelay(...). + AtomicInteger callScheduleReadEntriesWithDelayCnt = new AtomicInteger(0); + Mockito.doAnswer(inv -> { + callScheduleReadEntriesWithDelayCnt.getAndIncrement(); + return inv.callRealMethod(); + }).when(dispatcher).scheduleReadEntriesWithDelay(Mockito.any(), Mockito.any(), Mockito.anyLong()); + + // Make the count + 1 when call the readEntriesFailed(...). + AtomicInteger callReadEntriesFailed = new AtomicInteger(0); + Mockito.doAnswer(inv -> { + callReadEntriesFailed.getAndIncrement(); + return inv.callRealMethod(); + }).when(dispatcher).readEntriesFailed(Mockito.any(), Mockito.any()); + + Mockito.doReturn(false).when(cursor).isClosed(); + + // Mock the readEntriesOrWait(...) to simulate the cursor is closed. + Mockito.doAnswer(inv -> { + PersistentDispatcherMultipleConsumers dispatcher1 = inv.getArgument(2); + dispatcher1.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException("cursor closed"), + null); + return null; + }).when(cursor).asyncReadEntriesOrWait(Mockito.anyInt(), Mockito.anyLong(), Mockito.eq(dispatcher), + Mockito.any(), Mockito.any()); + + dispatcher.readMoreEntries(); + + // Verify: the readEntriesFailed should be called once and the scheduleReadEntriesWithDelay should not be called. + Assert.assertTrue(callReadEntriesFailed.get() == 1 && callScheduleReadEntriesWithDelayCnt.get() == 0); + + // Verify: the topic can be deleted successfully. + admin.topics().delete(topicName, false); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumerTest.java new file mode 100644 index 0000000000000..a4c9e26ffb853 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumerTest.java @@ -0,0 +1,127 @@ +/* + * 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.service.persistent; + +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.api.proto.CommandSubscribe; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class PersistentDispatcherSingleActiveConsumerTest extends ProducerConsumerBase { + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testSkipReadEntriesFromCloseCursor() throws Exception { + final String topicName = + BrokerTestUtil.newUniqueName("persistent://public/default/testSkipReadEntriesFromCloseCursor"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + for (int i = 0; i < 10; i++) { + producer.send("message-" + i); + } + producer.close(); + + // Get the dispatcher of the topic. + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService() + .getTopic(topicName, false).join().get(); + + ManagedCursor cursor = Mockito.mock(ManagedCursorImpl.class); + Mockito.doReturn(subscription).when(cursor).getName(); + Subscription sub = Mockito.mock(PersistentSubscription.class); + Mockito.doReturn(topic).when(sub).getTopic(); + // Mock the dispatcher. + PersistentDispatcherSingleActiveConsumer dispatcher = + Mockito.spy(new PersistentDispatcherSingleActiveConsumer(cursor, CommandSubscribe.SubType.Exclusive,0, topic, sub)); + + // Mock a consumer + Consumer consumer = Mockito.mock(Consumer.class); + consumer.getAvailablePermits(); + Mockito.doReturn(10).when(consumer).getAvailablePermits(); + Mockito.doReturn(10).when(consumer).getAvgMessagesPerEntry(); + Mockito.doReturn("test").when(consumer).consumerName(); + Mockito.doReturn(true).when(consumer).isWritable(); + Mockito.doReturn(false).when(consumer).readCompacted(); + + // Make the consumer as the active consumer. + Mockito.doReturn(consumer).when(dispatcher).getActiveConsumer(); + + // Make the count + 1 when call the scheduleReadEntriesWithDelay(...). + AtomicInteger callScheduleReadEntriesWithDelayCnt = new AtomicInteger(0); + Mockito.doAnswer(inv -> { + callScheduleReadEntriesWithDelayCnt.getAndIncrement(); + return inv.callRealMethod(); + }).when(dispatcher).scheduleReadEntriesWithDelay(Mockito.eq(consumer), Mockito.anyLong()); + + // Make the count + 1 when call the readEntriesFailed(...). + AtomicInteger callReadEntriesFailed = new AtomicInteger(0); + Mockito.doAnswer(inv -> { + callReadEntriesFailed.getAndIncrement(); + return inv.callRealMethod(); + }).when(dispatcher).readEntriesFailed(Mockito.any(), Mockito.any()); + + Mockito.doReturn(false).when(cursor).isClosed(); + + // Mock the readEntriesOrWait(...) to simulate the cursor is closed. + Mockito.doAnswer(inv -> { + PersistentDispatcherSingleActiveConsumer dispatcher1 = inv.getArgument(2); + dispatcher1.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException("cursor closed"), + null); + return null; + }).when(cursor).asyncReadEntriesOrWait(Mockito.anyInt(), Mockito.anyLong(), Mockito.eq(dispatcher), + Mockito.any(), Mockito.any()); + + dispatcher.readMoreEntries(consumer); + + // Verify: the readEntriesFailed should be called once and the scheduleReadEntriesWithDelay should not be called. + Assert.assertTrue(callReadEntriesFailed.get() == 1 && callScheduleReadEntriesWithDelayCnt.get() == 0); + + // Verify: the topic can be deleted successfully. + admin.topics().delete(topicName, false); + } +} From 9edaa8569deff9c0cbb41b261fee472603f3df4d Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sun, 18 Aug 2024 19:02:23 +0800 Subject: [PATCH 455/580] [fix] [broker] Topic can never be loaded up due to broker maintains a failed topic creation future (#23184) --- .../pulsar/broker/service/BrokerService.java | 15 ++- .../client/api/OrphanPersistentTopicTest.java | 95 +++++++++++++++++++ 2 files changed, 106 insertions(+), 4 deletions(-) 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 d13d3b3174a7a..338d5f420ca25 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 @@ -1664,6 +1664,7 @@ private void checkOwnershipAndCreatePersistentTopic(final String topic, boolean topicFuture.completeExceptionally(new ServiceUnitNotReadyException(msg)); } }).exceptionally(ex -> { + pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); topicFuture.completeExceptionally(ex); return null; }); @@ -1767,10 +1768,16 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { long topicLoadLatencyMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()) - topicCreateTimeMs; pulsarStats.recordTopicLoadTimeValue(topic, topicLoadLatencyMs); - if (topicFuture.isCompletedExceptionally()) { + if (!topicFuture.complete(Optional.of(persistentTopic))) { // Check create persistent topic timeout. - log.warn("{} future is already completed with failure {}, closing the" - + " topic", topic, FutureUtil.getException(topicFuture)); + if (topicFuture.isCompletedExceptionally()) { + log.warn("{} future is already completed with failure {}, closing" + + " the topic", topic, FutureUtil.getException(topicFuture)); + } else { + // It should not happen. + log.error("{} future is already completed by another thread, " + + "which is not expected. Closing the current one", topic); + } executor().submit(() -> { persistentTopic.close().whenComplete((ignore, ex) -> { topics.remove(topic, topicFuture); @@ -1782,7 +1789,6 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { }); } else { addTopicToStatsMaps(topicName, persistentTopic); - topicFuture.complete(Optional.of(persistentTopic)); } }) .exceptionally((ex) -> { @@ -1811,6 +1817,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { if (!createIfMissing && exception instanceof ManagedLedgerNotFoundException) { // We were just trying to load a topic and the topic doesn't exist + pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); loadFuture.completeExceptionally(exception); topicFuture.complete(Optional.empty()); } else { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java index b5af3cc6afd6c..f60aeb78387ad 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java @@ -19,13 +19,17 @@ package org.apache.pulsar.client.api; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.DEDUPLICATION_CURSOR_NAME; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import java.lang.reflect.Field; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -33,6 +37,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; @@ -47,6 +52,7 @@ import org.awaitility.reflect.WhiteboxImpl; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -219,4 +225,93 @@ public void testNoOrphanTopicIfInitFailed() throws Exception { consumer.close(); admin.topics().delete(tpName, false); } + + @DataProvider(name = "whetherTimeoutOrNot") + public Object[][] whetherTimeoutOrNot() { + return new Object[][] { + {true}, + {false} + }; + } + + @Test(timeOut = 60 * 1000, dataProvider = "whetherTimeoutOrNot") + public void testCheckOwnerShipFails(boolean injectTimeout) throws Exception { + if (injectTimeout) { + pulsar.getConfig().setTopicLoadTimeoutSeconds(5); + } + String ns = "public" + "/" + UUID.randomUUID().toString().replaceAll("-", ""); + String tpName = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp"); + admin.namespaces().createNamespace(ns); + admin.topics().createNonPartitionedTopic(tpName); + admin.namespaces().unload(ns); + + // Inject an error when calling "NamespaceService.isServiceUnitActiveAsync". + AtomicInteger failedTimes = new AtomicInteger(); + NamespaceService namespaceService = pulsar.getNamespaceService(); + doAnswer(invocation -> { + TopicName paramTp = (TopicName) invocation.getArguments()[0]; + if (paramTp.toString().equalsIgnoreCase(tpName) && failedTimes.incrementAndGet() <= 2) { + if (injectTimeout) { + Thread.sleep(10 * 1000); + } + log.info("Failed {} times", failedTimes.get()); + return CompletableFuture.failedFuture(new RuntimeException("mocked error")); + } + return invocation.callRealMethod(); + }).when(namespaceService).isServiceUnitActiveAsync(any(TopicName.class)); + + // Verify: the consumer can create successfully eventually. + Consumer consumer = pulsarClient.newConsumer().topic(tpName).subscriptionName("s1").subscribe(); + + // cleanup. + if (injectTimeout) { + pulsar.getConfig().setTopicLoadTimeoutSeconds(60); + } + consumer.close(); + admin.topics().delete(tpName); + } + + @Test(timeOut = 60 * 1000, dataProvider = "whetherTimeoutOrNot") + public void testTopicLoadAndDeleteAtTheSameTime(boolean injectTimeout) throws Exception { + if (injectTimeout) { + pulsar.getConfig().setTopicLoadTimeoutSeconds(5); + } + String ns = "public" + "/" + UUID.randomUUID().toString().replaceAll("-", ""); + String tpName = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp"); + admin.namespaces().createNamespace(ns); + admin.topics().createNonPartitionedTopic(tpName); + admin.namespaces().unload(ns); + + // Inject a race condition: load topic and delete topic execute at the same time. + AtomicInteger mockRaceConditionCounter = new AtomicInteger(); + NamespaceService namespaceService = pulsar.getNamespaceService(); + doAnswer(invocation -> { + TopicName paramTp = (TopicName) invocation.getArguments()[0]; + if (paramTp.toString().equalsIgnoreCase(tpName) && mockRaceConditionCounter.incrementAndGet() <= 1) { + if (injectTimeout) { + Thread.sleep(10 * 1000); + } + log.info("Race condition occurs {} times", mockRaceConditionCounter.get()); + pulsar.getManagedLedgerFactory().delete(TopicName.get(tpName).getPersistenceNamingEncoding()); + } + return invocation.callRealMethod(); + }).when(namespaceService).isServiceUnitActiveAsync(any(TopicName.class)); + + // Verify: the consumer create failed due to pulsar does not allow to create topic automatically. + try { + pulsar.getBrokerService().getTopic(tpName, false, Collections.emptyMap()).join(); + } catch (Exception ex) { + log.warn("Expected error", ex); + } + + // Verify: the consumer create successfully after allowing to create topic automatically. + Consumer consumer = pulsarClient.newConsumer().topic(tpName).subscriptionName("s1").subscribe(); + + // cleanup. + if (injectTimeout) { + pulsar.getConfig().setTopicLoadTimeoutSeconds(60); + } + consumer.close(); + admin.topics().delete(tpName); + } } From 2d46bfafab2567593067847775b2d672ed502144 Mon Sep 17 00:00:00 2001 From: Apurva007 Date: Tue, 20 Aug 2024 01:10:26 -0700 Subject: [PATCH 456/580] [improve] PIP-337: Implement SSL Factory Plugin to customize SSL Context and SSL Engine generation (#23110) Co-authored-by: Apurva Telang --- .../pulsar/broker/ServiceConfiguration.java | 19 + .../jetty/tls/JettySslContextFactory.java | 58 +-- .../jetty/tls/JettySslContextFactoryTest.java | 94 +++-- ...ettySslContextFactoryWithKeyStoreTest.java | 82 +++- .../apache/pulsar/broker/PulsarService.java | 10 +- .../broker/namespace/NamespaceService.java | 4 +- .../pulsar/broker/service/BrokerService.java | 30 +- .../service/PulsarChannelInitializer.java | 90 +++-- .../apache/pulsar/broker/web/WebService.java | 81 ++-- .../pulsar/compaction/CompactorTool.java | 4 +- .../pulsar/broker/PulsarServiceTest.java | 3 + .../pulsar/broker/admin/TopicsTest.java | 3 + .../broker/loadbalance/LoadBalancerTest.java | 12 + .../loadbalance/SimpleBrokerStartTest.java | 14 + .../SimpleLoadManagerImplTest.java | 15 + .../ExtensibleLoadManagerImplBaseTest.java | 10 + .../impl/BundleSplitterTaskTest.java | 11 + .../impl/ModularLoadManagerImplTest.java | 18 + .../broker/service/BrokerServiceTest.java | 4 +- .../service/BrokerServiceThrottlingTest.java | 2 +- ...econnectZKClientPulsarServiceBaseTest.java | 10 + .../broker/service/ClusterMigrationTest.java | 3 + ...licationWithConfigurationSyncTestBase.java | 10 + .../broker/service/NetworkErrorTestBase.java | 10 + .../api/InjectedClientCnxClientBuilder.java | 2 +- .../client/api/TlsProducerConsumerTest.java | 6 +- .../client/impl/ConnectionPoolTest.java | 34 +- .../pulsar/client/impl/PulsarTestClient.java | 2 +- .../pulsar/compaction/CompactorToolTest.java | 1 + .../security/MockedPulsarStandalone.java | 3 + .../client/admin/PulsarAdminBuilder.java | 14 + .../common/policies/data/ClusterData.java | 8 + .../internal/PulsarAdminBuilderImpl.java | 14 + .../internal/http/AsyncHttpConnector.java | 126 +++--- .../pulsar/client/api/ClientBuilder.java | 35 +- .../apache/pulsar/admin/cli/CmdClusters.java | 14 + .../pulsar/admin/cli/PulsarAdminTool.java | 8 +- .../pulsar/client/cli/PulsarClientTool.java | 7 + .../pulsar/client/impl/ClientBuilderImpl.java | 23 ++ .../pulsar/client/impl/ConnectionPool.java | 19 +- .../apache/pulsar/client/impl/HttpClient.java | 126 +++--- .../client/impl/PulsarChannelInitializer.java | 142 ++++--- .../pulsar/client/impl/PulsarClientImpl.java | 7 +- .../impl/conf/ClientConfigurationData.java | 14 + ...a => PulsarHttpAsyncSslEngineFactory.java} | 31 +- .../client/impl/ClientInitializationTest.java | 6 +- .../client/impl/PulsarClientImplTest.java | 4 +- .../common/policies/data/ClusterDataImpl.java | 28 ++ .../common/util/DefaultPulsarSslFactory.java | 366 ++++++++++++++++++ .../common/util/DefaultSslContextBuilder.java | 76 ---- .../util/NettyClientSslContextRefresher.java | 96 ----- .../util/NettyServerSslContextBuilder.java | 89 ----- .../common/util/PulsarSslConfiguration.java | 167 ++++++++ .../pulsar/common/util/PulsarSslFactory.java | 106 +++++ .../util/SslContextAutoRefreshBuilder.java | 100 ----- .../keystoretls/NetSslContextBuilder.java | 90 ----- .../NettySSLContextAutoRefreshBuilder.java | 155 -------- .../util/DefaultPulsarSslFactoryTest.java | 282 ++++++++++++++ .../util/FileModifiedTimeUpdaterTest.java | 24 +- .../common/util/netty/SslContextTest.java | 109 ++++-- .../functions/worker/rest/WorkerServer.java | 86 ++-- .../proxy/server/AdminProxyHandler.java | 137 +++++-- .../proxy/server/DirectProxyHandler.java | 82 ++-- .../proxy/server/ProxyConfiguration.java | 22 ++ .../pulsar/proxy/server/ProxyConnection.java | 2 +- .../pulsar/proxy/server/ProxyService.java | 13 +- .../server/ServiceChannelInitializer.java | 96 ++--- .../apache/pulsar/proxy/server/WebServer.java | 86 ++-- .../proxy/server/AdminProxyHandlerTest.java | 2 +- .../pulsar/proxy/server/ProxyParserTest.java | 2 +- .../server/ProxyServiceTlsStarterTest.java | 2 + .../apache/pulsar/proxy/server/ProxyTest.java | 2 +- .../pulsar/testclient/PerfClientUtils.java | 10 + .../testclient/PerformanceBaseArguments.java | 9 + .../pulsar/websocket/service/ProxyServer.java | 84 ++-- 75 files changed, 2264 insertions(+), 1302 deletions(-) rename pulsar-client/src/main/java/org/apache/pulsar/client/util/{WithSNISslEngineFactory.java => PulsarHttpAsyncSslEngineFactory.java} (53%) create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultPulsarSslFactory.java delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultSslContextBuilder.java delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyClientSslContextRefresher.java delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyServerSslContextBuilder.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/PulsarSslConfiguration.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/PulsarSslFactory.java delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/SslContextAutoRefreshBuilder.java delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NetSslContextBuilder.java delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NettySSLContextAutoRefreshBuilder.java create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/util/DefaultPulsarSslFactoryTest.java 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 26b2f99abf545..20addc3924bf3 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 @@ -49,6 +49,7 @@ import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.sasl.SaslConstants; +import org.apache.pulsar.common.util.DefaultPulsarSslFactory; import org.apache.pulsar.common.util.DirectMemoryUtils; import org.apache.pulsar.metadata.api.MetadataStoreFactory; import org.apache.pulsar.metadata.impl.ZKMetadataStore; @@ -1581,6 +1582,15 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "Specify whether Client certificates are required for TLS Reject.\n" + "the Connection if the Client Certificate is not trusted") private boolean tlsRequireTrustedClientCertOnConnect = false; + @FieldContext( + category = CATEGORY_TLS, + doc = "SSL Factory Plugin class to provide SSLEngine and SSLContext objects. The default " + + " class used is DefaultSslFactory.") + private String sslFactoryPlugin = DefaultPulsarSslFactory.class.getName(); + @FieldContext( + category = CATEGORY_TLS, + doc = "SSL Factory plugin configuration parameters.") + private String sslFactoryPluginParams = ""; /***** --- Authentication. --- ****/ @FieldContext( @@ -3546,6 +3556,15 @@ public double getLoadBalancerBandwidthOutResourceWeight() { + " used by the internal client to authenticate with Pulsar brokers" ) private Set brokerClientTlsProtocols = new TreeSet<>(); + @FieldContext( + category = CATEGORY_TLS, + doc = "SSL Factory Plugin class used by internal client to provide SSLEngine and SSLContext objects. " + + "The default class used is DefaultSslFactory.") + private String brokerClientSslFactoryPlugin = DefaultPulsarSslFactory.class.getName(); + @FieldContext( + category = CATEGORY_TLS, + doc = "SSL Factory plugin configuration parameters used by internal client.") + private String brokerClientSslFactoryPluginParams = ""; /* packages management service configurations (begin) */ diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/JettySslContextFactory.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/JettySslContextFactory.java index 46a86045995f9..0ac1b78ca993f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/JettySslContextFactory.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/jetty/tls/JettySslContextFactory.java @@ -21,10 +21,8 @@ import java.util.Set; import javax.net.ssl.SSLContext; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.common.util.DefaultSslContextBuilder; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.common.util.SecurityUtility; -import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder; -import org.apache.pulsar.common.util.keystoretls.NetSslContextBuilder; import org.eclipse.jetty.util.ssl.SslContextFactory; @Slf4j @@ -35,57 +33,21 @@ public class JettySslContextFactory { } } - public static SslContextFactory.Server createServerSslContextWithKeystore(String sslProviderString, - String keyStoreTypeString, - String keyStore, - String keyStorePassword, - boolean allowInsecureConnection, - String trustStoreTypeString, - String trustStore, - String trustStorePassword, - boolean requireTrustedClientCertOnConnect, - Set ciphers, - Set protocols, - long certRefreshInSec) { - NetSslContextBuilder sslCtxRefresher = new NetSslContextBuilder( - sslProviderString, - keyStoreTypeString, - keyStore, - keyStorePassword, - allowInsecureConnection, - trustStoreTypeString, - trustStore, - trustStorePassword, - requireTrustedClientCertOnConnect, - certRefreshInSec); - - return new JettySslContextFactory.Server(sslProviderString, sslCtxRefresher, + public static SslContextFactory.Server createSslContextFactory(String sslProviderString, + PulsarSslFactory pulsarSslFactory, + boolean requireTrustedClientCertOnConnect, + Set ciphers, Set protocols) { + return new JettySslContextFactory.Server(sslProviderString, pulsarSslFactory, requireTrustedClientCertOnConnect, ciphers, protocols); } - public static SslContextFactory createServerSslContext(String sslProviderString, boolean tlsAllowInsecureConnection, - String tlsTrustCertsFilePath, - String tlsCertificateFilePath, - String tlsKeyFilePath, - boolean tlsRequireTrustedClientCertOnConnect, - Set ciphers, - Set protocols, - long certRefreshInSec) { - DefaultSslContextBuilder sslCtxRefresher = - new DefaultSslContextBuilder(tlsAllowInsecureConnection, tlsTrustCertsFilePath, tlsCertificateFilePath, - tlsKeyFilePath, tlsRequireTrustedClientCertOnConnect, certRefreshInSec, sslProviderString); - - return new JettySslContextFactory.Server(sslProviderString, sslCtxRefresher, - tlsRequireTrustedClientCertOnConnect, ciphers, protocols); - } - private static class Server extends SslContextFactory.Server { - private final SslContextAutoRefreshBuilder sslCtxRefresher; + private final PulsarSslFactory pulsarSslFactory; - public Server(String sslProviderString, SslContextAutoRefreshBuilder sslCtxRefresher, + public Server(String sslProviderString, PulsarSslFactory pulsarSslFactory, boolean requireTrustedClientCertOnConnect, Set ciphers, Set protocols) { super(); - this.sslCtxRefresher = sslCtxRefresher; + this.pulsarSslFactory = pulsarSslFactory; if (ciphers != null && ciphers.size() > 0) { this.setIncludeCipherSuites(ciphers.toArray(new String[0])); @@ -110,7 +72,7 @@ public Server(String sslProviderString, SslContextAutoRefreshBuilder @Override public SSLContext getSslContext() { - return sslCtxRefresher.get(); + return this.pulsarSslFactory.getInternalSslContext(); } } } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryTest.java index 2f0c8b627d581..019627f52cbcf 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryTest.java @@ -42,6 +42,9 @@ import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.testng.annotations.Test; +import org.apache.pulsar.common.util.DefaultPulsarSslFactory; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; @Slf4j public class JettySslContextFactoryTest { @@ -51,16 +54,20 @@ public void testJettyTlsServerTls() throws Exception { @Cleanup("stop") Server server = new Server(); List connectors = new ArrayList<>(); - SslContextFactory factory = JettySslContextFactory.createServerSslContext( - null, - false, - Resources.getResource("ssl/my-ca/ca.pem").getPath(), - Resources.getResource("ssl/my-ca/server-ca.pem").getPath(), - Resources.getResource("ssl/my-ca/server-key.pem").getPath(), - true, - null, - null, - 600); + PulsarSslConfiguration sslConfiguration = PulsarSslConfiguration.builder() + .tlsTrustCertsFilePath(Resources.getResource("ssl/my-ca/ca.pem").getPath()) + .tlsCertificateFilePath(Resources.getResource("ssl/my-ca/server-ca.pem").getPath()) + .tlsKeyFilePath(Resources.getResource("ssl/my-ca/server-key.pem").getPath()) + .allowInsecureConnection(false) + .requireTrustedClientCertOnConnect(true) + .tlsEnabledWithKeystore(false) + .isHttps(true) + .build(); + PulsarSslFactory sslFactory = new DefaultPulsarSslFactory(); + sslFactory.initialize(sslConfiguration); + sslFactory.createInternalSslContext(); + SslContextFactory factory = JettySslContextFactory.createSslContextFactory(null, + sslFactory, true, null, null); ServerConnector connector = new ServerConnector(server, factory); connector.setPort(0); @@ -85,20 +92,30 @@ public void testJettyTlsServerInvalidTlsProtocol() throws Exception { @Cleanup("stop") Server server = new Server(); List connectors = new ArrayList<>(); - SslContextFactory factory = JettySslContextFactory.createServerSslContext( - null, - false, - Resources.getResource("ssl/my-ca/ca.pem").getPath(), - Resources.getResource("ssl/my-ca/server-ca.pem").getPath(), - Resources.getResource("ssl/my-ca/server-key.pem").getPath(), - true, - null, + PulsarSslConfiguration sslConfiguration = PulsarSslConfiguration.builder() + .tlsProtocols(new HashSet() { + { + this.add("TLSv1.3"); + } + }) + .tlsTrustCertsFilePath(Resources.getResource("ssl/my-ca/ca.pem").getPath()) + .tlsCertificateFilePath(Resources.getResource("ssl/my-ca/server-ca.pem").getPath()) + .tlsKeyFilePath(Resources.getResource("ssl/my-ca/server-key.pem").getPath()) + .allowInsecureConnection(false) + .requireTrustedClientCertOnConnect(true) + .tlsEnabledWithKeystore(false) + .isHttps(true) + .build(); + PulsarSslFactory sslFactory = new DefaultPulsarSslFactory(); + sslFactory.initialize(sslConfiguration); + sslFactory.createInternalSslContext(); + SslContextFactory factory = JettySslContextFactory.createSslContextFactory(null, + sslFactory, true, null, new HashSet() { { this.add("TLSv1.3"); } - }, - 600); + }); factory.setHostnameVerifier((s, sslSession) -> true); ServerConnector connector = new ServerConnector(server, factory); connector.setPort(0); @@ -123,13 +140,30 @@ public void testJettyTlsServerInvalidCipher() throws Exception { @Cleanup("stop") Server server = new Server(); List connectors = new ArrayList<>(); - SslContextFactory factory = JettySslContextFactory.createServerSslContext( - null, - false, - Resources.getResource("ssl/my-ca/ca.pem").getPath(), - Resources.getResource("ssl/my-ca/server-ca.pem").getPath(), - Resources.getResource("ssl/my-ca/server-key.pem").getPath(), - true, + PulsarSslConfiguration sslConfiguration = PulsarSslConfiguration.builder() + .tlsCiphers(new HashSet() { + { + this.add("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256"); + } + }) + .tlsProtocols(new HashSet() { + { + this.add("TLSv1.3"); + } + }) + .tlsTrustCertsFilePath(Resources.getResource("ssl/my-ca/ca.pem").getPath()) + .tlsCertificateFilePath(Resources.getResource("ssl/my-ca/server-ca.pem").getPath()) + .tlsKeyFilePath(Resources.getResource("ssl/my-ca/server-key.pem").getPath()) + .allowInsecureConnection(false) + .requireTrustedClientCertOnConnect(true) + .isHttps(true) + .tlsEnabledWithKeystore(false) + .build(); + PulsarSslFactory sslFactory = new DefaultPulsarSslFactory(); + sslFactory.initialize(sslConfiguration); + sslFactory.createInternalSslContext(); + SslContextFactory factory = JettySslContextFactory.createSslContextFactory(null, + sslFactory, true, new HashSet() { { this.add("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256"); @@ -137,11 +171,9 @@ public void testJettyTlsServerInvalidCipher() throws Exception { }, new HashSet() { { - this.add("TLSv1.2"); + this.add("TLSv1.3"); } - }, - 600); - + }); factory.setHostnameVerifier((s, sslSession) -> true); ServerConnector connector = new ServerConnector(server, factory); connector.setPort(0); diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryWithKeyStoreTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryWithKeyStoreTest.java index f08f62c480c00..30fbc50257d4c 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryWithKeyStoreTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/jetty/tls/JettySslContextFactoryWithKeyStoreTest.java @@ -43,6 +43,9 @@ import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.core.config.Configurator; +import org.apache.pulsar.common.util.DefaultPulsarSslFactory; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.util.ssl.SslContextFactory; @@ -66,10 +69,22 @@ public void testJettyTlsServerTls() throws Exception { @Cleanup("stop") Server server = new Server(); List connectors = new ArrayList<>(); - SslContextFactory.Server factory = JettySslContextFactory.createServerSslContextWithKeystore(null, - keyStoreType, brokerKeyStorePath, keyStorePassword, false, keyStoreType, - clientTrustStorePath, keyStorePassword, true, null, - null, 600); + PulsarSslConfiguration sslConfiguration = PulsarSslConfiguration.builder() + .tlsKeyStoreType(keyStoreType) + .tlsKeyStorePath(brokerKeyStorePath) + .tlsKeyStorePassword(keyStorePassword) + .tlsTrustStoreType(keyStoreType) + .tlsTrustStorePath(clientTrustStorePath) + .tlsTrustStorePassword(keyStorePassword) + .requireTrustedClientCertOnConnect(true) + .tlsEnabledWithKeystore(true) + .isHttps(true) + .build(); + PulsarSslFactory sslFactory = new DefaultPulsarSslFactory(); + sslFactory.initialize(sslConfiguration); + sslFactory.createInternalSslContext(); + SslContextFactory.Server factory = JettySslContextFactory.createSslContextFactory(null, + sslFactory, true, null, null); factory.setHostnameVerifier((s, sslSession) -> true); ServerConnector connector = new ServerConnector(server, factory); connector.setPort(0); @@ -95,14 +110,32 @@ public void testJettyTlsServerInvalidTlsProtocol() throws Exception { @Cleanup("stop") Server server = new Server(); List connectors = new ArrayList<>(); - SslContextFactory.Server factory = JettySslContextFactory.createServerSslContextWithKeystore(null, - keyStoreType, brokerKeyStorePath, keyStorePassword, false, keyStoreType, clientTrustStorePath, - keyStorePassword, true, null, + PulsarSslConfiguration sslConfiguration = PulsarSslConfiguration.builder() + .tlsKeyStoreType(keyStoreType) + .tlsKeyStorePath(brokerKeyStorePath) + .tlsKeyStorePassword(keyStorePassword) + .tlsTrustStoreType(keyStoreType) + .tlsTrustStorePath(clientTrustStorePath) + .tlsTrustStorePassword(keyStorePassword) + .tlsProtocols(new HashSet() { + { + this.add("TLSv1.3"); + } + }) + .requireTrustedClientCertOnConnect(true) + .tlsEnabledWithKeystore(true) + .isHttps(true) + .build(); + PulsarSslFactory sslFactory = new DefaultPulsarSslFactory(); + sslFactory.initialize(sslConfiguration); + sslFactory.createInternalSslContext(); + SslContextFactory.Server factory = JettySslContextFactory.createSslContextFactory(null, + sslFactory, true, null, new HashSet() { { this.add("TLSv1.3"); } - }, 600); + }); factory.setHostnameVerifier((s, sslSession) -> true); ServerConnector connector = new ServerConnector(server, factory); connector.setPort(0); @@ -127,9 +160,33 @@ public void testJettyTlsServerInvalidCipher() throws Exception { @Cleanup("stop") Server server = new Server(); List connectors = new ArrayList<>(); - SslContextFactory.Server factory = JettySslContextFactory.createServerSslContextWithKeystore(null, - keyStoreType, brokerKeyStorePath, keyStorePassword, false, keyStoreType, clientTrustStorePath, - keyStorePassword, true, new HashSet() { + PulsarSslConfiguration sslConfiguration = PulsarSslConfiguration.builder() + .tlsKeyStoreType(keyStoreType) + .tlsKeyStorePath(brokerKeyStorePath) + .tlsKeyStorePassword(keyStorePassword) + .tlsTrustStoreType(keyStoreType) + .tlsTrustStorePath(clientTrustStorePath) + .tlsTrustStorePassword(keyStorePassword) + .tlsCiphers(new HashSet() { + { + this.add("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256"); + } + }) + .tlsProtocols(new HashSet() { + { + this.add("TLSv1.3"); + } + }) + .requireTrustedClientCertOnConnect(true) + .tlsEnabledWithKeystore(true) + .isHttps(true) + .build(); + PulsarSslFactory sslFactory = new DefaultPulsarSslFactory(); + sslFactory.initialize(sslConfiguration); + sslFactory.createInternalSslContext(); + SslContextFactory.Server factory = JettySslContextFactory.createSslContextFactory(null, + sslFactory, true, + new HashSet() { { this.add("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256"); } @@ -137,8 +194,9 @@ public void testJettyTlsServerInvalidCipher() throws Exception { new HashSet() { { this.add("TLSv1.2"); + this.add("TLSv1.3"); } - }, 600); + }); factory.setHostnameVerifier((s, sslSession) -> true); ServerConnector connector = new ServerConnector(server, factory); connector.setPort(0); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 3d57a3bc01042..9e147517ac724 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -1689,6 +1689,8 @@ public synchronized PulsarClient getClient() throws PulsarServerException { conf.setTlsProtocols(this.getConfiguration().getBrokerClientTlsProtocols()); conf.setTlsAllowInsecureConnection(this.getConfiguration().isTlsAllowInsecureConnection()); conf.setTlsHostnameVerificationEnable(this.getConfiguration().isTlsHostnameVerificationEnabled()); + conf.setSslFactoryPlugin(this.getConfiguration().getBrokerClientSslFactoryPlugin()); + conf.setSslFactoryPluginParams(this.getConfiguration().getBrokerClientSslFactoryPluginParams()); if (this.getConfiguration().isBrokerClientTlsEnabledWithKeyStore()) { conf.setUseKeyStoreTls(true); conf.setTlsTrustStoreType(this.getConfiguration().getBrokerClientTlsTrustStoreType()); @@ -1739,15 +1741,17 @@ public synchronized PulsarAdmin getAdminClient() throws PulsarServerException { // Apply all arbitrary configuration. This must be called before setting any fields annotated as // @Secret on the ClientConfigurationData object because of the way they are serialized. // See https://github.com/apache/pulsar/issues/8509 for more information. - builder.loadConf(PropertiesUtils.filterAndMapProperties(config.getProperties(), "brokerClient_")); + builder.loadConf(PropertiesUtils.filterAndMapProperties(conf.getProperties(), "brokerClient_")); builder.authentication( conf.getBrokerClientAuthenticationPlugin(), conf.getBrokerClientAuthenticationParameters()); if (conf.isBrokerClientTlsEnabled()) { - builder.tlsCiphers(config.getBrokerClientTlsCiphers()) - .tlsProtocols(config.getBrokerClientTlsProtocols()); + builder.tlsCiphers(conf.getBrokerClientTlsCiphers()) + .tlsProtocols(conf.getBrokerClientTlsProtocols()) + .sslFactoryPlugin(conf.getBrokerClientSslFactoryPlugin()) + .sslFactoryPluginParams(conf.getBrokerClientSslFactoryPluginParams()); if (conf.isBrokerClientTlsEnabledWithKeyStore()) { builder.useKeyStoreTls(true).tlsTrustStoreType(conf.getBrokerClientTlsTrustStoreType()) .tlsTrustStorePath(conf.getBrokerClientTlsTrustStore()) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index f3fb17c02fcee..92188f5e6eeee 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -1677,7 +1677,9 @@ public PulsarClientImpl getNamespaceClient(ClusterDataImpl cluster) { .enableTls(true) .tlsTrustCertsFilePath(pulsar.getConfiguration().getBrokerClientTrustCertsFilePath()) .allowTlsInsecureConnection(pulsar.getConfiguration().isTlsAllowInsecureConnection()) - .enableTlsHostnameVerification(pulsar.getConfiguration().isTlsHostnameVerificationEnabled()); + .enableTlsHostnameVerification(pulsar.getConfiguration().isTlsHostnameVerificationEnabled()) + .sslFactoryPlugin(pulsar.getConfiguration().getBrokerClientSslFactoryPlugin()) + .sslFactoryPluginParams(pulsar.getConfiguration().getBrokerClientSslFactoryPluginParams()); } else { clientBuilder.serviceUrl(isNotBlank(cluster.getBrokerServiceUrl()) ? cluster.getBrokerServiceUrl() : cluster.getServiceUrl()); 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 338d5f420ca25..066bfc98cc0cf 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 @@ -1389,7 +1389,9 @@ public PulsarClient getReplicationClient(String cluster, Optional c data.getBrokerClientTrustCertsFilePath(), data.getBrokerClientKeyFilePath(), data.getBrokerClientCertificateFilePath(), - pulsar.getConfiguration().isTlsHostnameVerificationEnabled() + pulsar.getConfiguration().isTlsHostnameVerificationEnabled(), + data.getBrokerClientSslFactoryPlugin(), + data.getBrokerClientSslFactoryPluginParams() ); } else if (pulsar.getConfiguration().isBrokerClientTlsEnabled()) { configTlsSettings(clientBuilder, serviceUrlTls, @@ -1404,7 +1406,9 @@ public PulsarClient getReplicationClient(String cluster, Optional c pulsar.getConfiguration().getBrokerClientTrustCertsFilePath(), pulsar.getConfiguration().getBrokerClientKeyFilePath(), pulsar.getConfiguration().getBrokerClientCertificateFilePath(), - pulsar.getConfiguration().isTlsHostnameVerificationEnabled() + pulsar.getConfiguration().isTlsHostnameVerificationEnabled(), + pulsar.getConfiguration().getBrokerClientSslFactoryPlugin(), + pulsar.getConfiguration().getBrokerClientSslFactoryPluginParams() ); } else { clientBuilder.serviceUrl( @@ -1435,11 +1439,16 @@ private void configTlsSettings(ClientBuilder clientBuilder, String serviceUrl, String brokerClientTlsKeyStore, String brokerClientTlsKeyStorePassword, String brokerClientTrustCertsFilePath, String brokerClientKeyFilePath, String brokerClientCertificateFilePath, - boolean isTlsHostnameVerificationEnabled) { + boolean isTlsHostnameVerificationEnabled, String brokerClientSslFactoryPlugin, + String brokerClientSslFactoryPluginParams) { clientBuilder .serviceUrl(serviceUrl) .allowTlsInsecureConnection(isTlsAllowInsecureConnection) .enableTlsHostnameVerification(isTlsHostnameVerificationEnabled); + if (StringUtils.isNotBlank(brokerClientSslFactoryPlugin)) { + clientBuilder.sslFactoryPlugin(brokerClientSslFactoryPlugin) + .sslFactoryPluginParams(brokerClientSslFactoryPluginParams); + } if (brokerClientTlsEnabledWithKeyStore) { clientBuilder.useKeyStoreTls(true) .tlsTrustStoreType(brokerClientTlsTrustStoreType) @@ -1462,7 +1471,8 @@ private void configAdminTlsSettings(PulsarAdminBuilder adminBuilder, boolean bro String brokerClientTlsKeyStore, String brokerClientTlsKeyStorePassword, String brokerClientTrustCertsFilePath, String brokerClientKeyFilePath, String brokerClientCertificateFilePath, - boolean isTlsHostnameVerificationEnabled) { + boolean isTlsHostnameVerificationEnabled, String brokerClientSslFactoryPlugin, + String brokerClientSslFactoryPluginParams) { if (brokerClientTlsEnabledWithKeyStore) { adminBuilder.useKeyStoreTls(true) .tlsTrustStoreType(brokerClientTlsTrustStoreType) @@ -1477,7 +1487,9 @@ private void configAdminTlsSettings(PulsarAdminBuilder adminBuilder, boolean bro .tlsCertificateFilePath(brokerClientCertificateFilePath); } adminBuilder.allowTlsInsecureConnection(isTlsAllowInsecureConnection) - .enableTlsHostnameVerification(isTlsHostnameVerificationEnabled); + .enableTlsHostnameVerification(isTlsHostnameVerificationEnabled) + .sslFactoryPlugin(brokerClientSslFactoryPlugin) + .sslFactoryPluginParams(brokerClientSslFactoryPluginParams); } public PulsarAdmin getClusterPulsarAdmin(String cluster, Optional clusterDataOp) { @@ -1524,7 +1536,9 @@ public PulsarAdmin getClusterPulsarAdmin(String cluster, Optional c data.getBrokerClientTrustCertsFilePath(), data.getBrokerClientKeyFilePath(), data.getBrokerClientCertificateFilePath(), - pulsar.getConfiguration().isTlsHostnameVerificationEnabled() + pulsar.getConfiguration().isTlsHostnameVerificationEnabled(), + data.getBrokerClientSslFactoryPlugin(), + data.getBrokerClientSslFactoryPluginParams() ); } else if (conf.isBrokerClientTlsEnabled()) { configAdminTlsSettings(builder, @@ -1539,7 +1553,9 @@ public PulsarAdmin getClusterPulsarAdmin(String cluster, Optional c conf.getBrokerClientTrustCertsFilePath(), conf.getBrokerClientKeyFilePath(), conf.getBrokerClientCertificateFilePath(), - pulsar.getConfiguration().isTlsHostnameVerificationEnabled() + pulsar.getConfiguration().isTlsHostnameVerificationEnabled(), + conf.getBrokerClientSslFactoryPlugin(), + conf.getBrokerClientSslFactoryPluginParams() ); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java index f15f6d67766f1..3b78d5931599e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java @@ -25,9 +25,8 @@ import io.netty.handler.codec.LengthFieldBasedFrameDecoder; import io.netty.handler.flow.FlowControlHandler; import io.netty.handler.flush.FlushConsolidationHandler; -import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; -import io.netty.handler.ssl.SslProvider; +import java.util.concurrent.TimeUnit; import lombok.Builder; import lombok.Data; import lombok.extern.slf4j.Slf4j; @@ -36,9 +35,8 @@ import org.apache.pulsar.common.protocol.ByteBufPair; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.OptionalProxyProtocolDecoder; -import org.apache.pulsar.common.util.NettyServerSslContextBuilder; -import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder; -import org.apache.pulsar.common.util.keystoretls.NettySSLContextAutoRefreshBuilder; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; @Slf4j public class PulsarChannelInitializer extends ChannelInitializer { @@ -48,10 +46,8 @@ public class PulsarChannelInitializer extends ChannelInitializer private final PulsarService pulsar; private final String listenerName; private final boolean enableTls; - private final boolean tlsEnabledWithKeyStore; - private SslContextAutoRefreshBuilder sslCtxRefresher; private final ServiceConfiguration brokerConf; - private NettySSLContextAutoRefreshBuilder nettySSLContextAutoRefreshBuilder; + private PulsarSslFactory sslFactory; /** * @param pulsar @@ -65,40 +61,18 @@ public PulsarChannelInitializer(PulsarService pulsar, PulsarChannelOptions opts) this.listenerName = opts.getListenerName(); this.enableTls = opts.isEnableTLS(); ServiceConfiguration serviceConfig = pulsar.getConfiguration(); - this.tlsEnabledWithKeyStore = serviceConfig.isTlsEnabledWithKeyStore(); if (this.enableTls) { - if (tlsEnabledWithKeyStore) { - nettySSLContextAutoRefreshBuilder = new NettySSLContextAutoRefreshBuilder( - serviceConfig.getTlsProvider(), - serviceConfig.getTlsKeyStoreType(), - serviceConfig.getTlsKeyStore(), - serviceConfig.getTlsKeyStorePassword(), - serviceConfig.isTlsAllowInsecureConnection(), - serviceConfig.getTlsTrustStoreType(), - serviceConfig.getTlsTrustStore(), - serviceConfig.getTlsTrustStorePassword(), - serviceConfig.isTlsRequireTrustedClientCertOnConnect(), - serviceConfig.getTlsCiphers(), - serviceConfig.getTlsProtocols(), - serviceConfig.getTlsCertRefreshCheckDurationSec()); - } else { - SslProvider sslProvider = null; - if (serviceConfig.getTlsProvider() != null) { - sslProvider = SslProvider.valueOf(serviceConfig.getTlsProvider()); - } - sslCtxRefresher = new NettyServerSslContextBuilder( - sslProvider, - serviceConfig.isTlsAllowInsecureConnection(), - serviceConfig.getTlsTrustCertsFilePath(), - serviceConfig.getTlsCertificateFilePath(), - serviceConfig.getTlsKeyFilePath(), - serviceConfig.getTlsCiphers(), - serviceConfig.getTlsProtocols(), - serviceConfig.isTlsRequireTrustedClientCertOnConnect(), - serviceConfig.getTlsCertRefreshCheckDurationSec()); + PulsarSslConfiguration pulsarSslConfig = buildSslConfiguration(serviceConfig); + this.sslFactory = (PulsarSslFactory) Class.forName(serviceConfig.getSslFactoryPlugin()) + .getConstructor().newInstance(); + this.sslFactory.initialize(pulsarSslConfig); + this.sslFactory.createInternalSslContext(); + if (serviceConfig.getTlsCertRefreshCheckDurationSec() > 0) { + this.pulsar.getExecutor().scheduleWithFixedDelay(this::refreshSslContext, + serviceConfig.getTlsCertRefreshCheckDurationSec(), + serviceConfig.getTlsCertRefreshCheckDurationSec(), + TimeUnit.SECONDS); } - } else { - this.sslCtxRefresher = null; } this.brokerConf = pulsar.getConfiguration(); } @@ -110,12 +84,7 @@ protected void initChannel(SocketChannel ch) throws Exception { ch.config().setAutoRead(false); ch.pipeline().addLast("consolidation", new FlushConsolidationHandler(1024, true)); if (this.enableTls) { - if (this.tlsEnabledWithKeyStore) { - ch.pipeline().addLast(TLS_HANDLER, - new SslHandler(nettySSLContextAutoRefreshBuilder.get().createSSLEngine())); - } else { - ch.pipeline().addLast(TLS_HANDLER, sslCtxRefresher.get().newHandler(ch.alloc())); - } + ch.pipeline().addLast(TLS_HANDLER, new SslHandler(this.sslFactory.createServerSslEngine(ch.alloc()))); } ch.pipeline().addLast("ByteBufPairEncoder", ByteBufPair.getEncoder(this.enableTls)); @@ -161,4 +130,33 @@ public static class PulsarChannelOptions { */ private String listenerName; } + + protected PulsarSslConfiguration buildSslConfiguration(ServiceConfiguration serviceConfig) { + return PulsarSslConfiguration.builder() + .tlsKeyStoreType(serviceConfig.getTlsKeyStoreType()) + .tlsKeyStorePath(serviceConfig.getTlsKeyStore()) + .tlsKeyStorePassword(serviceConfig.getTlsKeyStorePassword()) + .tlsTrustStoreType(serviceConfig.getTlsTrustStoreType()) + .tlsTrustStorePath(serviceConfig.getTlsTrustStore()) + .tlsTrustStorePassword(serviceConfig.getTlsTrustStorePassword()) + .tlsCiphers(serviceConfig.getTlsCiphers()) + .tlsProtocols(serviceConfig.getTlsProtocols()) + .tlsTrustCertsFilePath(serviceConfig.getTlsTrustCertsFilePath()) + .tlsCertificateFilePath(serviceConfig.getTlsCertificateFilePath()) + .tlsKeyFilePath(serviceConfig.getTlsKeyFilePath()) + .allowInsecureConnection(serviceConfig.isTlsAllowInsecureConnection()) + .requireTrustedClientCertOnConnect(serviceConfig.isTlsRequireTrustedClientCertOnConnect()) + .tlsEnabledWithKeystore(serviceConfig.isTlsEnabledWithKeyStore()) + .tlsCustomParams(serviceConfig.getSslFactoryPluginParams()) + .serverMode(true) + .build(); + } + + protected void refreshSslContext() { + try { + this.sslFactory.update(); + } catch (Exception e) { + log.error("Failed to refresh SSL context", e); + } + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java index d95e88661ae8c..5f5e260890a02 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/WebService.java @@ -27,6 +27,8 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; import javax.servlet.DispatcherType; import javax.servlet.Filter; import javax.servlet.FilterChain; @@ -39,6 +41,8 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.jetty.tls.JettySslContextFactory; import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; @@ -93,6 +97,8 @@ public class WebService implements AutoCloseable { private final ServerConnector httpsConnector; private final FilterInitializer filterInitializer; private JettyStatisticsCollector jettyStatisticsCollector; + private PulsarSslFactory sslFactory; + private ScheduledFuture sslContextRefreshTask; @Getter private static final DynamicSkipUnknownPropertyHandler sharedUnknownPropertyHandler = @@ -144,34 +150,22 @@ public WebService(PulsarService pulsar) throws PulsarServerException { Optional tlsPort = config.getWebServicePortTls(); if (tlsPort.isPresent()) { try { - SslContextFactory sslCtxFactory; - if (config.isTlsEnabledWithKeyStore()) { - sslCtxFactory = JettySslContextFactory.createServerSslContextWithKeystore( - config.getWebServiceTlsProvider(), - config.getTlsKeyStoreType(), - config.getTlsKeyStore(), - config.getTlsKeyStorePassword(), - config.isTlsAllowInsecureConnection(), - config.getTlsTrustStoreType(), - config.getTlsTrustStore(), - config.getTlsTrustStorePassword(), - config.isTlsRequireTrustedClientCertOnConnect(), - config.getWebServiceTlsCiphers(), - config.getWebServiceTlsProtocols(), - config.getTlsCertRefreshCheckDurationSec() - ); - } else { - sslCtxFactory = JettySslContextFactory.createServerSslContext( - config.getWebServiceTlsProvider(), - config.isTlsAllowInsecureConnection(), - config.getTlsTrustCertsFilePath(), - config.getTlsCertificateFilePath(), - config.getTlsKeyFilePath(), - config.isTlsRequireTrustedClientCertOnConnect(), - config.getWebServiceTlsCiphers(), - config.getWebServiceTlsProtocols(), - config.getTlsCertRefreshCheckDurationSec()); + PulsarSslConfiguration sslConfiguration = buildSslConfiguration(config); + this.sslFactory = (PulsarSslFactory) Class.forName(config.getSslFactoryPlugin()) + .getConstructor().newInstance(); + this.sslFactory.initialize(sslConfiguration); + this.sslFactory.createInternalSslContext(); + if (config.getTlsCertRefreshCheckDurationSec() > 0) { + this.sslContextRefreshTask = this.pulsar.getExecutor() + .scheduleWithFixedDelay(this::refreshSslContext, + config.getTlsCertRefreshCheckDurationSec(), + config.getTlsCertRefreshCheckDurationSec(), + TimeUnit.SECONDS); } + SslContextFactory sslCtxFactory = + JettySslContextFactory.createSslContextFactory(config.getWebServiceTlsProvider(), + this.sslFactory, config.isTlsRequireTrustedClientCertOnConnect(), + config.getTlsCiphers(), config.getTlsProtocols()); List connectionFactories = new ArrayList<>(); if (config.isWebServiceHaProxyProtocolEnabled()) { connectionFactories.add(new ProxyConnectionFactory()); @@ -431,6 +425,9 @@ public void close() throws PulsarServerException { jettyStatisticsCollector = null; } webServiceExecutor.join(); + if (this.sslContextRefreshTask != null) { + this.sslContextRefreshTask.cancel(true); + } webExecutorThreadPoolStats.close(); this.executorStats.close(); log.info("Web service closed"); @@ -455,5 +452,35 @@ public Optional getListenPortHTTPS() { } } + protected PulsarSslConfiguration buildSslConfiguration(ServiceConfiguration serviceConfig) { + return PulsarSslConfiguration.builder() + .tlsKeyStoreType(serviceConfig.getTlsKeyStoreType()) + .tlsKeyStorePath(serviceConfig.getTlsKeyStore()) + .tlsKeyStorePassword(serviceConfig.getTlsKeyStorePassword()) + .tlsTrustStoreType(serviceConfig.getTlsTrustStoreType()) + .tlsTrustStorePath(serviceConfig.getTlsTrustStore()) + .tlsTrustStorePassword(serviceConfig.getTlsTrustStorePassword()) + .tlsCiphers(serviceConfig.getTlsCiphers()) + .tlsProtocols(serviceConfig.getTlsProtocols()) + .tlsTrustCertsFilePath(serviceConfig.getTlsTrustCertsFilePath()) + .tlsCertificateFilePath(serviceConfig.getTlsCertificateFilePath()) + .tlsKeyFilePath(serviceConfig.getTlsKeyFilePath()) + .allowInsecureConnection(serviceConfig.isTlsAllowInsecureConnection()) + .requireTrustedClientCertOnConnect(serviceConfig.isTlsRequireTrustedClientCertOnConnect()) + .tlsEnabledWithKeystore(serviceConfig.isTlsEnabledWithKeyStore()) + .tlsCustomParams(serviceConfig.getSslFactoryPluginParams()) + .serverMode(true) + .isHttps(true) + .build(); + } + + protected void refreshSslContext() { + try { + this.sslFactory.update(); + } catch (Exception e) { + log.error("Failed to refresh SSL context", e); + } + } + private static final Logger log = LoggerFactory.getLogger(WebService.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java index 7d35c2c0f7b9e..fe77db33692b9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java @@ -86,7 +86,9 @@ public static PulsarClient createClient(ServiceConfiguration brokerConfig) throw if (internalListener.getBrokerServiceUrlTls() != null && brokerConfig.isBrokerClientTlsEnabled()) { clientBuilder.serviceUrl(internalListener.getBrokerServiceUrlTls().toString()) .allowTlsInsecureConnection(brokerConfig.isTlsAllowInsecureConnection()) - .enableTlsHostnameVerification(brokerConfig.isTlsHostnameVerificationEnabled()); + .enableTlsHostnameVerification(brokerConfig.isTlsHostnameVerificationEnabled()) + .sslFactoryPlugin(brokerConfig.getBrokerClientSslFactoryPlugin()) + .sslFactoryPluginParams(brokerConfig.getBrokerClientSslFactoryPluginParams()); if (brokerConfig.isBrokerClientTlsEnabledWithKeyStore()) { clientBuilder.useKeyStoreTls(true) .tlsKeyStoreType(brokerConfig.getBrokerClientTlsKeyStoreType()) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java index 3bbf423da6ef3..dd2f9288071a5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java @@ -67,6 +67,9 @@ protected void doInitConf() throws Exception { conf.setWebServicePort(Optional.of(8081)); conf.setWebServicePortTls(Optional.of(8082)); } + conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsTest.java index c9457e1a8883f..8940fe4a1f3c3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsTest.java @@ -326,6 +326,9 @@ public void testLookUpWithRedirect() throws Exception { conf.setBrokerServicePortTls(Optional.of(0)); conf.setWebServicePort(Optional.of(0)); conf.setWebServicePortTls(Optional.of(0)); + conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); @Cleanup PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(conf); PulsarService pulsar2 = pulsarTestContext2.getPulsarService(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java index 95aafd84ae406..acd918b55fe1b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java @@ -63,6 +63,7 @@ import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; import org.apache.pulsar.policies.data.loadbalancer.ResourceUsage; import org.apache.pulsar.policies.data.loadbalancer.SystemResourceUsage; +import org.apache.pulsar.utils.ResourceUtils; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.ZooDefs; @@ -83,6 +84,14 @@ */ @Test(groups = "broker") public class LoadBalancerTest { + + public final static String CA_CERT_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/certs/ca.cert.pem"); + public final static String BROKER_CERT_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/server-keys/broker.cert.pem"); + public final static String BROKER_KEY_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/server-keys/broker.key-pk8.pem"); + LocalBookkeeperEnsemble bkEnsemble; private static final Logger log = LoggerFactory.getLogger(LoadBalancerTest.class); @@ -126,6 +135,9 @@ void setup() throws Exception { config.setLoadManagerClassName(SimpleLoadManagerImpl.class.getName()); config.setAdvertisedAddress(localhost+i); config.setLoadBalancerEnabled(false); + config.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + config.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + config.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); pulsarServices[i] = new PulsarService(config); pulsarServices[i].start(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleBrokerStartTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleBrokerStartTest.java index 28dde8b7f559d..31c8c9f3bccc1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleBrokerStartTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleBrokerStartTest.java @@ -20,6 +20,7 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.google.common.io.Resources; import java.nio.file.Files; import java.nio.file.Paths; import java.util.Optional; @@ -37,6 +38,13 @@ @Test(groups = "broker") public class SimpleBrokerStartTest { + final static String caCertPath = Resources.getResource("certificate-authority/certs/ca.cert.pem") + .getPath(); + final static String brokerCertPath = + Resources.getResource("certificate-authority/server-keys/broker.cert.pem").getPath(); + final static String brokerKeyPath = + Resources.getResource("certificate-authority/server-keys/broker.key-pk8.pem").getPath(); + public void testHasNICSpeed() throws Exception { if (!LinuxInfoUtils.isLinux()) { return; @@ -57,6 +65,9 @@ public void testHasNICSpeed() throws Exception { config.setBrokerServicePortTls(Optional.of(0)); config.setWebServicePortTls(Optional.of(0)); config.setAdvertisedAddress("localhost"); + config.setTlsTrustCertsFilePath(caCertPath); + config.setTlsCertificateFilePath(brokerCertPath); + config.setTlsKeyFilePath(brokerKeyPath); boolean hasNicSpeeds = LinuxInfoUtils.checkHasNicSpeeds(); if (hasNicSpeeds) { @Cleanup @@ -85,6 +96,9 @@ public void testNoNICSpeed() throws Exception { config.setBrokerServicePortTls(Optional.of(0)); config.setWebServicePortTls(Optional.of(0)); config.setAdvertisedAddress("localhost"); + config.setTlsTrustCertsFilePath(caCertPath); + config.setTlsCertificateFilePath(brokerCertPath); + config.setTlsKeyFilePath(brokerKeyPath); boolean hasNicSpeeds = LinuxInfoUtils.checkHasNicSpeeds(); if (!hasNicSpeeds) { @Cleanup diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java index acf096751d769..1e91230559b0a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java @@ -77,6 +77,7 @@ import org.apache.pulsar.policies.data.loadbalancer.ResourceUnitRanking; import org.apache.pulsar.policies.data.loadbalancer.ResourceUsage; import org.apache.pulsar.policies.data.loadbalancer.SystemResourceUsage; +import org.apache.pulsar.utils.ResourceUtils; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -86,6 +87,14 @@ @Slf4j @Test(groups = "broker") public class SimpleLoadManagerImplTest { + + public final static String CA_CERT_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/certs/ca.cert.pem"); + public final static String BROKER_CERT_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/server-keys/broker.cert.pem"); + public final static String BROKER_KEY_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/server-keys/broker.key-pk8.pem"); + LocalBookkeeperEnsemble bkEnsemble; URL url1; @@ -129,6 +138,9 @@ void setup() throws Exception { config1.setLoadManagerClassName(SimpleLoadManagerImpl.class.getName()); config1.setBrokerServicePortTls(Optional.of(0)); config1.setAdvertisedAddress("localhost"); + config1.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + config1.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + config1.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); pulsar1 = new PulsarService(config1); pulsar1.start(); @@ -150,6 +162,9 @@ void setup() throws Exception { config2.setLoadManagerClassName(SimpleLoadManagerImpl.class.getName()); config2.setBrokerServicePortTls(Optional.of(0)); config2.setWebServicePortTls(Optional.of(0)); + config2.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + config2.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + config2.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); config2.setAdvertisedAddress("localhost"); pulsar2 = new PulsarService(config2); pulsar2.start(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java index cce16061506a1..e9fafa9c30317 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java @@ -21,6 +21,7 @@ import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import com.google.common.collect.Sets; +import com.google.common.io.Resources; import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.commons.lang3.reflect.FieldUtils; @@ -44,6 +45,12 @@ public abstract class ExtensibleLoadManagerImplBaseTest extends MockedPulsarServiceBaseTest { + final static String caCertPath = Resources.getResource("certificate-authority/certs/ca.cert.pem").getPath(); + final static String brokerCertPath = + Resources.getResource("certificate-authority/server-keys/broker.cert.pem").getPath(); + final static String brokerKeyPath = + Resources.getResource("certificate-authority/server-keys/broker.key-pk8.pem").getPath(); + protected PulsarService pulsar1; protected PulsarService pulsar2; @@ -79,6 +86,9 @@ protected ServiceConfiguration updateConfig(ServiceConfiguration conf) { conf.setLoadBalancerDebugModeEnabled(true); conf.setWebServicePortTls(Optional.of(0)); conf.setBrokerServicePortTls(Optional.of(0)); + conf.setTlsCertificateFilePath(brokerCertPath); + conf.setTlsKeyFilePath(brokerKeyPath); + conf.setTlsTrustCertsFilePath(caCertPath); return conf; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTaskTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTaskTest.java index bc49352f41d21..74e692e3d7de0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTaskTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/BundleSplitterTaskTest.java @@ -28,6 +28,7 @@ import org.apache.pulsar.policies.data.loadbalancer.LocalBrokerData; import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; import org.apache.pulsar.policies.data.loadbalancer.TimeAverageMessageData; +import org.apache.pulsar.utils.ResourceUtils; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -45,6 +46,13 @@ @Test(groups = "broker") public class BundleSplitterTaskTest { + public final static String CA_CERT_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/certs/ca.cert.pem"); + public final static String BROKER_CERT_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/server-keys/broker.cert.pem"); + public final static String BROKER_KEY_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/server-keys/broker.key-pk8.pem"); + private LocalBookkeeperEnsemble bkEnsemble; private PulsarService pulsar; @@ -67,6 +75,9 @@ void setup() throws Exception { config.setBrokerServicePort(Optional.of(0)); config.setBrokerServicePortTls(Optional.of(0)); config.setWebServicePortTls(Optional.of(0)); + config.setTlsCertificateFilePath(CA_CERT_FILE_PATH); + config.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + config.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); pulsar = new PulsarService(config); pulsar.start(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java index 68bef8b241c7b..aceeefe304b62 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java @@ -99,6 +99,7 @@ import org.apache.pulsar.policies.data.loadbalancer.SystemResourceUsage; import org.apache.pulsar.policies.data.loadbalancer.TimeAverageBrokerData; import org.apache.pulsar.policies.data.loadbalancer.TimeAverageMessageData; +import org.apache.pulsar.utils.ResourceUtils; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.awaitility.Awaitility; import org.mockito.Mockito; @@ -111,6 +112,14 @@ @Slf4j @Test(groups = "broker") public class ModularLoadManagerImplTest { + + public final static String CA_CERT_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/certs/ca.cert.pem"); + public final static String BROKER_CERT_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/server-keys/broker.cert.pem"); + public final static String BROKER_KEY_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/server-keys/broker.key-pk8.pem"); + private LocalBookkeeperEnsemble bkEnsemble; private URL url1; @@ -180,6 +189,9 @@ void setup() throws Exception { config1.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config1.setBrokerServicePort(Optional.of(0)); config1.setBrokerServicePortTls(Optional.of(0)); + config1.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + config1.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + config1.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); pulsar1 = new PulsarService(config1); pulsar1.start(); @@ -200,6 +212,9 @@ void setup() throws Exception { config2.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config2.setBrokerServicePort(Optional.of(0)); config2.setBrokerServicePortTls(Optional.of(0)); + config2.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + config2.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + config2.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); pulsar2 = new PulsarService(config2); pulsar2.start(); @@ -215,6 +230,9 @@ void setup() throws Exception { config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config.setBrokerServicePort(Optional.of(0)); config.setBrokerServicePortTls(Optional.of(0)); + config.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + config.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + config.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); pulsar3 = new PulsarService(config); secondaryBrokerId = pulsar2.getBrokerId(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 172842b5ed3bf..aa236e09da99d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -748,7 +748,7 @@ public void testTlsEnabled() throws Exception { fail("should fail"); } catch (Exception e) { - assertTrue(e.getMessage().contains("General OpenSslEngine problem")); + assertTrue(e.getMessage().contains("unable to find valid certification path to requested target")); } finally { pulsarClient.close(); } @@ -1034,7 +1034,7 @@ protected void handlePartitionResponse(CommandPartitionedTopicMetadataResponse l } super.handlePartitionResponse(lookupResult); } - })) { + }, null)) { // for PMR // 2 lookup will succeed long reqId1 = reqId++; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java index c6a94833c4c62..ddf0fae13545e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java @@ -190,7 +190,7 @@ public void testLookupThrottlingForClientByBroker() throws Exception { EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(20, false, new DefaultThreadFactory("test-pool", Thread.currentThread().isDaemon())); ExecutorService executor = Executors.newFixedThreadPool(10); - try (ConnectionPool pool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoop)) { + try (ConnectionPool pool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoop, null)) { final int totalConsumers = 20; List> futures = new ArrayList<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java index a1cb4abc4c30b..787b4d3154e90 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CanReconnectZKClientPulsarServiceBaseTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service; import com.google.common.collect.Sets; +import com.google.common.io.Resources; import io.netty.channel.Channel; import java.net.URL; import java.nio.channels.SelectionKey; @@ -45,6 +46,12 @@ public abstract class CanReconnectZKClientPulsarServiceBaseTest extends TestRetrySupport { protected final String defaultTenant = "public"; protected final String defaultNamespace = defaultTenant + "/default"; + final static String caCertPath = Resources.getResource("certificate-authority/certs/ca.cert.pem") + .getPath(); + final static String brokerCertPath = + Resources.getResource("certificate-authority/server-keys/broker.cert.pem").getPath(); + final static String brokerKeyPath = + Resources.getResource("certificate-authority/server-keys/broker.key-pk8.pem").getPath(); protected int numberOfBookies = 3; protected final String clusterName = "r1"; protected URL url; @@ -188,6 +195,9 @@ private void setConfigDefaults(ServiceConfiguration config, String clusterName, config.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); config.setEnableReplicatedSubscriptions(true); config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); + config.setTlsTrustCertsFilePath(caCertPath); + config.setTlsCertificateFilePath(brokerCertPath); + config.setTlsKeyFilePath(brokerKeyPath); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java index 20e13023cacfb..8ec565f7d4566 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java @@ -1104,6 +1104,9 @@ protected void doInitConf() throws Exception { this.conf.setLoadManagerClassName(loadManagerClassName); this.conf.setWebServicePortTls(Optional.of(0)); this.conf.setBrokerServicePortTls(Optional.of(0)); + this.conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + this.conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + this.conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/GeoReplicationWithConfigurationSyncTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/GeoReplicationWithConfigurationSyncTestBase.java index 9b4dd5192e1ec..1362a046247d8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/GeoReplicationWithConfigurationSyncTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/GeoReplicationWithConfigurationSyncTestBase.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service; import com.google.common.collect.Sets; +import com.google.common.io.Resources; import java.net.URL; import java.util.Collections; import java.util.Optional; @@ -39,6 +40,12 @@ public abstract class GeoReplicationWithConfigurationSyncTestBase extends TestRe protected final String defaultTenant = "public"; protected final String defaultNamespace = defaultTenant + "/default"; + final static String caCertPath = Resources.getResource("certificate-authority/certs/ca.cert.pem") + .getPath(); + final static String brokerCertPath = + Resources.getResource("certificate-authority/server-keys/broker.cert.pem").getPath(); + final static String brokerKeyPath = + Resources.getResource("certificate-authority/server-keys/broker.key-pk8.pem").getPath(); protected final String cluster1 = "r1"; protected URL url1; @@ -175,6 +182,9 @@ protected void setConfigDefaults(ServiceConfiguration config, String clusterName config.setEnableReplicatedSubscriptions(true); config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); config.setLoadBalancerSheddingEnabled(false); + config.setTlsTrustCertsFilePath(caCertPath); + config.setTlsCertificateFilePath(brokerCertPath); + config.setTlsKeyFilePath(brokerKeyPath); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java index 742194d9b12a1..0161a4a63cfc6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java @@ -44,6 +44,7 @@ import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.tests.TestRetrySupport; +import org.apache.pulsar.utils.ResourceUtils; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.apache.pulsar.zookeeper.ZookeeperServerTest; import org.awaitility.reflect.WhiteboxImpl; @@ -51,6 +52,12 @@ @Slf4j public abstract class NetworkErrorTestBase extends TestRetrySupport { + protected final static String CA_CERT_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/certs/ca.cert.pem"); + protected final static String BROKER_CERT_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/server-keys/broker.cert.pem"); + protected final static String BROKER_KEY_FILE_PATH = + ResourceUtils.getAbsolutePath("certificate-authority/server-keys/broker.key-pk8.pem"); protected final String defaultTenant = "public"; protected final String defaultNamespace = defaultTenant + "/default"; protected final String cluster1 = "r1"; @@ -176,6 +183,9 @@ protected void setConfigDefaults(ServiceConfiguration config, String clusterName config.setForceDeleteNamespaceAllowed(true); config.setLoadManagerClassName(PreferBrokerModularLoadManager.class.getName()); config.setMetadataStoreSessionTimeoutMillis(5000); + config.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + config.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + config.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java index 13447e089eab8..288bdba9b3846 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java @@ -44,7 +44,7 @@ public static PulsarClientImpl create(final ClientBuilderImpl clientBuilder, // Inject into ClientCnx. ConnectionPool pool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoopGroup, - () -> clientCnxFactory.generate(conf, eventLoopGroup)); + () -> clientCnxFactory.generate(conf, eventLoopGroup), null); return new InjectedClientCnxPulsarClientImpl(conf, eventLoopGroup, pool); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java index 879289eb65dc8..44af37ca90f51 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TlsProducerConsumerTest.java @@ -199,6 +199,7 @@ public void testTlsCertsFromDynamicStreamExpiredAndRenewCert() throws Exception log.info("-- Starting {} test --", methodName); ClientBuilder clientBuilder = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrlTls()) .enableTls(true).allowTlsInsecureConnection(false) + .autoCertRefreshSeconds(1) .operationTimeout(1000, TimeUnit.MILLISECONDS); AtomicInteger certIndex = new AtomicInteger(1); AtomicInteger keyIndex = new AtomicInteger(0); @@ -223,7 +224,7 @@ public void testTlsCertsFromDynamicStreamExpiredAndRenewCert() throws Exception } catch (PulsarClientException e) { // Ok.. } - + sleepSeconds(2); certIndex.set(0); try { consumer = pulsarClient.newConsumer().topic("persistent://my-property/use/my-ns/my-topic1") @@ -232,8 +233,9 @@ public void testTlsCertsFromDynamicStreamExpiredAndRenewCert() throws Exception } catch (PulsarClientException e) { // Ok.. } - + sleepSeconds(2); trustStoreIndex.set(0); + sleepSeconds(2); consumer = pulsarClient.newConsumer().topic("persistent://my-property/use/my-ns/my-topic1") .subscriptionName("my-subscriber-name").subscribe(); consumer.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java index 1037019d608ab..12dc9690115a4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java @@ -28,9 +28,12 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.Supplier; import java.util.stream.IntStream; import io.netty.util.concurrent.Promise; +import lombok.Cleanup; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.PulsarClient; @@ -69,8 +72,12 @@ protected void cleanup() throws Exception { public void testSingleIpAddress() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, new DefaultThreadFactory("test")); + @Cleanup("shutdownNow") + ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("test-pulsar-client-scheduled")); ConnectionPool pool = - spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop); + spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop, + scheduledExecutorService); conf.setServiceUrl(serviceUrl); PulsarClientImpl client = new PulsarClientImpl(conf, eventLoop, pool); @@ -119,8 +126,12 @@ public void testSelectConnectionForSameProducer() throws Exception { @Test public void testDoubleIpAddress() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); + @Cleanup("shutdownNow") + ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("test-pulsar-client-scheduled")); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, new DefaultThreadFactory("test")); - ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop); + ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, + eventLoop, scheduledExecutorService); conf.setServiceUrl(serviceUrl); PulsarClientImpl client = new PulsarClientImpl(conf, eventLoop, pool); @@ -145,8 +156,12 @@ public void testNoConnectionPool() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setConnectionsPerBroker(0); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(8, false, new DefaultThreadFactory("test")); + @Cleanup("shutdownNow") + ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("test-pulsar-client-scheduled")); ConnectionPool pool = - spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop); + spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop, + scheduledExecutorService); InetSocketAddress brokerAddress = InetSocketAddress.createUnresolved("127.0.0.1", brokerPort); @@ -169,8 +184,12 @@ public void testEnableConnectionPool() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setConnectionsPerBroker(5); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(8, false, new DefaultThreadFactory("test")); + @Cleanup("shutdownNow") + ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("test-pulsar-client-scheduled")); ConnectionPool pool = - spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop); + spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop, + scheduledExecutorService); InetSocketAddress brokerAddress = InetSocketAddress.createUnresolved("127.0.0.1", brokerPort); @@ -193,8 +212,9 @@ public void testEnableConnectionPool() throws Exception { public void testSetProxyToTargetBrokerAddress() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setConnectionsPerBroker(1); - - + @Cleanup("shutdownNow") + ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("test-pulsar-client-scheduled")); EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(8, false, new DefaultThreadFactory("test")); @@ -240,7 +260,7 @@ protected void doResolveAll(SocketAddress socketAddress, Promise promise) throws ConnectionPool pool = spyWithClassAndConstructorArgs(ConnectionPool.class, InstrumentProvider.NOOP, conf, eventLoop, (Supplier) () -> new ClientCnx(InstrumentProvider.NOOP, conf, eventLoop), - Optional.of(resolver)); + Optional.of(resolver), scheduledExecutorService); ClientCnx cnx = pool.getConnection( diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java index 8a79eb502439f..f69cd576f9ac2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java @@ -81,7 +81,7 @@ public static PulsarTestClient create(ClientBuilder clientBuilder) throws Pulsar AtomicReference> clientCnxSupplierReference = new AtomicReference<>(); ConnectionPool connectionPool = new ConnectionPool(InstrumentProvider.NOOP, clientConfigurationData, eventLoopGroup, - () -> clientCnxSupplierReference.get().get()); + () -> clientCnxSupplierReference.get().get(), null); return new PulsarTestClient(clientConfigurationData, eventLoopGroup, connectionPool, clientCnxSupplierReference); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorToolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorToolTest.java index 72b8628cacaa8..101d0a10b4fd1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorToolTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorToolTest.java @@ -98,6 +98,7 @@ public void testUseTlsUrlWithPEM() throws PulsarClientException { verify(serviceConfiguration, times(1)).getBrokerClientKeyFilePath(); verify(serviceConfiguration, times(1)).getBrokerClientTrustCertsFilePath(); verify(serviceConfiguration, times(1)).getBrokerClientCertificateFilePath(); + serviceConfiguration.setBrokerClientTlsTrustStorePassword(MockedPulsarServiceBaseTest.BROKER_KEYSTORE_PW); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java b/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java index 4a7d71c2b4f3e..866018b32fb0c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java @@ -69,6 +69,9 @@ public abstract class MockedPulsarStandalone implements AutoCloseable { serviceConfiguration.setWebServicePortTls(Optional.of(0)); serviceConfiguration.setNumExecutorThreadPoolSize(5); serviceConfiguration.setExposeBundlesMetricsInPrometheus(true); + serviceConfiguration.setTlsTrustCertsFilePath(TLS_EC_TRUSTED_CERT_PATH); + serviceConfiguration.setTlsCertificateFilePath(TLS_EC_SERVER_CERT_PATH); + serviceConfiguration.setTlsKeyFilePath(TLS_EC_SERVER_KEY_PATH); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java index b26e5b2cec802..5c41d98b89dbc 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java @@ -290,6 +290,20 @@ PulsarAdminBuilder authentication(String authPluginClassName, Map tlsProtocols); + /** + * SSL Factory Plugin used to generate the SSL Context and SSLEngine. + * @param sslFactoryPlugin Name of the SSL Factory Class to be used. + * @return PulsarAdminBuilder + */ + PulsarAdminBuilder sslFactoryPlugin(String sslFactoryPlugin); + + /** + * Parameters used by the SSL Factory Plugin class. + * @param sslFactoryPluginParams String parameters to be used by the SSL Factory Class. + * @return + */ + PulsarAdminBuilder sslFactoryPluginParams(String sslFactoryPluginParams); + /** * This sets the connection time out for the pulsar admin client. * diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ClusterData.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ClusterData.java index 1f7126521c6d6..6aeed746db428 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ClusterData.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ClusterData.java @@ -65,6 +65,10 @@ public interface ClusterData { String getBrokerClientTlsKeyStore(); + String getBrokerClientSslFactoryPlugin(); + + String getBrokerClientSslFactoryPluginParams(); + String getListenerName(); interface Builder { @@ -112,6 +116,10 @@ interface Builder { Builder listenerName(String listenerName); + Builder brokerClientSslFactoryPlugin(String sslFactoryPlugin); + + Builder brokerClientSslFactoryPluginParams(String sslFactoryPluginParams); + ClusterData build(); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java index 9bfb4fc45f3b7..7f0b3ab9a4218 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java @@ -214,6 +214,20 @@ public PulsarAdminBuilder tlsCiphers(Set tlsCiphers) { return this; } + @Override + public PulsarAdminBuilder sslFactoryPlugin(String sslFactoryPlugin) { + if (StringUtils.isNotBlank(sslFactoryPlugin)) { + conf.setSslFactoryPlugin(sslFactoryPlugin); + } + return this; + } + + @Override + public PulsarAdminBuilder sslFactoryPluginParams(String sslFactoryPluginParams) { + conf.setSslFactoryPluginParams(sslFactoryPluginParams); + return this; + } + @Override public PulsarAdminBuilder tlsProtocols(Set tlsProtocols) { conf.setTlsProtocols(tlsProtocols); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java index 1423d52642027..de694534a9e25 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnector.java @@ -31,8 +31,6 @@ import io.netty.handler.codec.http.DefaultHttpHeaders; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpResponse; -import io.netty.handler.ssl.SslContext; -import io.netty.handler.ssl.SslProvider; import io.netty.util.concurrent.DefaultThreadFactory; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -48,9 +46,9 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.Supplier; -import javax.net.ssl.SSLContext; import javax.ws.rs.ProcessingException; import javax.ws.rs.client.Client; import javax.ws.rs.core.HttpHeaders; @@ -61,15 +59,14 @@ import org.apache.commons.lang3.Validate; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; -import org.apache.pulsar.client.api.AuthenticationDataProvider; -import org.apache.pulsar.client.api.KeyStoreParams; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.PulsarServiceNameResolver; import org.apache.pulsar.client.impl.ServiceNameResolver; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; -import org.apache.pulsar.client.util.WithSNISslEngineFactory; +import org.apache.pulsar.client.util.PulsarHttpAsyncSslEngineFactory; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.SecurityUtility; -import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.asynchttpclient.AsyncCompletionHandlerBase; import org.asynchttpclient.AsyncHandler; import org.asynchttpclient.AsyncHttpClient; @@ -82,7 +79,6 @@ import org.asynchttpclient.Response; import org.asynchttpclient.SslEngineFactory; import org.asynchttpclient.channel.DefaultKeepAliveStrategy; -import org.asynchttpclient.netty.ssl.JsseSslEngineFactory; import org.asynchttpclient.uri.Uri; import org.glassfish.jersey.client.ClientProperties; import org.glassfish.jersey.client.ClientRequest; @@ -105,8 +101,10 @@ public class AsyncHttpConnector implements Connector, AsyncHttpRequestExecutor { private final ServiceNameResolver serviceNameResolver; private final ScheduledExecutorService delayer = Executors.newScheduledThreadPool(1, new DefaultThreadFactory("delayer")); + private ScheduledExecutorService sslRefresher; private final boolean acceptGzipCompression; private final Map> concurrencyReducers = new ConcurrentHashMap<>(); + private PulsarSslFactory sslFactory; public AsyncHttpConnector(Client client, ClientConfigurationData conf, int autoCertRefreshTimeSeconds, boolean acceptGzipCompression) { @@ -185,68 +183,28 @@ protected AsyncHttpClient createAsyncHttpClient(AsyncHttpClientConfig asyncHttpC return new DefaultAsyncHttpClient(asyncHttpClientConfig); } + @SneakyThrows private void configureAsyncHttpClientSslEngineFactory(ClientConfigurationData conf, int autoCertRefreshTimeSeconds, DefaultAsyncHttpClientConfig.Builder confBuilder) throws GeneralSecurityException, IOException { // Set client key and certificate if available - AuthenticationDataProvider authData = conf.getAuthentication().getAuthData(); - - SslEngineFactory sslEngineFactory = null; - if (conf.isUseKeyStoreTls()) { - KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : - new KeyStoreParams(conf.getTlsKeyStoreType(), conf.getTlsKeyStorePath(), - conf.getTlsKeyStorePassword()); - - final SSLContext sslCtx = KeyStoreSSLContext.createClientSslContext( - conf.getSslProvider(), - params.getKeyStoreType(), - params.getKeyStorePath(), - params.getKeyStorePassword(), - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustStoreType(), - conf.getTlsTrustStorePath(), - conf.getTlsTrustStorePassword(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - - sslEngineFactory = new JsseSslEngineFactory(sslCtx); - confBuilder.setSslEngineFactory(sslEngineFactory); - } else { - SslProvider sslProvider = null; - if (conf.getSslProvider() != null) { - sslProvider = SslProvider.valueOf(conf.getSslProvider()); - } - SslContext sslCtx = null; - if (authData.hasDataForTls()) { - sslCtx = authData.getTlsTrustStoreStream() == null - ? SecurityUtility.createAutoRefreshSslContextForClient( - sslProvider, - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustCertsFilePath(), authData.getTlsCertificateFilePath(), - authData.getTlsPrivateKeyFilePath(), null, autoCertRefreshTimeSeconds, delayer) - : SecurityUtility.createNettySslContextForClient( - sslProvider, - conf.isTlsAllowInsecureConnection(), - authData.getTlsTrustStoreStream(), authData.getTlsCertificates(), - authData.getTlsPrivateKey(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - } else { - sslCtx = SecurityUtility.createNettySslContextForClient( - sslProvider, - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustCertsFilePath(), - conf.getTlsCertificateFilePath(), - conf.getTlsKeyFilePath(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - } - confBuilder.setSslContext(sslCtx); - if (!conf.isTlsHostnameVerificationEnable()) { - confBuilder.setSslEngineFactory(new WithSNISslEngineFactory(serviceNameResolver - .resolveHostUri().getHost())); - } + sslRefresher = Executors.newScheduledThreadPool(1, + new DefaultThreadFactory("pulsar-admin-ssl-refresher")); + PulsarSslConfiguration sslConfiguration = buildSslConfiguration(conf); + this.sslFactory = (PulsarSslFactory) Class.forName(conf.getSslFactoryPlugin()) + .getConstructor().newInstance(); + this.sslFactory.initialize(sslConfiguration); + this.sslFactory.createInternalSslContext(); + if (conf.getAutoCertRefreshSeconds() > 0) { + this.sslRefresher.scheduleWithFixedDelay(this::refreshSslContext, conf.getAutoCertRefreshSeconds(), + conf.getAutoCertRefreshSeconds(), TimeUnit.SECONDS); } + String hostname = conf.isTlsHostnameVerificationEnable() ? null : serviceNameResolver + .resolveHostUri().getHost(); + SslEngineFactory sslEngineFactory = new PulsarHttpAsyncSslEngineFactory(sslFactory, hostname); + confBuilder.setSslEngineFactory(sslEngineFactory); + confBuilder.setUseInsecureTrustManager(conf.isTlsAllowInsecureConnection()); + confBuilder.setDisableHttpsEndpointIdentificationAlgorithm(!conf.isTlsHostnameVerificationEnable()); } @Override @@ -548,9 +506,45 @@ public void close() { try { httpClient.close(); delayer.shutdownNow(); + if (sslRefresher != null) { + sslRefresher.shutdownNow(); + } } catch (IOException e) { log.warn("Failed to close http client", e); } } + protected PulsarSslConfiguration buildSslConfiguration(ClientConfigurationData conf) + throws PulsarClientException { + return PulsarSslConfiguration.builder() + .tlsProvider(conf.getSslProvider()) + .tlsKeyStoreType(conf.getTlsKeyStoreType()) + .tlsKeyStorePath(conf.getTlsKeyStorePath()) + .tlsKeyStorePassword(conf.getTlsKeyStorePassword()) + .tlsTrustStoreType(conf.getTlsTrustStoreType()) + .tlsTrustStorePath(conf.getTlsTrustStorePath()) + .tlsTrustStorePassword(conf.getTlsTrustStorePassword()) + .tlsCiphers(conf.getTlsCiphers()) + .tlsProtocols(conf.getTlsProtocols()) + .tlsTrustCertsFilePath(conf.getTlsTrustCertsFilePath()) + .tlsCertificateFilePath(conf.getTlsCertificateFilePath()) + .tlsKeyFilePath(conf.getTlsKeyFilePath()) + .allowInsecureConnection(conf.isTlsAllowInsecureConnection()) + .requireTrustedClientCertOnConnect(false) + .tlsEnabledWithKeystore(conf.isUseKeyStoreTls()) + .authData(conf.getAuthentication().getAuthData()) + .tlsCustomParams(conf.getSslFactoryPluginParams()) + .serverMode(false) + .isHttps(true) + .build(); + } + + protected void refreshSslContext() { + try { + this.sslFactory.update(); + } catch (Exception e) { + log.error("Failed to refresh SSL context", e); + } + } + } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java index 7b98fa57bf0de..4adf7d89b0e33 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java @@ -600,7 +600,7 @@ ClientBuilder authentication(String authPluginClassName, Map aut * * @param proxyServiceUrl proxy service url * @param proxyProtocol protocol to decide type of proxy routing eg: SNI-routing - * @return + * @return the client builder instance */ ClientBuilder proxyServiceUrl(String proxyServiceUrl, ProxyProtocol proxyProtocol); @@ -608,7 +608,7 @@ ClientBuilder authentication(String authPluginClassName, Map aut * If enable transaction, start the transactionCoordinatorClient with pulsar client. * * @param enableTransaction whether enable transaction feature - * @return + * @return the client builder instance */ ClientBuilder enableTransaction(boolean enableTransaction); @@ -616,35 +616,56 @@ ClientBuilder authentication(String authPluginClassName, Map aut * Set dns lookup bind address and port. * @param address dnsBindAddress * @param port dnsBindPort - * @return + * @return the client builder instance */ ClientBuilder dnsLookupBind(String address, int port); /** * Set dns lookup server addresses. * @param addresses dnsServerAddresses - * @return + * @return the client builder instance */ ClientBuilder dnsServerAddresses(List addresses); /** * Set socks5 proxy address. * @param socks5ProxyAddress - * @return + * @return the client builder instance */ ClientBuilder socks5ProxyAddress(InetSocketAddress socks5ProxyAddress); /** * Set socks5 proxy username. * @param socks5ProxyUsername - * @return + * @return the client builder instance */ ClientBuilder socks5ProxyUsername(String socks5ProxyUsername); /** * Set socks5 proxy password. * @param socks5ProxyPassword - * @return + * @return the client builder instance */ ClientBuilder socks5ProxyPassword(String socks5ProxyPassword); + + /** + * Set the SSL Factory Plugin for custom implementation to create SSL Context and SSLEngine. + * @param sslFactoryPlugin ssl factory class name + * @return the client builder instance + */ + ClientBuilder sslFactoryPlugin(String sslFactoryPlugin); + + /** + * Set the SSL Factory Plugin params for the ssl factory plugin to use. + * @param sslFactoryPluginParams Params in String format that will be inputted to the SSL Factory Plugin + * @return the client builder instance + */ + ClientBuilder sslFactoryPluginParams(String sslFactoryPluginParams); + + /** + * Set Cert Refresh interval in seconds. + * @param autoCertRefreshSeconds + * @return the client builder instance + */ + ClientBuilder autoCertRefreshSeconds(int autoCertRefreshSeconds); } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java index 14f9eeadbffb5..9f11b48513867 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdClusters.java @@ -364,6 +364,14 @@ private static class ClusterDetails { description = "path for the TLS certificate file", required = false) protected String brokerClientCertificateFilePath; + @Option(names = "--tls-factory-plugin", + description = "TLS Factory Plugin to be used to generate SSL Context and SSL Engine") + protected String brokerClientSslFactoryPlugin; + + @Option(names = "--tls-factory-plugin-params", + description = "Parameters used by the TLS Factory Plugin") + protected String brokerClientSslFactoryPluginParams; + @Option(names = "--listener-name", description = "listenerName when client would like to connect to cluster", required = false) protected String listenerName; @@ -440,6 +448,12 @@ protected ClusterData getClusterData() throws IOException { if (brokerClientCertificateFilePath != null) { builder.brokerClientCertificateFilePath(brokerClientCertificateFilePath); } + if (StringUtils.isNotBlank(brokerClientSslFactoryPlugin)) { + builder.brokerClientSslFactoryPlugin(brokerClientSslFactoryPlugin); + } + if (StringUtils.isNotBlank(brokerClientSslFactoryPluginParams)) { + builder.brokerClientSslFactoryPluginParams(brokerClientSslFactoryPluginParams); + } if (listenerName != null) { builder.listenerName(listenerName); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/PulsarAdminTool.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/PulsarAdminTool.java index 2cc74d2f13bac..cd79098f0c3e9 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/PulsarAdminTool.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/PulsarAdminTool.java @@ -37,6 +37,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; +import org.apache.pulsar.common.util.DefaultPulsarSslFactory; import org.apache.pulsar.common.util.ShutdownUtil; import org.apache.pulsar.internal.CommandHook; import org.apache.pulsar.internal.CommanderFactory; @@ -130,6 +131,9 @@ private static PulsarAdminBuilder createAdminBuilderFromProperties(Properties pr boolean tlsEnableHostnameVerification = Boolean.parseBoolean(properties .getProperty("tlsEnableHostnameVerification", "false")); final String tlsTrustCertsFilePath = properties.getProperty("tlsTrustCertsFilePath"); + final String sslFactoryPlugin = properties.getProperty("sslFactoryPlugin", + DefaultPulsarSslFactory.class.getName()); + final String sslFactoryPluginParams = properties.getProperty("sslFactoryPluginParams", ""); return PulsarAdmin.builder().allowTlsInsecureConnection(tlsAllowInsecureConnection) .enableTlsHostnameVerification(tlsEnableHostnameVerification) @@ -142,7 +146,9 @@ private static PulsarAdminBuilder createAdminBuilderFromProperties(Properties pr .tlsKeyStorePath(tlsKeyStorePath) .tlsKeyStorePassword(tlsKeyStorePassword) .tlsKeyFilePath(tlsKeyFilePath) - .tlsCertificateFilePath(tlsCertificateFilePath); + .tlsCertificateFilePath(tlsCertificateFilePath) + .sslFactoryPlugin(sslFactoryPlugin) + .sslFactoryPluginParams(sslFactoryPluginParams); } private void setupCommands(Properties properties) { diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/PulsarClientTool.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/PulsarClientTool.java index 567c8d201e4ed..98f129441733d 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/PulsarClientTool.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/PulsarClientTool.java @@ -111,6 +111,8 @@ public static class RootParams { String tlsKeyStoreType; String tlsKeyStorePath; String tlsKeyStorePassword; + String sslFactoryPlugin; + String sslFactoryPluginParams; protected final CommandLine commander; protected CmdProduce produceCommand; @@ -152,6 +154,8 @@ protected void initCommander(Properties properties) { this.tlsKeyStorePassword = properties.getProperty("tlsKeyStorePassword"); this.tlsKeyFilePath = properties.getProperty("tlsKeyFilePath"); this.tlsCertificateFilePath = properties.getProperty("tlsCertificateFilePath"); + this.sslFactoryPlugin = properties.getProperty("sslFactoryPlugin"); + this.sslFactoryPluginParams = properties.getProperty("sslFactoryPluginParams"); pulsarClientPropertiesProvider = PulsarClientPropertiesProvider.create(properties); commander.setDefaultValueProvider(pulsarClientPropertiesProvider); @@ -192,6 +196,9 @@ private int updateConfig() throws UnsupportedAuthenticationException { .tlsKeyStorePath(tlsKeyStorePath) .tlsKeyStorePassword(tlsKeyStorePassword); + clientBuilder.sslFactoryPlugin(sslFactoryPlugin) + .sslFactoryPluginParams(sslFactoryPluginParams); + if (isNotBlank(rootParams.proxyServiceURL)) { if (rootParams.proxyProtocol == null) { commander.getErr().println("proxy-protocol must be provided with proxy-url"); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java index 2548a52aa95a8..d9edc53b50e37 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java @@ -39,6 +39,7 @@ import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ConfigurationDataUtils; import org.apache.pulsar.common.tls.InetAddressUtils; +import org.apache.pulsar.common.util.DefaultPulsarSslFactory; public class ClientBuilderImpl implements ClientBuilder { ClientConfigurationData conf; @@ -431,6 +432,28 @@ public ClientBuilder socks5ProxyPassword(String socks5ProxyPassword) { return this; } + @Override + public ClientBuilder sslFactoryPlugin(String sslFactoryPlugin) { + if (StringUtils.isBlank(sslFactoryPlugin)) { + conf.setSslFactoryPlugin(DefaultPulsarSslFactory.class.getName()); + } else { + conf.setSslFactoryPlugin(sslFactoryPlugin); + } + return this; + } + + @Override + public ClientBuilder sslFactoryPluginParams(String sslFactoryPluginParams) { + conf.setSslFactoryPluginParams(sslFactoryPluginParams); + return this; + } + + @Override + public ClientBuilder autoCertRefreshSeconds(int autoCertRefreshSeconds) { + conf.setAutoCertRefreshSeconds(autoCertRefreshSeconds); + return this; + } + /** * Set the description. * diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java index 21575578e76f2..a6a809af8585b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionPool.java @@ -47,6 +47,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -103,20 +104,25 @@ private static class Key { } public ConnectionPool(InstrumentProvider instrumentProvider, - ClientConfigurationData conf, EventLoopGroup eventLoopGroup) throws PulsarClientException { - this(instrumentProvider, conf, eventLoopGroup, () -> new ClientCnx(instrumentProvider, conf, eventLoopGroup)); + ClientConfigurationData conf, EventLoopGroup eventLoopGroup, + ScheduledExecutorService scheduledExecutorService) throws PulsarClientException { + this(instrumentProvider, conf, eventLoopGroup, () -> new ClientCnx(instrumentProvider, conf, eventLoopGroup), + scheduledExecutorService); } public ConnectionPool(InstrumentProvider instrumentProvider, ClientConfigurationData conf, EventLoopGroup eventLoopGroup, - Supplier clientCnxSupplier) throws PulsarClientException { - this(instrumentProvider, conf, eventLoopGroup, clientCnxSupplier, Optional.empty()); + Supplier clientCnxSupplier, + ScheduledExecutorService scheduledExecutorService) throws PulsarClientException { + this(instrumentProvider, conf, eventLoopGroup, clientCnxSupplier, Optional.empty(), + scheduledExecutorService); } public ConnectionPool(InstrumentProvider instrumentProvider, ClientConfigurationData conf, EventLoopGroup eventLoopGroup, Supplier clientCnxSupplier, - Optional> addressResolver) + Optional> addressResolver, + ScheduledExecutorService scheduledExecutorService) throws PulsarClientException { this.eventLoopGroup = eventLoopGroup; this.clientConfig = conf; @@ -134,7 +140,8 @@ public ConnectionPool(InstrumentProvider instrumentProvider, bootstrap.option(ChannelOption.ALLOCATOR, PulsarByteBufAllocator.DEFAULT); try { - channelInitializerHandler = new PulsarChannelInitializer(conf, clientCnxSupplier); + channelInitializerHandler = new PulsarChannelInitializer(conf, clientCnxSupplier, + scheduledExecutorService); bootstrap.handler(channelInitializerHandler); } catch (Exception e) { log.error("Failed to create channel initializer"); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java index 38b8954377957..53796ff7a4bf5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpClient.java @@ -21,40 +21,39 @@ import io.netty.channel.EventLoopGroup; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpResponse; -import io.netty.handler.ssl.SslContext; -import io.netty.handler.ssl.SslProvider; import java.io.Closeable; import java.io.IOException; import java.net.HttpURLConnection; import java.net.InetSocketAddress; import java.net.URI; import java.net.URL; -import java.security.GeneralSecurityException; import java.util.Map; import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.CompletableFuture; -import javax.net.ssl.SSLContext; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; -import org.apache.pulsar.client.api.KeyStoreParams; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.NotFoundException; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; -import org.apache.pulsar.client.util.WithSNISslEngineFactory; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.client.util.PulsarHttpAsyncSslEngineFactory; import org.apache.pulsar.common.util.ObjectMapperFactory; -import org.apache.pulsar.common.util.SecurityUtility; -import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.asynchttpclient.AsyncHttpClient; import org.asynchttpclient.AsyncHttpClientConfig; import org.asynchttpclient.BoundRequestBuilder; import org.asynchttpclient.DefaultAsyncHttpClient; import org.asynchttpclient.DefaultAsyncHttpClientConfig; import org.asynchttpclient.Request; +import org.asynchttpclient.SslEngineFactory; import org.asynchttpclient.channel.DefaultKeepAliveStrategy; -import org.asynchttpclient.netty.ssl.JsseSslEngineFactory; @Slf4j @@ -66,6 +65,8 @@ public class HttpClient implements Closeable { protected final AsyncHttpClient httpClient; protected final ServiceNameResolver serviceNameResolver; protected final Authentication authentication; + protected ScheduledExecutorService executorService; + protected PulsarSslFactory sslFactory; protected HttpClient(ClientConfigurationData conf, EventLoopGroup eventLoopGroup) throws PulsarClientException { this.authentication = conf.getAuthentication(); @@ -92,65 +93,28 @@ public boolean keepAlive(InetSocketAddress remoteAddress, Request ahcRequest, if ("https".equals(serviceNameResolver.getServiceUri().getServiceName())) { try { // Set client key and certificate if available - AuthenticationDataProvider authData = authentication.getAuthData(); - - if (conf.isUseKeyStoreTls()) { - SSLContext sslCtx = null; - KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : - new KeyStoreParams(conf.getTlsKeyStoreType(), conf.getTlsKeyStorePath(), - conf.getTlsKeyStorePassword()); + this.executorService = Executors + .newSingleThreadScheduledExecutor(new ExecutorProvider + .ExtendedThreadFactory("httpclient-ssl-refresh")); + PulsarSslConfiguration sslConfiguration = buildSslConfiguration(conf); + this.sslFactory = (PulsarSslFactory) Class.forName(conf.getSslFactoryPlugin()) + .getConstructor().newInstance(); + this.sslFactory.initialize(sslConfiguration); + this.sslFactory.createInternalSslContext(); + if (conf.getAutoCertRefreshSeconds() > 0) { + this.executorService.scheduleWithFixedDelay(this::refreshSslContext, + conf.getAutoCertRefreshSeconds(), + conf.getAutoCertRefreshSeconds(), TimeUnit.SECONDS); + } + String hostname = conf.isTlsHostnameVerificationEnable() ? null : serviceNameResolver + .resolveHostUri().getHost(); + SslEngineFactory sslEngineFactory = new PulsarHttpAsyncSslEngineFactory(this.sslFactory, hostname); + confBuilder.setSslEngineFactory(sslEngineFactory); - sslCtx = KeyStoreSSLContext.createClientSslContext( - conf.getSslProvider(), - params.getKeyStoreType(), - params.getKeyStorePath(), - params.getKeyStorePassword(), - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustStoreType(), - conf.getTlsTrustStorePath(), - conf.getTlsTrustStorePassword(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - JsseSslEngineFactory sslEngineFactory = new JsseSslEngineFactory(sslCtx); - confBuilder.setSslEngineFactory(sslEngineFactory); - } else { - SslProvider sslProvider = null; - if (conf.getSslProvider() != null) { - sslProvider = SslProvider.valueOf(conf.getSslProvider()); - } - SslContext sslCtx = null; - if (authData.hasDataForTls()) { - sslCtx = authData.getTlsTrustStoreStream() == null - ? SecurityUtility.createNettySslContextForClient(sslProvider, - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustCertsFilePath(), authData.getTlsCertificates(), - authData.getTlsPrivateKey(), conf.getTlsCiphers(), conf.getTlsProtocols()) - : SecurityUtility.createNettySslContextForClient(sslProvider, - conf.isTlsAllowInsecureConnection(), - authData.getTlsTrustStoreStream(), authData.getTlsCertificates(), - authData.getTlsPrivateKey(), conf.getTlsCiphers(), conf.getTlsProtocols()); - } else { - sslCtx = SecurityUtility.createNettySslContextForClient( - sslProvider, - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustCertsFilePath(), - conf.getTlsCertificateFilePath(), - conf.getTlsKeyFilePath(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - } - confBuilder.setSslContext(sslCtx); - if (!conf.isTlsHostnameVerificationEnable()) { - confBuilder.setSslEngineFactory(new WithSNISslEngineFactory(serviceNameResolver - .resolveHostUri().getHost())); - } - } confBuilder.setUseInsecureTrustManager(conf.isTlsAllowInsecureConnection()); confBuilder.setDisableHttpsEndpointIdentificationAlgorithm(!conf.isTlsHostnameVerificationEnable()); - } catch (GeneralSecurityException e) { - throw new PulsarClientException.InvalidConfigurationException(e); } catch (Exception e) { throw new PulsarClientException.InvalidConfigurationException(e); } @@ -177,6 +141,9 @@ void setServiceUrl(String serviceUrl) throws PulsarClientException { @Override public void close() throws IOException { httpClient.close(); + if (executorService != null) { + executorService.shutdownNow(); + } } public CompletableFuture get(String path, Class clazz) { @@ -264,4 +231,37 @@ public CompletableFuture get(String path, Class clazz) { return future; } + + protected PulsarSslConfiguration buildSslConfiguration(ClientConfigurationData config) + throws PulsarClientException { + return PulsarSslConfiguration.builder() + .tlsProvider(config.getSslProvider()) + .tlsKeyStoreType(config.getTlsKeyStoreType()) + .tlsKeyStorePath(config.getTlsKeyStorePath()) + .tlsKeyStorePassword(config.getTlsKeyStorePassword()) + .tlsTrustStoreType(config.getTlsTrustStoreType()) + .tlsTrustStorePath(config.getTlsTrustStorePath()) + .tlsTrustStorePassword(config.getTlsTrustStorePassword()) + .tlsCiphers(config.getTlsCiphers()) + .tlsProtocols(config.getTlsProtocols()) + .tlsTrustCertsFilePath(config.getTlsTrustCertsFilePath()) + .tlsCertificateFilePath(config.getTlsCertificateFilePath()) + .tlsKeyFilePath(config.getTlsKeyFilePath()) + .allowInsecureConnection(config.isTlsAllowInsecureConnection()) + .requireTrustedClientCertOnConnect(false) + .tlsEnabledWithKeystore(config.isUseKeyStoreTls()) + .tlsCustomParams(config.getSslFactoryPluginParams()) + .authData(config.getAuthentication().getAuthData()) + .serverMode(false) + .isHttps(true) + .build(); + } + + protected void refreshSslContext() { + try { + this.sslFactory.update(); + } catch (Exception e) { + log.error("Failed to refresh SSL context", e); + } + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java index dff423d19fbef..5097c34e0b2fd 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarChannelInitializer.java @@ -25,25 +25,22 @@ import io.netty.handler.codec.LengthFieldBasedFrameDecoder; import io.netty.handler.flush.FlushConsolidationHandler; import io.netty.handler.proxy.Socks5ProxyHandler; -import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; -import io.netty.handler.ssl.SslProvider; import java.net.InetSocketAddress; import java.util.Objects; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; import lombok.Getter; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.client.api.AuthenticationDataProvider; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; -import org.apache.pulsar.client.util.ObjectCache; import org.apache.pulsar.common.protocol.ByteBufPair; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.common.util.SecurityUtility; -import org.apache.pulsar.common.util.keystoretls.NettySSLContextAutoRefreshBuilder; import org.apache.pulsar.common.util.netty.NettyFutureUtil; @Slf4j @@ -55,18 +52,16 @@ public class PulsarChannelInitializer extends ChannelInitializer @Getter private final boolean tlsEnabled; private final boolean tlsHostnameVerificationEnabled; - private final boolean tlsEnabledWithKeyStore; private final InetSocketAddress socks5ProxyAddress; private final String socks5ProxyUsername; private final String socks5ProxyPassword; - private final Supplier sslContextSupplier; - private NettySSLContextAutoRefreshBuilder nettySSLContextAutoRefreshBuilder; + private final PulsarSslFactory pulsarSslFactory; private static final long TLS_CERTIFICATE_CACHE_MILLIS = TimeUnit.MINUTES.toMillis(1); - public PulsarChannelInitializer(ClientConfigurationData conf, Supplier clientCnxSupplier) - throws Exception { + public PulsarChannelInitializer(ClientConfigurationData conf, Supplier clientCnxSupplier, + ScheduledExecutorService scheduledExecutorService) throws Exception { super(); this.clientCnxSupplier = clientCnxSupplier; this.tlsEnabled = conf.isUseTls(); @@ -75,71 +70,25 @@ public PulsarChannelInitializer(ClientConfigurationData conf, Supplier 0) { + scheduledExecutorService.scheduleWithFixedDelay(() -> this.refreshSslContext(conf), + conf.getAutoCertRefreshSeconds(), + conf.getAutoCertRefreshSeconds(), + TimeUnit.SECONDS); } - sslContextSupplier = new ObjectCache(() -> { - try { - SslProvider sslProvider = null; - if (conf.getSslProvider() != null) { - sslProvider = SslProvider.valueOf(conf.getSslProvider()); - } - - // Set client certificate if available - AuthenticationDataProvider authData = conf.getAuthentication().getAuthData(); - if (authData.hasDataForTls()) { - return authData.getTlsTrustStoreStream() == null - ? SecurityUtility.createNettySslContextForClient( - sslProvider, - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustCertsFilePath(), - authData.getTlsCertificates(), - authData.getTlsPrivateKey(), - conf.getTlsCiphers(), - conf.getTlsProtocols()) - : SecurityUtility.createNettySslContextForClient(sslProvider, - conf.isTlsAllowInsecureConnection(), - authData.getTlsTrustStoreStream(), - authData.getTlsCertificates(), authData.getTlsPrivateKey(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - } else { - return SecurityUtility.createNettySslContextForClient( - sslProvider, - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustCertsFilePath(), - conf.getTlsCertificateFilePath(), - conf.getTlsKeyFilePath(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - } - } catch (Exception e) { - throw new RuntimeException("Failed to create TLS context", e); - } - }, TLS_CERTIFICATE_CACHE_MILLIS, TimeUnit.MILLISECONDS); } else { - sslContextSupplier = null; + pulsarSslFactory = null; } } @@ -174,10 +123,8 @@ CompletableFuture initTls(Channel ch, InetSocketAddress sniHost) { CompletableFuture initTlsFuture = new CompletableFuture<>(); ch.eventLoop().execute(() -> { try { - SslHandler handler = tlsEnabledWithKeyStore - ? new SslHandler(nettySSLContextAutoRefreshBuilder.get() - .createSSLEngine(sniHost.getHostString(), sniHost.getPort())) - : sslContextSupplier.get().newHandler(ch.alloc(), sniHost.getHostString(), sniHost.getPort()); + SslHandler handler = new SslHandler(pulsarSslFactory + .createClientSslEngine(ch.alloc(), sniHost.getHostName(), sniHost.getPort())); if (tlsHostnameVerificationEnabled) { SecurityUtility.configureSSLHandler(handler); @@ -234,5 +181,48 @@ CompletableFuture initializeClientCnx(Channel ch, return ch; })); } + protected PulsarSslConfiguration buildSslConfiguration(ClientConfigurationData config) + throws PulsarClientException { + return PulsarSslConfiguration.builder() + .tlsProvider(config.getSslProvider()) + .tlsKeyStoreType(config.getTlsKeyStoreType()) + .tlsKeyStorePath(config.getTlsKeyStorePath()) + .tlsKeyStorePassword(config.getTlsKeyStorePassword()) + .tlsTrustStoreType(config.getTlsTrustStoreType()) + .tlsTrustStorePath(config.getTlsTrustStorePath()) + .tlsTrustStorePassword(config.getTlsTrustStorePassword()) + .tlsCiphers(config.getTlsCiphers()) + .tlsProtocols(config.getTlsProtocols()) + .tlsTrustCertsFilePath(config.getTlsTrustCertsFilePath()) + .tlsCertificateFilePath(config.getTlsCertificateFilePath()) + .tlsKeyFilePath(config.getTlsKeyFilePath()) + .allowInsecureConnection(config.isTlsAllowInsecureConnection()) + .requireTrustedClientCertOnConnect(false) + .tlsEnabledWithKeystore(config.isUseKeyStoreTls()) + .tlsCustomParams(config.getSslFactoryPluginParams()) + .authData(config.getAuthentication().getAuthData()) + .serverMode(false) + .build(); + } + + protected void refreshSslContext(ClientConfigurationData conf) { + try { + try { + if (conf.isUseKeyStoreTls()) { + this.pulsarSslFactory.getInternalSslContext(); + } else { + this.pulsarSslFactory.getInternalNettySslContext(); + } + } catch (Exception e) { + log.error("SSL Context is not initialized", e); + PulsarSslConfiguration sslConfiguration = buildSslConfiguration(conf); + this.pulsarSslFactory.initialize(sslConfiguration); + } + this.pulsarSslFactory.update(); + } catch (Exception e) { + log.error("Failed to refresh SSL context", e); + } + } + } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index d37c3a10e1607..a63ade280efc3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -206,16 +206,17 @@ private PulsarClientImpl(ClientConfigurationData conf, EventLoopGroup eventLoopG this.instrumentProvider = new InstrumentProvider(conf.getOpenTelemetry()); clientClock = conf.getClock(); conf.getAuthentication().start(); + this.scheduledExecutorProvider = scheduledExecutorProvider != null ? scheduledExecutorProvider : + new ScheduledExecutorProvider(conf.getNumIoThreads(), "pulsar-client-scheduled"); connectionPoolReference = connectionPool != null ? connectionPool : - new ConnectionPool(instrumentProvider, conf, this.eventLoopGroup); + new ConnectionPool(instrumentProvider, conf, this.eventLoopGroup, + (ScheduledExecutorService) this.scheduledExecutorProvider.getExecutor()); this.cnxPool = connectionPoolReference; this.externalExecutorProvider = externalExecutorProvider != null ? externalExecutorProvider : new ExecutorProvider(conf.getNumListenerThreads(), "pulsar-external-listener"); this.internalExecutorProvider = internalExecutorProvider != null ? internalExecutorProvider : new ExecutorProvider(conf.getNumIoThreads(), "pulsar-client-internal"); - this.scheduledExecutorProvider = scheduledExecutorProvider != null ? scheduledExecutorProvider : - new ScheduledExecutorProvider(conf.getNumIoThreads(), "pulsar-client-scheduled"); if (conf.getServiceUrl().startsWith("http")) { lookup = new HttpLookupService(instrumentProvider, conf, this.eventLoopGroup); } else { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java index 237c6b5aebc3c..e2713644af641 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java @@ -43,6 +43,8 @@ import org.apache.pulsar.client.api.ServiceUrlProvider; import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; import org.apache.pulsar.client.util.Secret; +import org.apache.pulsar.common.util.DefaultPulsarSslFactory; + /** * This is a simple holder of the client configuration values. @@ -179,6 +181,18 @@ public class ClientConfigurationData implements Serializable, Cloneable { value = "Whether the hostname is validated when the client creates a TLS connection with brokers." ) private boolean tlsHostnameVerificationEnable = false; + + @ApiModelProperty( + name = "sslFactoryPlugin", + value = "SSL Factory Plugin class to provide SSLEngine and SSLContext objects. The default " + + " class used is DefaultPulsarSslFactory.") + private String sslFactoryPlugin = DefaultPulsarSslFactory.class.getName(); + + @ApiModelProperty( + name = "sslFactoryPluginParams", + value = "SSL Factory plugin configuration parameters.") + private String sslFactoryPluginParams = ""; + @ApiModelProperty( name = "concurrentLookupRequest", value = "The number of concurrent lookup requests that can be sent on each broker connection. " diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/util/WithSNISslEngineFactory.java b/pulsar-client/src/main/java/org/apache/pulsar/client/util/PulsarHttpAsyncSslEngineFactory.java similarity index 53% rename from pulsar-client/src/main/java/org/apache/pulsar/client/util/WithSNISslEngineFactory.java rename to pulsar-client/src/main/java/org/apache/pulsar/client/util/PulsarHttpAsyncSslEngineFactory.java index d950e68271bcd..ddf034bbb098e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/util/WithSNISslEngineFactory.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/util/PulsarHttpAsyncSslEngineFactory.java @@ -20,23 +20,42 @@ import java.util.Collections; import javax.net.ssl.SNIHostName; +import javax.net.ssl.SSLContext; import javax.net.ssl.SSLEngine; import javax.net.ssl.SSLParameters; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.asynchttpclient.AsyncHttpClientConfig; import org.asynchttpclient.netty.ssl.DefaultSslEngineFactory; -public class WithSNISslEngineFactory extends DefaultSslEngineFactory { +public class PulsarHttpAsyncSslEngineFactory extends DefaultSslEngineFactory { + + private final PulsarSslFactory pulsarSslFactory; private final String host; - public WithSNISslEngineFactory(String host) { + public PulsarHttpAsyncSslEngineFactory(PulsarSslFactory pulsarSslFactory, String host) { + this.pulsarSslFactory = pulsarSslFactory; this.host = host; } @Override protected void configureSslEngine(SSLEngine sslEngine, AsyncHttpClientConfig config) { super.configureSslEngine(sslEngine, config); - SSLParameters params = sslEngine.getSSLParameters(); - params.setServerNames(Collections.singletonList(new SNIHostName(host))); - sslEngine.setSSLParameters(params); + if (StringUtils.isNotBlank(host)) { + SSLParameters parameters = sslEngine.getSSLParameters(); + parameters.setServerNames(Collections.singletonList(new SNIHostName(host))); + sslEngine.setSSLParameters(parameters); + } } -} + + @Override + public SSLEngine newSslEngine(AsyncHttpClientConfig config, String peerHost, int peerPort) { + SSLContext sslContext = this.pulsarSslFactory.getInternalSslContext(); + SSLEngine sslEngine = config.isDisableHttpsEndpointIdentificationAlgorithm() + ? sslContext.createSSLEngine() : + sslContext.createSSLEngine(domain(peerHost), peerPort); + configureSslEngine(sslEngine, config); + return sslEngine; + } + +} \ No newline at end of file diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientInitializationTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientInitializationTest.java index 3b92f362ca188..2682d011cd0c5 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientInitializationTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientInitializationTest.java @@ -19,8 +19,8 @@ package org.apache.pulsar.client.impl; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; import lombok.Cleanup; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.PulsarClient; @@ -40,9 +40,7 @@ public void testInitializeAuthWithTls() throws PulsarClientException { .authentication(auth) .build(); - // Auth should only be started, though we shouldn't have tried to get credentials yet (until we first attempt to - // connect). verify(auth).start(); - verifyNoMoreInteractions(auth); + verify(auth, times(1)).getAuthData(); } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java index 103254a6b90a4..4481de9f1e65f 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java @@ -182,7 +182,7 @@ public void testInitializeWithTimer() throws PulsarClientException { ClientConfigurationData conf = new ClientConfigurationData(); @Cleanup("shutdownGracefully") EventLoopGroup eventLoop = EventLoopUtil.newEventLoopGroup(1, false, new DefaultThreadFactory("test")); - ConnectionPool pool = Mockito.spy(new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoop)); + ConnectionPool pool = Mockito.spy(new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoop, null)); conf.setServiceUrl("pulsar://localhost:6650"); HashedWheelTimer timer = new HashedWheelTimer(); @@ -207,7 +207,7 @@ public void testResourceCleanup() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setServiceUrl(""); initializeEventLoopGroup(conf); - try (ConnectionPool connectionPool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoopGroup)) { + try (ConnectionPool connectionPool = new ConnectionPool(InstrumentProvider.NOOP, conf, eventLoopGroup, null)) { assertThrows(() -> new PulsarClientImpl(conf, eventLoopGroup, connectionPool)); } finally { // Externally passed eventLoopGroup should not be shutdown. diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterDataImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterDataImpl.java index 6cb5a0034e938..b887fe0a5861b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterDataImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/ClusterDataImpl.java @@ -28,6 +28,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.ProxyProtocol; +import org.apache.pulsar.common.util.DefaultPulsarSslFactory; import org.apache.pulsar.common.util.URIPreconditions; /** @@ -170,6 +171,16 @@ public final class ClusterDataImpl implements ClusterData, Cloneable { + "used by the internal client to authenticate with Pulsar brokers" ) private String brokerClientCertificateFilePath; + @ApiModelProperty( + name = "brokerClientSslFactoryPlugin", + value = "SSL Factory plugin used by internal client to generate the SSL Context and Engine" + ) + private String brokerClientSslFactoryPlugin; + @ApiModelProperty( + name = "brokerClientSslFactoryPluginParams", + value = "Parameters used by the internal client's SSL factory plugin to generate the SSL Context and Engine" + ) + private String brokerClientSslFactoryPluginParams; @ApiModelProperty( name = "listenerName", value = "listenerName when client would like to connect to cluster", @@ -205,6 +216,8 @@ public ClusterDataImplBuilder clone() { .brokerClientTrustCertsFilePath(brokerClientTrustCertsFilePath) .brokerClientCertificateFilePath(brokerClientCertificateFilePath) .brokerClientKeyFilePath(brokerClientKeyFilePath) + .brokerClientSslFactoryPlugin(brokerClientSslFactoryPlugin) + .brokerClientSslFactoryPluginParams(brokerClientSslFactoryPluginParams) .listenerName(listenerName); } @@ -231,6 +244,8 @@ public static class ClusterDataImplBuilder implements ClusterData.Builder { private String brokerClientCertificateFilePath; private String brokerClientKeyFilePath; private String brokerClientTrustCertsFilePath; + private String brokerClientSslFactoryPlugin = DefaultPulsarSslFactory.class.getName(); + private String brokerClientSslFactoryPluginParams; private String listenerName; ClusterDataImplBuilder() { @@ -346,6 +361,17 @@ public ClusterDataImplBuilder brokerClientKeyFilePath(String keyFilePath) { return this; } + @Override + public ClusterDataImplBuilder brokerClientSslFactoryPlugin(String sslFactoryPlugin) { + this.brokerClientSslFactoryPlugin = sslFactoryPlugin; + return this; + } + + @Override + public ClusterDataImplBuilder brokerClientSslFactoryPluginParams(String sslFactoryPluginParams) { + this.brokerClientSslFactoryPluginParams = sslFactoryPluginParams; + return this; + } public ClusterDataImplBuilder listenerName(String listenerName) { this.listenerName = listenerName; @@ -375,6 +401,8 @@ public ClusterDataImpl build() { brokerClientTrustCertsFilePath, brokerClientKeyFilePath, brokerClientCertificateFilePath, + brokerClientSslFactoryPlugin, + brokerClientSslFactoryPluginParams, listenerName); } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultPulsarSslFactory.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultPulsarSslFactory.java new file mode 100644 index 0000000000000..9be16f835b28b --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultPulsarSslFactory.java @@ -0,0 +1,366 @@ +/* + * 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; + +import io.netty.buffer.ByteBufAllocator; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslProvider; +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.concurrent.NotThreadSafe; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLParameters; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.api.AuthenticationDataProvider; +import org.apache.pulsar.client.api.KeyStoreParams; +import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext; + +/** + * Default Implementation of {@link PulsarSslFactory}. This factory loads file based certificates to create SSLContext + * and SSL Engines. This class is not thread safe. It has been integrated into the pulsar code base as a single writer, + * multiple readers pattern. + */ +@NotThreadSafe +public class DefaultPulsarSslFactory implements PulsarSslFactory { + + private PulsarSslConfiguration config; + private final AtomicReference internalSslContext = new AtomicReference<>(); + private final AtomicReference internalNettySslContext = new AtomicReference<>(); + + protected FileModifiedTimeUpdater tlsKeyStore; + protected FileModifiedTimeUpdater tlsTrustStore; + protected FileModifiedTimeUpdater tlsTrustCertsFilePath; + protected FileModifiedTimeUpdater tlsCertificateFilePath; + protected FileModifiedTimeUpdater tlsKeyFilePath; + protected AuthenticationDataProvider authData; + protected boolean isTlsTrustStoreStreamProvided; + protected final String[] defaultSslEnabledProtocols = {"TLSv1.3", "TLSv1.2"}; + protected String tlsKeystoreType; + protected String tlsKeystorePath; + protected String tlsKeystorePassword; + + /** + * Initializes the DefaultPulsarSslFactory. + * + * @param config {@link PulsarSslConfiguration} object required for initialization. + * + */ + @Override + public void initialize(PulsarSslConfiguration config) { + this.config = config; + AuthenticationDataProvider authData = this.config.getAuthData(); + if (this.config.isTlsEnabledWithKeystore()) { + if (authData != null && authData.hasDataForTls()) { + KeyStoreParams authParams = authData.getTlsKeyStoreParams(); + if (authParams != null) { + this.tlsKeystoreType = authParams.getKeyStoreType(); + this.tlsKeystorePath = authParams.getKeyStorePath(); + this.tlsKeystorePassword = authParams.getKeyStorePassword(); + } + } + if (this.tlsKeystoreType == null) { + this.tlsKeystoreType = this.config.getTlsKeyStoreType(); + } + if (this.tlsKeystorePath == null) { + this.tlsKeystorePath = this.config.getTlsKeyStorePath(); + } + if (this.tlsKeystorePassword == null) { + this.tlsKeystorePassword = this.config.getTlsKeyStorePassword(); + } + this.tlsKeyStore = new FileModifiedTimeUpdater(this.tlsKeystorePath); + this.tlsTrustStore = new FileModifiedTimeUpdater(this.config.getTlsTrustStorePath()); + } else { + if (authData != null && authData.hasDataForTls()) { + if (authData.getTlsTrustStoreStream() != null) { + this.isTlsTrustStoreStreamProvided = true; + } else { + this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(this.config.getTlsTrustCertsFilePath()); + } + this.authData = authData; + } else { + this.tlsCertificateFilePath = new FileModifiedTimeUpdater(this.config.getTlsCertificateFilePath()); + this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(this.config.getTlsTrustCertsFilePath()); + this.tlsKeyFilePath = new FileModifiedTimeUpdater(this.config.getTlsKeyFilePath()); + } + } + } + + /** + * Creates a Client {@link SSLEngine} utilizing the peer hostname, peer port and {@link PulsarSslConfiguration} + * object provided during initialization. + * + * @param peerHost the name of the peer host + * @param peerPort the port number of the peer + * @return {@link SSLEngine} + */ + @Override + public SSLEngine createClientSslEngine(ByteBufAllocator buf, String peerHost, int peerPort) { + return createSSLEngine(buf, peerHost, peerPort, NetworkMode.CLIENT); + } + + /** + * Creates a Server {@link SSLEngine} utilizing the {@link PulsarSslConfiguration} object provided during + * initialization. + * + * @return {@link SSLEngine} + */ + @Override + public SSLEngine createServerSslEngine(ByteBufAllocator buf) { + return createSSLEngine(buf, "", 0, NetworkMode.SERVER); + } + + /** + * Returns a boolean value based on if the underlying certificate files have been modified since it was last read. + * + * @return {@code true} if the underlying certificates have been modified indicating that + * the SSL Context should be refreshed. + */ + @Override + public boolean needsUpdate() { + if (this.config.isTlsEnabledWithKeystore()) { + return (this.tlsKeyStore != null && this.tlsKeyStore.checkAndRefresh()) + || (this.tlsTrustStore != null && this.tlsTrustStore.checkAndRefresh()); + } else { + if (this.authData != null && this.authData.hasDataForTls()) { + return true; + } else { + return this.tlsTrustCertsFilePath.checkAndRefresh() || this.tlsCertificateFilePath.checkAndRefresh() + || this.tlsKeyFilePath.checkAndRefresh(); + } + } + } + + /** + * Creates a {@link SSLContext} object and saves it internally. + * + * @throws Exception If there were any issues generating the {@link SSLContext} + */ + @Override + public void createInternalSslContext() throws Exception { + if (this.config.isTlsEnabledWithKeystore()) { + this.internalSslContext.set(buildKeystoreSslContext(this.config.isServerMode())); + } else { + if (this.config.isHttps()) { + this.internalSslContext.set(buildSslContext()); + } else { + this.internalNettySslContext.set(buildNettySslContext()); + } + } + } + + + /** + * Get the internally stored {@link SSLContext}. + * + * @return {@link SSLContext} + * @throws RuntimeException if the {@link SSLContext} object has not yet been initialized. + */ + @Override + public SSLContext getInternalSslContext() { + if (this.internalSslContext.get() == null) { + throw new RuntimeException("Internal SSL context is not initialized. " + + "Please call createInternalSslContext() first."); + } + return this.internalSslContext.get(); + } + + /** + * Get the internally stored {@link SslContext}. + * + * @return {@link SslContext} + * @throws RuntimeException if the {@link SslContext} object has not yet been initialized. + */ + public SslContext getInternalNettySslContext() { + if (this.internalNettySslContext.get() == null) { + throw new RuntimeException("Internal SSL context is not initialized. " + + "Please call createInternalSslContext() first."); + } + return this.internalNettySslContext.get(); + } + + private SSLContext buildKeystoreSslContext(boolean isServerMode) throws GeneralSecurityException, IOException { + KeyStoreSSLContext keyStoreSSLContext; + if (isServerMode) { + keyStoreSSLContext = KeyStoreSSLContext.createServerKeyStoreSslContext(this.config.getTlsProvider(), + this.tlsKeystoreType, this.tlsKeyStore.getFileName(), + this.tlsKeystorePassword, this.config.isAllowInsecureConnection(), + this.config.getTlsTrustStoreType(), this.tlsTrustStore.getFileName(), + this.config.getTlsTrustStorePassword(), this.config.isRequireTrustedClientCertOnConnect(), + this.config.getTlsCiphers(), this.config.getTlsProtocols()); + } else { + keyStoreSSLContext = KeyStoreSSLContext.createClientKeyStoreSslContext(this.config.getTlsProvider(), + this.tlsKeystoreType, this.tlsKeyStore.getFileName(), + this.tlsKeystorePassword, this.config.isAllowInsecureConnection(), + this.config.getTlsTrustStoreType(), this.tlsTrustStore.getFileName(), + this.config.getTlsTrustStorePassword(), this.config.getTlsCiphers(), + this.config.getTlsProtocols()); + } + return keyStoreSSLContext.createSSLContext(); + } + + private SSLContext buildSslContext() throws GeneralSecurityException { + if (this.authData != null && this.authData.hasDataForTls()) { + if (this.isTlsTrustStoreStreamProvided) { + return SecurityUtility.createSslContext(this.config.isAllowInsecureConnection(), + SecurityUtility.loadCertificatesFromPemStream(this.authData.getTlsTrustStoreStream()), + this.authData.getTlsCertificates(), + this.authData.getTlsPrivateKey(), + this.config.getTlsProvider()); + } else { + if (this.authData.getTlsCertificates() != null) { + return SecurityUtility.createSslContext(this.config.isAllowInsecureConnection(), + SecurityUtility.loadCertificatesFromPemFile(this.tlsTrustCertsFilePath.getFileName()), + this.authData.getTlsCertificates(), + this.authData.getTlsPrivateKey(), + this.config.getTlsProvider()); + } else { + return SecurityUtility.createSslContext(this.config.isAllowInsecureConnection(), + this.tlsTrustCertsFilePath.getFileName(), + this.authData.getTlsCertificateFilePath(), + this.authData.getTlsPrivateKeyFilePath(), + this.config.getTlsProvider() + ); + } + } + } else { + return SecurityUtility.createSslContext(this.config.isAllowInsecureConnection(), + this.tlsTrustCertsFilePath.getFileName(), + this.tlsCertificateFilePath.getFileName(), + this.tlsKeyFilePath.getFileName(), + this.config.getTlsProvider()); + } + } + + private SslContext buildNettySslContext() throws GeneralSecurityException, IOException { + SslProvider sslProvider = null; + if (StringUtils.isNotBlank(this.config.getTlsProvider())) { + sslProvider = SslProvider.valueOf(this.config.getTlsProvider()); + } + if (this.authData != null && this.authData.hasDataForTls()) { + if (this.isTlsTrustStoreStreamProvided) { + return SecurityUtility.createNettySslContextForClient(sslProvider, + this.config.isAllowInsecureConnection(), + this.authData.getTlsTrustStoreStream(), + this.authData.getTlsCertificates(), + this.authData.getTlsPrivateKey(), + this.config.getTlsCiphers(), + this.config.getTlsProtocols()); + } else { + if (this.authData.getTlsCertificates() != null) { + return SecurityUtility.createNettySslContextForClient(sslProvider, + this.config.isAllowInsecureConnection(), + this.tlsTrustCertsFilePath.getFileName(), + this.authData.getTlsCertificates(), + this.authData.getTlsPrivateKey(), + this.config.getTlsCiphers(), + this.config.getTlsProtocols()); + } else { + return SecurityUtility.createNettySslContextForClient(sslProvider, + this.config.isAllowInsecureConnection(), + this.tlsTrustCertsFilePath.getFileName(), + this.authData.getTlsCertificateFilePath(), + this.authData.getTlsPrivateKeyFilePath(), + this.config.getTlsCiphers(), + this.config.getTlsProtocols()); + } + } + } else { + if (this.config.isServerMode()) { + return SecurityUtility.createNettySslContextForServer(sslProvider, + this.config.isAllowInsecureConnection(), + this.tlsTrustCertsFilePath.getFileName(), + this.tlsCertificateFilePath.getFileName(), + this.tlsKeyFilePath.getFileName(), + this.config.getTlsCiphers(), + this.config.getTlsProtocols(), + this.config.isRequireTrustedClientCertOnConnect()); + } else { + return SecurityUtility.createNettySslContextForClient(sslProvider, + this.config.isAllowInsecureConnection(), + this.tlsTrustCertsFilePath.getFileName(), + this.tlsCertificateFilePath.getFileName(), + this.tlsKeyFilePath.getFileName(), + this.config.getTlsCiphers(), + this.config.getTlsProtocols()); + } + } + } + + private SSLEngine createSSLEngine(ByteBufAllocator buf, String peerHost, int peerPort, NetworkMode mode) { + SSLEngine sslEngine; + SSLParameters sslParams; + SSLContext sslContext = this.internalSslContext.get(); + SslContext nettySslContext = this.internalNettySslContext.get(); + validateSslContext(sslContext, nettySslContext); + if (mode == NetworkMode.CLIENT) { + if (sslContext != null) { + sslEngine = sslContext.createSSLEngine(peerHost, peerPort); + } else { + sslEngine = nettySslContext.newEngine(buf, peerHost, peerPort); + } + sslEngine.setUseClientMode(true); + sslParams = sslEngine.getSSLParameters(); + } else { + if (sslContext != null) { + sslEngine = sslContext.createSSLEngine(); + } else { + sslEngine = nettySslContext.newEngine(buf); + } + sslEngine.setUseClientMode(false); + sslParams = sslEngine.getSSLParameters(); + if (this.config.isRequireTrustedClientCertOnConnect()) { + sslParams.setNeedClientAuth(true); + } else { + sslParams.setWantClientAuth(true); + } + } + if (this.config.getTlsProtocols() != null && !this.config.getTlsProtocols().isEmpty()) { + sslParams.setProtocols(this.config.getTlsProtocols().toArray(new String[0])); + } else { + sslParams.setProtocols(defaultSslEnabledProtocols); + } + if (this.config.getTlsCiphers() != null && !this.config.getTlsCiphers().isEmpty()) { + sslParams.setCipherSuites(this.config.getTlsCiphers().toArray(new String[0])); + } + sslEngine.setSSLParameters(sslParams); + return sslEngine; + } + + private void validateSslContext(SSLContext sslContext, SslContext nettySslContext) { + if (sslContext == null && nettySslContext == null) { + throw new RuntimeException("Internal SSL context is not initialized. " + + "Please call createInternalSslContext() first."); + } + } + + /** + * Clean any resources that may have been created. + * @throws Exception if any resources failed to be cleaned. + */ + @Override + public void close() throws Exception { + // noop + } + + private enum NetworkMode { + CLIENT, SERVER + } +} \ No newline at end of file diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultSslContextBuilder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultSslContextBuilder.java deleted file mode 100644 index ab5f41c6bbf8d..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/DefaultSslContextBuilder.java +++ /dev/null @@ -1,76 +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; - -import java.security.GeneralSecurityException; -import javax.net.ssl.SSLContext; - -@SuppressWarnings("checkstyle:JavadocType") -public class DefaultSslContextBuilder extends SslContextAutoRefreshBuilder { - private volatile SSLContext sslContext; - - protected final boolean tlsAllowInsecureConnection; - protected final FileModifiedTimeUpdater tlsTrustCertsFilePath, tlsCertificateFilePath, tlsKeyFilePath; - protected final boolean tlsRequireTrustedClientCertOnConnect; - private final String providerName; - - public DefaultSslContextBuilder(boolean allowInsecure, String trustCertsFilePath, String certificateFilePath, - String keyFilePath, boolean requireTrustedClientCertOnConnect, - long certRefreshInSec) { - super(certRefreshInSec); - this.tlsAllowInsecureConnection = allowInsecure; - this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(trustCertsFilePath); - this.tlsCertificateFilePath = new FileModifiedTimeUpdater(certificateFilePath); - this.tlsKeyFilePath = new FileModifiedTimeUpdater(keyFilePath); - this.tlsRequireTrustedClientCertOnConnect = requireTrustedClientCertOnConnect; - this.providerName = null; - } - - public DefaultSslContextBuilder(boolean allowInsecure, String trustCertsFilePath, String certificateFilePath, - String keyFilePath, boolean requireTrustedClientCertOnConnect, - long certRefreshInSec, String providerName) { - super(certRefreshInSec); - this.tlsAllowInsecureConnection = allowInsecure; - this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(trustCertsFilePath); - this.tlsCertificateFilePath = new FileModifiedTimeUpdater(certificateFilePath); - this.tlsKeyFilePath = new FileModifiedTimeUpdater(keyFilePath); - this.tlsRequireTrustedClientCertOnConnect = requireTrustedClientCertOnConnect; - this.providerName = providerName; - } - - @Override - public synchronized SSLContext update() throws GeneralSecurityException { - this.sslContext = SecurityUtility.createSslContext(tlsAllowInsecureConnection, - tlsTrustCertsFilePath.getFileName(), tlsCertificateFilePath.getFileName(), - tlsKeyFilePath.getFileName(), this.providerName); - return this.sslContext; - } - - @Override - public SSLContext getSslContext() { - return this.sslContext; - } - - @Override - public boolean needUpdate() { - return tlsTrustCertsFilePath.checkAndRefresh() - || tlsCertificateFilePath.checkAndRefresh() - || tlsKeyFilePath.checkAndRefresh(); - } -} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyClientSslContextRefresher.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyClientSslContextRefresher.java deleted file mode 100644 index 828cf35121d7e..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyClientSslContextRefresher.java +++ /dev/null @@ -1,96 +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; - -import io.netty.handler.ssl.SslContext; -import io.netty.handler.ssl.SslProvider; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.security.GeneralSecurityException; -import java.security.cert.X509Certificate; -import java.util.Set; -import javax.net.ssl.SSLException; -import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.client.api.AuthenticationDataProvider; - -/** - * SSL context builder for Netty Client side. - */ -@Slf4j -public class NettyClientSslContextRefresher extends SslContextAutoRefreshBuilder { - private volatile SslContext sslNettyContext; - private final boolean tlsAllowInsecureConnection; - protected final FileModifiedTimeUpdater tlsTrustCertsFilePath; - protected final FileModifiedTimeUpdater tlsCertsFilePath; - protected final FileModifiedTimeUpdater tlsPrivateKeyFilePath; - private final AuthenticationDataProvider authData; - private final SslProvider sslProvider; - private final Set ciphers; - private final Set protocols; - - public NettyClientSslContextRefresher(SslProvider sslProvider, boolean allowInsecure, - String trustCertsFilePath, - AuthenticationDataProvider authData, - Set ciphers, - Set protocols, - long delayInSeconds) { - super(delayInSeconds); - this.tlsAllowInsecureConnection = allowInsecure; - this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(trustCertsFilePath); - this.authData = authData; - this.tlsCertsFilePath = new FileModifiedTimeUpdater( - authData != null ? authData.getTlsCertificateFilePath() : null); - this.tlsPrivateKeyFilePath = new FileModifiedTimeUpdater( - authData != null ? authData.getTlsPrivateKeyFilePath() : null); - this.sslProvider = sslProvider; - this.ciphers = ciphers; - this.protocols = protocols; - } - - @Override - public synchronized SslContext update() - throws SSLException, FileNotFoundException, GeneralSecurityException, IOException { - if (authData != null && authData.hasDataForTls()) { - this.sslNettyContext = authData.getTlsTrustStoreStream() == null - ? SecurityUtility.createNettySslContextForClient(this.sslProvider, this.tlsAllowInsecureConnection, - tlsTrustCertsFilePath.getFileName(), (X509Certificate[]) authData.getTlsCertificates(), - authData.getTlsPrivateKey(), this.ciphers, this.protocols) - : SecurityUtility.createNettySslContextForClient(this.sslProvider, this.tlsAllowInsecureConnection, - authData.getTlsTrustStoreStream(), (X509Certificate[]) authData.getTlsCertificates(), - authData.getTlsPrivateKey(), this.ciphers, this.protocols); - } else { - this.sslNettyContext = - SecurityUtility.createNettySslContextForClient(this.sslProvider, this.tlsAllowInsecureConnection, - this.tlsTrustCertsFilePath.getFileName(), this.ciphers, this.protocols); - } - return this.sslNettyContext; - } - - @Override - public SslContext getSslContext() { - return this.sslNettyContext; - } - - @Override - public boolean needUpdate() { - return tlsTrustCertsFilePath.checkAndRefresh() || tlsCertsFilePath.checkAndRefresh() - || tlsPrivateKeyFilePath.checkAndRefresh(); - - } -} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyServerSslContextBuilder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyServerSslContextBuilder.java deleted file mode 100644 index eda61be3f87c2..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/NettyServerSslContextBuilder.java +++ /dev/null @@ -1,89 +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; - -import io.netty.handler.ssl.SslContext; -import io.netty.handler.ssl.SslProvider; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.security.GeneralSecurityException; -import java.util.Set; -import javax.net.ssl.SSLException; - -/** - * SSL context builder for Netty Server side. - */ -public class NettyServerSslContextBuilder extends SslContextAutoRefreshBuilder { - private volatile SslContext sslNettyContext; - - protected final boolean tlsAllowInsecureConnection; - protected final FileModifiedTimeUpdater tlsTrustCertsFilePath, tlsCertificateFilePath, tlsKeyFilePath; - protected final Set tlsCiphers; - protected final Set tlsProtocols; - protected final boolean tlsRequireTrustedClientCertOnConnect; - protected final SslProvider sslProvider; - - public NettyServerSslContextBuilder(boolean allowInsecure, String trustCertsFilePath, - String certificateFilePath, - String keyFilePath, Set ciphers, Set protocols, - boolean requireTrustedClientCertOnConnect, - long delayInSeconds) { - this(null, allowInsecure, trustCertsFilePath, certificateFilePath, keyFilePath, ciphers, protocols, - requireTrustedClientCertOnConnect, delayInSeconds); - } - - public NettyServerSslContextBuilder(SslProvider sslProvider, boolean allowInsecure, String trustCertsFilePath, - String certificateFilePath, - String keyFilePath, Set ciphers, Set protocols, - boolean requireTrustedClientCertOnConnect, - long delayInSeconds) { - super(delayInSeconds); - this.tlsAllowInsecureConnection = allowInsecure; - this.tlsTrustCertsFilePath = new FileModifiedTimeUpdater(trustCertsFilePath); - this.tlsCertificateFilePath = new FileModifiedTimeUpdater(certificateFilePath); - this.tlsKeyFilePath = new FileModifiedTimeUpdater(keyFilePath); - this.tlsCiphers = ciphers; - this.tlsProtocols = protocols; - this.tlsRequireTrustedClientCertOnConnect = requireTrustedClientCertOnConnect; - this.sslProvider = sslProvider; - } - - @Override - public synchronized SslContext update() - throws SSLException, FileNotFoundException, GeneralSecurityException, IOException { - this.sslNettyContext = - SecurityUtility.createNettySslContextForServer(this.sslProvider, tlsAllowInsecureConnection, - tlsTrustCertsFilePath.getFileName(), tlsCertificateFilePath.getFileName(), - tlsKeyFilePath.getFileName(), - tlsCiphers, tlsProtocols, tlsRequireTrustedClientCertOnConnect); - return this.sslNettyContext; - } - - @Override - public SslContext getSslContext() { - return this.sslNettyContext; - } - - @Override - public boolean needUpdate() { - return tlsTrustCertsFilePath.checkAndRefresh() - || tlsCertificateFilePath.checkAndRefresh() - || tlsKeyFilePath.checkAndRefresh(); - } -} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/PulsarSslConfiguration.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/PulsarSslConfiguration.java new file mode 100644 index 0000000000000..f71888009bf4c --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/PulsarSslConfiguration.java @@ -0,0 +1,167 @@ +/* + * 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; + +import io.swagger.annotations.ApiModelProperty; +import java.io.Serializable; +import java.util.Set; +import lombok.Builder; +import lombok.Getter; +import lombok.ToString; +import org.apache.pulsar.client.api.AuthenticationDataProvider; + +/** + * Pulsar SSL Configuration Object to be used by all Pulsar Server and Client Components. + */ +@Builder +@Getter +@ToString +public class PulsarSslConfiguration implements Serializable, Cloneable { + + private static final long serialVersionUID = 1L; + + @ApiModelProperty( + name = "tlsCiphers", + value = "TLS ciphers to be used", + required = true + ) + private Set tlsCiphers; + + @ApiModelProperty( + name = "tlsProtocols", + value = "TLS protocols to be used", + required = true + ) + private Set tlsProtocols; + + @ApiModelProperty( + name = "allowInsecureConnection", + value = "Insecure Connections are allowed", + required = true + ) + private boolean allowInsecureConnection; + + @ApiModelProperty( + name = "requireTrustedClientCertOnConnect", + value = "Require trusted client certificate on connect", + required = true + ) + private boolean requireTrustedClientCertOnConnect; + + @ApiModelProperty( + name = "authData", + value = "Authentication Data Provider utilized by the Client for identification" + ) + private AuthenticationDataProvider authData; + + @ApiModelProperty( + name = "tlsCustomParams", + value = "Custom Parameters required by Pulsar SSL factory plugins" + ) + private String tlsCustomParams; + + @ApiModelProperty( + name = "tlsProvider", + value = "TLS Provider to be used" + ) + private String tlsProvider; + + @ApiModelProperty( + name = "tlsTrustStoreType", + value = "TLS Trust Store Type to be used" + ) + private String tlsTrustStoreType; + + @ApiModelProperty( + name = "tlsTrustStorePath", + value = "TLS Trust Store Path" + ) + private String tlsTrustStorePath; + + @ApiModelProperty( + name = "tlsTrustStorePassword", + value = "TLS Trust Store Password" + ) + private String tlsTrustStorePassword; + + @ApiModelProperty( + name = "tlsTrustCertsFilePath", + value = " TLS Trust certificates file path" + ) + private String tlsTrustCertsFilePath; + + @ApiModelProperty( + name = "tlsCertificateFilePath", + value = "Path for the TLS Certificate file" + ) + private String tlsCertificateFilePath; + + @ApiModelProperty( + name = "tlsKeyFilePath", + value = "Path for TLS Private key file" + ) + private String tlsKeyFilePath; + + @ApiModelProperty( + name = "tlsKeyStoreType", + value = "TLS Key Store Type to be used" + ) + private String tlsKeyStoreType; + + @ApiModelProperty( + name = "tlsKeyStorePath", + value = "TLS Key Store Path" + ) + private String tlsKeyStorePath; + + @ApiModelProperty( + name = "tlsKeyStorePassword", + value = "TLS Key Store Password" + ) + private String tlsKeyStorePassword; + + @ApiModelProperty( + name = "isTlsEnabledWithKeystore", + value = "TLS configuration enabled with key store configs" + ) + private boolean tlsEnabledWithKeystore; + + @ApiModelProperty( + name = "isServerMode", + value = "Is the SSL Configuration for a Server or Client", + required = true + ) + private boolean serverMode; + + @ApiModelProperty( + name = "isHttps", + value = "Is the SSL Configuration for a Http client or Server" + ) + private boolean isHttps; + + @Override + public PulsarSslConfiguration clone() { + try { + return (PulsarSslConfiguration) super.clone(); + } catch (CloneNotSupportedException e) { + throw new RuntimeException("Failed to clone PulsarSslConfiguration", e); + } + } + +} \ No newline at end of file diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/PulsarSslFactory.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/PulsarSslFactory.java new file mode 100644 index 0000000000000..bccbbbe5b2516 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/PulsarSslFactory.java @@ -0,0 +1,106 @@ +/* + * 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; + +import io.netty.buffer.ByteBufAllocator; +import io.netty.handler.ssl.SslContext; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; + +/** + * Factory for generating SSL Context and SSL Engine using {@link PulsarSslConfiguration}. + */ +public interface PulsarSslFactory extends AutoCloseable { + + /** + * Initializes the PulsarSslFactory. + * @param config {@link PulsarSslConfiguration} object required for initialization + */ + void initialize(PulsarSslConfiguration config); + + /** + * Creates a Client {@link SSLEngine} utilizing {@link ByteBufAllocator} object, the peer hostname, peer port and + * {@link PulsarSslConfiguration} object provided during initialization. + * + * @param buf The ByteBufAllocator required for netty connections. This can be passed as {@code null} if utilized + * for web connections. + * @param peerHost the name of the peer host + * @param peerPort the port number of the peer + * @return {@link SSLEngine} + */ + SSLEngine createClientSslEngine(ByteBufAllocator buf, String peerHost, int peerPort); + + /** + * Creates a Server {@link SSLEngine} utilizing the {@link ByteBufAllocator} object and + * {@link PulsarSslConfiguration} object provided during initialization. + * + * @param buf The ByteBufAllocator required for netty connections. This can be passed as {@code null} if utilized + * for web connections. + * @return {@link SSLEngine} + */ + SSLEngine createServerSslEngine(ByteBufAllocator buf); + + /** + * Returns a boolean value indicating {@link SSLContext} or {@link SslContext} should be refreshed. + * + * @return {@code true} if {@link SSLContext} or {@link SslContext} should be refreshed. + */ + boolean needsUpdate(); + + /** + * Update the internal {@link SSLContext} or {@link SslContext}. + * @throws Exception if there are any issues generating the new {@link SSLContext} or {@link SslContext} + */ + default void update() throws Exception { + if (this.needsUpdate()) { + this.createInternalSslContext(); + } + } + + /** + * Creates the following: + * 1. {@link SslContext} if netty connections are being created for Non-Keystore based TLS configurations. + * 2. {@link SSLContext} if netty connections are being created for Keystore based TLS configurations. It will + * also create it for all web connections irrespective of it being Keystore or Non-Keystore based TLS + * configurations. + * + * @throws Exception if there are any issues creating the new {@link SSLContext} or {@link SslContext} + */ + void createInternalSslContext() throws Exception; + + /** + * Get the internally stored {@link SSLContext}. It will be used in the following scenarios: + * 1. Netty connection creations for keystore based TLS configurations + * 2. All Web connections + * + * @return {@link SSLContext} + * @throws RuntimeException if the {@link SSLContext} object has not yet been initialized. + */ + SSLContext getInternalSslContext(); + + /** + * Get the internally stored {@link SslContext}. It will be used to create Netty Connections for non-keystore based + * tls configurations. + * + * @return {@link SslContext} + * @throws RuntimeException if the {@link SslContext} object has not yet been initialized. + */ + SslContext getInternalNettySslContext(); + +} \ No newline at end of file diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SslContextAutoRefreshBuilder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SslContextAutoRefreshBuilder.java deleted file mode 100644 index 8c8f580046448..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SslContextAutoRefreshBuilder.java +++ /dev/null @@ -1,100 +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; - -import java.io.IOException; -import java.security.GeneralSecurityException; -import java.util.concurrent.TimeUnit; -import lombok.extern.slf4j.Slf4j; - -/** - * Auto refresher and builder of SSLContext. - * - * @param - * type of SSLContext - */ -@Slf4j -public abstract class SslContextAutoRefreshBuilder { - protected final long refreshTime; - protected long lastRefreshTime; - - public SslContextAutoRefreshBuilder( - long certRefreshInSec) { - this.refreshTime = TimeUnit.SECONDS.toMillis(certRefreshInSec); - this.lastRefreshTime = -1; - - if (log.isDebugEnabled()) { - log.debug("Certs will be refreshed every {} seconds", certRefreshInSec); - } - } - - /** - * updates and returns cached SSLContext. - * - * @return - * @throws GeneralSecurityException - * @throws IOException - */ - protected abstract T update() throws GeneralSecurityException, IOException; - - /** - * Returns cached SSLContext. - * - * @return - */ - protected abstract T getSslContext(); - - /** - * Returns whether the key files modified after a refresh time, and context need update. - * - * @return true if files modified - */ - protected abstract boolean needUpdate(); - - /** - * It updates SSLContext at every configured refresh time and returns updated SSLContext. - * - * @return - */ - public T get() { - T ctx = getSslContext(); - if (ctx == null) { - try { - update(); - lastRefreshTime = System.currentTimeMillis(); - return getSslContext(); - } catch (GeneralSecurityException | IOException e) { - log.error("Exception while trying to refresh ssl Context {}", e.getMessage(), e); - } - } else { - long now = System.currentTimeMillis(); - if (refreshTime <= 0 || now > (lastRefreshTime + refreshTime)) { - if (needUpdate()) { - try { - ctx = update(); - lastRefreshTime = now; - } catch (GeneralSecurityException | IOException e) { - log.error("Exception while trying to refresh ssl Context {} ", e.getMessage(), e); - } - } - } - } - return ctx; - } -} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NetSslContextBuilder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NetSslContextBuilder.java deleted file mode 100644 index 3d4d4e72546ea..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NetSslContextBuilder.java +++ /dev/null @@ -1,90 +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.keystoretls; - -import java.io.IOException; -import java.security.GeneralSecurityException; -import javax.net.ssl.SSLContext; -import org.apache.pulsar.common.util.FileModifiedTimeUpdater; -import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder; - -/** - * Similar to `DefaultSslContextBuilder`, which build `javax.net.ssl.SSLContext` for web service. - */ -public class NetSslContextBuilder extends SslContextAutoRefreshBuilder { - private volatile SSLContext sslContext; - - protected final boolean tlsAllowInsecureConnection; - protected final boolean tlsRequireTrustedClientCertOnConnect; - - protected final String tlsProvider; - protected final String tlsKeyStoreType; - protected final String tlsKeyStorePassword; - protected final FileModifiedTimeUpdater tlsKeyStore; - protected final String tlsTrustStoreType; - protected final String tlsTrustStorePassword; - protected final FileModifiedTimeUpdater tlsTrustStore; - - public NetSslContextBuilder(String sslProviderString, - String keyStoreTypeString, - String keyStore, - String keyStorePasswordPath, - boolean allowInsecureConnection, - String trustStoreTypeString, - String trustStore, - String trustStorePasswordPath, - boolean requireTrustedClientCertOnConnect, - long certRefreshInSec) { - super(certRefreshInSec); - - this.tlsAllowInsecureConnection = allowInsecureConnection; - this.tlsProvider = sslProviderString; - this.tlsKeyStoreType = keyStoreTypeString; - this.tlsKeyStore = new FileModifiedTimeUpdater(keyStore); - this.tlsKeyStorePassword = keyStorePasswordPath; - - this.tlsTrustStoreType = trustStoreTypeString; - this.tlsTrustStore = new FileModifiedTimeUpdater(trustStore); - this.tlsTrustStorePassword = trustStorePasswordPath; - - this.tlsRequireTrustedClientCertOnConnect = requireTrustedClientCertOnConnect; - } - - @Override - public synchronized SSLContext update() - throws GeneralSecurityException, IOException { - this.sslContext = KeyStoreSSLContext.createServerSslContext(tlsProvider, - tlsKeyStoreType, tlsKeyStore.getFileName(), tlsKeyStorePassword, - tlsAllowInsecureConnection, - tlsTrustStoreType, tlsTrustStore.getFileName(), tlsTrustStorePassword, - tlsRequireTrustedClientCertOnConnect); - return this.sslContext; - } - - @Override - public SSLContext getSslContext() { - return this.sslContext; - } - - @Override - public boolean needUpdate() { - return tlsKeyStore.checkAndRefresh() - || tlsTrustStore.checkAndRefresh(); - } -} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NettySSLContextAutoRefreshBuilder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NettySSLContextAutoRefreshBuilder.java deleted file mode 100644 index 6d0cfb108bd0e..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/keystoretls/NettySSLContextAutoRefreshBuilder.java +++ /dev/null @@ -1,155 +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.keystoretls; - -import java.io.IOException; -import java.security.GeneralSecurityException; -import java.util.Set; -import org.apache.pulsar.client.api.AuthenticationDataProvider; -import org.apache.pulsar.client.api.KeyStoreParams; -import org.apache.pulsar.common.util.FileModifiedTimeUpdater; -import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder; - -/** - * SSL context builder for Netty. - */ -public class NettySSLContextAutoRefreshBuilder extends SslContextAutoRefreshBuilder { - private volatile KeyStoreSSLContext keyStoreSSLContext; - - protected final boolean tlsAllowInsecureConnection; - protected final Set tlsCiphers; - protected final Set tlsProtocols; - protected boolean tlsRequireTrustedClientCertOnConnect; - - protected final String tlsProvider; - protected final String tlsTrustStoreType; - protected final String tlsTrustStorePassword; - protected final FileModifiedTimeUpdater tlsTrustStore; - - // client context not need keystore at start time, keyStore is passed in by authData. - protected String tlsKeyStoreType; - protected String tlsKeyStorePassword; - protected FileModifiedTimeUpdater tlsKeyStore; - - protected final boolean isServer; - - // for server - public NettySSLContextAutoRefreshBuilder(String sslProviderString, - String keyStoreTypeString, - String keyStore, - String keyStorePassword, - boolean allowInsecureConnection, - String trustStoreTypeString, - String trustStore, - String trustStorePassword, - boolean requireTrustedClientCertOnConnect, - Set ciphers, - Set protocols, - long certRefreshInSec) { - super(certRefreshInSec); - - this.tlsAllowInsecureConnection = allowInsecureConnection; - this.tlsProvider = sslProviderString; - - this.tlsKeyStoreType = keyStoreTypeString; - this.tlsKeyStore = new FileModifiedTimeUpdater(keyStore); - this.tlsKeyStorePassword = keyStorePassword; - - this.tlsTrustStoreType = trustStoreTypeString; - this.tlsTrustStore = new FileModifiedTimeUpdater(trustStore); - this.tlsTrustStorePassword = trustStorePassword; - - this.tlsRequireTrustedClientCertOnConnect = requireTrustedClientCertOnConnect; - this.tlsCiphers = ciphers; - this.tlsProtocols = protocols; - - this.isServer = true; - } - - // for client - public NettySSLContextAutoRefreshBuilder(String sslProviderString, - boolean allowInsecureConnection, - String trustStoreTypeString, - String trustStore, - String trustStorePassword, - String keyStoreTypeString, - String keyStore, - String keyStorePassword, - Set ciphers, - Set protocols, - long certRefreshInSec, - AuthenticationDataProvider authData) { - super(certRefreshInSec); - - this.tlsAllowInsecureConnection = allowInsecureConnection; - this.tlsProvider = sslProviderString; - - if (authData != null) { - KeyStoreParams authParams = authData.getTlsKeyStoreParams(); - if (authParams != null) { - keyStoreTypeString = authParams.getKeyStoreType(); - keyStore = authParams.getKeyStorePath(); - keyStorePassword = authParams.getKeyStorePassword(); - } - } - this.tlsKeyStoreType = keyStoreTypeString; - this.tlsKeyStore = new FileModifiedTimeUpdater(keyStore); - this.tlsKeyStorePassword = keyStorePassword; - - this.tlsTrustStoreType = trustStoreTypeString; - this.tlsTrustStore = new FileModifiedTimeUpdater(trustStore); - this.tlsTrustStorePassword = trustStorePassword; - - this.tlsCiphers = ciphers; - this.tlsProtocols = protocols; - - this.isServer = false; - } - - @Override - public synchronized KeyStoreSSLContext update() throws GeneralSecurityException, IOException { - if (isServer) { - this.keyStoreSSLContext = KeyStoreSSLContext.createServerKeyStoreSslContext(tlsProvider, - tlsKeyStoreType, tlsKeyStore.getFileName(), tlsKeyStorePassword, - tlsAllowInsecureConnection, - tlsTrustStoreType, tlsTrustStore.getFileName(), tlsTrustStorePassword, - tlsRequireTrustedClientCertOnConnect, tlsCiphers, tlsProtocols); - } else { - this.keyStoreSSLContext = KeyStoreSSLContext.createClientKeyStoreSslContext(tlsProvider, - tlsKeyStoreType, - tlsKeyStore.getFileName(), - tlsKeyStorePassword, - tlsAllowInsecureConnection, - tlsTrustStoreType, tlsTrustStore.getFileName(), tlsTrustStorePassword, - tlsCiphers, tlsProtocols); - } - return this.keyStoreSSLContext; - } - - @Override - public KeyStoreSSLContext getSslContext() { - return this.keyStoreSSLContext; - } - - @Override - public boolean needUpdate() { - return (tlsKeyStore != null && tlsKeyStore.checkAndRefresh()) - || (tlsTrustStore != null && tlsTrustStore.checkAndRefresh()); - } -} diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/DefaultPulsarSslFactoryTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/DefaultPulsarSslFactoryTest.java new file mode 100644 index 0000000000000..34cf3a97ce803 --- /dev/null +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/DefaultPulsarSslFactoryTest.java @@ -0,0 +1,282 @@ +/* + * 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; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertThrows; +import com.google.common.io.Resources; +import io.netty.buffer.ByteBufAllocator; +import io.netty.handler.ssl.OpenSslEngine; +import java.io.File; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; +import javax.net.ssl.SSLEngine; +import org.testng.annotations.Test; + +public class DefaultPulsarSslFactoryTest { + + public final static String KEYSTORE_FILE_PATH = + getAbsolutePath("certificate-authority/jks/broker.keystore.jks"); + public final static String TRUSTSTORE_FILE_PATH = + getAbsolutePath("certificate-authority/jks/broker.truststore.jks"); + public final static String TRUSTSTORE_NO_PASSWORD_FILE_PATH = + getAbsolutePath("certificate-authority/jks/broker.truststore.nopassword.jks"); + public final static String KEYSTORE_PW = "111111"; + public final static String TRUSTSTORE_PW = "111111"; + public final static String KEYSTORE_TYPE = "JKS"; + + public final static String CA_CERT_FILE_PATH = + getAbsolutePath("certificate-authority/certs/ca.cert.pem"); + public final static String CERT_FILE_PATH = + getAbsolutePath("certificate-authority/server-keys/broker.cert.pem"); + public final static String KEY_FILE_PATH = + getAbsolutePath("certificate-authority/server-keys/broker.key-pk8.pem"); + + @Test + public void sslContextCreationUsingKeystoreTest() throws Exception { + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsEnabledWithKeystore(true) + .tlsKeyStoreType(KEYSTORE_TYPE) + .tlsKeyStorePath(KEYSTORE_FILE_PATH) + .tlsKeyStorePassword(KEYSTORE_PW) + .tlsTrustStorePath(TRUSTSTORE_FILE_PATH) + .tlsTrustStorePassword(TRUSTSTORE_PW) + .build(); + PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + assertNotNull(pulsarSslFactory.getInternalSslContext()); + assertThrows(RuntimeException.class, pulsarSslFactory::getInternalNettySslContext); + } + + @Test + public void sslContextCreationUsingPasswordLessTruststoreTest() throws Exception { + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsEnabledWithKeystore(true) + .tlsKeyStoreType(KEYSTORE_TYPE) + .tlsKeyStorePath(KEYSTORE_FILE_PATH) + .tlsKeyStorePassword(KEYSTORE_PW) + .tlsTrustStorePath(TRUSTSTORE_NO_PASSWORD_FILE_PATH) + .build(); + PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + assertNotNull(pulsarSslFactory.getInternalSslContext()); + assertThrows(RuntimeException.class, pulsarSslFactory::getInternalNettySslContext); + } + + @Test + public void sslContextCreationUsingTlsCertsTest() throws Exception { + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsCertificateFilePath(CERT_FILE_PATH) + .tlsKeyFilePath(KEY_FILE_PATH) + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH) + .build(); + PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + assertNotNull(pulsarSslFactory.getInternalNettySslContext()); + assertThrows(RuntimeException.class, pulsarSslFactory::getInternalSslContext); + } + + @Test + public void sslContextCreationUsingOnlyCACertsTest() throws Exception { + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH) + .build(); + PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + assertNotNull(pulsarSslFactory.getInternalNettySslContext()); + assertThrows(RuntimeException.class, pulsarSslFactory::getInternalSslContext); + } + + @Test + public void sslContextCreationForWebClientConnections() throws Exception { + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsCertificateFilePath(CERT_FILE_PATH) + .tlsKeyFilePath(KEY_FILE_PATH) + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH) + .isHttps(true) + .build(); + PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + assertNotNull(pulsarSslFactory.getInternalSslContext()); + assertThrows(RuntimeException.class, pulsarSslFactory::getInternalNettySslContext); + } + + @Test + public void sslContextCreationForWebServerConnectionsTest() throws Exception { + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsCertificateFilePath(CERT_FILE_PATH) + .tlsKeyFilePath(KEY_FILE_PATH) + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH) + .isHttps(true) + .serverMode(true) + .build(); + PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + assertNotNull(pulsarSslFactory.getInternalSslContext()); + assertThrows(RuntimeException.class, pulsarSslFactory::getInternalNettySslContext); + } + + @Test + public void sslEngineCreationWithEnabledProtocolsAndCiphersForOpenSSLTest() throws Exception { + Set ciphers = new HashSet<>(); + ciphers.add("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256"); + ciphers.add("TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256"); + Set protocols = new HashSet<>(); + protocols.add("TLSv1.2"); + protocols.add("TLSv1"); + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsCertificateFilePath(CERT_FILE_PATH) + .tlsKeyFilePath(KEY_FILE_PATH) + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH) + .tlsCiphers(ciphers) + .tlsProtocols(protocols) + .serverMode(true) + .build(); + PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + assertNotNull(pulsarSslFactory.getInternalNettySslContext()); + assertThrows(RuntimeException.class, pulsarSslFactory::getInternalSslContext); + SSLEngine sslEngine = pulsarSslFactory.createServerSslEngine(ByteBufAllocator.DEFAULT); + /* Adding SSLv2Hello protocol only during expected checks as Netty adds it as part of the + ReferenceCountedOpenSslEngine's setEnabledProtocols method. The reasoning is that OpenSSL currently has no + way to disable this protocol. + */ + protocols.add("SSLv2Hello"); + assertEquals(new HashSet<>(Arrays.asList(sslEngine.getEnabledProtocols())), protocols); + assertEquals(new HashSet<>(Arrays.asList(sslEngine.getEnabledCipherSuites())), ciphers); + assert(!sslEngine.getUseClientMode()); + } + + @Test + public void sslEngineCreationWithEnabledProtocolsAndCiphersForWebTest() throws Exception { + Set ciphers = new HashSet<>(); + ciphers.add("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256"); + ciphers.add("TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256"); + Set protocols = new HashSet<>(); + protocols.add("TLSv1.2"); + protocols.add("TLSv1"); + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsCertificateFilePath(CERT_FILE_PATH) + .tlsKeyFilePath(KEY_FILE_PATH) + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH) + .tlsCiphers(ciphers) + .tlsProtocols(protocols) + .isHttps(true) + .serverMode(true) + .build(); + PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + assertNotNull(pulsarSslFactory.getInternalSslContext()); + assertThrows(RuntimeException.class, pulsarSslFactory::getInternalNettySslContext); + SSLEngine sslEngine = pulsarSslFactory.createServerSslEngine(ByteBufAllocator.DEFAULT); + assertEquals(new HashSet<>(Arrays.asList(sslEngine.getEnabledProtocols())), protocols); + assertEquals(new HashSet<>(Arrays.asList(sslEngine.getEnabledCipherSuites())), ciphers); + assert(!sslEngine.getUseClientMode()); + } + + @Test + public void sslContextCreationAsOpenSslTlsProvider() throws Exception { + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsProvider("OPENSSL") + .tlsCertificateFilePath(CERT_FILE_PATH) + .tlsKeyFilePath(KEY_FILE_PATH) + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH) + .serverMode(true) + .build(); + PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + assertNotNull(pulsarSslFactory.getInternalNettySslContext()); + assertThrows(RuntimeException.class, pulsarSslFactory::getInternalSslContext); + SSLEngine sslEngine = pulsarSslFactory.createServerSslEngine(ByteBufAllocator.DEFAULT); + assert(sslEngine instanceof OpenSslEngine); + } + + @Test + public void sslContextCreationAsJDKTlsProvider() throws Exception { + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsProvider("JDK") + .tlsCertificateFilePath(CERT_FILE_PATH) + .tlsKeyFilePath(KEY_FILE_PATH) + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH) + .serverMode(true) + .build(); + PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + assertNotNull(pulsarSslFactory.getInternalNettySslContext()); + assertThrows(RuntimeException.class, pulsarSslFactory::getInternalSslContext); + SSLEngine sslEngine = pulsarSslFactory.createServerSslEngine(ByteBufAllocator.DEFAULT); + assert (!(sslEngine instanceof OpenSslEngine)); + } + + @Test + public void sslEngineMutualAuthEnabledTest() throws Exception { + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsProvider("JDK") + .tlsCertificateFilePath(CERT_FILE_PATH) + .tlsKeyFilePath(KEY_FILE_PATH) + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH) + .requireTrustedClientCertOnConnect(true) + .serverMode(true) + .build(); + PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + assertNotNull(pulsarSslFactory.getInternalNettySslContext()); + assertThrows(RuntimeException.class, pulsarSslFactory::getInternalSslContext); + SSLEngine sslEngine = pulsarSslFactory.createServerSslEngine(ByteBufAllocator.DEFAULT); + assert(sslEngine.getNeedClientAuth()); + } + + @Test + public void sslEngineSniClientTest() throws Exception { + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsCertificateFilePath(CERT_FILE_PATH) + .tlsKeyFilePath(KEY_FILE_PATH) + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH) + .build(); + PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + assertNotNull(pulsarSslFactory.getInternalNettySslContext()); + assertThrows(RuntimeException.class, pulsarSslFactory::getInternalSslContext); + SSLEngine sslEngine = pulsarSslFactory.createClientSslEngine(ByteBufAllocator.DEFAULT, "localhost", + 1234); + assertEquals(sslEngine.getPeerHost(), "localhost"); + assertEquals(sslEngine.getPeerPort(), 1234); + } + + + + private static String getAbsolutePath(String resourceName) { + return new File(Resources.getResource(resourceName).getPath()).getAbsolutePath(); + } + +} diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/FileModifiedTimeUpdaterTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/FileModifiedTimeUpdaterTest.java index 1d950078d21c5..a41c9ceb2cbb7 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/FileModifiedTimeUpdaterTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/FileModifiedTimeUpdaterTest.java @@ -48,6 +48,11 @@ public BasicAuthenticationData(String authParam) { this.authParam = authParam; } + @Override + public boolean hasDataForTls() { + return true; + } + public boolean hasDataFromCommand() { return true; } @@ -107,14 +112,17 @@ public void testNettyClientSslContextRefresher() throws Exception { createFile(Paths.get(certFile)); provider.certFilePath = certFile; provider.keyFilePath = certFile; - NettyClientSslContextRefresher refresher = new NettyClientSslContextRefresher(null, false, certFile, - provider, null, null, 1); - Thread.sleep(5000); - Paths.get(certFile).toFile().delete(); - // update the file - createFile(Paths.get(certFile)); - Awaitility.await().atMost(30, TimeUnit.SECONDS).until(()-> refresher.needUpdate()); - assertTrue(refresher.needUpdate()); + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .allowInsecureConnection(false).tlsTrustCertsFilePath(certFile).authData(provider).build(); + try (PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory()) { + pulsarSslFactory.initialize(pulsarSslConfiguration); + Thread.sleep(5000); + Paths.get(certFile).toFile().delete(); + // update the file + createFile(Paths.get(certFile)); + Awaitility.await().atMost(30, TimeUnit.SECONDS).until(pulsarSslFactory::needsUpdate); + assertTrue(pulsarSslFactory.needsUpdate()); + } } } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/netty/SslContextTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/netty/SslContextTest.java index 303df5a003278..120fee9319db7 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/netty/SslContextTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/netty/SslContextTest.java @@ -21,16 +21,14 @@ import static org.testng.Assert.assertThrows; import com.google.common.io.Resources; import io.netty.handler.ssl.SslProvider; -import java.io.IOException; -import java.security.GeneralSecurityException; import java.util.HashSet; import java.util.Set; import javax.net.ssl.SSLException; import org.apache.pulsar.client.api.AuthenticationDataProvider; import org.apache.pulsar.client.api.KeyStoreParams; -import org.apache.pulsar.common.util.NettyClientSslContextRefresher; -import org.apache.pulsar.common.util.NettyServerSslContextBuilder; -import org.apache.pulsar.common.util.keystoretls.NettySSLContextAutoRefreshBuilder; +import org.apache.pulsar.common.util.DefaultPulsarSslFactory; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -84,13 +82,22 @@ public static Object[] getCipher() { @Test(dataProvider = "cipherDataProvider") public void testServerKeyStoreSSLContext(Set cipher) throws Exception { - NettySSLContextAutoRefreshBuilder contextAutoRefreshBuilder = new NettySSLContextAutoRefreshBuilder( - null, - keyStoreType, brokerKeyStorePath, keyStorePassword, false, - keyStoreType, brokerTrustStorePath, keyStorePassword, - true, cipher, - null, 600); - contextAutoRefreshBuilder.update(); + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .tlsEnabledWithKeystore(true) + .tlsKeyStoreType(keyStoreType) + .tlsKeyStorePath(brokerKeyStorePath) + .tlsKeyStorePassword(keyStorePassword) + .allowInsecureConnection(false) + .tlsTrustStoreType(keyStoreType) + .tlsTrustStorePath(brokerTrustStorePath) + .tlsTrustStorePassword(keyStorePassword) + .requireTrustedClientCertOnConnect(true) + .tlsCiphers(cipher) + .build(); + try (PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory()) { + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + } } private static class ClientAuthenticationData implements AuthenticationDataProvider { @@ -102,45 +109,67 @@ public KeyStoreParams getTlsKeyStoreParams() { @Test(dataProvider = "cipherDataProvider") public void testClientKeyStoreSSLContext(Set cipher) throws Exception { - NettySSLContextAutoRefreshBuilder contextAutoRefreshBuilder = new NettySSLContextAutoRefreshBuilder( - null, - false, - keyStoreType, brokerTrustStorePath, keyStorePassword, - null, null, null, - cipher, null, 0, new ClientAuthenticationData()); - contextAutoRefreshBuilder.update(); + PulsarSslConfiguration pulsarSslConfiguration = PulsarSslConfiguration.builder() + .allowInsecureConnection(false) + .tlsEnabledWithKeystore(true) + .tlsTrustStoreType(keyStoreType) + .tlsTrustStorePath(brokerTrustStorePath) + .tlsTrustStorePassword(keyStorePassword) + .tlsCiphers(cipher) + .authData(new ClientAuthenticationData()) + .build(); + try (PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory()) { + pulsarSslFactory.initialize(pulsarSslConfiguration); + pulsarSslFactory.createInternalSslContext(); + } } @Test(dataProvider = "caCertSslContextDataProvider") public void testServerCaCertSslContextWithSslProvider(SslProvider sslProvider, Set ciphers) - throws GeneralSecurityException, IOException { - NettyServerSslContextBuilder sslContext = new NettyServerSslContextBuilder(sslProvider, - true, - caCertPath, brokerCertPath, brokerKeyPath, - ciphers, - null, - true, 60); - if (ciphers != null) { - if (sslProvider == null || sslProvider == SslProvider.OPENSSL) { - assertThrows(SSLException.class, sslContext::update); - return; + throws Exception { + try (PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory()) { + PulsarSslConfiguration.PulsarSslConfigurationBuilder builder = PulsarSslConfiguration.builder() + .tlsTrustCertsFilePath(caCertPath) + .tlsCertificateFilePath(brokerCertPath) + .tlsKeyFilePath(brokerKeyPath) + .tlsCiphers(ciphers) + .requireTrustedClientCertOnConnect(true); + if (sslProvider != null) { + builder.tlsProvider(sslProvider.name()); } + PulsarSslConfiguration pulsarSslConfiguration = builder.build(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + + if (ciphers != null) { + if (sslProvider == null || sslProvider == SslProvider.OPENSSL) { + assertThrows(SSLException.class, pulsarSslFactory::createInternalSslContext); + return; + } + } + pulsarSslFactory.createInternalSslContext(); } - sslContext.update(); } @Test(dataProvider = "caCertSslContextDataProvider") public void testClientCaCertSslContextWithSslProvider(SslProvider sslProvider, Set ciphers) - throws GeneralSecurityException, IOException { - NettyClientSslContextRefresher sslContext = new NettyClientSslContextRefresher(sslProvider, - true, caCertPath, - null, ciphers, null, 0); - if (ciphers != null) { - if (sslProvider == null || sslProvider == SslProvider.OPENSSL) { - assertThrows(SSLException.class, sslContext::update); - return; + throws Exception { + try (PulsarSslFactory pulsarSslFactory = new DefaultPulsarSslFactory()) { + PulsarSslConfiguration.PulsarSslConfigurationBuilder builder = PulsarSslConfiguration.builder() + .allowInsecureConnection(true) + .tlsTrustCertsFilePath(caCertPath) + .tlsCiphers(ciphers); + if (sslProvider != null) { + builder.tlsProvider(sslProvider.name()); + } + PulsarSslConfiguration pulsarSslConfiguration = builder.build(); + pulsarSslFactory.initialize(pulsarSslConfiguration); + if (ciphers != null) { + if (sslProvider == null || sslProvider == SslProvider.OPENSSL) { + assertThrows(SSLException.class, pulsarSslFactory::createInternalSslContext); + return; + } } + pulsarSslFactory.createInternalSslContext(); } - sslContext.update(); } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java index 1d8c66a57df53..4f01f17174e31 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java @@ -24,6 +24,9 @@ import java.util.EnumSet; import java.util.List; import java.util.Optional; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import javax.servlet.DispatcherType; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.authentication.AuthenticationService; @@ -31,6 +34,10 @@ import org.apache.pulsar.broker.web.JettyRequestLogFactory; import org.apache.pulsar.broker.web.RateLimitingFilter; import org.apache.pulsar.broker.web.WebExecutorThreadPool; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.util.DefaultPulsarSslFactory; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.functions.worker.PulsarWorkerOpenTelemetry; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; @@ -75,6 +82,8 @@ public class WorkerServer { private ServerConnector httpsConnector; private final FilterInitializer filterInitializer; + private PulsarSslFactory sslFactory; + private ScheduledExecutorService scheduledExecutorService; public WorkerServer(WorkerService workerService, AuthenticationService authenticationService) { this.workerConfig = workerService.getWorkerConfig(); @@ -155,35 +164,22 @@ private void init() { if (this.workerConfig.getTlsEnabled()) { log.info("Configuring https server on port={}", this.workerConfig.getWorkerPortTls()); try { - SslContextFactory sslCtxFactory; - if (workerConfig.isTlsEnabledWithKeyStore()) { - sslCtxFactory = JettySslContextFactory.createServerSslContextWithKeystore( - workerConfig.getTlsProvider(), - workerConfig.getTlsKeyStoreType(), - workerConfig.getTlsKeyStore(), - workerConfig.getTlsKeyStorePassword(), - workerConfig.isTlsAllowInsecureConnection(), - workerConfig.getTlsTrustStoreType(), - workerConfig.getTlsTrustStore(), - workerConfig.getTlsTrustStorePassword(), - workerConfig.isTlsRequireTrustedClientCertOnConnect(), - workerConfig.getWebServiceTlsCiphers(), - workerConfig.getWebServiceTlsProtocols(), - workerConfig.getTlsCertRefreshCheckDurationSec() - ); - } else { - sslCtxFactory = JettySslContextFactory.createServerSslContext( - workerConfig.getTlsProvider(), - workerConfig.isTlsAllowInsecureConnection(), - workerConfig.getTlsTrustCertsFilePath(), - workerConfig.getTlsCertificateFilePath(), - workerConfig.getTlsKeyFilePath(), - workerConfig.isTlsRequireTrustedClientCertOnConnect(), - workerConfig.getWebServiceTlsCiphers(), - workerConfig.getWebServiceTlsProtocols(), - workerConfig.getTlsCertRefreshCheckDurationSec() - ); - } + PulsarSslConfiguration sslConfiguration = buildSslConfiguration(workerConfig); + this.sslFactory = new DefaultPulsarSslFactory(); + this.sslFactory.initialize(sslConfiguration); + this.sslFactory.createInternalSslContext(); + this.scheduledExecutorService = Executors + .newSingleThreadScheduledExecutor(new ExecutorProvider + .ExtendedThreadFactory("functions-worker-web-ssl-refresh")); + this.scheduledExecutorService.scheduleWithFixedDelay(this::refreshSslContext, + workerConfig.getTlsCertRefreshCheckDurationSec(), + workerConfig.getTlsCertRefreshCheckDurationSec(), + TimeUnit.SECONDS); + SslContextFactory sslCtxFactory = + JettySslContextFactory.createSslContextFactory(this.workerConfig.getTlsProvider(), + this.sslFactory, this.workerConfig.isTlsRequireTrustedClientCertOnConnect(), + this.workerConfig.getWebServiceTlsCiphers(), + this.workerConfig.getWebServiceTlsProtocols()); List connectionFactories = new ArrayList<>(); if (workerConfig.isWebServiceHaProxyProtocolEnabled()) { connectionFactories.add(new ProxyConnectionFactory()); @@ -288,6 +284,9 @@ public void stop() { log.warn("Error stopping function web-server executor", e); } } + if (this.scheduledExecutorService != null) { + this.scheduledExecutorService.shutdownNow(); + } } public Optional getListenPortHTTP() { @@ -305,4 +304,33 @@ public Optional getListenPortHTTPS() { return Optional.empty(); } } + + protected void refreshSslContext() { + try { + this.sslFactory.update(); + } catch (Exception e) { + log.error("Failed to refresh SSL context", e); + } + } + + protected PulsarSslConfiguration buildSslConfiguration(WorkerConfig config) { + return PulsarSslConfiguration.builder() + .tlsKeyStoreType(config.getTlsKeyStoreType()) + .tlsKeyStorePath(config.getTlsKeyStore()) + .tlsKeyStorePassword(config.getTlsKeyStorePassword()) + .tlsTrustStoreType(config.getTlsTrustStoreType()) + .tlsTrustStorePath(config.getTlsTrustStore()) + .tlsTrustStorePassword(config.getTlsTrustStorePassword()) + .tlsCiphers(config.getWebServiceTlsCiphers()) + .tlsProtocols(config.getWebServiceTlsProtocols()) + .tlsTrustCertsFilePath(config.getTlsTrustCertsFilePath()) + .tlsCertificateFilePath(config.getTlsCertificateFilePath()) + .tlsKeyFilePath(config.getTlsKeyFilePath()) + .allowInsecureConnection(config.isTlsAllowInsecureConnection()) + .requireTrustedClientCertOnConnect(config.isTlsRequireTrustedClientCertOnConnect()) + .tlsEnabledWithKeystore(config.isTlsEnabledWithKeyStore()) + .serverMode(true) + .isHttps(true) + .build(); + } } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java index 0108b770249a0..54b6db5198c57 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java @@ -24,13 +24,15 @@ import java.io.InputStream; import java.net.URI; import java.nio.ByteBuffer; -import java.security.cert.X509Certificate; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.Set; import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import javax.net.ssl.SSLContext; import javax.servlet.ServletConfig; import javax.servlet.ServletException; @@ -39,10 +41,10 @@ import org.apache.pulsar.broker.web.AuthenticationFilter; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; -import org.apache.pulsar.client.api.KeyStoreParams; import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.common.util.SecurityUtility; -import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.client.HttpRequest; @@ -88,6 +90,8 @@ class AdminProxyHandler extends ProxyServlet { private final Authentication proxyClientAuthentication; private final String brokerWebServiceUrl; private final String functionWorkerWebServiceUrl; + private PulsarSslFactory pulsarSslFactory; + private ScheduledExecutorService sslContextRefresher; AdminProxyHandler(ProxyConfiguration config, BrokerDiscoveryProvider discoveryProvider, Authentication proxyClientAuthentication) { @@ -98,7 +102,16 @@ class AdminProxyHandler extends ProxyServlet { : config.getBrokerWebServiceURL(); this.functionWorkerWebServiceUrl = config.isTlsEnabledWithBroker() ? config.getFunctionWorkerWebServiceURLTLS() : config.getFunctionWorkerWebServiceURL(); - + if (config.isTlsEnabledWithBroker()) { + this.pulsarSslFactory = createPulsarSslFactory(); + this.sslContextRefresher = Executors.newSingleThreadScheduledExecutor( + new ExecutorProvider.ExtendedThreadFactory("pulsar-proxy-admin-handler-ssl-refresh")); + if (config.getTlsCertRefreshCheckDurationSec() > 0) { + this.sslContextRefresher.scheduleWithFixedDelay(this::refreshSslContext, + config.getTlsCertRefreshCheckDurationSec(), config.getTlsCertRefreshCheckDurationSec(), + TimeUnit.SECONDS); + } + } super.setTimeout(config.getHttpProxyTimeout()); } @@ -259,44 +272,7 @@ protected HttpClient newHttpClient() { try { if (config.isTlsEnabledWithBroker()) { try { - X509Certificate[] trustCertificates = SecurityUtility - .loadCertificatesFromPemFile(config.getBrokerClientTrustCertsFilePath()); - - SSLContext sslCtx; - AuthenticationDataProvider authData = proxyClientAuthentication.getAuthData(); - if (config.isBrokerClientTlsEnabledWithKeyStore()) { - KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : null; - sslCtx = KeyStoreSSLContext.createClientSslContext( - config.getBrokerClientSslProvider(), - params != null ? params.getKeyStoreType() : null, - params != null ? params.getKeyStorePath() : null, - params != null ? params.getKeyStorePassword() : null, - config.isTlsAllowInsecureConnection(), - config.getBrokerClientTlsTrustStoreType(), - config.getBrokerClientTlsTrustStore(), - config.getBrokerClientTlsTrustStorePassword(), - config.getBrokerClientTlsCiphers(), - config.getBrokerClientTlsProtocols()); - } else { - if (authData.hasDataForTls()) { - sslCtx = SecurityUtility.createSslContext( - config.isTlsAllowInsecureConnection(), - trustCertificates, - authData.getTlsCertificates(), - authData.getTlsPrivateKey(), - config.getBrokerClientSslProvider() - ); - } else { - sslCtx = SecurityUtility.createSslContext( - config.isTlsAllowInsecureConnection(), - trustCertificates, - config.getBrokerClientSslProvider() - ); - } - } - - SslContextFactory contextFactory = new SslContextFactory.Client(); - contextFactory.setSslContext(sslCtx); + SslContextFactory contextFactory = new Client(this.pulsarSslFactory); if (!config.isTlsHostnameVerificationEnabled()) { contextFactory.setEndpointIdentificationAlgorithm(null); } @@ -379,4 +355,79 @@ protected void addProxyHeaders(HttpServletRequest clientRequest, Request proxyRe proxyRequest.header(ORIGINAL_PRINCIPAL_HEADER, user); } } + + private static class Client extends SslContextFactory.Client { + + private final PulsarSslFactory sslFactory; + + public Client(PulsarSslFactory sslFactory) { + super(); + this.sslFactory = sslFactory; + } + + @Override + public SSLContext getSslContext() { + return this.sslFactory.getInternalSslContext(); + } + } + + protected PulsarSslConfiguration buildSslConfiguration(AuthenticationDataProvider authData) { + return PulsarSslConfiguration.builder() + .tlsProvider(config.getBrokerClientSslProvider()) + .tlsKeyStoreType(config.getBrokerClientTlsKeyStoreType()) + .tlsKeyStorePath(config.getBrokerClientTlsKeyStore()) + .tlsKeyStorePassword(config.getBrokerClientTlsKeyStorePassword()) + .tlsTrustStoreType(config.getBrokerClientTlsTrustStoreType()) + .tlsTrustStorePath(config.getBrokerClientTlsTrustStore()) + .tlsTrustStorePassword(config.getBrokerClientTlsTrustStorePassword()) + .tlsCiphers(config.getBrokerClientTlsCiphers()) + .tlsProtocols(config.getBrokerClientTlsProtocols()) + .tlsTrustCertsFilePath(config.getBrokerClientTrustCertsFilePath()) + .tlsCertificateFilePath(config.getBrokerClientCertificateFilePath()) + .tlsKeyFilePath(config.getBrokerClientKeyFilePath()) + .allowInsecureConnection(config.isTlsAllowInsecureConnection()) + .requireTrustedClientCertOnConnect(false) + .tlsEnabledWithKeystore(config.isBrokerClientTlsEnabledWithKeyStore()) + .tlsCustomParams(config.getBrokerClientSslFactoryPluginParams()) + .authData(authData) + .serverMode(false) + .isHttps(true) + .build(); + } + + protected PulsarSslFactory createPulsarSslFactory() { + try { + try { + AuthenticationDataProvider authData = proxyClientAuthentication.getAuthData(); + PulsarSslConfiguration pulsarSslConfiguration = buildSslConfiguration(authData); + PulsarSslFactory sslFactory = + (PulsarSslFactory) Class.forName(config.getBrokerClientSslFactoryPlugin()) + .getConstructor().newInstance(); + sslFactory.initialize(pulsarSslConfiguration); + sslFactory.createInternalSslContext(); + return sslFactory; + } catch (Exception e) { + LOG.error("Failed to create Pulsar SSLFactory ", e); + throw new PulsarClientException.InvalidConfigurationException(e.getMessage()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + protected void refreshSslContext() { + try { + this.pulsarSslFactory.update(); + } catch (Exception e) { + LOG.error("Failed to refresh SSL context", e); + } + } + + @Override + public void destroy() { + super.destroy(); + if (this.sslContextRefresher != null) { + this.sslContextRefresher.shutdownNow(); + } + } } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java index 4678db82c6e55..407c93074a0fc 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DirectProxyHandler.java @@ -40,15 +40,14 @@ import io.netty.handler.codec.haproxy.HAProxyProtocolVersion; import io.netty.handler.codec.haproxy.HAProxyProxiedProtocol; import io.netty.handler.flush.FlushConsolidationHandler; -import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; -import io.netty.handler.ssl.SslProvider; import io.netty.handler.timeout.ReadTimeoutHandler; import io.netty.util.CharsetUtil; import java.net.InetSocketAddress; import java.util.Arrays; import java.util.concurrent.TimeUnit; import lombok.Getter; +import lombok.SneakyThrows; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; @@ -60,10 +59,9 @@ import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.PulsarDecoder; import org.apache.pulsar.common.stats.Rate; -import org.apache.pulsar.common.util.NettyClientSslContextRefresher; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.common.util.SecurityUtility; -import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder; -import org.apache.pulsar.common.util.keystoretls.NettySSLContextAutoRefreshBuilder; import org.apache.pulsar.common.util.netty.NettyChannelUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,11 +86,10 @@ public class DirectProxyHandler { private final ProxyService service; private final Runnable onHandshakeCompleteAction; private final boolean tlsHostnameVerificationEnabled; - private final boolean tlsEnabledWithKeyStore; final boolean tlsEnabledWithBroker; - private final SslContextAutoRefreshBuilder clientSslCtxRefresher; - private final NettySSLContextAutoRefreshBuilder clientSSLContextAutoRefreshBuilder; + private PulsarSslFactory sslFactory; + @SneakyThrows public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection) { this.service = service; this.authentication = proxyConnection.getClientAuthentication(); @@ -104,7 +101,6 @@ public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection) this.clientAuthMethod = proxyConnection.clientAuthMethod; this.tlsEnabledWithBroker = service.getConfiguration().isTlsEnabledWithBroker(); this.tlsHostnameVerificationEnabled = service.getConfiguration().isTlsHostnameVerificationEnabled(); - this.tlsEnabledWithKeyStore = service.getConfiguration().isTlsEnabledWithKeyStore(); this.onHandshakeCompleteAction = proxyConnection::cancelKeepAliveTask; ProxyConfiguration config = service.getConfiguration(); @@ -118,41 +114,11 @@ public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection) throw new RuntimeException(e); } } - - if (tlsEnabledWithKeyStore) { - clientSSLContextAutoRefreshBuilder = new NettySSLContextAutoRefreshBuilder( - config.getBrokerClientSslProvider(), - config.isTlsAllowInsecureConnection(), - config.getBrokerClientTlsTrustStoreType(), - config.getBrokerClientTlsTrustStore(), - config.getBrokerClientTlsTrustStorePassword(), - config.getBrokerClientTlsKeyStoreType(), - config.getBrokerClientTlsKeyStore(), - config.getBrokerClientTlsKeyStorePassword(), - config.getBrokerClientTlsCiphers(), - config.getBrokerClientTlsProtocols(), - config.getTlsCertRefreshCheckDurationSec(), - authData); - clientSslCtxRefresher = null; - } else { - SslProvider sslProvider = null; - if (config.getBrokerClientSslProvider() != null) { - sslProvider = SslProvider.valueOf(config.getBrokerClientSslProvider()); - } - clientSslCtxRefresher = new NettyClientSslContextRefresher( - sslProvider, - config.isTlsAllowInsecureConnection(), - config.getBrokerClientTrustCertsFilePath(), - authData, - config.getBrokerClientTlsCiphers(), - config.getBrokerClientTlsProtocols(), - config.getTlsCertRefreshCheckDurationSec() - ); - clientSSLContextAutoRefreshBuilder = null; - } - } else { - clientSSLContextAutoRefreshBuilder = null; - clientSslCtxRefresher = null; + PulsarSslConfiguration sslConfiguration = buildSslConfiguration(config, authData); + this.sslFactory = (PulsarSslFactory) Class.forName(config.getSslFactoryPlugin()) + .getConstructor().newInstance(); + this.sslFactory.initialize(sslConfiguration); + this.sslFactory.createInternalSslContext(); } } @@ -193,9 +159,7 @@ protected void initChannel(SocketChannel ch) { if (tlsEnabledWithBroker) { String host = targetBrokerAddress.getHostString(); int port = targetBrokerAddress.getPort(); - SslHandler handler = tlsEnabledWithKeyStore - ? new SslHandler(clientSSLContextAutoRefreshBuilder.get().createSSLEngine(host, port)) - : clientSslCtxRefresher.get().newHandler(ch.alloc(), host, port); + SslHandler handler = new SslHandler(sslFactory.createClientSslEngine(ch.alloc(), host, port)); if (tlsHostnameVerificationEnabled) { SecurityUtility.configureSSLHandler(handler); } @@ -499,5 +463,29 @@ private void writeAndFlush(ByteBuf cmd) { NettyChannelUtil.writeAndFlushWithVoidPromise(outboundChannel, cmd); } + protected PulsarSslConfiguration buildSslConfiguration(ProxyConfiguration config, + AuthenticationDataProvider authData) { + return PulsarSslConfiguration.builder() + .tlsProvider(config.getBrokerClientSslProvider()) + .tlsKeyStoreType(config.getBrokerClientTlsKeyStoreType()) + .tlsKeyStorePath(config.getBrokerClientTlsKeyStore()) + .tlsKeyStorePassword(config.getBrokerClientTlsKeyStorePassword()) + .tlsTrustStoreType(config.getBrokerClientTlsTrustStoreType()) + .tlsTrustStorePath(config.getBrokerClientTlsTrustStore()) + .tlsTrustStorePassword(config.getBrokerClientTlsTrustStorePassword()) + .tlsCiphers(config.getBrokerClientTlsCiphers()) + .tlsProtocols(config.getBrokerClientTlsProtocols()) + .tlsTrustCertsFilePath(config.getBrokerClientTrustCertsFilePath()) + .tlsCertificateFilePath(config.getBrokerClientCertificateFilePath()) + .tlsKeyFilePath(config.getBrokerClientKeyFilePath()) + .allowInsecureConnection(config.isTlsAllowInsecureConnection()) + .requireTrustedClientCertOnConnect(false) + .tlsEnabledWithKeystore(config.isBrokerClientTlsEnabledWithKeyStore()) + .tlsCustomParams(config.getBrokerClientSslFactoryPluginParams()) + .authData(authData) + .serverMode(false) + .build(); + } + private static final Logger log = LoggerFactory.getLogger(DirectProxyHandler.class); } 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 d65408748f432..b9360e403f6f4 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 @@ -42,6 +42,8 @@ import org.apache.pulsar.common.nar.NarClassLoader; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.sasl.SaslConstants; +import org.apache.pulsar.common.util.DefaultPulsarSslFactory; + @Getter @Setter @@ -614,6 +616,16 @@ public class ProxyConfiguration implements PulsarConfiguration { ) private String tlsTrustStorePassword = null; + @FieldContext( + category = CATEGORY_TLS, + doc = "SSL Factory Plugin class to provide SSLEngine and SSLContext objects. The default " + + " class used is DefaultSslFactory.") + private String sslFactoryPlugin = DefaultPulsarSslFactory.class.getName(); + @FieldContext( + category = CATEGORY_TLS, + doc = "SSL Factory plugin configuration parameters.") + private String sslFactoryPluginParams = ""; + /** * KeyStore TLS config variables used for proxy to auth with broker. */ @@ -683,6 +695,16 @@ public class ProxyConfiguration implements PulsarConfiguration { ) private Set brokerClientTlsProtocols = new TreeSet<>(); + @FieldContext( + category = CATEGORY_TLS, + doc = "SSL Factory Plugin class used by internal client to provide SSLEngine and SSLContext objects. " + + "The default class used is DefaultSslFactory.") + private String brokerClientSslFactoryPlugin = DefaultPulsarSslFactory.class.getName(); + @FieldContext( + category = CATEGORY_TLS, + doc = "SSL Factory plugin configuration parameters used by internal client.") + private String brokerClientSslFactoryPluginParams = ""; + // HTTP @FieldContext( diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java index d58fe46e0063a..f8b5d0844509e 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java @@ -398,7 +398,7 @@ private synchronized void completeConnect() throws PulsarClientException { if (this.connectionPool == null) { this.connectionPool = new ConnectionPool(InstrumentProvider.NOOP, clientConf, service.getWorkerGroup(), clientCnxSupplier, - Optional.of(dnsAddressResolverGroup.getResolver(service.getWorkerGroup().next()))); + Optional.of(dnsAddressResolverGroup.getResolver(service.getWorkerGroup().next())), null); } else { LOG.error("BUG! Connection Pool has already been created for proxy connection to {} state {} role {}", remoteAddress, state, clientAuthRole); 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 5cf01d6668b9b..4ee15fd7124a6 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 @@ -119,6 +119,7 @@ public class ProxyService implements Closeable { protected boolean proxyZeroCopyModeEnabled; private final ScheduledExecutorService statsExecutor; + private ScheduledExecutorService sslContextRefresher; static final Gauge ACTIVE_CONNECTIONS = Gauge .build("pulsar_proxy_active_connections", "Number of connections currently active in the proxy").create() @@ -245,7 +246,7 @@ public void start() throws Exception { proxyZeroCopyModeEnabled = true; } - bootstrap.childHandler(new ServiceChannelInitializer(this, proxyConfig, false)); + bootstrap.childHandler(new ServiceChannelInitializer(this, proxyConfig, false, null)); // Bind and start to accept incoming connections. if (proxyConfig.getServicePort().isPresent()) { try { @@ -258,8 +259,12 @@ public void start() throws Exception { } if (proxyConfig.getServicePortTls().isPresent()) { + this.sslContextRefresher = Executors + .newSingleThreadScheduledExecutor( + new DefaultThreadFactory("proxy-ssl-context-refresher")); ServerBootstrap tlsBootstrap = bootstrap.clone(); - tlsBootstrap.childHandler(new ServiceChannelInitializer(this, proxyConfig, true)); + tlsBootstrap.childHandler(new ServiceChannelInitializer(this, proxyConfig, true, + sslContextRefresher)); listenChannelTls = tlsBootstrap.bind(proxyConfig.getBindAddress(), proxyConfig.getServicePortTls().get()).sync().channel(); LOG.info("Started Pulsar TLS Proxy on {}", listenChannelTls.localAddress()); @@ -389,6 +394,10 @@ public void close() throws IOException { discoveryProvider.close(); } + if (this.sslContextRefresher != null) { + this.sslContextRefresher.shutdownNow(); + } + if (statsExecutor != null) { statsExecutor.shutdownNow(); } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java index 19f4002ad52ce..728d27c815ff3 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ServiceChannelInitializer.java @@ -22,22 +22,23 @@ import io.netty.channel.socket.SocketChannel; import io.netty.handler.codec.LengthFieldBasedFrameDecoder; import io.netty.handler.flush.FlushConsolidationHandler; -import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; -import io.netty.handler.ssl.SslProvider; import io.netty.handler.timeout.ReadTimeoutHandler; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.OptionalProxyProtocolDecoder; -import org.apache.pulsar.common.util.NettyServerSslContextBuilder; -import org.apache.pulsar.common.util.SslContextAutoRefreshBuilder; -import org.apache.pulsar.common.util.keystoretls.NettySSLContextAutoRefreshBuilder; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Initialize service channel handlers. * */ public class ServiceChannelInitializer extends ChannelInitializer { + private static final Logger log = LoggerFactory.getLogger(ServiceChannelInitializer.class); public static final String TLS_HANDLER = "tls"; private final ProxyService proxyService; @@ -46,10 +47,10 @@ public class ServiceChannelInitializer extends ChannelInitializer private final int brokerProxyReadTimeoutMs; private final int maxMessageSize; - private SslContextAutoRefreshBuilder serverSslCtxRefresher; - private NettySSLContextAutoRefreshBuilder serverSSLContextAutoRefreshBuilder; + private PulsarSslFactory sslFactory; - public ServiceChannelInitializer(ProxyService proxyService, ProxyConfiguration serviceConfig, boolean enableTls) + public ServiceChannelInitializer(ProxyService proxyService, ProxyConfiguration serviceConfig, + boolean enableTls, ScheduledExecutorService sslContextRefresher) throws Exception { super(); this.proxyService = proxyService; @@ -59,36 +60,16 @@ public ServiceChannelInitializer(ProxyService proxyService, ProxyConfiguration s this.maxMessageSize = serviceConfig.getMaxMessageSize(); if (enableTls) { - if (tlsEnabledWithKeyStore) { - serverSSLContextAutoRefreshBuilder = new NettySSLContextAutoRefreshBuilder( - serviceConfig.getTlsProvider(), - serviceConfig.getTlsKeyStoreType(), - serviceConfig.getTlsKeyStore(), - serviceConfig.getTlsKeyStorePassword(), - serviceConfig.isTlsAllowInsecureConnection(), - serviceConfig.getTlsTrustStoreType(), - serviceConfig.getTlsTrustStore(), - serviceConfig.getTlsTrustStorePassword(), - serviceConfig.isTlsRequireTrustedClientCertOnConnect(), - serviceConfig.getTlsCiphers(), - serviceConfig.getTlsProtocols(), - serviceConfig.getTlsCertRefreshCheckDurationSec()); - } else { - SslProvider sslProvider = null; - if (serviceConfig.getTlsProvider() != null) { - sslProvider = SslProvider.valueOf(serviceConfig.getTlsProvider()); - } - serverSslCtxRefresher = new NettyServerSslContextBuilder( - sslProvider, - serviceConfig.isTlsAllowInsecureConnection(), - serviceConfig.getTlsTrustCertsFilePath(), serviceConfig.getTlsCertificateFilePath(), - serviceConfig.getTlsKeyFilePath(), serviceConfig.getTlsCiphers(), - serviceConfig.getTlsProtocols(), - serviceConfig.isTlsRequireTrustedClientCertOnConnect(), - serviceConfig.getTlsCertRefreshCheckDurationSec()); + PulsarSslConfiguration sslConfiguration = buildSslConfiguration(serviceConfig); + this.sslFactory = (PulsarSslFactory) Class.forName(serviceConfig.getSslFactoryPlugin()) + .getConstructor().newInstance(); + this.sslFactory.initialize(sslConfiguration); + this.sslFactory.createInternalSslContext(); + if (serviceConfig.getTlsCertRefreshCheckDurationSec() > 0) { + sslContextRefresher.scheduleWithFixedDelay(this::refreshSslContext, + serviceConfig.getTlsCertRefreshCheckDurationSec(), + serviceConfig.getTlsCertRefreshCheckDurationSec(), TimeUnit.SECONDS); } - } else { - this.serverSslCtxRefresher = null; } } @@ -96,14 +77,8 @@ public ServiceChannelInitializer(ProxyService proxyService, ProxyConfiguration s protected void initChannel(SocketChannel ch) throws Exception { ch.pipeline().addLast("consolidation", new FlushConsolidationHandler(1024, true)); - if (serverSslCtxRefresher != null && this.enableTls) { - SslContext sslContext = serverSslCtxRefresher.get(); - if (sslContext != null) { - ch.pipeline().addLast(TLS_HANDLER, sslContext.newHandler(ch.alloc())); - } - } else if (this.tlsEnabledWithKeyStore && serverSSLContextAutoRefreshBuilder != null) { - ch.pipeline().addLast(TLS_HANDLER, - new SslHandler(serverSSLContextAutoRefreshBuilder.get().createSSLEngine())); + if (this.enableTls) { + ch.pipeline().addLast(TLS_HANDLER, new SslHandler(this.sslFactory.createServerSslEngine(ch.alloc()))); } if (brokerProxyReadTimeoutMs > 0) { ch.pipeline().addLast("readTimeoutHandler", @@ -117,4 +92,35 @@ protected void initChannel(SocketChannel ch) throws Exception { ch.pipeline().addLast("handler", new ProxyConnection(proxyService, proxyService.getDnsAddressResolverGroup())); } + + protected PulsarSslConfiguration buildSslConfiguration(ProxyConfiguration config) { + return PulsarSslConfiguration.builder() + .tlsProvider(config.getTlsProvider()) + .tlsKeyStoreType(config.getTlsKeyStoreType()) + .tlsKeyStorePath(config.getTlsKeyStore()) + .tlsKeyStorePassword(config.getTlsKeyStorePassword()) + .tlsTrustStoreType(config.getTlsTrustStoreType()) + .tlsTrustStorePath(config.getTlsTrustStore()) + .tlsTrustStorePassword(config.getTlsTrustStorePassword()) + .tlsCiphers(config.getTlsCiphers()) + .tlsProtocols(config.getTlsProtocols()) + .tlsTrustCertsFilePath(config.getTlsTrustCertsFilePath()) + .tlsCertificateFilePath(config.getTlsCertificateFilePath()) + .tlsKeyFilePath(config.getTlsKeyFilePath()) + .allowInsecureConnection(config.isTlsAllowInsecureConnection()) + .requireTrustedClientCertOnConnect(config.isTlsRequireTrustedClientCertOnConnect()) + .tlsEnabledWithKeystore(config.isTlsEnabledWithKeyStore()) + .tlsCustomParams(config.getSslFactoryPluginParams()) + .authData(null) + .serverMode(true) + .build(); + } + + protected void refreshSslContext() { + try { + this.sslFactory.update(); + } catch (Exception e) { + log.error("Failed to refresh SSL context", e); + } + } } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java index ad94f1b65a092..3c472135bdfb0 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java @@ -29,6 +29,9 @@ import java.util.EnumSet; import java.util.List; import java.util.Optional; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import javax.servlet.DispatcherType; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.authentication.AuthenticationService; @@ -37,6 +40,9 @@ import org.apache.pulsar.broker.web.JsonMapperProvider; import org.apache.pulsar.broker.web.RateLimitingFilter; import org.apache.pulsar.broker.web.WebExecutorThreadPool; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.jetty.tls.JettySslContextFactory; import org.apache.pulsar.proxy.stats.PulsarProxyOpenTelemetry; import org.eclipse.jetty.server.ConnectionFactory; @@ -85,6 +91,9 @@ public class WebServer { private ServerConnector connector; private ServerConnector connectorTls; + private ScheduledExecutorService sslRefreshScheduledExecutor; + private PulsarSslFactory sslFactory; + private final FilterInitializer filterInitializer; public WebServer(ProxyConfiguration config, AuthenticationService authenticationService) { @@ -121,34 +130,22 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication } if (config.getWebServicePortTls().isPresent()) { try { - SslContextFactory sslCtxFactory; - if (config.isTlsEnabledWithKeyStore()) { - sslCtxFactory = JettySslContextFactory.createServerSslContextWithKeystore( - config.getWebServiceTlsProvider(), - config.getTlsKeyStoreType(), - config.getTlsKeyStore(), - config.getTlsKeyStorePassword(), - config.isTlsAllowInsecureConnection(), - config.getTlsTrustStoreType(), - config.getTlsTrustStore(), - config.getTlsTrustStorePassword(), - config.isTlsRequireTrustedClientCertOnConnect(), - config.getWebServiceTlsCiphers(), - config.getWebServiceTlsProtocols(), - config.getTlsCertRefreshCheckDurationSec() - ); - } else { - sslCtxFactory = JettySslContextFactory.createServerSslContext( - config.getWebServiceTlsProvider(), - config.isTlsAllowInsecureConnection(), - config.getTlsTrustCertsFilePath(), - config.getTlsCertificateFilePath(), - config.getTlsKeyFilePath(), - config.isTlsRequireTrustedClientCertOnConnect(), - config.getWebServiceTlsCiphers(), - config.getWebServiceTlsProtocols(), - config.getTlsCertRefreshCheckDurationSec()); + this.sslRefreshScheduledExecutor = Executors.newSingleThreadScheduledExecutor( + new ExecutorProvider.ExtendedThreadFactory("pulsar-proxy-web-server-tls-refresh")); + PulsarSslConfiguration sslConfiguration = buildSslConfiguration(config); + this.sslFactory = (PulsarSslFactory) Class.forName(config.getSslFactoryPlugin()) + .getConstructor().newInstance(); + this.sslFactory.initialize(sslConfiguration); + this.sslFactory.createInternalSslContext(); + if (config.getTlsCertRefreshCheckDurationSec() > 0) { + sslRefreshScheduledExecutor.scheduleWithFixedDelay(this::refreshSslContext, + config.getTlsCertRefreshCheckDurationSec(), + config.getTlsCertRefreshCheckDurationSec(), TimeUnit.SECONDS); } + SslContextFactory sslCtxFactory = + JettySslContextFactory.createSslContextFactory(config.getTlsProvider(), + sslFactory, config.isTlsRequireTrustedClientCertOnConnect(), + config.getWebServiceTlsCiphers(), config.getWebServiceTlsProtocols()); List connectionFactories = new ArrayList<>(); if (config.isWebServiceHaProxyProtocolEnabled()) { connectionFactories.add(new ProxyConnectionFactory()); @@ -363,6 +360,9 @@ public void start() throws Exception { } public void stop() throws Exception { + if (this.sslRefreshScheduledExecutor != null) { + this.sslRefreshScheduledExecutor.shutdownNow(); + } server.stop(); webServiceExecutor.stop(); log.info("Server stopped successfully"); @@ -388,5 +388,37 @@ public Optional getListenPortHTTPS() { } } + protected PulsarSslConfiguration buildSslConfiguration(ProxyConfiguration config) { + return PulsarSslConfiguration.builder() + .tlsProvider(config.getTlsProvider()) + .tlsKeyStoreType(config.getTlsKeyStoreType()) + .tlsKeyStorePath(config.getTlsKeyStore()) + .tlsKeyStorePassword(config.getTlsKeyStorePassword()) + .tlsTrustStoreType(config.getTlsTrustStoreType()) + .tlsTrustStorePath(config.getTlsTrustStore()) + .tlsTrustStorePassword(config.getTlsTrustStorePassword()) + .tlsCiphers(config.getTlsCiphers()) + .tlsProtocols(config.getTlsProtocols()) + .tlsTrustCertsFilePath(config.getTlsTrustCertsFilePath()) + .tlsCertificateFilePath(config.getTlsCertificateFilePath()) + .tlsKeyFilePath(config.getTlsKeyFilePath()) + .allowInsecureConnection(config.isTlsAllowInsecureConnection()) + .requireTrustedClientCertOnConnect(config.isTlsRequireTrustedClientCertOnConnect()) + .tlsEnabledWithKeystore(config.isTlsEnabledWithKeyStore()) + .tlsCustomParams(config.getSslFactoryPluginParams()) + .authData(null) + .serverMode(true) + .isHttps(true) + .build(); + } + + protected void refreshSslContext() { + try { + this.sslFactory.update(); + } catch (Exception e) { + log.error("Failed to refresh SSL context", e); + } + } + private static final Logger log = LoggerFactory.getLogger(WebServer.class); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java index 4f925618e8a79..fdf9242c9f3d8 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java @@ -43,7 +43,7 @@ public class AdminProxyHandlerTest { private AdminProxyHandler adminProxyHandler; @BeforeClass - public void setupMocks() throws ServletException { + public void setupMocks() throws Exception { // given HttpClient httpClient = mock(HttpClient.class); adminProxyHandler = new AdminProxyHandler(mock(ProxyConfiguration.class), diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java index 583ab7000e54f..ee0f8010b7d79 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java @@ -268,7 +268,7 @@ protected void handleActiveConsumerChange(CommandActiveConsumerChange change) { throw new UnsupportedOperationException(); } }; - }); + }, null); return new PulsarClientImpl(conf, eventLoopGroup, cnxPool); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java index 770424d93747c..1148234be624c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java @@ -60,6 +60,8 @@ protected void setup() throws Exception { serviceStarter.getConfig().setBrokerServiceURLTLS(pulsar.getBrokerServiceUrlTls()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); serviceStarter.getConfig().setBrokerClientTrustCertsFilePath(CA_CERT_FILE_PATH); + serviceStarter.getConfig().setBrokerClientCertificateFilePath(BROKER_CERT_FILE_PATH); + serviceStarter.getConfig().setBrokerClientKeyFilePath(BROKER_KEY_FILE_PATH); serviceStarter.getConfig().setServicePort(Optional.empty()); serviceStarter.getConfig().setServicePortTls(Optional.of(0)); serviceStarter.getConfig().setWebServicePort(Optional.of(0)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java index e101eb4ff7a2b..4c0cd49d74fa7 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java @@ -392,7 +392,7 @@ protected void handleActiveConsumerChange(CommandActiveConsumerChange change) { throw new UnsupportedOperationException(); } }; - }); + }, null); registerCloseable(cnxPool); return new PulsarClientImpl(conf, eventLoopGroup, cnxPool); diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java index b6b3d805edc75..6bf73e705d16c 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerfClientUtils.java @@ -85,6 +85,11 @@ public static ClientBuilder createClientBuilderFromArguments(PerformanceBaseArgu clientBuilder.authentication(arguments.authPluginClassName, arguments.authParams); } + if (isNotBlank(arguments.sslfactoryPlugin)) { + clientBuilder.sslFactoryPlugin(arguments.sslfactoryPlugin) + .sslFactoryPluginParams(arguments.sslFactoryPluginParams); + } + if (arguments.tlsAllowInsecureConnection != null) { clientBuilder.allowTlsInsecureConnection(arguments.tlsAllowInsecureConnection); } @@ -111,6 +116,11 @@ public static PulsarAdminBuilder createAdminBuilderFromArguments(PerformanceBase pulsarAdminBuilder.authentication(arguments.authPluginClassName, arguments.authParams); } + if (isNotBlank(arguments.sslfactoryPlugin)) { + pulsarAdminBuilder.sslFactoryPlugin(arguments.sslfactoryPlugin) + .sslFactoryPluginParams(arguments.sslFactoryPluginParams); + } + if (arguments.tlsAllowInsecureConnection != null) { pulsarAdminBuilder.allowTlsInsecureConnection(arguments.tlsAllowInsecureConnection); } diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceBaseArguments.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceBaseArguments.java index 3c4b831332281..ee79066c32f90 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceBaseArguments.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceBaseArguments.java @@ -44,6 +44,15 @@ public abstract class PerformanceBaseArguments extends CmdBase{ + "or \"{\"key1\":\"val1\",\"key2\":\"val2\"}\".", descriptionKey = "authParams") public String authParams; + @Option(names = { "--ssl-factory-plugin" }, description = "Pulsar SSL Factory plugin class name", + descriptionKey = "sslFactoryPlugin") + public String sslfactoryPlugin; + + @Option(names = { "--ssl-factory-plugin-params" }, + description = "Pulsar SSL Factory Plugin parameters in the format: " + + "\"{\"key1\":\"val1\",\"key2\":\"val2\"}\".", descriptionKey = "sslFactoryPluginParams") + public String sslFactoryPluginParams; + @Option(names = { "--trust-cert-file" }, description = "Path for the trusted TLS certificate file", descriptionKey = "tlsTrustCertsFilePath") diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java index bbb34a3e3f73d..e7523252bd960 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java @@ -24,6 +24,9 @@ import java.util.EnumSet; import java.util.List; import java.util.Optional; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import javax.servlet.DispatcherType; import javax.servlet.Servlet; @@ -34,6 +37,10 @@ import org.apache.pulsar.broker.web.JsonMapperProvider; import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.util.DefaultPulsarSslFactory; +import org.apache.pulsar.common.util.PulsarSslConfiguration; +import org.apache.pulsar.common.util.PulsarSslFactory; import org.apache.pulsar.jetty.tls.JettySslContextFactory; import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; @@ -70,6 +77,8 @@ public class ProxyServer { private ServerConnector connector; private ServerConnector connectorTls; + private PulsarSslFactory sslFactory; + private ScheduledExecutorService scheduledExecutorService; public ProxyServer(WebSocketProxyConfiguration config) throws PulsarClientException, MalformedURLException, PulsarServerException { @@ -102,34 +111,23 @@ public ProxyServer(WebSocketProxyConfiguration config) // TLS enabled connector if (config.getWebServicePortTls().isPresent()) { try { - SslContextFactory sslCtxFactory; - if (config.isTlsEnabledWithKeyStore()) { - sslCtxFactory = JettySslContextFactory.createServerSslContextWithKeystore( - config.getTlsProvider(), - config.getTlsKeyStoreType(), - config.getTlsKeyStore(), - config.getTlsKeyStorePassword(), - config.isTlsAllowInsecureConnection(), - config.getTlsTrustStoreType(), - config.getTlsTrustStore(), - config.getTlsTrustStorePassword(), - config.isTlsRequireTrustedClientCertOnConnect(), - config.getWebServiceTlsCiphers(), - config.getWebServiceTlsProtocols(), - config.getTlsCertRefreshCheckDurationSec() - ); - } else { - sslCtxFactory = JettySslContextFactory.createServerSslContext( - config.getTlsProvider(), - config.isTlsAllowInsecureConnection(), - config.getTlsTrustCertsFilePath(), - config.getTlsCertificateFilePath(), - config.getTlsKeyFilePath(), - config.isTlsRequireTrustedClientCertOnConnect(), - config.getWebServiceTlsCiphers(), - config.getWebServiceTlsProtocols(), - config.getTlsCertRefreshCheckDurationSec()); + PulsarSslConfiguration sslConfiguration = buildSslConfiguration(config); + this.sslFactory = new DefaultPulsarSslFactory(); + this.sslFactory.initialize(sslConfiguration); + this.sslFactory.createInternalSslContext(); + this.scheduledExecutorService = Executors + .newSingleThreadScheduledExecutor(new ExecutorProvider + .ExtendedThreadFactory("proxy-websocket-ssl-refresh")); + if (config.getTlsCertRefreshCheckDurationSec() > 0) { + this.scheduledExecutorService.scheduleWithFixedDelay(this::refreshSslContext, + config.getTlsCertRefreshCheckDurationSec(), + config.getTlsCertRefreshCheckDurationSec(), + TimeUnit.SECONDS); } + SslContextFactory sslCtxFactory = + JettySslContextFactory.createSslContextFactory(config.getTlsProvider(), + sslFactory, config.isTlsRequireTrustedClientCertOnConnect(), + config.getWebServiceTlsCiphers(), config.getWebServiceTlsProtocols()); List connectionFactories = new ArrayList<>(); if (config.isWebServiceHaProxyProtocolEnabled()) { connectionFactories.add(new ProxyConnectionFactory()); @@ -223,6 +221,9 @@ public void start() throws PulsarServerException { public void stop() throws Exception { server.stop(); executorService.stop(); + if (scheduledExecutorService != null) { + scheduledExecutorService.shutdownNow(); + } } public Optional getListenPortHTTP() { @@ -241,5 +242,34 @@ public Optional getListenPortHTTPS() { } } + protected PulsarSslConfiguration buildSslConfiguration(WebSocketProxyConfiguration config) { + return PulsarSslConfiguration.builder() + .tlsKeyStoreType(config.getTlsKeyStoreType()) + .tlsKeyStorePath(config.getTlsKeyStore()) + .tlsKeyStorePassword(config.getTlsKeyStorePassword()) + .tlsTrustStoreType(config.getTlsTrustStoreType()) + .tlsTrustStorePath(config.getTlsTrustStore()) + .tlsTrustStorePassword(config.getTlsTrustStorePassword()) + .tlsCiphers(config.getWebServiceTlsCiphers()) + .tlsProtocols(config.getWebServiceTlsProtocols()) + .tlsTrustCertsFilePath(config.getTlsTrustCertsFilePath()) + .tlsCertificateFilePath(config.getTlsCertificateFilePath()) + .tlsKeyFilePath(config.getTlsKeyFilePath()) + .allowInsecureConnection(config.isTlsAllowInsecureConnection()) + .requireTrustedClientCertOnConnect(config.isTlsRequireTrustedClientCertOnConnect()) + .tlsEnabledWithKeystore(config.isTlsEnabledWithKeyStore()) + .serverMode(true) + .isHttps(true) + .build(); + } + + protected void refreshSslContext() { + try { + this.sslFactory.update(); + } catch (Exception e) { + log.error("Failed to refresh SSL context", e); + } + } + private static final Logger log = LoggerFactory.getLogger(ProxyServer.class); } From acc463f4b2f0648f5e3c9c146124a9223ec909f7 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Tue, 20 Aug 2024 18:31:35 +0800 Subject: [PATCH 457/580] [cleanup][broker] delete ConcurrentSortedLongPairSet. (#23202) --- .../ConcurrentSortedLongPairSet.java | 215 ------------- .../ConcurrentSortedLongPairSetTest.java | 291 ------------------ 2 files changed, 506 deletions(-) delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentSortedLongPairSet.java delete mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentSortedLongPairSetTest.java diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentSortedLongPairSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentSortedLongPairSet.java deleted file mode 100644 index 0718a4f81a61f..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentSortedLongPairSet.java +++ /dev/null @@ -1,215 +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.collections; - -import java.util.NavigableMap; -import java.util.NavigableSet; -import java.util.Set; -import java.util.TreeSet; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.commons.lang.mutable.MutableInt; -import org.apache.commons.lang.mutable.MutableLong; -import org.apache.pulsar.common.util.collections.ConcurrentLongPairSet.LongPair; -import org.apache.pulsar.common.util.collections.ConcurrentLongPairSet.LongPairConsumer; - -/** - * Sorted concurrent {@link LongPairSet} which is not fully accurate in sorting. - * - * {@link ConcurrentSortedLongPairSet} creates separate {@link ConcurrentLongPairSet} for unique first-key of - * inserted item. So, it can iterate over all items by sorting on item's first key. However, item's second key will not - * be sorted. eg: - * - *
- *  insert: (1,2), (1,4), (2,1), (1,5), (2,6)
- *  while iterating set will first read all the entries for items whose first-key=1 and then first-key=2.
- *  output: (1,4), (1,5), (1,2), (2,6), (2,1)
- * 
- * - *

This map can be expensive and not recommended if set has to store large number of unique item.first's key - * because set has to create that many {@link ConcurrentLongPairSet} objects. - */ -public class ConcurrentSortedLongPairSet implements LongPairSet { - - protected final NavigableMap longPairSets = new ConcurrentSkipListMap<>(); - private final int expectedItems; - private final int concurrencyLevel; - /** - * If {@link #longPairSets} adds and removes the item-set frequently then it allocates and removes - * {@link ConcurrentLongPairSet} for the same item multiple times which can lead to gc-puases. To avoid such - * situation, avoid removing empty LogPairSet until it reaches max limit. - */ - private final int maxAllowedSetOnRemove; - private final boolean autoShrink; - private static final int DEFAULT_MAX_ALLOWED_SET_ON_REMOVE = 10; - - public ConcurrentSortedLongPairSet() { - this(16, 1, DEFAULT_MAX_ALLOWED_SET_ON_REMOVE); - } - - public ConcurrentSortedLongPairSet(int expectedItems) { - this(expectedItems, 1, DEFAULT_MAX_ALLOWED_SET_ON_REMOVE); - } - - public ConcurrentSortedLongPairSet(int expectedItems, int concurrencyLevel) { - this(expectedItems, concurrencyLevel, DEFAULT_MAX_ALLOWED_SET_ON_REMOVE); - } - - public ConcurrentSortedLongPairSet(int expectedItems, int concurrencyLevel, boolean autoShrink) { - this(expectedItems, concurrencyLevel, DEFAULT_MAX_ALLOWED_SET_ON_REMOVE, autoShrink); - } - - public ConcurrentSortedLongPairSet(int expectedItems, int concurrencyLevel, int maxAllowedSetOnRemove) { - this(expectedItems, concurrencyLevel, maxAllowedSetOnRemove, false); - } - - public ConcurrentSortedLongPairSet(int expectedItems, int concurrencyLevel, int maxAllowedSetOnRemove, - boolean autoShrink) { - this.expectedItems = expectedItems; - this.concurrencyLevel = concurrencyLevel; - this.maxAllowedSetOnRemove = maxAllowedSetOnRemove; - this.autoShrink = autoShrink; - } - - @Override - public boolean add(long item1, long item2) { - ConcurrentLongPairSet messagesToReplay = longPairSets.computeIfAbsent(item1, - (key) -> ConcurrentLongPairSet.newBuilder() - .expectedItems(expectedItems) - .concurrencyLevel(concurrencyLevel) - .autoShrink(autoShrink) - .build()); - return messagesToReplay.add(item1, item2); - } - - @Override - public boolean remove(long item1, long item2) { - ConcurrentLongPairSet messagesToReplay = longPairSets.get(item1); - if (messagesToReplay != null) { - boolean removed = messagesToReplay.remove(item1, item2); - if (messagesToReplay.isEmpty() && longPairSets.size() > maxAllowedSetOnRemove) { - longPairSets.remove(item1, messagesToReplay); - } - return removed; - } - return false; - } - - @Override - public int removeIf(LongPairPredicate filter) { - MutableInt removedValues = new MutableInt(0); - longPairSets.forEach((item1, longPairSet) -> { - removedValues.add(longPairSet.removeIf(filter)); - if (longPairSet.isEmpty() && longPairSets.size() > maxAllowedSetOnRemove) { - longPairSets.remove(item1, longPairSet); - } - }); - return removedValues.intValue(); - } - - @Override - public Set items() { - return items((int) this.size()); - } - - @Override - public void forEach(LongPairConsumer processor) { - longPairSets.forEach((__, longPairSet) -> longPairSet.forEach(processor)); - } - - @Override - public Set items(int numberOfItems) { - return items(numberOfItems, (item1, item2) -> new LongPair(item1, item2)); - } - - @Override - public Set items(int numberOfItems, LongPairFunction longPairConverter) { - NavigableSet items = new TreeSet<>(); - forEach((i1, i2) -> { - items.add(longPairConverter.apply(i1, i2)); - if (items.size() > numberOfItems) { - items.pollLast(); - } - }); - return items; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append('{'); - final AtomicBoolean first = new AtomicBoolean(true); - longPairSets.forEach((key, longPairSet) -> { - longPairSet.forEach((item1, item2) -> { - if (!first.getAndSet(false)) { - sb.append(", "); - } - sb.append('['); - sb.append(item1); - sb.append(':'); - sb.append(item2); - sb.append(']'); - }); - }); - sb.append('}'); - return sb.toString(); - } - - @Override - public boolean isEmpty() { - if (longPairSets.isEmpty()) { - return true; - } - for (ConcurrentLongPairSet subSet : longPairSets.values()) { - if (!subSet.isEmpty()) { - return false; - } - } - return true; - } - - @Override - public void clear() { - longPairSets.clear(); - } - - @Override - public long size() { - MutableLong size = new MutableLong(0); - longPairSets.forEach((__, longPairSet) -> size.add(longPairSet.size())); - return size.longValue(); - } - - @Override - public long capacity() { - MutableLong capacity = new MutableLong(0); - longPairSets.forEach((__, longPairSet) -> capacity.add(longPairSet.capacity())); - return capacity.longValue(); - } - - @Override - public boolean contains(long item1, long item2) { - ConcurrentLongPairSet longPairSet = longPairSets.get(item1); - if (longPairSet != null) { - return longPairSet.contains(item1, item2); - } - return false; - } - -} \ No newline at end of file diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentSortedLongPairSetTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentSortedLongPairSetTest.java deleted file mode 100644 index eff49883215d7..0000000000000 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentSortedLongPairSetTest.java +++ /dev/null @@ -1,291 +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.collections; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNotEquals; -import static org.testng.Assert.assertTrue; -import com.google.common.collect.ComparisonChain; -import com.google.common.collect.Lists; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import lombok.Cleanup; -import org.apache.pulsar.common.util.collections.ConcurrentLongPairSet.LongPair; -import org.testng.annotations.Test; - -public class ConcurrentSortedLongPairSetTest { - - @Test - public void simpleInsertions() { - LongPairSet set = new ConcurrentSortedLongPairSet(16); - - assertTrue(set.isEmpty()); - assertTrue(set.add(1, 1)); - assertFalse(set.isEmpty()); - - assertTrue(set.add(2, 2)); - assertTrue(set.add(3, 3)); - - assertEquals(set.size(), 3); - - assertTrue(set.contains(1, 1)); - assertEquals(set.size(), 3); - - assertTrue(set.remove(1, 1)); - assertEquals(set.size(), 2); - assertFalse(set.contains(1, 1)); - assertFalse(set.contains(5, 5)); - assertEquals(set.size(), 2); - - assertTrue(set.add(1, 1)); - assertEquals(set.size(), 3); - assertFalse(set.add(1, 1)); - assertEquals(set.size(), 3); - } - - @Test - public void testRemove() { - LongPairSet set = new ConcurrentSortedLongPairSet(16); - - assertTrue(set.isEmpty()); - assertTrue(set.add(1, 1)); - assertFalse(set.isEmpty()); - - assertFalse(set.remove(1, 0)); - assertFalse(set.isEmpty()); - assertTrue(set.remove(1, 1)); - assertTrue(set.isEmpty()); - } - - @Test - public void concurrentInsertions() throws Throwable { - LongPairSet set = new ConcurrentSortedLongPairSet(16); - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - - final int nThreads = 8; - final int N = 1000; - - List> futures = new ArrayList<>(); - for (int i = 0; i < nThreads; i++) { - final int threadIdx = i; - - futures.add(executor.submit(() -> { - Random random = new Random(); - - for (int j = 0; j < N; j++) { - long key = random.nextLong(); - // Ensure keys are unique - key -= key % (threadIdx + 1); - key = Math.abs(key); - set.add(key, key); - } - })); - } - - for (Future future : futures) { - future.get(); - } - - assertEquals(set.size(), N * nThreads); - } - - @Test - public void testIteration() { - LongPairSet set = new ConcurrentSortedLongPairSet(16); - - for (int i = 0; i < 10; i++) { - for (int j = 0; j < 10; j++) { - set.add(i, j); - } - } - - for (int i = 0; i < 10; i++) { - final int firstKey = i; - Set longSetResult = set.items(10); - assertEquals(longSetResult.size(), 10); - longSetResult.forEach(longPair -> { - assertEquals(firstKey, longPair.first); - }); - set.removeIf((item1, item2) -> item1 == firstKey); - } - - } - - @Test - public void testRemoval() { - LongPairSet set = new ConcurrentSortedLongPairSet(16); - - set.add(0, 0); - set.add(1, 1); - set.add(3, 3); - set.add(6, 6); - set.add(7, 7); - - List values = new ArrayList<>(set.items()); - values.sort(null); - assertEquals(values, Lists.newArrayList(new LongPair(0, 0), new LongPair(1, 1), new LongPair(3, 3), - new LongPair(6, 6), new LongPair(7, 7))); - - set.forEach((first, second) -> { - if (first < 5) { - set.remove(first, second); - } - }); - assertEquals(set.size(), values.size() - 3); - values = new ArrayList<>(set.items()); - values.sort(null); - assertEquals(values, Lists.newArrayList(new LongPair(6, 6), new LongPair(7, 7))); - } - - @Test - public void testIfRemoval() { - LongPairSet set = new ConcurrentSortedLongPairSet(16, 1, 1); - - set.add(0, 0); - set.add(1, 1); - set.add(3, 3); - set.add(6, 6); - set.add(7, 7); - - List values = new ArrayList<>(set.items()); - values.sort(null); - assertEquals(values, Lists.newArrayList(new LongPair(0, 0), new LongPair(1, 1), new LongPair(3, 3), - new LongPair(6, 6), new LongPair(7, 7))); - - int removeItems = set.removeIf((first, second) -> first < 5); - - assertEquals(3, removeItems); - assertEquals(set.size(), values.size() - 3); - values = new ArrayList<>(set.items()); - values.sort(null); - assertEquals(values, Lists.newArrayList(new LongPair(6, 6), new LongPair(7, 7))); - - set = new ConcurrentSortedLongPairSet(128, 2, true); - set.add(2, 2); - set.add(1, 3); - set.add(3, 1); - set.add(2, 1); - set.add(3, 2); - set.add(1, 2); - set.add(1, 1); - removeItems = set.removeIf((ledgerId, entryId) -> { - return ComparisonChain.start().compare(ledgerId, 1).compare(entryId, 3) - .result() <= 0; - }); - assertEquals(removeItems, 3); - } - - @Test - public void testItems() { - LongPairSet set = new ConcurrentSortedLongPairSet(16); - - int n = 100; - int limit = 10; - for (int i = 0; i < n; i++) { - set.add(i, i); - } - - Set items = set.items(); - Set limitItems = set.items(limit); - assertEquals(items.size(), n); - assertEquals(limitItems.size(), limit); - - int totalRemovedItems = set.removeIf((first, second) -> limitItems.contains((new LongPair(first, second)))); - assertEquals(limitItems.size(), totalRemovedItems); - assertEquals(set.size(), n - limit); - } - - @Test - public void testEqualsObjects() { - - LongPairSet set = new ConcurrentSortedLongPairSet(16); - - long t1 = 1; - long t2 = 2; - long t1_b = 1; - assertEquals(t1, t1_b); - assertNotEquals(t2, t1); - assertNotEquals(t2, t1_b); - - set.add(t1, t1); - assertTrue(set.contains(t1, t1)); - assertTrue(set.contains(t1_b, t1_b)); - assertFalse(set.contains(t2, t2)); - - assertTrue(set.remove(t1_b, t1_b)); - assertFalse(set.contains(t1, t1)); - assertFalse(set.contains(t1_b, t1_b)); - } - - @Test - public void testToString() { - - LongPairSet set = new ConcurrentSortedLongPairSet(16); - - set.add(0, 0); - set.add(1, 1); - set.add(3, 3); - final String toString = "{[0:0], [1:1], [3:3]}"; - System.out.println(set.toString()); - assertEquals(set.toString(), toString); - } - - @Test - public void testIsEmpty() { - LongPairSet set = new ConcurrentSortedLongPairSet(); - assertTrue(set.isEmpty()); - set.add(1, 1); - assertFalse(set.isEmpty()); - } - - @Test - public void testShrink() { - LongPairSet set = new ConcurrentSortedLongPairSet(2, 1, true); - set.add(0, 0); - assertTrue(set.capacity() == 4); - set.add(0, 1); - assertTrue(set.capacity() == 4); - set.add(1, 1); - assertTrue(set.capacity() == 8); - set.add(1, 2); - assertTrue(set.capacity() == 8); - set.add(1, 3); - set.add(1, 4); - set.add(1, 5); - assertTrue(set.capacity() == 12); - set.remove(1, 5); - // not shrink - assertTrue(set.capacity() == 12); - set.remove(1, 4); - // the internal map does not keep shrinking at every remove() operation - assertTrue(set.capacity() == 12); - set.remove(1, 3); - set.remove(1, 2); - set.remove(1, 1); - // shrink - assertTrue(set.capacity() == 8); - } -} From 94e1341d1e299bc93b809fc3046b70390de43592 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Tue, 20 Aug 2024 23:32:22 +0800 Subject: [PATCH 458/580] [improve][broker] Optimize high CPU usage when consuming from topics with ongoing txn (#23189) --- .../org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index f99ee957e025a..e808c31bc89f1 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -953,7 +953,7 @@ public void asyncReadEntriesWithSkipOrWait(int maxEntries, long maxSizeBytes, Re int numberOfEntriesToRead = applyMaxSizeCap(maxEntries, maxSizeBytes); - if (hasMoreEntries()) { + if (hasMoreEntries() && maxPosition.compareTo(readPosition) >= 0) { // If we have available entries, we can read them immediately if (log.isDebugEnabled()) { log.debug("[{}] [{}] Read entries immediately", ledger.getName(), name); From a605ea32c7e6813bd37ef73198ed8706d88d4b1a Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Wed, 21 Aug 2024 03:43:38 +0800 Subject: [PATCH 459/580] [cleanup] Cleanup some duplicated code (#23204) --- .../mledger/impl/ManagedLedgerImpl.java | 46 +------------------ .../service/persistent/PersistentTopic.java | 8 +--- .../broker/service/PersistentTopicTest.java | 19 ++++---- .../pulsar/broker/service/ServerCnxTest.java | 6 +-- .../persistent/MessageDuplicationTest.java | 21 +++++---- 5 files changed, 28 insertions(+), 72 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 2f60eeff2fbd3..5756d6e952480 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -687,37 +687,7 @@ public Position addEntry(byte[] data, int numberOfMessages) throws InterruptedEx @Override public Position addEntry(byte[] data, int offset, int length) throws InterruptedException, ManagedLedgerException { - final CountDownLatch counter = new CountDownLatch(1); - // Result list will contain the status exception and the resulting - // position - class Result { - ManagedLedgerException status = null; - Position position = null; - } - final Result result = new Result(); - - asyncAddEntry(data, offset, length, new AddEntryCallback() { - @Override - public void addComplete(Position position, ByteBuf entryData, Object ctx) { - result.position = position; - counter.countDown(); - } - - @Override - public void addFailed(ManagedLedgerException exception, Object ctx) { - result.status = exception; - counter.countDown(); - } - }, null); - - counter.await(); - - if (result.status != null) { - log.error("[{}] Error adding entry", name, result.status); - throw result.status; - } - - return result.position; + return addEntry(data, 1, offset, length); } @Override @@ -777,19 +747,7 @@ public void asyncAddEntry(final byte[] data, int numberOfMessages, int offset, i @Override public void asyncAddEntry(ByteBuf buffer, AddEntryCallback callback, Object ctx) { - if (log.isDebugEnabled()) { - log.debug("[{}] asyncAddEntry size={} state={}", name, buffer.readableBytes(), state); - } - - // retain buffer in this thread - buffer.retain(); - - // Jump to specific thread to avoid contention from writers writing from different threads - executor.execute(() -> { - OpAddEntry addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, callback, ctx, - currentLedgerTimeoutTriggered); - internalAsyncAddEntry(addOperation); - }); + asyncAddEntry(buffer, 1, callback, ctx); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index c26725deaeab5..146ac05d695d5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -683,12 +683,8 @@ public void updateSubscribeRateLimiter() { } private void asyncAddEntry(ByteBuf headersAndPayload, PublishContext publishContext) { - if (brokerService.isBrokerEntryMetadataEnabled()) { - ledger.asyncAddEntry(headersAndPayload, - (int) publishContext.getNumberOfMessages(), this, publishContext); - } else { - ledger.asyncAddEntry(headersAndPayload, this, publishContext); - } + ledger.asyncAddEntry(headersAndPayload, + (int) publishContext.getNumberOfMessages(), this, publishContext); } public void asyncReadEntry(Position position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { 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 f2ed015bd1e67..f9171e883613b 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 @@ -24,6 +24,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.atLeast; @@ -294,11 +295,11 @@ public void testPublishMessage() throws Exception { doAnswer(invocationOnMock -> { final ByteBuf payload = (ByteBuf) invocationOnMock.getArguments()[0]; - final AddEntryCallback callback = (AddEntryCallback) invocationOnMock.getArguments()[1]; - final Topic.PublishContext ctx = (Topic.PublishContext) invocationOnMock.getArguments()[2]; + final AddEntryCallback callback = (AddEntryCallback) invocationOnMock.getArguments()[2]; + final Topic.PublishContext ctx = (Topic.PublishContext) invocationOnMock.getArguments()[3]; callback.addComplete(PositionFactory.LATEST, payload, ctx); return null; - }).when(ledgerMock).asyncAddEntry(any(ByteBuf.class), any(AddEntryCallback.class), any()); + }).when(ledgerMock).asyncAddEntry(any(ByteBuf.class), anyInt(), any(AddEntryCallback.class), any()); PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); long lastMaxReadPositionMovedForwardTimestamp = topic.getLastMaxReadPositionMovedForwardTimestamp(); @@ -377,10 +378,10 @@ public void testPublishMessageMLFailure() throws Exception { // override asyncAddEntry callback to return error doAnswer((Answer) invocationOnMock -> { - ((AddEntryCallback) invocationOnMock.getArguments()[1]).addFailed( - new ManagedLedgerException("Managed ledger failure"), invocationOnMock.getArguments()[2]); + ((AddEntryCallback) invocationOnMock.getArguments()[2]).addFailed( + new ManagedLedgerException("Managed ledger failure"), invocationOnMock.getArguments()[3]); return null; - }).when(ledgerMock).asyncAddEntry(any(ByteBuf.class), any(AddEntryCallback.class), any()); + }).when(ledgerMock).asyncAddEntry(any(ByteBuf.class), anyInt(), any(AddEntryCallback.class), any()); topic.publishMessage(payload, (exception, ledgerId, entryId) -> { if (exception == null) { @@ -1421,11 +1422,11 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { // call addComplete on ledger asyncAddEntry doAnswer(invocationOnMock -> { - ((AddEntryCallback) invocationOnMock.getArguments()[1]).addComplete(PositionFactory.create(1, 1), + ((AddEntryCallback) invocationOnMock.getArguments()[2]).addComplete(PositionFactory.create(1, 1), null, - invocationOnMock.getArguments()[2]); + invocationOnMock.getArguments()[3]); return null; - }).when(ledgerMock).asyncAddEntry(any(ByteBuf.class), any(AddEntryCallback.class), any()); + }).when(ledgerMock).asyncAddEntry(any(ByteBuf.class), anyInt(), any(AddEntryCallback.class), any()); // call openCursorComplete on cursor asyncOpen doAnswer(invocationOnMock -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index 03115d79af0a0..42b52d901e32f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -2943,12 +2943,12 @@ private void setupMLAsyncCallbackMocks() { // call addComplete on ledger asyncAddEntry doAnswer((Answer) invocationOnMock -> { - ((AddEntryCallback) invocationOnMock.getArguments()[1]).addComplete( + ((AddEntryCallback) invocationOnMock.getArguments()[2]).addComplete( PositionFactory.create(-1, -1), null, - invocationOnMock.getArguments()[2]); + invocationOnMock.getArguments()[3]); return null; - }).when(ledgerMock).asyncAddEntry(any(ByteBuf.class), any(AddEntryCallback.class), any()); + }).when(ledgerMock).asyncAddEntry(any(ByteBuf.class), anyInt(), any(AddEntryCallback.class), any()); doAnswer((Answer) invocationOnMock -> true).when(cursorMock).isDurable(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java index 4957cc998e327..e7dcbc602134c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java @@ -21,6 +21,7 @@ import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs; import static org.apache.pulsar.common.protocol.Commands.serializeMetadataAndPayload; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; @@ -284,7 +285,7 @@ public void testIsDuplicateWithFailure() { persistentTopic.publishMessage(byteBuf1, publishContext1); persistentTopic.addComplete(PositionFactory.create(0, 1), null, publishContext1); - verify(managedLedger, times(1)).asyncAddEntry(any(ByteBuf.class), any(), any()); + verify(managedLedger, times(1)).asyncAddEntry(any(ByteBuf.class), anyInt(), any(), any()); Long lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName1); assertNotNull(lastSequenceIdPushed); assertEquals(lastSequenceIdPushed.longValue(), 0); @@ -294,7 +295,7 @@ public void testIsDuplicateWithFailure() { persistentTopic.publishMessage(byteBuf2, publishContext2); persistentTopic.addComplete(PositionFactory.create(0, 2), null, publishContext2); - verify(managedLedger, times(2)).asyncAddEntry(any(ByteBuf.class), any(), any()); + verify(managedLedger, times(2)).asyncAddEntry(any(ByteBuf.class), anyInt(), any(), any()); lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName2); assertNotNull(lastSequenceIdPushed); assertEquals(lastSequenceIdPushed.longValue(), 1); @@ -306,7 +307,7 @@ public void testIsDuplicateWithFailure() { publishContext1 = getPublishContext(producerName1, 1); persistentTopic.publishMessage(byteBuf1, publishContext1); persistentTopic.addComplete(PositionFactory.create(0, 3), null, publishContext1); - verify(managedLedger, times(3)).asyncAddEntry(any(ByteBuf.class), any(), any()); + verify(managedLedger, times(3)).asyncAddEntry(any(ByteBuf.class), anyInt(), any(), any()); lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName1); assertNotNull(lastSequenceIdPushed); assertEquals(lastSequenceIdPushed.longValue(), 1); @@ -318,7 +319,7 @@ public void testIsDuplicateWithFailure() { publishContext1 = getPublishContext(producerName1, 5); persistentTopic.publishMessage(byteBuf1, publishContext1); persistentTopic.addComplete(PositionFactory.create(0, 4), null, publishContext1); - verify(managedLedger, times(4)).asyncAddEntry(any(ByteBuf.class), any(), any()); + verify(managedLedger, times(4)).asyncAddEntry(any(ByteBuf.class), anyInt(), any(), any()); lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName1); assertNotNull(lastSequenceIdPushed); assertEquals(lastSequenceIdPushed.longValue(), 5); @@ -330,7 +331,7 @@ public void testIsDuplicateWithFailure() { byteBuf1 = getMessage(producerName1, 0); publishContext1 = getPublishContext(producerName1, 0); persistentTopic.publishMessage(byteBuf1, publishContext1); - verify(managedLedger, times(4)).asyncAddEntry(any(ByteBuf.class), any(), any()); + verify(managedLedger, times(4)).asyncAddEntry(any(ByteBuf.class), anyInt(), any(), any()); lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName1); assertNotNull(lastSequenceIdPushed); assertEquals(lastSequenceIdPushed.longValue(), 5); @@ -341,7 +342,7 @@ public void testIsDuplicateWithFailure() { publishContext1 = getPublishContext(producerName1, 6); // don't complete message persistentTopic.publishMessage(byteBuf1, publishContext1); - verify(managedLedger, times(5)).asyncAddEntry(any(ByteBuf.class), any(), any()); + verify(managedLedger, times(5)).asyncAddEntry(any(ByteBuf.class), anyInt(), any(), any()); lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName1); assertNotNull(lastSequenceIdPushed); assertEquals(lastSequenceIdPushed.longValue(), 6); @@ -353,7 +354,7 @@ public void testIsDuplicateWithFailure() { byteBuf1 = getMessage(producerName1, 6); publishContext1 = getPublishContext(producerName1, 6); persistentTopic.publishMessage(byteBuf1, publishContext1); - verify(managedLedger, times(5)).asyncAddEntry(any(ByteBuf.class), any(), any()); + verify(managedLedger, times(5)).asyncAddEntry(any(ByteBuf.class), anyInt(), any(), any()); verify(publishContext1, times(1)).completed(any(MessageDeduplication.MessageDupUnknownException.class), eq(-1L), eq(-1L)); // complete seq 6 message eventually @@ -363,7 +364,7 @@ public void testIsDuplicateWithFailure() { byteBuf1 = getMessage(producerName1, 7); publishContext1 = getPublishContext(producerName1, 7); persistentTopic.publishMessage(byteBuf1, publishContext1); - verify(managedLedger, times(6)).asyncAddEntry(any(ByteBuf.class), any(), any()); + verify(managedLedger, times(6)).asyncAddEntry(any(ByteBuf.class), anyInt(), any(), any()); persistentTopic.addFailed(new ManagedLedgerException("test"), publishContext1); // check highestSequencedPushed is reset @@ -383,7 +384,7 @@ public void testIsDuplicateWithFailure() { byteBuf1 = getMessage(producerName1, 6); publishContext1 = getPublishContext(producerName1, 6); persistentTopic.publishMessage(byteBuf1, publishContext1); - verify(managedLedger, times(6)).asyncAddEntry(any(ByteBuf.class), any(), any()); + verify(managedLedger, times(6)).asyncAddEntry(any(ByteBuf.class), anyInt(), any(), any()); verify(publishContext1, times(1)).completed(eq(null), eq(-1L), eq(-1L)); lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName1); assertNotNull(lastSequenceIdPushed); @@ -393,7 +394,7 @@ public void testIsDuplicateWithFailure() { byteBuf1 = getMessage(producerName1, 8); publishContext1 = getPublishContext(producerName1, 8); persistentTopic.publishMessage(byteBuf1, publishContext1); - verify(managedLedger, times(7)).asyncAddEntry(any(ByteBuf.class), any(), any()); + verify(managedLedger, times(7)).asyncAddEntry(any(ByteBuf.class), anyInt(), any(), any()); persistentTopic.addComplete(PositionFactory.create(0, 5), null, publishContext1); lastSequenceIdPushed = messageDeduplication.highestSequencedPushed.get(producerName1); assertNotNull(lastSequenceIdPushed); From 18cb458f73b8967eb7c1f1f133e0b7f9f1a60e93 Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Wed, 21 Aug 2024 09:41:53 +0800 Subject: [PATCH 460/580] [improve][broker] Add callback parameters to the SendCallback.sendComplete (#23196) --- .../NonPersistentReplicator.java | 3 +- .../persistent/PersistentReplicator.java | 3 +- .../client/impl/ProduceWithMessageIdTest.java | 85 ++++++++++++++++++- .../impl/BatchMessageContainerImpl.java | 2 +- .../pulsar/client/impl/OpSendMsgStats.java | 38 +++++++++ .../client/impl/OpSendMsgStatsImpl.java | 73 ++++++++++++++++ .../pulsar/client/impl/ProducerImpl.java | 34 +++++--- .../pulsar/client/impl/SendCallback.java | 5 +- 8 files changed, 224 insertions(+), 19 deletions(-) create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/OpSendMsgStats.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/OpSendMsgStatsImpl.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java index 6441230fad87b..45b4ebf6e17cc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java @@ -33,6 +33,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.impl.MessageImpl; +import org.apache.pulsar.client.impl.OpSendMsgStats; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.SendCallback; @@ -173,7 +174,7 @@ private static final class ProducerSendCallback implements SendCallback { private MessageImpl msg; @Override - public void sendComplete(Exception exception) { + public void sendComplete(Throwable exception, OpSendMsgStats opSendMsgStats) { if (exception != null) { log.error("[{}] Error producing on remote broker", replicator.replicatorId, exception); } else { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index 33e883ab9406a..b3d7546beed81 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -60,6 +60,7 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.MessageImpl; +import org.apache.pulsar.client.impl.OpSendMsgStats; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.SendCallback; @@ -377,7 +378,7 @@ protected static final class ProducerSendCallback implements SendCallback { private MessageImpl msg; @Override - public void sendComplete(Exception exception) { + public void sendComplete(Throwable exception, OpSendMsgStats opSendMsgStats) { if (exception != null && !(exception instanceof PulsarClientException.InvalidMessageException)) { log.error("[{}] Error producing on remote broker", replicator.replicatorId, exception); // cursor should be rewinded since it was incremented when readMoreEntries diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProduceWithMessageIdTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProduceWithMessageIdTest.java index b8efdeb99696a..45f9a9c52e871 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProduceWithMessageIdTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProduceWithMessageIdTest.java @@ -18,14 +18,18 @@ */ package org.apache.pulsar.client.impl; +import static org.apache.pulsar.client.impl.AbstractBatchMessageContainer.INITIAL_BATCH_BUFFER_SIZE; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MockBrokerService; +import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.api.proto.MessageMetadata; @@ -38,13 +42,20 @@ @Test(groups = "broker-impl") @Slf4j -public class ProduceWithMessageIdTest { +public class ProduceWithMessageIdTest extends ProducerConsumerBase { MockBrokerService mockBrokerService; @BeforeClass(alwaysRun = true) - public void setup() { + public void setup() throws Exception { mockBrokerService = new MockBrokerService(); mockBrokerService.start(); + super.internalSetup(); + super.producerBaseSetup(); + } + + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); } @AfterClass(alwaysRun = true) @@ -86,7 +97,7 @@ public void testSend() throws Exception { AtomicBoolean result = new AtomicBoolean(false); producer.sendAsync(msg, new SendCallback() { @Override - public void sendComplete(Exception e) { + public void sendComplete(Throwable e, OpSendMsgStats opSendMsgStats) { log.info("sendComplete", e); result.set(e == null); } @@ -115,4 +126,72 @@ public CompletableFuture getFuture() { // the result is true only if broker received right message id. Awaitility.await().untilTrue(result); } + + @Test + public void sendWithCallBack() throws Exception { + + int batchSize = 10; + + String topic = "persistent://public/default/testSendWithCallBack"; + ProducerImpl producer = + (ProducerImpl) pulsarClient.newProducer().topic(topic) + .enableBatching(true) + .batchingMaxMessages(batchSize) + .create(); + + CountDownLatch cdl = new CountDownLatch(1); + AtomicReference sendMsgStats = new AtomicReference<>(); + SendCallback sendComplete = new SendCallback() { + @Override + public void sendComplete(Throwable e, OpSendMsgStats opSendMsgStats) { + log.info("sendComplete", e); + if (e == null){ + cdl.countDown(); + sendMsgStats.set(opSendMsgStats); + } + } + + @Override + public void addCallback(MessageImpl msg, SendCallback scb) { + + } + + @Override + public SendCallback getNextSendCallback() { + return null; + } + + @Override + public MessageImpl getNextMessage() { + return null; + } + + @Override + public CompletableFuture getFuture() { + return null; + } + }; + int totalReadabled = 0; + int totalUncompressedSize = 0; + for (int i = 0; i < batchSize; i++) { + MessageMetadata metadata = new MessageMetadata(); + ByteBuffer buffer = ByteBuffer.wrap("data".getBytes(StandardCharsets.UTF_8)); + MessageImpl msg = MessageImpl.create(metadata, buffer, Schema.BYTES, topic); + msg.getDataBuffer().retain(); + totalReadabled += msg.getDataBuffer().readableBytes(); + totalUncompressedSize += msg.getUncompressedSize(); + producer.sendAsync(msg, sendComplete); + } + + cdl.await(); + OpSendMsgStats opSendMsgStats = sendMsgStats.get(); + Assert.assertEquals(opSendMsgStats.getUncompressedSize(), totalUncompressedSize + INITIAL_BATCH_BUFFER_SIZE); + Assert.assertEquals(opSendMsgStats.getSequenceId(), 0); + Assert.assertEquals(opSendMsgStats.getRetryCount(), 1); + Assert.assertEquals(opSendMsgStats.getBatchSizeByte(), totalReadabled); + Assert.assertEquals(opSendMsgStats.getNumMessagesInBatch(), batchSize); + Assert.assertEquals(opSendMsgStats.getHighestSequenceId(), batchSize-1); + Assert.assertEquals(opSendMsgStats.getTotalChunks(), 0); + Assert.assertEquals(opSendMsgStats.getChunkId(), -1); + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java index a3c9d1bc9ab48..44f1fb274655a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageContainerImpl.java @@ -229,7 +229,7 @@ public void discard(Exception ex) { try { // Need to protect ourselves from any exception being thrown in the future handler from the application if (firstCallback != null) { - firstCallback.sendComplete(ex); + firstCallback.sendComplete(ex, null); } if (batchedMessageMetadataAndPayload != null) { ReferenceCountUtil.safeRelease(batchedMessageMetadataAndPayload); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/OpSendMsgStats.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/OpSendMsgStats.java new file mode 100644 index 0000000000000..dc28df50f2886 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/OpSendMsgStats.java @@ -0,0 +1,38 @@ +/* + * 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.impl; + + +public interface OpSendMsgStats { + long getUncompressedSize(); + + long getSequenceId(); + + int getRetryCount(); + + long getBatchSizeByte(); + + int getNumMessagesInBatch(); + + long getHighestSequenceId(); + + int getTotalChunks(); + + int getChunkId(); +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/OpSendMsgStatsImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/OpSendMsgStatsImpl.java new file mode 100644 index 0000000000000..41bb742776caa --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/OpSendMsgStatsImpl.java @@ -0,0 +1,73 @@ +/* + * 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.impl; + +import lombok.Builder; + +@Builder +public class OpSendMsgStatsImpl implements OpSendMsgStats { + private long uncompressedSize; + private long sequenceId; + private int retryCount; + private long batchSizeByte; + private int numMessagesInBatch; + private long highestSequenceId; + private int totalChunks; + private int chunkId; + + @Override + public long getUncompressedSize() { + return uncompressedSize; + } + + @Override + public long getSequenceId() { + return sequenceId; + } + + @Override + public int getRetryCount() { + return retryCount; + } + + @Override + public long getBatchSizeByte() { + return batchSizeByte; + } + + @Override + public int getNumMessagesInBatch() { + return numMessagesInBatch; + } + + @Override + public long getHighestSequenceId() { + return highestSequenceId; + } + + @Override + public int getTotalChunks() { + return totalChunks; + } + + @Override + public int getChunkId() { + return chunkId; + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 6d5a81454631f..5c46057ae308d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -414,7 +414,7 @@ public MessageImpl getNextMessage() { } @Override - public void sendComplete(Exception e) { + public void sendComplete(Throwable e, OpSendMsgStats opSendMsgStats) { SendCallback loopingCallback = this; MessageImpl loopingMsg = currentMsg; while (loopingCallback != null) { @@ -424,7 +424,7 @@ public void sendComplete(Exception e) { } } - private void onSendComplete(Exception e, SendCallback sendCallback, MessageImpl msg) { + private void onSendComplete(Throwable e, SendCallback sendCallback, MessageImpl msg) { long createdAt = (sendCallback instanceof ProducerImpl.DefaultSendMessageCallback) ? ((DefaultSendMessageCallback) sendCallback).createdAt : this.createdAt; long latencyNanos = System.nanoTime() - createdAt; @@ -842,7 +842,7 @@ private void tryRegisterSchema(ClientCnx cnx, MessageImpl msg, SendCallback call log.warn("[{}] [{}] GetOrCreateSchema error", topic, producerName, t); if (t instanceof PulsarClientException.IncompatibleSchemaException) { msg.setSchemaState(MessageImpl.SchemaState.Broken); - callback.sendComplete((PulsarClientException.IncompatibleSchemaException) t); + callback.sendComplete(t, null); } } else { log.info("[{}] [{}] GetOrCreateSchema succeed", topic, producerName); @@ -985,19 +985,19 @@ private boolean isValidProducerState(SendCallback callback, long sequenceId) { case Closing: case Closed: callback.sendComplete( - new PulsarClientException.AlreadyClosedException("Producer already closed", sequenceId)); + new PulsarClientException.AlreadyClosedException("Producer already closed", sequenceId), null); return false; case ProducerFenced: - callback.sendComplete(new PulsarClientException.ProducerFencedException("Producer was fenced")); + callback.sendComplete(new PulsarClientException.ProducerFencedException("Producer was fenced"), null); return false; case Terminated: callback.sendComplete( - new PulsarClientException.TopicTerminatedException("Topic was terminated", sequenceId)); + new PulsarClientException.TopicTerminatedException("Topic was terminated", sequenceId), null); return false; case Failed: case Uninitialized: default: - callback.sendComplete(new PulsarClientException.NotConnectedException(sequenceId)); + callback.sendComplete(new PulsarClientException.NotConnectedException(sequenceId), null); return false; } } @@ -1012,20 +1012,20 @@ private boolean canEnqueueRequest(SendCallback callback, long sequenceId, int pa } else { if (!semaphore.map(Semaphore::tryAcquire).orElse(true)) { callback.sendComplete(new PulsarClientException.ProducerQueueIsFullError( - "Producer send queue is full", sequenceId)); + "Producer send queue is full", sequenceId), null); return false; } if (!client.getMemoryLimitController().tryReserveMemory(payloadSize)) { semaphore.ifPresent(Semaphore::release); callback.sendComplete(new PulsarClientException.MemoryBufferIsFullError( - "Client memory buffer is full", sequenceId)); + "Client memory buffer is full", sequenceId), null); return false; } } } catch (InterruptedException e) { Thread.currentThread().interrupt(); - callback.sendComplete(new PulsarClientException(e, sequenceId)); + callback.sendComplete(new PulsarClientException(e, sequenceId), null); return false; } @@ -1302,7 +1302,7 @@ private void releaseSemaphoreForSendOp(OpSendMsg op) { private void completeCallbackAndReleaseSemaphore(long payloadSize, SendCallback callback, Exception exception) { semaphore.ifPresent(Semaphore::release); client.getMemoryLimitController().releaseMemory(payloadSize); - callback.sendComplete(exception); + callback.sendComplete(exception, null); } /** @@ -1595,7 +1595,17 @@ void sendComplete(final Exception e) { rpcLatencyHistogram.recordFailure(now - this.lastSentAt); } - callback.sendComplete(finalEx); + OpSendMsgStats opSendMsgStats = OpSendMsgStatsImpl.builder() + .uncompressedSize(uncompressedSize) + .sequenceId(sequenceId) + .retryCount(retryCount) + .batchSizeByte(batchSizeByte) + .numMessagesInBatch(numMessagesInBatch) + .highestSequenceId(highestSequenceId) + .totalChunks(totalChunks) + .chunkId(chunkId) + .build(); + callback.sendComplete(finalEx, opSendMsgStats); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SendCallback.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SendCallback.java index 369bb34a29a79..f55d7ae79129c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SendCallback.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SendCallback.java @@ -20,18 +20,21 @@ import java.util.concurrent.CompletableFuture; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.common.classification.InterfaceStability; /** * */ +@InterfaceStability.Evolving public interface SendCallback { /** * invoked when send operation completes. * * @param e + * @param opSendMsgStats stats associated with the send operation */ - void sendComplete(Exception e); + void sendComplete(Throwable e, OpSendMsgStats opSendMsgStats); /** * used to specify a callback to be invoked on completion of a send operation for individual messages sent in a From b661ec82d20adb71c0fe00ad115ec1ab71327880 Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Wed, 21 Aug 2024 11:06:35 +0800 Subject: [PATCH 461/580] [improve][broker] Improve pulsar_topic_load_failed metric to record correct failed time (#23199) Co-authored-by: fanjianye --- .../apache/pulsar/broker/service/BrokerService.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 066bfc98cc0cf..e5248d45d4226 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 @@ -1586,6 +1586,11 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S Duration.ofSeconds(pulsar.getConfiguration().getTopicLoadTimeoutSeconds()), executor(), () -> FAILED_TO_LOAD_TOPIC_TIMEOUT_EXCEPTION); + topicFuture.exceptionally(t -> { + pulsarStats.recordTopicLoadFailed(); + return null; + }); + checkTopicNsOwnership(topic) .thenRun(() -> { final Semaphore topicLoadSemaphore = topicLoadRequestSemaphore.get(); @@ -1700,11 +1705,6 @@ private void createPersistentTopic(final String topic, boolean createIfMissing, TopicName topicName = TopicName.get(topic); final long topicCreateTimeMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()); - topicFuture.exceptionally(t -> { - pulsarStats.recordTopicLoadFailed(); - return null; - }); - if (isTransactionInternalName(topicName)) { String msg = String.format("Can not create transaction system topic %s", topic); log.warn(msg); From 66e1a06b247cd032872e5ab454ff8c8c6cd98550 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 21 Aug 2024 14:17:44 +0800 Subject: [PATCH 462/580] [improve][client] Don't print info logs for each schema loaded by client (#23206) --- .../client/impl/schema/reader/MultiVersionAvroReader.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/reader/MultiVersionAvroReader.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/reader/MultiVersionAvroReader.java index 0ca847917eeca..85d4d63a1b136 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/reader/MultiVersionAvroReader.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/reader/MultiVersionAvroReader.java @@ -44,9 +44,11 @@ public MultiVersionAvroReader(Schema readerSchema, ClassLoader pojoClassLoader, protected SchemaReader loadReader(BytesSchemaVersion schemaVersion) { SchemaInfo schemaInfo = getSchemaInfoByVersion(schemaVersion.get()); if (schemaInfo != null) { - LOG.info("Load schema reader for version({}), schema is : {}, schemaInfo: {}", - SchemaUtils.getStringSchemaVersion(schemaVersion.get()), - schemaInfo.getSchemaDefinition(), schemaInfo.toString()); + if (LOG.isDebugEnabled()) { + LOG.debug("Load schema reader for version({}), schema is : {}, schemaInfo: {}", + SchemaUtils.getStringSchemaVersion(schemaVersion.get()), + schemaInfo.getSchemaDefinition(), schemaInfo); + } boolean jsr310ConversionEnabled = getJsr310ConversionEnabledFromSchemaInfo(schemaInfo); return new AvroReader<>(parseAvroSchema(schemaInfo.getSchemaDefinition()), readerSchema, pojoClassLoader, jsr310ConversionEnabled); From 0a5cb51a2f010d6771ae0ae0fd259d002cca20da Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 22 Aug 2024 11:13:55 +0800 Subject: [PATCH 463/580] [fix] [log] Do not print warn log when concurrently publishing and switching ledgers (#23209) --- .../mledger/impl/ManagedLedgerImpl.java | 5 +- ...ProducerConsumerMLInitializeDelayTest.java | 70 +++++++++++++++++++ 2 files changed, 73 insertions(+), 2 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 5756d6e952480..92c55e572a49a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -1645,8 +1645,9 @@ void createNewOpAddEntryForNewLedger() { if (existsOp.ledger != null) { existsOp = existsOp.duplicateAndClose(currentLedgerTimeoutTriggered); } else { - // This scenario should not happen. - log.warn("[{}] An OpAddEntry's ledger is empty.", name); + // It may happen when the following operations execute at the same time, so it is expected. + // - Adding entry. + // - Switching ledger. existsOp.setTimeoutTriggered(currentLedgerTimeoutTriggered); } existsOp.setLedger(currentLedger); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java index ab4e063ae3d83..7c7665a5bd3e4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java @@ -20,12 +20,19 @@ import com.carrotsearch.hppc.ObjectSet; import java.time.Duration; +import java.util.ArrayList; +import java.util.LinkedHashSet; import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicName; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; @@ -105,4 +112,67 @@ public void testConsumerListMatchesConsumerSet() throws Exception { // cleanup. client.close(); } + + @Test(timeOut = 30 * 1000) + public void testConcurrentlyOfPublishAndSwitchLedger() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subscription, MessageId.earliest); + // Make ledger switches faster. + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get(); + ManagedLedgerConfig config = persistentTopic.getManagedLedger().getConfig(); + config.setMaxEntriesPerLedger(2); + config.setMinimumRolloverTime(0, TimeUnit.MILLISECONDS); + // Inject a delay for switching ledgers, so publishing requests will be push in to the pending queue. + AtomicInteger delayTimes = new AtomicInteger(); + mockZooKeeper.delay(10, (op, s) -> { + if (op.toString().equals("SET") && s.contains(TopicName.get(topicName).getPersistenceNamingEncoding())) { + return delayTimes.incrementAndGet() == 1; + } + return false; + }); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).enableBatching(false) + .create(); + List> sendRequests = new ArrayList<>(); + List msgsSent = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + String msg = i + ""; + sendRequests.add(producer.sendAsync(i + "")); + msgsSent.add(msg); + } + // Verify: + // - All messages were sent. + // - The order of messages are correct. + Set msgIds = new LinkedHashSet<>(); + MessageIdImpl previousMsgId = null; + for (CompletableFuture msgId : sendRequests) { + Assert.assertNotNull(msgId.join()); + MessageIdImpl messageIdImpl = (MessageIdImpl) msgId.join(); + if (previousMsgId != null) { + Assert.assertTrue(messageIdImpl.compareTo(previousMsgId) > 0); + } + msgIds.add(String.format("%s:%s", messageIdImpl.getLedgerId(), messageIdImpl.getEntryId())); + previousMsgId = messageIdImpl; + } + Assert.assertEquals(msgIds.size(), 100); + log.info("messages were sent: {}", msgIds.toString()); + List msgsReceived = new ArrayList<>(); + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionName(subscription).subscribe(); + while (true) { + Message receivedMsg = consumer.receive(2, TimeUnit.SECONDS); + if (receivedMsg == null) { + break; + } + msgsReceived.add(receivedMsg.getValue()); + } + Assert.assertEquals(msgsReceived, msgsSent); + + // cleanup. + consumer.close(); + producer.close(); + admin.topics().delete(topicName); + } } From 44f986014e4d314a4a52484856c7dfb2d89ea3c1 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 23 Aug 2024 08:31:49 +0800 Subject: [PATCH 464/580] [improve] [broker] Phase 1 of PIP-370 support disable create topics on remote cluster through replication (#23169) --- conf/broker.conf | 10 + conf/standalone.conf | 10 + .../pulsar/broker/ServiceConfiguration.java | 5 + .../pulsar/broker/admin/AdminResource.java | 10 +- .../admin/impl/PersistentTopicsBase.java | 6 + .../broker/service/AbstractReplicator.java | 16 +- .../persistent/GeoPersistentReplicator.java | 29 +++ ...opicToRemoteClusterForReplicationTest.java | 208 ++++++++++++++++++ .../pulsar/broker/service/StandaloneTest.java | 1 + .../naming/ServiceConfigurationTest.java | 5 + .../configurations/pulsar_broker_test.conf | 1 + .../pulsar_broker_test_standalone.conf | 1 + .../pulsar/client/impl/PulsarClientImpl.java | 2 +- 13 files changed, 296 insertions(+), 8 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DisabledCreateTopicToRemoteClusterForReplicationTest.java diff --git a/conf/broker.conf b/conf/broker.conf index 3c956bdd86dab..e5d8a32e7171c 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1549,6 +1549,16 @@ replicatorPrefix=pulsar.repl # due to missing ZooKeeper watch (disable with value 0) replicationPolicyCheckDurationSeconds=600 +# Whether the internal replication of the local cluster will trigger topic auto-creation on the remote cluster. +# 1. After enabling namespace-level Geo-Replication: whether the local broker will create topics on the remote +# cluster automatically when calling `pulsar-admin topics create-partitioned-topic`. +# 2. When enabling topic-level Geo-Replication on a partitioned topic: whether the local broker will create topics on +# the remote cluster. +# 3. Whether the internal Geo-Replicator in the local cluster will trigger non-persistent topic auto-creation for +# remote clusters. +# It is not a dynamic config, the default value is "true" to preserve backward-compatible behavior. +createTopicToRemoteClusterForReplication=true + # Default message retention time. # 0 means retention is disabled. -1 means data is not removed by time quota. defaultRetentionTimeInMinutes=0 diff --git a/conf/standalone.conf b/conf/standalone.conf index 635b31ac38def..30b39af8869d4 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -949,6 +949,16 @@ replicationProducerQueueSize=1000 # due to missing ZooKeeper watch (disable with value 0) replicationPolicyCheckDurationSeconds=600 +# Whether the internal replication of the local cluster will trigger topic auto-creation on the remote cluster. +# 1. After enabling namespace-level Geo-Replication: whether the local broker will create topics on the remote +# cluster automatically when calling `pulsar-admin topics create-partitioned-topic`. +# 2. When enabling topic-level Geo-Replication on a partitioned topic: whether the local broker will create topics on +# the remote cluster. +# 3. Whether the internal Geo-Replicator in the local cluster will trigger non-persistent topic auto-creation for +# remote clusters. +# It is not a dynamic config, the default value is "true" to preserve backward-compatible behavior. +createTopicToRemoteClusterForReplication=true + # Default message retention time. 0 means retention is disabled. -1 means data is not removed by time quota defaultRetentionTimeInMinutes=0 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 20addc3924bf3..c836879b075f1 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 @@ -2889,6 +2889,11 @@ public double getLoadBalancerBandwidthOutResourceWeight() { + "inconsistency due to missing ZooKeeper watch (disable with value 0)" ) private int replicationPolicyCheckDurationSeconds = 600; + @FieldContext( + category = CATEGORY_REPLICATION, + doc = "Whether the internal replicator will trigger topic auto-creation on the remote cluster." + ) + private boolean createTopicToRemoteClusterForReplication = true; @Deprecated @FieldContext( category = CATEGORY_REPLICATION, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index 1f43aeaa668bc..497af71955158 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -609,11 +609,15 @@ protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int n .thenCompose(__ -> provisionPartitionedTopicPath(numPartitions, createLocalTopicOnly, properties)) .thenCompose(__ -> tryCreatePartitionsAsync(numPartitions)) .thenRun(() -> { - if (!createLocalTopicOnly && topicName.isGlobal()) { + if (!createLocalTopicOnly && topicName.isGlobal() + && pulsar().getConfig().isCreateTopicToRemoteClusterForReplication()) { internalCreatePartitionedTopicToReplicatedClustersInBackground(numPartitions); + log.info("[{}] Successfully created partitioned for topic {} for the remote clusters", + clientAppId()); + } else { + log.info("[{}] Skip creating partitioned for topic {} for the remote clusters", + clientAppId(), topicName); } - log.info("[{}] Successfully created partitions for topic {} in cluster {}", - clientAppId(), topicName, pulsar().getConfiguration().getClusterName()); asyncResponse.resume(Response.noContent().build()); }) .exceptionally(ex -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 747031df7a0af..40e74f83e986d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -3332,6 +3332,12 @@ protected CompletableFuture internalSetReplicationClusters(List cl } return FutureUtil.waitForAll(futures); }).thenCompose(__ -> { + if (!pulsar().getConfig().isCreateTopicToRemoteClusterForReplication()) { + log.info("[{}] Skip creating partitioned for topic {} for the remote clusters {}", + clientAppId(), topicName, replicationClusters.stream().filter(v -> + !pulsar().getConfig().getClusterName().equals(v)).collect(Collectors.toList())); + return CompletableFuture.completedFuture(null); + } // Sync to create partitioned topic on the remote cluster if needed. TopicName topicNameWithoutPartition = TopicName.get(topicName.getPartitionedTopicName()); return pulsar().getPulsarResources().getNamespaceResources().getPartitionedTopicResources() diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index 424263720f012..34fd9f17f6ea6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -159,6 +159,10 @@ public String getRemoteCluster() { return remoteCluster; } + protected CompletableFuture prepareCreateProducer() { + return CompletableFuture.completedFuture(null); + } + public void startProducer() { // Guarantee only one task call "producerBuilder.createAsync()". Pair setStartingRes = compareSetAndGetState(State.Disconnected, State.Starting); @@ -185,12 +189,15 @@ public void startProducer() { } log.info("[{}] Starting replicator", replicatorId); + // Force only replicate messages to a non-partitioned topic, to avoid auto-create a partitioned topic on // the remote cluster. - ProducerBuilderImpl builderImpl = (ProducerBuilderImpl) producerBuilder; - builderImpl.getConf().setNonPartitionedTopicExpected(true); - producerBuilder.createAsync().thenAccept(producer -> { - setProducerAndTriggerReadEntries(producer); + prepareCreateProducer().thenCompose(ignore -> { + ProducerBuilderImpl builderImpl = (ProducerBuilderImpl) producerBuilder; + builderImpl.getConf().setNonPartitionedTopicExpected(true); + return producerBuilder.createAsync().thenAccept(producer -> { + setProducerAndTriggerReadEntries(producer); + }); }).exceptionally(ex -> { Pair setDisconnectedRes = compareSetAndGetState(State.Starting, State.Disconnected); if (setDisconnectedRes.getLeft()) { @@ -215,6 +222,7 @@ public void startProducer() { } return null; }); + } /*** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java index 1d9df2bcccda3..cd5b2ba721215 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java @@ -26,11 +26,13 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.protocol.Markers; import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.util.FutureUtil; @Slf4j public class GeoPersistentReplicator extends PersistentReplicator { @@ -50,6 +52,33 @@ protected String getProducerName() { return getReplicatorName(replicatorPrefix, localCluster) + REPL_PRODUCER_NAME_DELIMITER + remoteCluster; } + @Override + protected CompletableFuture prepareCreateProducer() { + if (brokerService.getPulsar().getConfig().isCreateTopicToRemoteClusterForReplication()) { + return CompletableFuture.completedFuture(null); + } else { + CompletableFuture topicCheckFuture = new CompletableFuture<>(); + replicationClient.getPartitionedTopicMetadata(localTopic.getName(), false, false) + .whenComplete((metadata, ex) -> { + if (ex == null) { + if (metadata.partitions == 0) { + topicCheckFuture.complete(null); + } else { + String errorMsg = String.format("{} Can not create the replicator due to the partitions in the" + + " remote cluster is not 0, but is %s", + replicatorId, metadata.partitions); + log.error(errorMsg); + topicCheckFuture.completeExceptionally( + new PulsarClientException.NotAllowedException(errorMsg)); + } + } else { + topicCheckFuture.completeExceptionally(FutureUtil.unwrapCompletionException(ex)); + } + }); + return topicCheckFuture; + } + } + @Override protected boolean replicateEntries(List entries) { boolean atLeastOneMessageSentForReplication = false; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DisabledCreateTopicToRemoteClusterForReplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DisabledCreateTopicToRemoteClusterForReplicationTest.java new file mode 100644 index 0000000000000..0f8db4aaa7316 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DisabledCreateTopicToRemoteClusterForReplicationTest.java @@ -0,0 +1,208 @@ +/* + * 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.service; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.util.Arrays; +import java.util.HashSet; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.apache.pulsar.zookeeper.ZookeeperServerTest; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class DisabledCreateTopicToRemoteClusterForReplicationTest extends OneWayReplicatorTestBase { + + @Override + @BeforeClass(alwaysRun = true, timeOut = 300000) + public void setup() throws Exception { + super.setup(); + admin1.namespaces().setRetention(replicatedNamespace, new RetentionPolicies(300, 1024)); + admin2.namespaces().setRetention(replicatedNamespace, new RetentionPolicies(300, 1024)); + admin1.namespaces().setRetention(nonReplicatedNamespace, new RetentionPolicies(300, 1024)); + admin2.namespaces().setRetention(nonReplicatedNamespace, new RetentionPolicies(300, 1024)); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + @Override + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, + LocalBookkeeperEnsemble bookkeeperEnsemble, ZookeeperServerTest brokerConfigZk) { + super.setConfigDefaults(config, clusterName, bookkeeperEnsemble, brokerConfigZk); + config.setCreateTopicToRemoteClusterForReplication(false); + config.setReplicationStartAt("earliest"); + } + + @Test + public void testCreatePartitionedTopicWithNsReplication() throws Exception { + String ns = defaultTenant + "/" + UUID.randomUUID().toString().replace("-", ""); + admin1.namespaces().createNamespace(ns); + admin2.namespaces().createNamespace(ns); + admin1.namespaces().setRetention(ns, new RetentionPolicies(3600, -1)); + admin2.namespaces().setRetention(ns, new RetentionPolicies(3600, -1)); + + // Create non-partitioned topic. + // Enable replication. + final String tp = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp_"); + final String part1 = TopicName.get(tp).getPartition(0).toString(); + admin1.topics().createPartitionedTopic(tp, 1); + admin1.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1, cluster2))); + + // Trigger and wait for replicator starts. + String msgValue = "msg-1"; + Producer producer1 = client1.newProducer(Schema.STRING).topic(tp).create(); + producer1.send(msgValue); + producer1.close(); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(part1, false).join().get(); + assertFalse(topicPart1.getReplicators().isEmpty()); + }); + + // Verify: there is no topic with the same name on the remote cluster. + try { + admin2.topics().getPartitionedTopicMetadata(tp); + fail("Expected a not found ex"); + } catch (PulsarAdminException.NotFoundException ex) { + // expected. + } + + // Verify: after creating the topic on the remote cluster, all things are fine. + admin2.topics().createPartitionedTopic(tp, 1); + Consumer consumer2 = client2.newConsumer(Schema.STRING).topic(tp).isAckReceiptEnabled(true) + .subscriptionName("s1").subscribe(); + assertEquals(consumer2.receive(10, TimeUnit.SECONDS).getValue(), msgValue); + consumer2.close(); + + // cleanup. + admin1.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1))); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(part1, false).join().get(); + assertTrue(topicPart1.getReplicators().isEmpty()); + }); + admin1.topics().deletePartitionedTopic(tp, false); + admin2.topics().deletePartitionedTopic(tp, false); + admin1.namespaces().deleteNamespace(ns); + admin2.namespaces().deleteNamespace(ns); + } + + @Test + public void testEnableTopicReplication() throws Exception { + String ns = nonReplicatedNamespace; + + // Create non-partitioned topic. + // Enable replication. + final String tp = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp_"); + final String part1 = TopicName.get(tp).getPartition(0).toString(); + admin1.topics().createPartitionedTopic(tp, 1); + admin1.topics().setReplicationClusters(tp, Arrays.asList(cluster1, cluster2)); + + // Trigger and wait for replicator starts. + Producer p1 = client1.newProducer(Schema.STRING).topic(tp).create(); + p1.send("msg-1"); + p1.close(); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(part1, false).join().get(); + assertFalse(topicPart1.getReplicators().isEmpty()); + }); + + // Verify: there is no topic with the same name on the remote cluster. + try { + admin2.topics().getPartitionedTopicMetadata(tp); + fail("Expected a not found ex"); + } catch (PulsarAdminException.NotFoundException ex) { + // expected. + } + + // Verify: after creating the topic on the remote cluster, all things are fine. + admin2.topics().createPartitionedTopic(tp, 1); + waitReplicatorStarted(part1); + + // cleanup. + admin1.topics().setReplicationClusters(tp, Arrays.asList(cluster1)); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(part1, false).join().get(); + assertTrue(topicPart1.getReplicators().isEmpty()); + }); + admin1.topics().deletePartitionedTopic(tp, false); + admin2.topics().deletePartitionedTopic(tp, false); + } + + @Test + public void testNonPartitionedTopic() throws Exception { + String ns = nonReplicatedNamespace; + + // Create non-partitioned topic. + // Enable replication. + final String tp = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp_"); + admin1.topics().createNonPartitionedTopic(tp); + admin1.topics().setReplicationClusters(tp, Arrays.asList(cluster1, cluster2)); + + // Trigger and wait for replicator starts. + Producer p1 = client1.newProducer(Schema.STRING).topic(tp).create(); + p1.send("msg-1"); + p1.close(); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(tp, false).join().get(); + assertFalse(topicPart1.getReplicators().isEmpty()); + }); + + // Verify: there is no topic with the same name on the remote cluster. + try { + admin2.topics().getPartitionedTopicMetadata(tp); + fail("Expected a not found ex"); + } catch (PulsarAdminException.NotFoundException ex) { + // expected. + } + + // Verify: after creating the topic on the remote cluster, all things are fine. + admin2.topics().createNonPartitionedTopic(tp); + waitReplicatorStarted(tp); + + // cleanup. + admin1.topics().setReplicationClusters(tp, Arrays.asList(cluster1)); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(tp, false).join().get(); + assertTrue(topicPart1.getReplicators().isEmpty()); + }); + admin1.topics().delete(tp, false); + admin2.topics().delete(tp, false); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java index e95b9410f4d12..541408b781be2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java @@ -65,5 +65,6 @@ public void testInitialize() throws Exception { assertEquals(standalone.getConfig().isDispatcherPauseOnAckStatePersistentEnabled(), true); assertEquals(standalone.getConfig().getMaxSecondsToClearTopicNameCache(), 1); assertEquals(standalone.getConfig().getTopicNameCacheMaxCapacity(), 200); + assertEquals(standalone.getConfig().isCreateTopicToRemoteClusterForReplication(), true); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java index c64c54d2d191c..77bb36eb68de1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java @@ -76,6 +76,7 @@ public void testInit() throws Exception { assertEquals(config.isDispatcherPauseOnAckStatePersistentEnabled(), true); assertEquals(config.getMaxSecondsToClearTopicNameCache(), 1); assertEquals(config.getTopicNameCacheMaxCapacity(), 200); + assertEquals(config.isCreateTopicToRemoteClusterForReplication(), false); OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.create(config.getProperties()); assertEquals(offloadPolicies.getManagedLedgerOffloadedReadPriority().getValue(), "bookkeeper-first"); } @@ -293,6 +294,7 @@ public void testTransactionBatchConfigurations() throws Exception{ assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxSize(), 1024 * 1024 * 4); assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxDelayInMillis(), 1); assertEquals(configuration.isDispatcherPauseOnAckStatePersistentEnabled(), false); + assertEquals(configuration.isCreateTopicToRemoteClusterForReplication(), true); } // pulsar_broker_test.conf. try (InputStream inputStream = this.getClass().getClassLoader().getResourceAsStream(fileName)) { @@ -306,6 +308,7 @@ public void testTransactionBatchConfigurations() throws Exception{ assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxSize(), 55); assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxDelayInMillis(), 66); assertEquals(configuration.isDispatcherPauseOnAckStatePersistentEnabled(), true); + assertEquals(configuration.isCreateTopicToRemoteClusterForReplication(), false); } // string input stream. StringBuilder stringBuilder = new StringBuilder(); @@ -318,6 +321,7 @@ public void testTransactionBatchConfigurations() throws Exception{ stringBuilder.append("transactionPendingAckBatchedWriteMaxSize=1025").append(System.lineSeparator()); stringBuilder.append("transactionPendingAckBatchedWriteMaxDelayInMillis=20").append(System.lineSeparator()); stringBuilder.append("dispatcherPauseOnAckStatePersistentEnabled=true").append(System.lineSeparator()); + stringBuilder.append("createTopicToRemoteClusterForReplication=false").append(System.lineSeparator()); try(ByteArrayInputStream inputStream = new ByteArrayInputStream(stringBuilder.toString().getBytes(StandardCharsets.UTF_8))){ configuration = PulsarConfigurationLoader.create(inputStream, ServiceConfiguration.class); @@ -330,6 +334,7 @@ public void testTransactionBatchConfigurations() throws Exception{ assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxSize(), 1025); assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxDelayInMillis(), 20); assertEquals(configuration.isDispatcherPauseOnAckStatePersistentEnabled(), true); + assertEquals(configuration.isCreateTopicToRemoteClusterForReplication(), false); } } diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf index f344a3e3f63da..0fdb29e06866f 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf @@ -106,3 +106,4 @@ transactionPendingAckBatchedWriteMaxSize=55 transactionPendingAckBatchedWriteMaxDelayInMillis=66 topicNameCacheMaxCapacity=200 maxSecondsToClearTopicNameCache=1 +createTopicToRemoteClusterForReplication=false diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf index c520512e77bf9..d3f9430f29b48 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf @@ -97,3 +97,4 @@ maxMessagePublishBufferSizeInMB=-1 dispatcherPauseOnAckStatePersistentEnabled=true topicNameCacheMaxCapacity=200 maxSecondsToClearTopicNameCache=1 +createTopicToRemoteClusterForReplication=true diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index a63ade280efc3..ae28d835fd22f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -391,7 +391,7 @@ private CompletableFuture checkPartitions(String topic, boolean forceNo getPartitionedTopicMetadata(topic, !forceNoPartitioned, true).thenAccept(metadata -> { if (forceNoPartitioned && metadata.partitions > 0) { String errorMsg = String.format("Can not create the producer[%s] for the topic[%s] that contains %s" - + " partitions, but the producer does not support for a partitioned topic.", + + " partitions b,ut the producer does not support for a partitioned topic.", producerNameForLog, topic, metadata.partitions); log.error(errorMsg); checkPartitions.completeExceptionally( From 1c495e190b3c569e9dfd44acef2a697c93a1f771 Mon Sep 17 00:00:00 2001 From: Marek Czajkowski <76772327+marekczajkowski@users.noreply.github.com> Date: Fri, 23 Aug 2024 16:49:15 +0200 Subject: [PATCH 465/580] [feat] PIP-352: Event time based compaction (#22517) --- conf/broker.conf | 6 + conf/standalone.conf | 6 + .../pulsar/client/impl/RawBatchConverter.java | 33 +- .../compaction/AbstractTwoPhaseCompactor.java | 439 ++++++++++++++++ .../pulsar/compaction/CompactorTool.java | 2 +- .../EventTimeCompactionServiceFactory.java | 33 ++ .../compaction/EventTimeOrderCompactor.java | 161 ++++++ .../compaction/MessageCompactionData.java | 23 + .../compaction/PublishingOrderCompactor.java | 127 +++++ .../PulsarCompactionServiceFactory.java | 2 +- .../StrategicTwoPhaseCompactor.java | 2 +- .../pulsar/compaction/TwoPhaseCompactor.java | 470 ------------------ .../compaction/CompactionRetentionTest.java | 4 +- .../pulsar/compaction/CompactionTest.java | 8 +- .../pulsar/compaction/CompactorTest.java | 6 +- .../EventTimeOrderCompactorTest.java | 201 ++++++++ .../compaction/StrategicCompactionTest.java | 2 +- .../TopicCompactionServiceTest.java | 4 +- .../pulsar/io/PulsarFunctionE2ETest.java | 4 +- .../apache/pulsar/io/PulsarSinkE2ETest.java | 4 +- 20 files changed, 1037 insertions(+), 500 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/compaction/AbstractTwoPhaseCompactor.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/compaction/EventTimeCompactionServiceFactory.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/compaction/EventTimeOrderCompactor.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/compaction/MessageCompactionData.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/compaction/PublishingOrderCompactor.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/compaction/EventTimeOrderCompactorTest.java diff --git a/conf/broker.conf b/conf/broker.conf index e5d8a32e7171c..fc32246adea1f 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -563,6 +563,12 @@ brokerServiceCompactionPhaseOneLoopTimeInSeconds=30 # Whether retain null-key message during topic compaction topicCompactionRetainNullKey=false +# Class name of the factory that implements the topic compaction service. +# If value is "org.apache.pulsar.compaction.EventTimeCompactionServiceFactory", +# will create topic compaction service based on message eventTime. +# By default compaction service is based on message publishing order. +compactionServiceFactoryClassName=org.apache.pulsar.compaction.PulsarCompactionServiceFactory + # Whether to enable the delayed delivery for messages. # If disabled, messages will be immediately delivered and there will # be no tracking overhead. diff --git a/conf/standalone.conf b/conf/standalone.conf index 30b39af8869d4..ae696410d86bf 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -1318,3 +1318,9 @@ disableBrokerInterceptors=true # Whether retain null-key message during topic compaction topicCompactionRetainNullKey=false + +# Class name of the factory that implements the topic compaction service. +# If value is "org.apache.pulsar.compaction.EventTimeCompactionServiceFactory", +# will create topic compaction service based on message eventTime. +# By default compaction service is based on message publishing order. +compactionServiceFactoryClassName=org.apache.pulsar.compaction.PulsarCompactionServiceFactory \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java index 4c24f6d303668..f41a7aedd59b2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java @@ -38,6 +38,7 @@ import org.apache.pulsar.common.compression.CompressionCodec; import org.apache.pulsar.common.compression.CompressionCodecProvider; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.compaction.MessageCompactionData; public class RawBatchConverter { @@ -51,8 +52,8 @@ public static boolean isReadableBatch(MessageMetadata metadata) { return metadata.hasNumMessagesInBatch() && metadata.getEncryptionKeysCount() == 0; } - public static List> extractIdsAndKeysAndSize(RawMessage msg) - throws IOException { + public static List extractMessageCompactionData(RawMessage msg) + throws IOException { checkArgument(msg.getMessageIdData().getBatchIndex() == -1); ByteBuf payload = msg.getHeadersAndPayload(); @@ -64,25 +65,35 @@ public static List> extractIdsAndKey int uncompressedSize = metadata.getUncompressedSize(); ByteBuf uncompressedPayload = codec.decode(payload, uncompressedSize); - List> idsAndKeysAndSize = new ArrayList<>(); + List messageCompactionDataList = new ArrayList<>(); SingleMessageMetadata smm = new SingleMessageMetadata(); for (int i = 0; i < batchSize; i++) { ByteBuf singleMessagePayload = Commands.deSerializeSingleMessageInBatch(uncompressedPayload, - smm, - 0, batchSize); + smm, + 0, batchSize); MessageId id = new BatchMessageIdImpl(msg.getMessageIdData().getLedgerId(), - msg.getMessageIdData().getEntryId(), - msg.getMessageIdData().getPartition(), - i); + msg.getMessageIdData().getEntryId(), + msg.getMessageIdData().getPartition(), + i); if (!smm.isCompactedOut()) { - idsAndKeysAndSize.add(ImmutableTriple.of(id, - smm.hasPartitionKey() ? smm.getPartitionKey() : null, - smm.hasPayloadSize() ? smm.getPayloadSize() : 0)); + messageCompactionDataList.add(new MessageCompactionData(id, + smm.hasPartitionKey() ? smm.getPartitionKey() : null, + smm.hasPayloadSize() ? smm.getPayloadSize() : 0, smm.getEventTime())); } singleMessagePayload.release(); } uncompressedPayload.release(); + return messageCompactionDataList; + } + + public static List> extractIdsAndKeysAndSize( + RawMessage msg) + throws IOException { + List> idsAndKeysAndSize = new ArrayList<>(); + for (MessageCompactionData mcd : extractMessageCompactionData(msg)) { + idsAndKeysAndSize.add(ImmutableTriple.of(mcd.messageId(), mcd.key(), mcd.payloadSize())); + } return idsAndKeysAndSize; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/AbstractTwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/AbstractTwoPhaseCompactor.java new file mode 100644 index 0000000000000..5b03f270251a0 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/AbstractTwoPhaseCompactor.java @@ -0,0 +1,439 @@ +/* + * 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.compaction; + +import io.netty.buffer.ByteBuf; +import java.io.IOException; +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.function.BiPredicate; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.mledger.impl.LedgerMetadataUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.RawMessage; +import org.apache.pulsar.client.api.RawReader; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.impl.RawBatchConverter; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.protocol.Markers; +import org.apache.pulsar.common.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Compaction will go through the topic in two passes. The first pass + * selects latest offset for each key in the topic. Then the second pass + * writes these values to a ledger. + * + *

The two passes are required to avoid holding the payloads of each of + * the latest values in memory, as the payload can be many orders of + * magnitude larger than a message id. + */ +public abstract class AbstractTwoPhaseCompactor extends Compactor { + + private static final Logger log = LoggerFactory.getLogger(AbstractTwoPhaseCompactor.class); + protected static final int MAX_OUTSTANDING = 500; + protected final Duration phaseOneLoopReadTimeout; + protected final boolean topicCompactionRetainNullKey; + + public AbstractTwoPhaseCompactor(ServiceConfiguration conf, + PulsarClient pulsar, + BookKeeper bk, + ScheduledExecutorService scheduler) { + super(conf, pulsar, bk, scheduler); + phaseOneLoopReadTimeout = Duration.ofSeconds( + conf.getBrokerServiceCompactionPhaseOneLoopTimeInSeconds()); + topicCompactionRetainNullKey = conf.isTopicCompactionRetainNullKey(); + } + + protected abstract Map toLatestMessageIdForKey(Map latestForKey); + + protected abstract boolean compactMessage(String topic, Map latestForKey, + RawMessage m, MessageId id); + + + protected abstract boolean compactBatchMessage(String topic, Map latestForKey, + RawMessage m, + MessageMetadata metadata, MessageId id); + + @Override + protected CompletableFuture doCompaction(RawReader reader, BookKeeper bk) { + return reader.hasMessageAvailableAsync() + .thenCompose(available -> { + if (available) { + return phaseOne(reader).thenCompose( + (r) -> phaseTwo(reader, r.from, r.to, r.lastReadId, toLatestMessageIdForKey(r.latestForKey), bk)); + } else { + log.info("Skip compaction of the empty topic {}", reader.getTopic()); + return CompletableFuture.completedFuture(-1L); + } + }); + } + + private CompletableFuture phaseOne(RawReader reader) { + Map latestForKey = new HashMap<>(); + CompletableFuture loopPromise = new CompletableFuture<>(); + + reader.getLastMessageIdAsync() + .thenAccept(lastMessageId -> { + log.info("Commencing phase one of compaction for {}, reading to {}", + reader.getTopic(), lastMessageId); + // Each entry is processed as a whole, discard the batchIndex part deliberately. + MessageIdImpl lastImpl = (MessageIdImpl) lastMessageId; + MessageIdImpl lastEntryMessageId = new MessageIdImpl(lastImpl.getLedgerId(), + lastImpl.getEntryId(), + lastImpl.getPartitionIndex()); + phaseOneLoop(reader, Optional.empty(), Optional.empty(), lastEntryMessageId, latestForKey, + loopPromise); + }).exceptionally(ex -> { + loopPromise.completeExceptionally(ex); + return null; + }); + + return loopPromise; + } + + private void phaseOneLoop(RawReader reader, + Optional firstMessageId, + Optional toMessageId, + MessageId lastMessageId, + Map latestForKey, + CompletableFuture loopPromise) { + if (loopPromise.isDone()) { + return; + } + CompletableFuture future = reader.readNextAsync(); + FutureUtil.addTimeoutHandling(future, + phaseOneLoopReadTimeout, scheduler, + () -> FutureUtil.createTimeoutException("Timeout", getClass(), "phaseOneLoop(...)")); + + future.thenAcceptAsync(m -> { + try (m) { + MessageId id = m.getMessageId(); + boolean deletedMessage = false; + mxBean.addCompactionReadOp(reader.getTopic(), m.getHeadersAndPayload().readableBytes()); + MessageMetadata metadata = Commands.parseMessageMetadata(m.getHeadersAndPayload()); + if (Markers.isServerOnlyMarker(metadata)) { + mxBean.addCompactionRemovedEvent(reader.getTopic()); + deletedMessage = true; + } else if (RawBatchConverter.isReadableBatch(metadata)) { + deletedMessage = compactBatchMessage(reader.getTopic(), latestForKey, m, metadata, id); + } else { + deletedMessage = compactMessage(reader.getTopic(), latestForKey, m, id); + } + MessageId first = firstMessageId.orElse(deletedMessage ? null : id); + MessageId to = deletedMessage ? toMessageId.orElse(null) : id; + if (id.compareTo(lastMessageId) == 0) { + loopPromise.complete(new PhaseOneResult(first == null ? id : first, to == null ? id : to, + lastMessageId, latestForKey)); + } else { + phaseOneLoop(reader, + Optional.ofNullable(first), + Optional.ofNullable(to), + lastMessageId, + latestForKey, loopPromise); + } + } + }, scheduler).exceptionally(ex -> { + loopPromise.completeExceptionally(ex); + return null; + }); + } + + private CompletableFuture phaseTwo(RawReader reader, MessageId from, MessageId to, + MessageId lastReadId, + Map latestForKey, BookKeeper bk) { + Map metadata = + LedgerMetadataUtils.buildMetadataForCompactedLedger(reader.getTopic(), to.toByteArray()); + return createLedger(bk, metadata).thenCompose((ledger) -> { + log.info( + "Commencing phase two of compaction for {}, from {} to {}, compacting {} keys to ledger {}", + reader.getTopic(), from, to, latestForKey.size(), ledger.getId()); + return phaseTwoSeekThenLoop(reader, from, to, lastReadId, latestForKey, bk, ledger); + }); + } + + private CompletableFuture phaseTwoSeekThenLoop(RawReader reader, MessageId from, + MessageId to, + MessageId lastReadId, Map latestForKey, BookKeeper bk, + LedgerHandle ledger) { + CompletableFuture promise = new CompletableFuture<>(); + + reader.seekAsync(from).thenCompose((v) -> { + Semaphore outstanding = new Semaphore(MAX_OUTSTANDING); + CompletableFuture loopPromise = new CompletableFuture<>(); + phaseTwoLoop(reader, to, latestForKey, ledger, outstanding, loopPromise, MessageId.earliest); + return loopPromise; + }).thenCompose((v) -> closeLedger(ledger)) + .thenCompose((v) -> reader.acknowledgeCumulativeAsync(lastReadId, + Map.of(COMPACTED_TOPIC_LEDGER_PROPERTY, ledger.getId()))) + .whenComplete((res, exception) -> { + if (exception != null) { + deleteLedger(bk, ledger).whenComplete((res2, exception2) -> { + if (exception2 != null) { + log.warn("Cleanup of ledger {} for failed", ledger, exception2); + } + // complete with original exception + promise.completeExceptionally(exception); + }); + } else { + promise.complete(ledger.getId()); + } + }); + return promise; + } + + private void phaseTwoLoop(RawReader reader, MessageId to, Map latestForKey, + LedgerHandle lh, Semaphore outstanding, CompletableFuture promise, + MessageId lastCompactedMessageId) { + if (promise.isDone()) { + return; + } + reader.readNextAsync().thenAcceptAsync(m -> { + if (promise.isDone()) { + m.close(); + return; + } + + if (m.getMessageId().compareTo(lastCompactedMessageId) <= 0) { + m.close(); + phaseTwoLoop(reader, to, latestForKey, lh, outstanding, promise, lastCompactedMessageId); + return; + } + + try { + MessageId id = m.getMessageId(); + Optional messageToAdd = Optional.empty(); + mxBean.addCompactionReadOp(reader.getTopic(), m.getHeadersAndPayload().readableBytes()); + MessageMetadata metadata = Commands.parseMessageMetadata(m.getHeadersAndPayload()); + if (Markers.isServerOnlyMarker(metadata)) { + messageToAdd = Optional.empty(); + } else if (RawBatchConverter.isReadableBatch(metadata)) { + try { + messageToAdd = rebatchMessage(reader.getTopic(), + m, (key, subid) -> subid.equals(latestForKey.get(key)), + topicCompactionRetainNullKey); + } catch (IOException ioe) { + log.info("Error decoding batch for message {}. Whole batch will be included in output", + id, ioe); + messageToAdd = Optional.of(m); + } + } else { + Pair keyAndSize = extractKeyAndSize(m); + MessageId msg; + if (keyAndSize == null) { + messageToAdd = topicCompactionRetainNullKey ? Optional.of(m) : Optional.empty(); + } else if ((msg = latestForKey.get(keyAndSize.getLeft())) != null + && msg.equals(id)) { // consider message only if present into latestForKey map + if (keyAndSize.getRight() <= 0) { + promise.completeExceptionally(new IllegalArgumentException( + "Compaction phase found empty record from sorted key-map")); + } + messageToAdd = Optional.of(m); + } + } + + if (messageToAdd.isPresent()) { + RawMessage message = messageToAdd.get(); + try { + outstanding.acquire(); + CompletableFuture addFuture = addToCompactedLedger(lh, message, reader.getTopic()) + .whenComplete((res, exception2) -> { + outstanding.release(); + if (exception2 != null) { + promise.completeExceptionally(exception2); + } + }); + if (to.equals(id)) { + // make sure all inflight writes have finished + outstanding.acquire(MAX_OUTSTANDING); + addFuture.whenComplete((res, exception2) -> { + if (exception2 == null) { + promise.complete(null); + } + }); + return; + } + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + promise.completeExceptionally(ie); + } finally { + if (message != m) { + message.close(); + } + } + } else if (to.equals(id)) { + // Reached to last-id and phase-one found it deleted-message while iterating on ledger so, + // not present under latestForKey. Complete the compaction. + try { + // make sure all inflight writes have finished + outstanding.acquire(MAX_OUTSTANDING); + promise.complete(null); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + promise.completeExceptionally(e); + } + return; + } + phaseTwoLoop(reader, to, latestForKey, lh, outstanding, promise, m.getMessageId()); + } finally { + m.close(); + } + }, scheduler).exceptionally(ex -> { + promise.completeExceptionally(ex); + return null; + }); + } + + protected CompletableFuture createLedger(BookKeeper bk, + Map metadata) { + CompletableFuture bkf = new CompletableFuture<>(); + + try { + bk.asyncCreateLedger(conf.getManagedLedgerDefaultEnsembleSize(), + conf.getManagedLedgerDefaultWriteQuorum(), + conf.getManagedLedgerDefaultAckQuorum(), + Compactor.COMPACTED_TOPIC_LEDGER_DIGEST_TYPE, + Compactor.COMPACTED_TOPIC_LEDGER_PASSWORD, + (rc, ledger, ctx) -> { + if (rc != BKException.Code.OK) { + bkf.completeExceptionally(BKException.create(rc)); + } else { + bkf.complete(ledger); + } + }, null, metadata); + } catch (Throwable t) { + log.error("Encountered unexpected error when creating compaction ledger", t); + return FutureUtil.failedFuture(t); + } + return bkf; + } + + protected CompletableFuture deleteLedger(BookKeeper bk, LedgerHandle lh) { + CompletableFuture bkf = new CompletableFuture<>(); + try { + bk.asyncDeleteLedger(lh.getId(), + (rc, ctx) -> { + if (rc != BKException.Code.OK) { + bkf.completeExceptionally(BKException.create(rc)); + } else { + bkf.complete(null); + } + }, null); + } catch (Throwable t) { + return FutureUtil.failedFuture(t); + } + return bkf; + } + + protected CompletableFuture closeLedger(LedgerHandle lh) { + CompletableFuture bkf = new CompletableFuture<>(); + try { + lh.asyncClose((rc, ledger, ctx) -> { + if (rc != BKException.Code.OK) { + bkf.completeExceptionally(BKException.create(rc)); + } else { + bkf.complete(null); + } + }, null); + } catch (Throwable t) { + return FutureUtil.failedFuture(t); + } + return bkf; + } + + private CompletableFuture addToCompactedLedger(LedgerHandle lh, RawMessage m, + String topic) { + CompletableFuture bkf = new CompletableFuture<>(); + ByteBuf serialized = m.serialize(); + try { + mxBean.addCompactionWriteOp(topic, m.getHeadersAndPayload().readableBytes()); + long start = System.nanoTime(); + lh.asyncAddEntry(serialized, + (rc, ledger, eid, ctx) -> { + mxBean.addCompactionLatencyOp(topic, System.nanoTime() - start, TimeUnit.NANOSECONDS); + if (rc != BKException.Code.OK) { + bkf.completeExceptionally(BKException.create(rc)); + } else { + bkf.complete(null); + } + }, null); + } catch (Throwable t) { + return FutureUtil.failedFuture(t); + } + return bkf; + } + + protected Pair extractKeyAndSize(RawMessage m) { + ByteBuf headersAndPayload = m.getHeadersAndPayload(); + MessageMetadata msgMetadata = Commands.parseMessageMetadata(headersAndPayload); + if (msgMetadata.hasPartitionKey()) { + int size = headersAndPayload.readableBytes(); + if (msgMetadata.hasUncompressedSize()) { + size = msgMetadata.getUncompressedSize(); + } + return Pair.of(msgMetadata.getPartitionKey(), size); + } else { + return null; + } + } + + + protected Optional rebatchMessage(String topic, RawMessage msg, + BiPredicate filter, + boolean retainNullKey) + throws IOException { + if (log.isDebugEnabled()) { + log.debug("Rebatching message {} for topic {}", msg.getMessageId(), topic); + } + return RawBatchConverter.rebatchMessage(msg, filter, retainNullKey); + } + + protected static class PhaseOneResult { + + final MessageId from; + final MessageId to; // last undeleted messageId + final MessageId lastReadId; // last read messageId + final Map latestForKey; + + PhaseOneResult(MessageId from, MessageId to, MessageId lastReadId, + Map latestForKey) { + this.from = from; + this.to = to; + this.lastReadId = lastReadId; + this.latestForKey = latestForKey; + } + } + + public long getPhaseOneLoopReadTimeoutInSeconds() { + return phaseOneLoopReadTimeout.getSeconds(); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java index fe77db33692b9..ba68e07cf5b0d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java @@ -172,7 +172,7 @@ public static void main(String[] args) throws Exception { @Cleanup PulsarClient pulsar = createClient(brokerConfig); - Compactor compactor = new TwoPhaseCompactor(brokerConfig, pulsar, bk, scheduler); + Compactor compactor = new PublishingOrderCompactor(brokerConfig, pulsar, bk, scheduler); long ledgerId = compactor.compact(arguments.topic).get(); log.info("Compaction of topic {} complete. Compacted to ledger {}", arguments.topic, ledgerId); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/EventTimeCompactionServiceFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/EventTimeCompactionServiceFactory.java new file mode 100644 index 0000000000000..383c7b1aeedd6 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/EventTimeCompactionServiceFactory.java @@ -0,0 +1,33 @@ +/* + * 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.compaction; + +import org.apache.pulsar.broker.PulsarServerException; +import org.apache.pulsar.broker.PulsarService; + +public class EventTimeCompactionServiceFactory extends PulsarCompactionServiceFactory { + + @Override + protected Compactor newCompactor() throws PulsarServerException { + PulsarService pulsarService = getPulsarService(); + return new EventTimeOrderCompactor(pulsarService.getConfiguration(), + pulsarService.getClient(), pulsarService.getBookKeeperClient(), + pulsarService.getCompactorExecutor()); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/EventTimeOrderCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/EventTimeOrderCompactor.java new file mode 100644 index 0000000000000..2cd19ba15d608 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/EventTimeOrderCompactor.java @@ -0,0 +1,161 @@ +/* + * 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.compaction; + +import io.netty.buffer.ByteBuf; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.RawMessage; +import org.apache.pulsar.client.impl.RawBatchConverter; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.protocol.Commands; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class EventTimeOrderCompactor extends AbstractTwoPhaseCompactor> { + + private static final Logger log = LoggerFactory.getLogger(EventTimeOrderCompactor.class); + + public EventTimeOrderCompactor(ServiceConfiguration conf, + PulsarClient pulsar, + BookKeeper bk, + ScheduledExecutorService scheduler) { + super(conf, pulsar, bk, scheduler); + } + + @Override + protected Map toLatestMessageIdForKey( + Map> latestForKey) { + return latestForKey.entrySet() + .stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + entry -> entry.getValue().getLeft())); + } + + @Override + protected boolean compactMessage(String topic, Map> latestForKey, + RawMessage m, MessageId id) { + boolean deletedMessage = false; + boolean replaceMessage = false; + MessageCompactionData mcd = extractMessageCompactionData(m); + + if (mcd != null) { + boolean newer = Optional.ofNullable(latestForKey.get(mcd.key())) + .map(Pair::getRight) + .map(latestEventTime -> mcd.eventTime() != null + && mcd.eventTime() >= latestEventTime).orElse(true); + if (newer) { + if (mcd.payloadSize() > 0) { + Pair old = latestForKey.put(mcd.key(), + new ImmutablePair<>(mcd.messageId(), mcd.eventTime())); + replaceMessage = old != null; + } else { + deletedMessage = true; + latestForKey.remove(mcd.key()); + } + } + } else { + if (!topicCompactionRetainNullKey) { + deletedMessage = true; + } + } + if (replaceMessage || deletedMessage) { + mxBean.addCompactionRemovedEvent(topic); + } + return deletedMessage; + } + + @Override + protected boolean compactBatchMessage(String topic, Map> latestForKey, RawMessage m, + MessageMetadata metadata, MessageId id) { + boolean deletedMessage = false; + try { + int numMessagesInBatch = metadata.getNumMessagesInBatch(); + int deleteCnt = 0; + + for (MessageCompactionData mcd : extractMessageCompactionDataFromBatch(m)) { + if (mcd.key() == null) { + if (!topicCompactionRetainNullKey) { + // record delete null-key message event + deleteCnt++; + mxBean.addCompactionRemovedEvent(topic); + } + continue; + } + + boolean newer = Optional.ofNullable(latestForKey.get(mcd.key())) + .map(Pair::getRight) + .map(latestEventTime -> mcd.eventTime() != null + && mcd.eventTime() > latestEventTime).orElse(true); + if (newer) { + if (mcd.payloadSize() > 0) { + Pair old = latestForKey.put(mcd.key(), + new ImmutablePair<>(mcd.messageId(), mcd.eventTime())); + if (old != null) { + mxBean.addCompactionRemovedEvent(topic); + } + } else { + latestForKey.remove(mcd.key()); + deleteCnt++; + mxBean.addCompactionRemovedEvent(topic); + } + } + } + + if (deleteCnt == numMessagesInBatch) { + deletedMessage = true; + } + } catch (IOException ioe) { + log.info("Error decoding batch for message {}. Whole batch will be included in output", + id, ioe); + } + return deletedMessage; + } + + protected MessageCompactionData extractMessageCompactionData(RawMessage m) { + ByteBuf headersAndPayload = m.getHeadersAndPayload(); + MessageMetadata msgMetadata = Commands.parseMessageMetadata(headersAndPayload); + if (msgMetadata.hasPartitionKey()) { + int size = headersAndPayload.readableBytes(); + if (msgMetadata.hasUncompressedSize()) { + size = msgMetadata.getUncompressedSize(); + } + return new MessageCompactionData(m.getMessageId(), msgMetadata.getPartitionKey(), + size, msgMetadata.getEventTime()); + } else { + return null; + } + } + + private List extractMessageCompactionDataFromBatch(RawMessage msg) + throws IOException { + return RawBatchConverter.extractMessageCompactionData(msg); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/MessageCompactionData.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/MessageCompactionData.java new file mode 100644 index 0000000000000..03800273a806e --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/MessageCompactionData.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. + */ +package org.apache.pulsar.compaction; + +import org.apache.pulsar.client.api.MessageId; + +public record MessageCompactionData (MessageId messageId, String key, Integer payloadSize, Long eventTime) {} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PublishingOrderCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PublishingOrderCompactor.java new file mode 100644 index 0000000000000..a825c0782fbf9 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PublishingOrderCompactor.java @@ -0,0 +1,127 @@ +/* + * 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.compaction; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.commons.lang3.tuple.ImmutableTriple; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.RawMessage; +import org.apache.pulsar.client.impl.RawBatchConverter; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class PublishingOrderCompactor extends AbstractTwoPhaseCompactor { + + private static final Logger log = LoggerFactory.getLogger(PublishingOrderCompactor.class); + + public PublishingOrderCompactor(ServiceConfiguration conf, + PulsarClient pulsar, + BookKeeper bk, + ScheduledExecutorService scheduler) { + super(conf, pulsar, bk, scheduler); + } + + @Override + protected Map toLatestMessageIdForKey(Map latestForKey) { + return latestForKey; + } + + @Override + protected boolean compactMessage(String topic, Map latestForKey, + RawMessage m, MessageId id) { + boolean deletedMessage = false; + boolean replaceMessage = false; + Pair keyAndSize = extractKeyAndSize(m); + if (keyAndSize != null) { + if (keyAndSize.getRight() > 0) { + MessageId old = latestForKey.put(keyAndSize.getLeft(), id); + replaceMessage = old != null; + } else { + deletedMessage = true; + latestForKey.remove(keyAndSize.getLeft()); + } + } else { + if (!topicCompactionRetainNullKey) { + deletedMessage = true; + } + } + if (replaceMessage || deletedMessage) { + mxBean.addCompactionRemovedEvent(topic); + } + return deletedMessage; + } + + @Override + protected boolean compactBatchMessage(String topic, Map latestForKey, + RawMessage m, MessageMetadata metadata, MessageId id) { + boolean deletedMessage = false; + try { + int numMessagesInBatch = metadata.getNumMessagesInBatch(); + int deleteCnt = 0; + for (ImmutableTriple e : extractIdsAndKeysAndSizeFromBatch( + m)) { + if (e != null) { + if (e.getMiddle() == null) { + if (!topicCompactionRetainNullKey) { + // record delete null-key message event + deleteCnt++; + mxBean.addCompactionRemovedEvent(topic); + } + continue; + } + if (e.getRight() > 0) { + MessageId old = latestForKey.put(e.getMiddle(), e.getLeft()); + if (old != null) { + mxBean.addCompactionRemovedEvent(topic); + } + } else { + latestForKey.remove(e.getMiddle()); + deleteCnt++; + mxBean.addCompactionRemovedEvent(topic); + } + } + } + if (deleteCnt == numMessagesInBatch) { + deletedMessage = true; + } + } catch (IOException ioe) { + log.info( + "Error decoding batch for message {}. Whole batch will be included in output", + id, ioe); + } + + return deletedMessage; + } + + protected List> extractIdsAndKeysAndSizeFromBatch( + RawMessage msg) + throws IOException { + return RawBatchConverter.extractIdsAndKeysAndSize(msg); + } + +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PulsarCompactionServiceFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PulsarCompactionServiceFactory.java index 424733ad58158..90132461b4c4a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PulsarCompactionServiceFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PulsarCompactionServiceFactory.java @@ -54,7 +54,7 @@ public Compactor getNullableCompactor() { } protected Compactor newCompactor() throws PulsarServerException { - return new TwoPhaseCompactor(pulsarService.getConfiguration(), + return new PublishingOrderCompactor(pulsarService.getConfiguration(), pulsarService.getClient(), pulsarService.getBookKeeperClient(), pulsarService.getCompactorExecutor()); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java index fefa2ee959cc5..1b54092d9aa4f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java @@ -59,7 +59,7 @@ *

As the first pass caches the entire message(not just offset) for each key into a map, * this compaction could be memory intensive if the message payload is large. */ -public class StrategicTwoPhaseCompactor extends TwoPhaseCompactor { +public class StrategicTwoPhaseCompactor extends PublishingOrderCompactor { private static final Logger log = LoggerFactory.getLogger(StrategicTwoPhaseCompactor.class); private static final int MAX_OUTSTANDING = 500; private static final int MAX_READER_RECONNECT_WAITING_TIME_IN_MILLIS = 20 * 1000; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java deleted file mode 100644 index 647c34a94ad81..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java +++ /dev/null @@ -1,470 +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.compaction; - -import io.netty.buffer.ByteBuf; -import java.io.IOException; -import java.time.Duration; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; -import java.util.function.BiPredicate; -import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.client.BookKeeper; -import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.mledger.impl.LedgerMetadataUtils; -import org.apache.commons.lang3.tuple.ImmutableTriple; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.RawMessage; -import org.apache.pulsar.client.api.RawReader; -import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.client.impl.RawBatchConverter; -import org.apache.pulsar.common.api.proto.MessageMetadata; -import org.apache.pulsar.common.protocol.Commands; -import org.apache.pulsar.common.protocol.Markers; -import org.apache.pulsar.common.util.FutureUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Compaction will go through the topic in two passes. The first pass - * selects latest offset for each key in the topic. Then the second pass - * writes these values to a ledger. - * - *

The two passes are required to avoid holding the payloads of each of - * the latest values in memory, as the payload can be many orders of - * magnitude larger than a message id. -*/ -public class TwoPhaseCompactor extends Compactor { - private static final Logger log = LoggerFactory.getLogger(TwoPhaseCompactor.class); - private static final int MAX_OUTSTANDING = 500; - private final Duration phaseOneLoopReadTimeout; - private final boolean topicCompactionRetainNullKey; - - public TwoPhaseCompactor(ServiceConfiguration conf, - PulsarClient pulsar, - BookKeeper bk, - ScheduledExecutorService scheduler) { - super(conf, pulsar, bk, scheduler); - phaseOneLoopReadTimeout = Duration.ofSeconds(conf.getBrokerServiceCompactionPhaseOneLoopTimeInSeconds()); - topicCompactionRetainNullKey = conf.isTopicCompactionRetainNullKey(); - } - - @Override - protected CompletableFuture doCompaction(RawReader reader, BookKeeper bk) { - return reader.hasMessageAvailableAsync() - .thenCompose(available -> { - if (available) { - return phaseOne(reader).thenCompose( - (r) -> phaseTwo(reader, r.from, r.to, r.lastReadId, r.latestForKey, bk)); - } else { - log.info("Skip compaction of the empty topic {}", reader.getTopic()); - return CompletableFuture.completedFuture(-1L); - } - }); - } - - private CompletableFuture phaseOne(RawReader reader) { - Map latestForKey = new HashMap<>(); - CompletableFuture loopPromise = new CompletableFuture<>(); - - reader.getLastMessageIdAsync() - .thenAccept(lastMessageId -> { - log.info("Commencing phase one of compaction for {}, reading to {}", - reader.getTopic(), lastMessageId); - // Each entry is processed as a whole, discard the batchIndex part deliberately. - MessageIdImpl lastImpl = (MessageIdImpl) lastMessageId; - MessageIdImpl lastEntryMessageId = new MessageIdImpl(lastImpl.getLedgerId(), lastImpl.getEntryId(), - lastImpl.getPartitionIndex()); - phaseOneLoop(reader, Optional.empty(), Optional.empty(), lastEntryMessageId, latestForKey, - loopPromise); - }).exceptionally(ex -> { - loopPromise.completeExceptionally(ex); - return null; - }); - - return loopPromise; - } - - private void phaseOneLoop(RawReader reader, - Optional firstMessageId, - Optional toMessageId, - MessageId lastMessageId, - Map latestForKey, - CompletableFuture loopPromise) { - if (loopPromise.isDone()) { - return; - } - CompletableFuture future = reader.readNextAsync(); - FutureUtil.addTimeoutHandling(future, - phaseOneLoopReadTimeout, scheduler, - () -> FutureUtil.createTimeoutException("Timeout", getClass(), "phaseOneLoop(...)")); - - future.thenAcceptAsync(m -> { - try (m) { - MessageId id = m.getMessageId(); - boolean deletedMessage = false; - boolean replaceMessage = false; - mxBean.addCompactionReadOp(reader.getTopic(), m.getHeadersAndPayload().readableBytes()); - MessageMetadata metadata = Commands.parseMessageMetadata(m.getHeadersAndPayload()); - if (Markers.isServerOnlyMarker(metadata)) { - mxBean.addCompactionRemovedEvent(reader.getTopic()); - deletedMessage = true; - } else if (RawBatchConverter.isReadableBatch(metadata)) { - try { - int numMessagesInBatch = metadata.getNumMessagesInBatch(); - int deleteCnt = 0; - for (ImmutableTriple e : extractIdsAndKeysAndSizeFromBatch(m)) { - if (e != null) { - if (e.getMiddle() == null) { - if (!topicCompactionRetainNullKey) { - // record delete null-key message event - deleteCnt++; - mxBean.addCompactionRemovedEvent(reader.getTopic()); - } - continue; - } - if (e.getRight() > 0) { - MessageId old = latestForKey.put(e.getMiddle(), e.getLeft()); - if (old != null) { - mxBean.addCompactionRemovedEvent(reader.getTopic()); - } - } else { - latestForKey.remove(e.getMiddle()); - deleteCnt++; - mxBean.addCompactionRemovedEvent(reader.getTopic()); - } - } - } - if (deleteCnt == numMessagesInBatch) { - deletedMessage = true; - } - } catch (IOException ioe) { - log.info("Error decoding batch for message {}. Whole batch will be included in output", - id, ioe); - } - } else { - Pair keyAndSize = extractKeyAndSize(m); - if (keyAndSize != null) { - if (keyAndSize.getRight() > 0) { - MessageId old = latestForKey.put(keyAndSize.getLeft(), id); - replaceMessage = old != null; - } else { - deletedMessage = true; - latestForKey.remove(keyAndSize.getLeft()); - } - } else { - if (!topicCompactionRetainNullKey) { - deletedMessage = true; - } - } - if (replaceMessage || deletedMessage) { - mxBean.addCompactionRemovedEvent(reader.getTopic()); - } - } - MessageId first = firstMessageId.orElse(deletedMessage ? null : id); - MessageId to = deletedMessage ? toMessageId.orElse(null) : id; - if (id.compareTo(lastMessageId) == 0) { - loopPromise.complete(new PhaseOneResult(first == null ? id : first, to == null ? id : to, - lastMessageId, latestForKey)); - } else { - phaseOneLoop(reader, - Optional.ofNullable(first), - Optional.ofNullable(to), - lastMessageId, - latestForKey, loopPromise); - } - } - }, scheduler).exceptionally(ex -> { - loopPromise.completeExceptionally(ex); - return null; - }); - } - - private CompletableFuture phaseTwo(RawReader reader, MessageId from, MessageId to, MessageId lastReadId, - Map latestForKey, BookKeeper bk) { - Map metadata = - LedgerMetadataUtils.buildMetadataForCompactedLedger(reader.getTopic(), to.toByteArray()); - return createLedger(bk, metadata).thenCompose((ledger) -> { - log.info("Commencing phase two of compaction for {}, from {} to {}, compacting {} keys to ledger {}", - reader.getTopic(), from, to, latestForKey.size(), ledger.getId()); - return phaseTwoSeekThenLoop(reader, from, to, lastReadId, latestForKey, bk, ledger); - }); - } - - private CompletableFuture phaseTwoSeekThenLoop(RawReader reader, MessageId from, MessageId to, - MessageId lastReadId, Map latestForKey, BookKeeper bk, LedgerHandle ledger) { - CompletableFuture promise = new CompletableFuture<>(); - - reader.seekAsync(from).thenCompose((v) -> { - Semaphore outstanding = new Semaphore(MAX_OUTSTANDING); - CompletableFuture loopPromise = new CompletableFuture<>(); - phaseTwoLoop(reader, to, latestForKey, ledger, outstanding, loopPromise, MessageId.earliest); - return loopPromise; - }).thenCompose((v) -> closeLedger(ledger)) - .thenCompose((v) -> reader.acknowledgeCumulativeAsync(lastReadId, - Map.of(COMPACTED_TOPIC_LEDGER_PROPERTY, ledger.getId()))) - .whenComplete((res, exception) -> { - if (exception != null) { - deleteLedger(bk, ledger).whenComplete((res2, exception2) -> { - if (exception2 != null) { - log.warn("Cleanup of ledger {} for failed", ledger, exception2); - } - // complete with original exception - promise.completeExceptionally(exception); - }); - } else { - promise.complete(ledger.getId()); - } - }); - return promise; - } - - private void phaseTwoLoop(RawReader reader, MessageId to, Map latestForKey, - LedgerHandle lh, Semaphore outstanding, CompletableFuture promise, - MessageId lastCompactedMessageId) { - if (promise.isDone()) { - return; - } - reader.readNextAsync().thenAcceptAsync(m -> { - if (promise.isDone()) { - m.close(); - return; - } - - if (m.getMessageId().compareTo(lastCompactedMessageId) <= 0) { - m.close(); - phaseTwoLoop(reader, to, latestForKey, lh, outstanding, promise, lastCompactedMessageId); - return; - } - - try { - MessageId id = m.getMessageId(); - Optional messageToAdd = Optional.empty(); - mxBean.addCompactionReadOp(reader.getTopic(), m.getHeadersAndPayload().readableBytes()); - MessageMetadata metadata = Commands.parseMessageMetadata(m.getHeadersAndPayload()); - if (Markers.isServerOnlyMarker(metadata)) { - messageToAdd = Optional.empty(); - } else if (RawBatchConverter.isReadableBatch(metadata)) { - try { - messageToAdd = rebatchMessage(reader.getTopic(), - m, (key, subid) -> subid.equals(latestForKey.get(key)), topicCompactionRetainNullKey); - } catch (IOException ioe) { - log.info("Error decoding batch for message {}. Whole batch will be included in output", - id, ioe); - messageToAdd = Optional.of(m); - } - } else { - Pair keyAndSize = extractKeyAndSize(m); - MessageId msg; - if (keyAndSize == null) { - messageToAdd = topicCompactionRetainNullKey ? Optional.of(m) : Optional.empty(); - } else if ((msg = latestForKey.get(keyAndSize.getLeft())) != null - && msg.equals(id)) { // consider message only if present into latestForKey map - if (keyAndSize.getRight() <= 0) { - promise.completeExceptionally(new IllegalArgumentException( - "Compaction phase found empty record from sorted key-map")); - } - messageToAdd = Optional.of(m); - } - } - - if (messageToAdd.isPresent()) { - RawMessage message = messageToAdd.get(); - try { - outstanding.acquire(); - CompletableFuture addFuture = addToCompactedLedger(lh, message, reader.getTopic()) - .whenComplete((res, exception2) -> { - outstanding.release(); - if (exception2 != null) { - promise.completeExceptionally(exception2); - } - }); - if (to.equals(id)) { - // make sure all inflight writes have finished - outstanding.acquire(MAX_OUTSTANDING); - addFuture.whenComplete((res, exception2) -> { - if (exception2 == null) { - promise.complete(null); - } - }); - return; - } - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - promise.completeExceptionally(ie); - } finally { - if (message != m) { - message.close(); - } - } - } else if (to.equals(id)) { - // Reached to last-id and phase-one found it deleted-message while iterating on ledger so, - // not present under latestForKey. Complete the compaction. - try { - // make sure all inflight writes have finished - outstanding.acquire(MAX_OUTSTANDING); - promise.complete(null); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - promise.completeExceptionally(e); - } - return; - } - phaseTwoLoop(reader, to, latestForKey, lh, outstanding, promise, m.getMessageId()); - } finally { - m.close(); - } - }, scheduler).exceptionally(ex -> { - promise.completeExceptionally(ex); - return null; - }); - } - - protected CompletableFuture createLedger(BookKeeper bk, Map metadata) { - CompletableFuture bkf = new CompletableFuture<>(); - - try { - bk.asyncCreateLedger(conf.getManagedLedgerDefaultEnsembleSize(), - conf.getManagedLedgerDefaultWriteQuorum(), - conf.getManagedLedgerDefaultAckQuorum(), - Compactor.COMPACTED_TOPIC_LEDGER_DIGEST_TYPE, - Compactor.COMPACTED_TOPIC_LEDGER_PASSWORD, - (rc, ledger, ctx) -> { - if (rc != BKException.Code.OK) { - bkf.completeExceptionally(BKException.create(rc)); - } else { - bkf.complete(ledger); - } - }, null, metadata); - } catch (Throwable t) { - log.error("Encountered unexpected error when creating compaction ledger", t); - return FutureUtil.failedFuture(t); - } - return bkf; - } - - protected CompletableFuture deleteLedger(BookKeeper bk, LedgerHandle lh) { - CompletableFuture bkf = new CompletableFuture<>(); - try { - bk.asyncDeleteLedger(lh.getId(), - (rc, ctx) -> { - if (rc != BKException.Code.OK) { - bkf.completeExceptionally(BKException.create(rc)); - } else { - bkf.complete(null); - } - }, null); - } catch (Throwable t) { - return FutureUtil.failedFuture(t); - } - return bkf; - } - - protected CompletableFuture closeLedger(LedgerHandle lh) { - CompletableFuture bkf = new CompletableFuture<>(); - try { - lh.asyncClose((rc, ledger, ctx) -> { - if (rc != BKException.Code.OK) { - bkf.completeExceptionally(BKException.create(rc)); - } else { - bkf.complete(null); - } - }, null); - } catch (Throwable t) { - return FutureUtil.failedFuture(t); - } - return bkf; - } - - private CompletableFuture addToCompactedLedger(LedgerHandle lh, RawMessage m, String topic) { - CompletableFuture bkf = new CompletableFuture<>(); - ByteBuf serialized = m.serialize(); - try { - mxBean.addCompactionWriteOp(topic, m.getHeadersAndPayload().readableBytes()); - long start = System.nanoTime(); - lh.asyncAddEntry(serialized, - (rc, ledger, eid, ctx) -> { - mxBean.addCompactionLatencyOp(topic, System.nanoTime() - start, TimeUnit.NANOSECONDS); - if (rc != BKException.Code.OK) { - bkf.completeExceptionally(BKException.create(rc)); - } else { - bkf.complete(null); - } - }, null); - } catch (Throwable t) { - return FutureUtil.failedFuture(t); - } - return bkf; - } - - protected Pair extractKeyAndSize(RawMessage m) { - ByteBuf headersAndPayload = m.getHeadersAndPayload(); - MessageMetadata msgMetadata = Commands.parseMessageMetadata(headersAndPayload); - if (msgMetadata.hasPartitionKey()) { - int size = headersAndPayload.readableBytes(); - if (msgMetadata.hasUncompressedSize()) { - size = msgMetadata.getUncompressedSize(); - } - return Pair.of(msgMetadata.getPartitionKey(), size); - } else { - return null; - } - } - - protected List> extractIdsAndKeysAndSizeFromBatch(RawMessage msg) - throws IOException { - return RawBatchConverter.extractIdsAndKeysAndSize(msg); - } - - protected Optional rebatchMessage(String topic, RawMessage msg, BiPredicate filter, - boolean retainNullKey) - throws IOException { - if (log.isDebugEnabled()) { - log.debug("Rebatching message {} for topic {}", msg.getMessageId(), topic); - } - return RawBatchConverter.rebatchMessage(msg, filter, retainNullKey); - } - - private static class PhaseOneResult { - final MessageId from; - final MessageId to; // last undeleted messageId - final MessageId lastReadId; // last read messageId - final Map latestForKey; - - PhaseOneResult(MessageId from, MessageId to, MessageId lastReadId, Map latestForKey) { - this.from = from; - this.to = to; - this.lastReadId = lastReadId; - this.latestForKey = latestForKey; - } - } - - public long getPhaseOneLoopReadTimeoutInSeconds() { - return phaseOneLoopReadTimeout.getSeconds(); - } -} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java index ac1ba6bc814b1..45dc30d21df64 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java @@ -62,7 +62,7 @@ public class CompactionRetentionTest extends MockedPulsarServiceBaseTest { protected ScheduledExecutorService compactionScheduler; protected BookKeeper bk; - private TwoPhaseCompactor compactor; + private PublishingOrderCompactor compactor; @BeforeMethod @Override @@ -79,7 +79,7 @@ public void setup() throws Exception { compactionScheduler = Executors.newSingleThreadScheduledExecutor( new ThreadFactoryBuilder().setNameFormat("compaction-%d").setDaemon(true).build()); bk = pulsar.getBookKeeperClientFactory().create(this.conf, null, null, Optional.empty(), null).get(); - compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); + compactor = new PublishingOrderCompactor(conf, pulsarClient, bk, compactionScheduler); } @AfterMethod(alwaysRun = true) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java index 0cf32859e3dd6..19f42a7e0570f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java @@ -109,7 +109,7 @@ public class CompactionTest extends MockedPulsarServiceBaseTest { protected ScheduledExecutorService compactionScheduler; protected BookKeeper bk; - private TwoPhaseCompactor compactor; + private PublishingOrderCompactor compactor; @BeforeMethod @Override @@ -124,7 +124,7 @@ public void setup() throws Exception { compactionScheduler = Executors.newSingleThreadScheduledExecutor( new ThreadFactoryBuilder().setNameFormat("compaction-%d").setDaemon(true).build()); bk = pulsar.getBookKeeperClientFactory().create(this.conf, null, null, Optional.empty(), null).get(); - compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); + compactor = new PublishingOrderCompactor(conf, pulsarClient, bk, compactionScheduler); } @AfterMethod(alwaysRun = true) @@ -147,7 +147,7 @@ protected long compact(String topic, CryptoKeyReader cryptoKeyReader) return compactor.compact(topic).get(); } - protected TwoPhaseCompactor getCompactor() { + protected PublishingOrderCompactor getCompactor() { return compactor; } @@ -656,7 +656,7 @@ public static Object[][] retainNullKey() { public void testKeyLessMessagesPassThrough(boolean retainNullKey) throws Exception { conf.setTopicCompactionRetainNullKey(retainNullKey); restartBroker(); - FieldUtils.writeDeclaredField(compactor, "topicCompactionRetainNullKey", retainNullKey, true); + FieldUtils.writeField(compactor, "topicCompactionRetainNullKey", retainNullKey, true); String topic = "persistent://my-property/use/my-ns/my-topic1"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java index 1c09dc0d6434c..5cf7d33200d66 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java @@ -101,7 +101,7 @@ public void setup() throws Exception { new ThreadFactoryBuilder().setNameFormat("compactor").setDaemon(true).build()); bk = pulsar.getBookKeeperClientFactory().create( this.conf, null, null, Optional.empty(), null).get(); - compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); + compactor = new PublishingOrderCompactor(conf, pulsarClient, bk, compactionScheduler); } @@ -127,7 +127,7 @@ protected Compactor getCompactor() { return compactor; } - private List compactAndVerify(String topic, Map expected, boolean checkMetrics) + protected List compactAndVerify(String topic, Map expected, boolean checkMetrics) throws Exception { long compactedLedgerId = compact(topic); @@ -361,7 +361,7 @@ public void testPhaseOneLoopTimeConfiguration() { PulsarClientImpl mockClient = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); when(mockClient.getCnxPool()).thenReturn(connectionPool); - TwoPhaseCompactor compactor = new TwoPhaseCompactor(configuration, mockClient, + PublishingOrderCompactor compactor = new PublishingOrderCompactor(configuration, mockClient, Mockito.mock(BookKeeper.class), compactionScheduler); Assert.assertEquals(compactor.getPhaseOneLoopReadTimeoutInSeconds(), 60); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/EventTimeOrderCompactorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/EventTimeOrderCompactorTest.java new file mode 100644 index 0000000000000..8fba0983123ee --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/EventTimeOrderCompactorTest.java @@ -0,0 +1,201 @@ +/* + * 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.compaction; + +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricDoubleSumValue; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import io.opentelemetry.api.common.Attributes; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Reader; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.ConnectionPool; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; +import org.awaitility.Awaitility; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker-compaction") +public class EventTimeOrderCompactorTest extends CompactorTest { + + private EventTimeOrderCompactor compactor; + + @BeforeMethod + @Override + public void setup() throws Exception { + super.setup(); + compactor = new EventTimeOrderCompactor(conf, pulsarClient, bk, compactionScheduler); + } + + @Override + protected long compact(String topic) throws ExecutionException, InterruptedException { + return compactor.compact(topic).get(); + } + + @Override + protected Compactor getCompactor() { + return compactor; + } + + @Test + public void testCompactedOutByEventTime() throws Exception { + String topicName = BrokerTestUtil.newUniqueName("persistent://my-property/use/my-ns/testCompactedOutByEventTime"); + this.restartBroker(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .enableBatching(true).topic(topicName).batchingMaxMessages(3).create(); + + producer.newMessage().key("K1").value("V1").eventTime(1L).sendAsync(); + producer.newMessage().key("K2").value("V2").eventTime(1L).sendAsync(); + producer.newMessage().key("K2").value(null).eventTime(2L).sendAsync(); + producer.flush(); + + admin.topics().triggerCompaction(topicName); + + Awaitility.await().untilAsserted(() -> { + Assert.assertEquals(admin.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + var attributes = Attributes.builder() + .put(OpenTelemetryAttributes.PULSAR_DOMAIN, "persistent") + .put(OpenTelemetryAttributes.PULSAR_TENANT, "my-property") + .put(OpenTelemetryAttributes.PULSAR_NAMESPACE, "my-property/use/my-ns") + .put(OpenTelemetryAttributes.PULSAR_TOPIC, topicName) + .build(); + var metrics = pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_REMOVED_COUNTER, attributes, 1); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_OPERATION_COUNTER, Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_COMPACTION_STATUS, "success") + .build(), + 1); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_OPERATION_COUNTER, Attributes.builder() + .putAll(attributes) + .put(OpenTelemetryAttributes.PULSAR_COMPACTION_STATUS, "failure") + .build(), + 0); + assertMetricDoubleSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_DURATION_SECONDS, attributes, + actual -> assertThat(actual).isPositive()); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_BYTES_IN_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_BYTES_OUT_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_ENTRIES_COUNTER, attributes, 1); + assertMetricLongSumValue(metrics, OpenTelemetryTopicStats.COMPACTION_BYTES_COUNTER, attributes, + actual -> assertThat(actual).isPositive()); + + producer.newMessage().key("K1").eventTime(2L).value("V1-2").sendAsync(); + producer.flush(); + + admin.topics().triggerCompaction(topicName); + + Awaitility.await().untilAsserted(() -> { + Assert.assertEquals(admin.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + @Cleanup + Reader reader = pulsarClient.newReader(Schema.STRING) + .subscriptionName("reader-test") + .topic(topicName) + .readCompacted(true) + .startMessageId(MessageId.earliest) + .create(); + while (reader.hasMessageAvailable()) { + Message message = reader.readNext(3, TimeUnit.SECONDS); + Assert.assertEquals(message.getEventTime(), 2L); + } + } + + @Test + public void testCompactWithEventTimeAddCompact() throws Exception { + String topic = "persistent://my-property/use/my-ns/my-topic1"; + + @Cleanup + Producer producer = pulsarClient.newProducer().topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + + Map expected = new HashMap<>(); + + producer.newMessage() + .key("a") + .eventTime(1L) + .value("A_1".getBytes()) + .send(); + producer.newMessage() + .key("b") + .eventTime(1L) + .value("B_1".getBytes()) + .send(); + producer.newMessage() + .key("a") + .eventTime(2L) + .value("A_2".getBytes()) + .send(); + expected.put("a", "A_2".getBytes()); + expected.put("b", "B_1".getBytes()); + + compactAndVerify(topic, new HashMap<>(expected), false); + + producer.newMessage() + .key("b") + .eventTime(2L) + .value("B_2".getBytes()) + .send(); + expected.put("b", "B_2".getBytes()); + + compactAndVerify(topic, expected, false); + } + + @Override + @Test + public void testPhaseOneLoopTimeConfiguration() { + ServiceConfiguration configuration = new ServiceConfiguration(); + configuration.setBrokerServiceCompactionPhaseOneLoopTimeInSeconds(60); + PulsarClientImpl mockClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(mockClient.getCnxPool()).thenReturn(connectionPool); + EventTimeOrderCompactor compactor = new EventTimeOrderCompactor(configuration, mockClient, + Mockito.mock(BookKeeper.class), compactionScheduler); + Assert.assertEquals(compactor.getPhaseOneLoopReadTimeoutInSeconds(), 60); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java index 54563431052eb..d1ff46cbc02d5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java @@ -74,7 +74,7 @@ protected long compact(String topic, CryptoKeyReader cryptoKeyReader) } @Override - protected TwoPhaseCompactor getCompactor() { + protected PublishingOrderCompactor getCompactor() { return compactor; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/TopicCompactionServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/TopicCompactionServiceTest.java index 2aa09309d3931..9f33479ce4cab 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/TopicCompactionServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/TopicCompactionServiceTest.java @@ -53,7 +53,7 @@ public class TopicCompactionServiceTest extends MockedPulsarServiceBaseTest { protected ScheduledExecutorService compactionScheduler; protected BookKeeper bk; - private TwoPhaseCompactor compactor; + private PublishingOrderCompactor compactor; @BeforeMethod @Override @@ -73,7 +73,7 @@ public void setup() throws Exception { new ThreadFactoryBuilder().setNameFormat("compactor").setDaemon(true).build()); bk = pulsar.getBookKeeperClientFactory().create( this.conf, null, null, Optional.empty(), null).get(); - compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); + compactor = new PublishingOrderCompactor(conf, pulsarClient, bk, compactionScheduler); } @AfterMethod(alwaysRun = true) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java index ab4925bfeb8de..74c2a93b84e9f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionE2ETest.java @@ -62,7 +62,7 @@ import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TopicStats; -import org.apache.pulsar.compaction.TwoPhaseCompactor; +import org.apache.pulsar.compaction.PublishingOrderCompactor; import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.utils.FunctionCommon; @@ -259,7 +259,7 @@ public void testReadCompactedFunction() throws Exception { @Cleanup("shutdownNow") ScheduledExecutorService compactionScheduler = Executors.newSingleThreadScheduledExecutor( new ThreadFactoryBuilder().setNameFormat("compactor").setDaemon(true).build()); - TwoPhaseCompactor twoPhaseCompactor = new TwoPhaseCompactor(config, + PublishingOrderCompactor twoPhaseCompactor = new PublishingOrderCompactor(config, pulsarClient, pulsar.getBookKeeperClient(), compactionScheduler); twoPhaseCompactor.compact(sourceTopic).get(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSinkE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSinkE2ETest.java index 7edc87bb996d4..be2b377a9cff5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSinkE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSinkE2ETest.java @@ -49,7 +49,7 @@ import org.apache.pulsar.common.policies.data.SinkStatus; import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TopicStats; -import org.apache.pulsar.compaction.TwoPhaseCompactor; +import org.apache.pulsar.compaction.PublishingOrderCompactor; import org.apache.pulsar.functions.LocalRunner; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.utils.FunctionCommon; @@ -107,7 +107,7 @@ public void testReadCompactedSink() throws Exception { @Cleanup("shutdownNow") ScheduledExecutorService compactionScheduler = Executors.newSingleThreadScheduledExecutor( new ThreadFactoryBuilder().setNameFormat("compactor").setDaemon(true).build()); - TwoPhaseCompactor twoPhaseCompactor = new TwoPhaseCompactor(config, + PublishingOrderCompactor twoPhaseCompactor = new PublishingOrderCompactor(config, pulsarClient, pulsar.getBookKeeperClient(), compactionScheduler); twoPhaseCompactor.compact(sourceTopic).get(); From 09a16c26974408de270bcaaf6162b0e2a9a6d203 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 26 Aug 2024 10:05:21 +0800 Subject: [PATCH 466/580] [improve] [broker] Part 2 of PIP-370: add metrics "pulsar_replication_disconnected_count" (#23213) --- pip/pip-370.md | 7 +- .../prometheus/AggregatedNamespaceStats.java | 1 + .../AggregatedReplicationStats.java | 3 + .../prometheus/NamespaceStatsAggregator.java | 8 +- .../broker/stats/prometheus/TopicStats.java | 2 + .../broker/service/OneWayReplicatorTest.java | 121 ++++++++++++++++++ .../OneWayReplicatorUsingGlobalZKTest.java | 6 + .../AggregatedNamespaceStatsTest.java | 2 + 8 files changed, 145 insertions(+), 5 deletions(-) diff --git a/pip/pip-370.md b/pip/pip-370.md index 6699846cee105..a29d556143200 100644 --- a/pip/pip-370.md +++ b/pip/pip-370.md @@ -85,10 +85,9 @@ For each metric provide: * Attributes (labels) * Unit --> -| Name | Description | Attributes | Units| -| --- | --- | --- | --- | -| `pulsar_broker_replication_count` | Counter. The number of topics enabled replication. | cluster | - | -| `pulsar_broker_replication_disconnected_count` | Counter. The number of topics that enabled replication and its replicator failed to connect | cluster | - | +| Name | Description | Attributes | Units| +| --- |---------------------------------------------------------------------------------------------|---------------------------| --- | +| `pulsar_replication_disconnected_count` | Counter. The number of replicators. | cluster, namespace, topic | - | # Monitoring diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java index 3975cd89cfa6b..85ff15c915aa7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java @@ -133,6 +133,7 @@ void updateStats(TopicStats stats) { replStats.replicationBacklog += as.replicationBacklog; replStats.msgRateExpired += as.msgRateExpired; replStats.connectedCount += as.connectedCount; + replStats.disconnectedCount += as.disconnectedCount; replStats.replicationDelayInSeconds += as.replicationDelayInSeconds; }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedReplicationStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedReplicationStats.java index 78f33f874e998..82668de6c35f7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedReplicationStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedReplicationStats.java @@ -41,6 +41,9 @@ public class AggregatedReplicationStats { /** The count of replication-subscriber up and running to replicate to remote cluster. */ public long connectedCount; + /** The count of replication-subscriber that failed to start to replicate to remote cluster. */ + public long disconnectedCount; + /** Time in seconds from the time a message was produced to the time when it is about to be replicated. */ public long replicationDelayInSeconds; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java index a229ef54c795d..f0d11167e65fe 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java @@ -303,7 +303,11 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include aggReplStats.msgThroughputOut += replStats.msgThroughputOut; aggReplStats.replicationBacklog += replStats.replicationBacklog; aggReplStats.msgRateExpired += replStats.msgRateExpired; - aggReplStats.connectedCount += replStats.connected ? 1 : 0; + if (replStats.connected) { + aggReplStats.connectedCount += 1; + } else { + aggReplStats.disconnectedCount += 1; + } aggReplStats.replicationDelayInSeconds += replStats.replicationDelayInSeconds; }); @@ -510,6 +514,8 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat replStats -> replStats.replicationBacklog, cluster, namespace); writeReplicationStat(stream, "pulsar_replication_connected_count", stats, replStats -> replStats.connectedCount, cluster, namespace); + writeReplicationStat(stream, "pulsar_replication_disconnected_count", stats, + replStats -> replStats.disconnectedCount, cluster, namespace); writeReplicationStat(stream, "pulsar_replication_rate_expired", stats, replStats -> replStats.msgRateExpired, cluster, namespace); writeReplicationStat(stream, "pulsar_replication_delay_in_seconds", stats, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java index 9eb4077225ca1..013b528731060 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java @@ -408,6 +408,8 @@ public static void printTopicStats(PrometheusMetricStreams stream, TopicStats st cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); writeMetric(stream, "pulsar_replication_connected_count", replStats.connectedCount, cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_replication_disconnected_count", replStats.disconnectedCount, + cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); writeMetric(stream, "pulsar_replication_rate_expired", replStats.msgRateExpired, cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); writeMetric(stream, "pulsar_replication_delay_in_seconds", replStats.replicationDelayInSeconds, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index 1745d4dc90f3b..74604dd990c54 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -31,6 +31,7 @@ import static org.testng.Assert.fail; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import io.netty.util.concurrent.FastThreadLocalThread; import java.lang.reflect.Field; @@ -67,6 +68,7 @@ import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -91,6 +93,8 @@ import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; +import org.glassfish.jersey.client.JerseyClient; +import org.glassfish.jersey.client.JerseyClientBuilder; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -1160,4 +1164,121 @@ public void testDifferentTopicCreationRule(ReplicationMode replicationMode) thro admin1.namespaces().deleteNamespace(ns); admin2.namespaces().deleteNamespace(ns); } + + @Test + public void testReplicationCountMetrics() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); + // 1.Create topic, does not enable replication now. + admin1.topics().createNonPartitionedTopic(topicName); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + + // We inject an error to make the internal producer fail to connect. + final AtomicInteger createProducerCounter = new AtomicInteger(); + final AtomicBoolean failedCreateProducer = new AtomicBoolean(true); + Runnable taskToClearInjection = injectMockReplicatorProducerBuilder((producerCnf, originalProducer) -> { + if (topicName.equals(producerCnf.getTopicName())) { + // There is a switch to determine create producer successfully or not. + if (failedCreateProducer.get()) { + log.info("Retry create replicator.producer count: {}", createProducerCounter); + // Release producer and fail callback. + originalProducer.closeAsync(); + throw new RuntimeException("mock error"); + } + return originalProducer; + } + return originalProducer; + }); + + // 2.Enable replication. + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); + + // Verify: metrics. + // Cluster level: + // - pulsar_replication_connected_count + // - pulsar_replication_disconnected_count + // Namespace level: + // - pulsar_replication_connected_count + // - pulsar_replication_disconnected_count + // Topic level: + // - pulsar_replication_connected_count + // - pulsar_replication_disconnected_count + JerseyClient httpClient = JerseyClientBuilder.createClient(); + Awaitility.await().untilAsserted(() -> { + int topicConnected = 0; + int topicDisconnected = 0; + + String response = httpClient.target(pulsar1.getWebServiceAddress()).path("/metrics/") + .request().get(String.class); + Multimap metricMap = PrometheusMetricsClient.parseMetrics(response); + if (!metricMap.containsKey("pulsar_replication_disconnected_count")) { + fail("Expected 1 disconnected replicator."); + } + for (PrometheusMetricsClient.Metric metric : metricMap.get("pulsar_replication_connected_count")) { + if (cluster1.equals(metric.tags.get("cluster")) + && nonReplicatedNamespace.equals(metric.tags.get("namespace")) + && topicName.equals(metric.tags.get("topic"))) { + topicConnected += Double.valueOf(metric.value).intValue(); + } + } + for (PrometheusMetricsClient.Metric metric : metricMap.get("pulsar_replication_disconnected_count")) { + if (cluster1.equals(metric.tags.get("cluster")) + && nonReplicatedNamespace.equals(metric.tags.get("namespace")) + && topicName.equals(metric.tags.get("topic"))) { + topicDisconnected += Double.valueOf(metric.value).intValue(); + } + } + log.info("{}, {},", topicConnected, topicDisconnected); + assertEquals(topicConnected, 0); + assertEquals(topicDisconnected, 1); + }); + + // Let replicator connect successfully. + failedCreateProducer.set(false); + // Verify: metrics. + // Cluster level: + // - pulsar_replication_connected_count + // - pulsar_replication_disconnected_count + // Namespace level: + // - pulsar_replication_connected_count + // - pulsar_replication_disconnected_count + // Topic level: + // - pulsar_replication_connected_count + // - pulsar_replication_disconnected_count + Awaitility.await().atMost(Duration.ofSeconds(130)).untilAsserted(() -> { + int topicConnected = 0; + int topicDisconnected = 0; + + String response = httpClient.target(pulsar1.getWebServiceAddress()).path("/metrics/") + .request().get(String.class); + Multimap metricMap = PrometheusMetricsClient.parseMetrics(response); + if (!metricMap.containsKey("pulsar_replication_disconnected_count")) { + fail("Expected 1 disconnected replicator."); + } + for (PrometheusMetricsClient.Metric metric : metricMap.get("pulsar_replication_connected_count")) { + if (cluster1.equals(metric.tags.get("cluster")) + && nonReplicatedNamespace.equals(metric.tags.get("namespace")) + && topicName.equals(metric.tags.get("topic"))) { + topicConnected += Double.valueOf(metric.value).intValue(); + } + } + for (PrometheusMetricsClient.Metric metric : metricMap.get("pulsar_replication_disconnected_count")) { + if (cluster1.equals(metric.tags.get("cluster")) + && nonReplicatedNamespace.equals(metric.tags.get("namespace")) + && topicName.equals(metric.tags.get("topic"))) { + topicDisconnected += Double.valueOf(metric.value).intValue(); + } + } + log.info("{}, {}", topicConnected, topicDisconnected); + assertEquals(topicConnected, 1); + assertEquals(topicDisconnected, 0); + }); + + // cleanup. + taskToClearInjection.run(); + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); + waitReplicatorStopped(topicName); + admin1.topics().delete(topicName, false); + admin2.topics().delete(topicName, false); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java index 34810bbe9057b..d99969fbaa7e5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java @@ -167,4 +167,10 @@ public void testConfigReplicationStartAt() throws Exception { public void testDifferentTopicCreationRule(ReplicationMode replicationMode) throws Exception { super.testDifferentTopicCreationRule(replicationMode); } + + @Test(enabled = false) + @Override + public void testReplicationCountMetrics() throws Exception { + super.testReplicationCountMetrics(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java index 0e12d75f74fa0..11358eb1e2c1c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java @@ -101,6 +101,7 @@ public void testSimpleAggregation() { replStats2.msgThroughputOut = 1536.0; replStats2.replicationBacklog = 99; replStats2.connectedCount = 1; + replStats2.disconnectedCount = 2; replStats2.msgRateExpired = 3.0; replStats2.replicationDelayInSeconds = 20; topicStats2.replicationStats.put(namespace, replStats2); @@ -148,6 +149,7 @@ public void testSimpleAggregation() { assertEquals(nsReplStats.msgThroughputOut, 1792.0); assertEquals(nsReplStats.replicationBacklog, 100); assertEquals(nsReplStats.connectedCount, 1); + assertEquals(nsReplStats.disconnectedCount, 2); assertEquals(nsReplStats.msgRateExpired, 6.0); assertEquals(nsReplStats.replicationDelayInSeconds, 40); From a6029ad2e1b63553f6a68f91cd65ed4be963a051 Mon Sep 17 00:00:00 2001 From: guan46 <48884472+guan46@users.noreply.github.com> Date: Mon, 26 Aug 2024 18:37:38 +0800 Subject: [PATCH 467/580] [cleanup][test]delete invalid method in PrometheusMetricsTest (#23218) Co-authored-by: account guanyue --- .../apache/pulsar/broker/stats/PrometheusMetricsTest.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index 81c0acba44046..e7f86d542a074 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -1960,13 +1960,6 @@ public void testSplitTopicAndPartitionLabel() throws Exception { consumer2.close(); } - private void compareCompactionStateCount(List cm, double count) { - assertEquals(cm.size(), 1); - assertEquals(cm.get(0).tags.get("cluster"), "test"); - assertEquals(cm.get(0).tags.get("broker"), "localhost"); - assertEquals(cm.get(0).value, count); - } - @Test public void testMetricsGroupedByTypeDefinitions() throws Exception { Producer p1 = pulsarClient.newProducer().topic("persistent://my-property/use/my-ns/my-topic1").create(); From cd3519aea7c9f341e40b1343112b0b7d41a6c508 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov <8622884+dlg99@users.noreply.github.com> Date: Mon, 26 Aug 2024 13:32:58 -0700 Subject: [PATCH 468/580] [fix] StatsOutputStream: add string write function (#308) (#23227) Co-authored-by: Paul Gier --- .../pulsar/common/util/SimpleTextOutputStream.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java index 9bf6302f50f02..d3f319bd958ba 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java @@ -56,6 +56,15 @@ public SimpleTextOutputStream write(char c) { return this; } + public SimpleTextOutputStream write(String s) { + if (s == null) { + return this; + } + + buffer.writeCharSequence(s, CharsetUtil.UTF_8); + return this; + } + public SimpleTextOutputStream write(CharSequence s) { if (s == null) { return this; From d9bd6b004edf3aa8d170205fde024bff97ee05ce Mon Sep 17 00:00:00 2001 From: "Canwei.Luo" Date: Tue, 27 Aug 2024 20:00:33 +0800 Subject: [PATCH 469/580] [improve][broker]Change the log level to reduce repeated error logs (#23192) --- .../pulsar/broker/loadbalance/LinuxInfoUtils.java | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java index 9cf861a8e85cf..b63f0fe85b20c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Locale; import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.Stream; import lombok.AllArgsConstructor; @@ -54,7 +55,8 @@ public class LinuxInfoUtils { // NIC type private static final int ARPHRD_ETHER = 1; private static final String NIC_SPEED_TEMPLATE = "/sys/class/net/%s/speed"; - + private static final long errLogPrintedFrequencyInReadingNicLimits = 1000; + private static final AtomicLong failedCounterInReadingNicLimits = new AtomicLong(0); private static Object /*jdk.internal.platform.Metrics*/ metrics; private static Method getMetricsProviderMethod; private static Method getCpuQuotaMethod; @@ -251,7 +253,15 @@ public static double getTotalNicLimit(List nics, BitRateUnit bitRateUnit try { return readDoubleFromFile(getReplacedNICPath(NIC_SPEED_TEMPLATE, nicPath)); } catch (IOException e) { - log.error("[LinuxInfo] Failed to get total nic limit.", e); + // ERROR-level logs about NIC rate limiting reading failures are periodically printed but not + // continuously printed + if (failedCounterInReadingNicLimits.getAndIncrement() % errLogPrintedFrequencyInReadingNicLimits == 0) { + log.error("[LinuxInfo] Failed to get the nic limit of {}.", nicPath, e); + } else { + if (log.isDebugEnabled()) { + log.debug("[LinuxInfo] Failed to get the nic limit of {}.", nicPath, e); + } + } return 0d; } }).sum(), BitRateUnit.Megabit); From 9a97c843a46e23a0811e2172991cd00a3af642c0 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 28 Aug 2024 09:56:05 +0800 Subject: [PATCH 470/580] [feat][broker] PIP-368: Support lookup based on the lookup properties (#23223) PIP: https://github.com/apache/pulsar/pull/23075 ### Motivation This is the implementation for the PIP: https://github.com/apache/pulsar/pull/23075 Currently, the lookup process uses only the topic name as its parameter. However, to enhance this process, it's beneficial for clients to provide additional information. This could be done by introducing the `lookupProperties` field in the client configuration. Clients can then share these properties with the broker during lookup. On the broker side, the broker could also contain some properties that are used for the lookup. We can also support the lookupProperties for the broker. The broker can use these properties to make a better decision on which broker to return. Here is the rack-aware lookup scenario for using the client properties for the lookup: Assuming there are two brokers that broker-0 configures the lookup property "rack" with "A" and broker-1 configures the lookup property "rack" with "B". By using the lookup properties, clients can supply rack information during the lookup, enabling the broker to identify and connect them to the nearest broker within the same rack. If a client that configures the "rack" property with "A" connects to a lookup broker, the customized load manager can determine broker-0 as the owner broker since the broker and the client have the same rack property. ### Modifications - Add new configuration `lookupProperties` to the client. While looking up the broker, the client will send the properties to the broker through `CommandLookupTopic` request. - Add `properties` field to the `CommandLookupTopic`. - Add `lookupProperties` to the `LookupOptions`. The Load Manager implementation can access the `properties` through `LookupOptions` to make a better decision on which broker to return. - Introduce a new broker configuration `lookupPropertyPrefix`. Any broker configuration properties that start with the `lookupPropertyPrefix` will be included into the `BrokerLookupData` and be persisted in the metadata store. The broker can use these properties during the lookup. Co-authored-by: Yunze Xu --- .../pulsar/broker/ServiceConfiguration.java | 19 +++ .../extensions/BrokerRegistryImpl.java | 3 +- .../extensions/data/BrokerLookupData.java | 3 +- .../pulsar/broker/lookup/TopicLookupBase.java | 8 +- .../broker/namespace/LookupOptions.java | 2 + .../pulsar/broker/service/ServerCnx.java | 13 ++- .../extensions/data/BrokerLookupDataTest.java | 4 +- .../filter/BrokerFilterTestBase.java | 3 +- .../BrokerIsolationPoliciesFilterTest.java | 3 +- .../manager/RedirectManagerTest.java | 4 +- .../scheduler/TransferShedderTest.java | 3 +- .../client/api/LookupPropertiesTest.java | 110 ++++++++++++++++++ .../naming/ServiceConfigurationTest.java | 14 +++ .../pulsar/client/api/ClientBuilder.java | 12 ++ .../client/impl/BinaryProtoLookupService.java | 3 +- .../pulsar/client/impl/ClientBuilderImpl.java | 6 + .../impl/conf/ClientConfigurationData.java | 11 ++ .../impl/BinaryProtoLookupServiceTest.java | 2 + .../pulsar/common/protocol/Commands.java | 8 +- pulsar-common/src/main/proto/PulsarApi.proto | 2 + 20 files changed, 220 insertions(+), 13 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/LookupPropertiesTest.java 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 c836879b075f1..6488ace991e2f 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 @@ -21,9 +21,11 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Properties; @@ -2946,6 +2948,13 @@ public double getLoadBalancerBandwidthOutResourceWeight() { @com.fasterxml.jackson.annotation.JsonIgnore private Properties properties = new Properties(); + @FieldContext( + category = CATEGORY_SERVER, + doc = "The properties whose name starts with this prefix will be uploaded to the metadata store for " + + " the topic lookup" + ) + private String lookupPropertyPrefix = "lookup."; + @FieldContext( dynamic = true, category = CATEGORY_SERVER, @@ -3743,4 +3752,14 @@ public int getTopicOrderedExecutorThreadNum() { public boolean isSystemTopicAndTopicLevelPoliciesEnabled() { return topicLevelPoliciesEnabled && systemTopicEnabled; } + + public Map lookupProperties() { + final var map = new HashMap(); + properties.forEach((key, value) -> { + if (key instanceof String && value instanceof String && ((String) key).startsWith(lookupPropertyPrefix)) { + map.put((String) key, (String) value); + } + }); + return map; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java index 18e30ddf922d0..5db11d40c33ff 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java @@ -94,7 +94,8 @@ public BrokerRegistryImpl(PulsarService pulsar) { pulsar.getConfiguration().isEnableNonPersistentTopics(), conf.getLoadManagerClassName(), System.currentTimeMillis(), - pulsar.getBrokerVersion()); + pulsar.getBrokerVersion(), + pulsar.getConfig().lookupProperties()); this.state = State.Init; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java index 50a2b70404039..5d982076bd609 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java @@ -41,7 +41,8 @@ public record BrokerLookupData (String webServiceUrl, boolean nonPersistentTopicsEnabled, String loadManagerClassName, long startTimestamp, - String brokerVersion) implements ServiceLookupData { + String brokerVersion, + Map properties) implements ServiceLookupData { @Override public String getWebServiceUrl() { return this.webServiceUrl(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java index 9a05c3d992aaf..42f145d32aab1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/lookup/TopicLookupBase.java @@ -24,6 +24,8 @@ import java.net.InetAddress; import java.net.URI; import java.net.URISyntaxException; +import java.util.Collections; +import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import javax.ws.rs.Encoded; @@ -180,7 +182,7 @@ protected String internalGetNamespaceBundle(TopicName topicName) { public static CompletableFuture lookupTopicAsync(PulsarService pulsarService, TopicName topicName, boolean authoritative, String clientAppId, AuthenticationDataSource authenticationData, long requestId) { return lookupTopicAsync(pulsarService, topicName, authoritative, clientAppId, - authenticationData, requestId, null); + authenticationData, requestId, null, Collections.emptyMap()); } /** @@ -208,7 +210,8 @@ public static CompletableFuture lookupTopicAsync(PulsarService pulsarSe public static CompletableFuture lookupTopicAsync(PulsarService pulsarService, TopicName topicName, boolean authoritative, String clientAppId, AuthenticationDataSource authenticationData, - long requestId, final String advertisedListenerName) { + long requestId, final String advertisedListenerName, + Map properties) { final CompletableFuture validationFuture = new CompletableFuture<>(); final CompletableFuture lookupfuture = new CompletableFuture<>(); @@ -299,6 +302,7 @@ public static CompletableFuture lookupTopicAsync(PulsarService pulsarSe .authoritative(authoritative) .advertisedListenerName(advertisedListenerName) .loadTopicsInBundle(true) + .properties(properties) .build(); pulsarService.getNamespaceService().getBrokerServiceUrlAsync(topicName, options) .thenAccept(lookupResult -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/LookupOptions.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/LookupOptions.java index 431266682c51c..be5450646329d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/LookupOptions.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/LookupOptions.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.namespace; +import java.util.Map; import lombok.Builder; import lombok.Data; import org.apache.commons.lang3.StringUtils; @@ -46,6 +47,7 @@ public class LookupOptions { private final boolean requestHttps; private final String advertisedListenerName; + private final Map properties; public boolean hasAdvertisedListenerName() { return StringUtils.isNotBlank(advertisedListenerName); 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 2f9e9b2a1ac2d..d1fe9776e079d 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 @@ -47,6 +47,7 @@ import java.net.SocketAddress; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.IdentityHashMap; import java.util.List; import java.util.Map; @@ -544,9 +545,19 @@ protected void handleLookup(CommandLookupTopic lookup) { isTopicOperationAllowed(topicName, TopicOperation.LOOKUP, authenticationData, originalAuthData).thenApply( isAuthorized -> { if (isAuthorized) { + final Map properties; + if (lookup.getPropertiesCount() > 0) { + properties = new HashMap<>(); + for (int i = 0; i < lookup.getPropertiesCount(); i++) { + final var keyValue = lookup.getPropertyAt(i); + properties.put(keyValue.getKey(), keyValue.getValue()); + } + } else { + properties = Collections.emptyMap(); + } lookupTopicAsync(getBrokerService().pulsar(), topicName, authoritative, getPrincipal(), getAuthenticationData(), - requestId, advertisedListenerName).handle((lookupResponse, ex) -> { + requestId, advertisedListenerName, properties).handle((lookupResponse, ex) -> { if (ex == null) { writeAndFlush(lookupResponse); } else { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java index 66e8c917d1fc5..0a9742fd76175 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java @@ -24,6 +24,7 @@ import java.net.URI; import java.net.URISyntaxException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -58,7 +59,8 @@ public void testConstructors() throws PulsarServerException, URISyntaxException BrokerLookupData lookupData = new BrokerLookupData( webServiceUrl, webServiceUrlTls, pulsarServiceUrl, pulsarServiceUrlTls, advertisedListeners, protocols, true, true, - ExtensibleLoadManagerImpl.class.getName(), System.currentTimeMillis(),"3.0"); + ExtensibleLoadManagerImpl.class.getName(), System.currentTimeMillis(),"3.0", + Collections.emptyMap()); assertEquals(webServiceUrl, lookupData.webServiceUrl()); assertEquals(webServiceUrlTls, lookupData.webServiceUrlTls()); assertEquals(pulsarServiceUrl, lookupData.pulsarServiceUrl()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerFilterTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerFilterTestBase.java index a120ef473e9a5..ab0065e0aa5ba 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerFilterTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerFilterTestBase.java @@ -22,6 +22,7 @@ import static org.mockito.Mockito.mock; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -136,6 +137,6 @@ public BrokerLookupData getLookupData(String version, String loadManagerClassNam return new BrokerLookupData( webServiceUrl, webServiceUrlTls, pulsarServiceUrl, pulsarServiceUrlTls, advertisedListeners, protocols, true, true, - loadManagerClassName, -1, version); + loadManagerClassName, -1, version, Collections.emptyMap()); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerIsolationPoliciesFilterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerIsolationPoliciesFilterTest.java index 87aaf4bac7fae..d3553bd25d1fa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerIsolationPoliciesFilterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerIsolationPoliciesFilterTest.java @@ -28,6 +28,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Set; @@ -218,7 +219,7 @@ public BrokerLookupData getLookupData(boolean persistentTopicsEnabled, webServiceUrl, webServiceUrlTls, pulsarServiceUrl, pulsarServiceUrlTls, advertisedListeners, protocols, persistentTopicsEnabled, nonPersistentTopicsEnabled, - ExtensibleLoadManagerImpl.class.getName(), System.currentTimeMillis(), "3.0.0"); + ExtensibleLoadManagerImpl.class.getName(), System.currentTimeMillis(), "3.0.0", Collections.emptyMap()); } public LoadManagerContext getContext() { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/RedirectManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/RedirectManagerTest.java index cbf77b59d5ad6..f2e9cf86868e2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/RedirectManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/RedirectManagerTest.java @@ -33,6 +33,8 @@ import org.apache.pulsar.broker.lookup.LookupResult; import org.apache.pulsar.policies.data.loadbalancer.AdvertisedListener; import org.testng.annotations.Test; + +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -106,6 +108,6 @@ public BrokerLookupData getLookupData(String broker, String loadManagerClassName return new BrokerLookupData( webServiceUrl, webServiceUrlTls, pulsarServiceUrl, pulsarServiceUrlTls, advertisedListeners, protocols, true, true, - loadManagerClassName, startTimeStamp, "3.0.0"); + loadManagerClassName, startTimeStamp, "3.0.0", Collections.emptyMap()); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java index efca2880949f2..48bef15b5f80a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/scheduler/TransferShedderTest.java @@ -46,6 +46,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -697,7 +698,7 @@ public BrokerLookupData getLookupData() { webServiceUrl, webServiceUrlTls, pulsarServiceUrl, pulsarServiceUrlTls, advertisedListeners, protocols, true, true, - conf.getLoadManagerClassName(), System.currentTimeMillis(), "3.0.0"); + conf.getLoadManagerClassName(), System.currentTimeMillis(), "3.0.0", Collections.emptyMap()); } private void setIsolationPolicies(SimpleResourceAllocationPolicies policies, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/LookupPropertiesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/LookupPropertiesTest.java new file mode 100644 index 0000000000000..cb8b2d1e526af --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/LookupPropertiesTest.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.api; + +import java.net.InetSocketAddress; +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.MultiBrokerBaseTest; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.broker.namespace.LookupOptions; +import org.apache.pulsar.client.impl.PartitionedProducerImpl; +import org.apache.pulsar.client.impl.ProducerImpl; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.naming.ServiceUnitId; +import org.testng.Assert; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class LookupPropertiesTest extends MultiBrokerBaseTest { + + private static final String BROKER_KEY = "lookup.broker.id"; + private static final String CLIENT_KEY = "broker.id"; + + @Override + protected void startBroker() throws Exception { + addCustomConfigs(conf, 0); + super.startBroker(); + } + + @Override + protected ServiceConfiguration createConfForAdditionalBroker(int additionalBrokerIndex) { + return addCustomConfigs(getDefaultConf(), additionalBrokerIndex + 10); + } + + private static ServiceConfiguration addCustomConfigs(ServiceConfiguration config, int index) { + config.setDefaultNumberOfNamespaceBundles(16); + config.setLoadBalancerAutoBundleSplitEnabled(false); + config.setLoadManagerClassName(BrokerIdAwareLoadManager.class.getName()); + config.setLoadBalancerAverageResourceUsageDifferenceThresholdPercentage(100); + config.setLoadBalancerDebugModeEnabled(true); + config.setBrokerShutdownTimeoutMs(1000); + final var properties = new Properties(); + properties.setProperty(BROKER_KEY, "broker-" + index); + config.setProperties(properties); + return config; + } + + @Test + public void testLookupProperty() throws Exception { + final var topic = "test-lookup-property"; + admin.topics().createPartitionedTopic(topic, 16); + @Cleanup final var client = (PulsarClientImpl) PulsarClient.builder() + .serviceUrl(pulsar.getBrokerServiceUrl()) + .lookupProperties( + Collections.singletonMap(CLIENT_KEY, "broker-10")) // broker-10 refers to additionalBrokers[0] + .build(); + @Cleanup final var producer = (PartitionedProducerImpl) client.newProducer().topic(topic).create(); + Assert.assertNotNull(producer); + final var connections = producer.getProducers().stream().map(ProducerImpl::getClientCnx) + .collect(Collectors.toSet()); + Assert.assertEquals(connections.size(), 1); + final var port = ((InetSocketAddress) connections.stream().findAny().orElseThrow().ctx().channel() + .remoteAddress()).getPort(); + Assert.assertEquals(port, additionalBrokers.get(0).getBrokerListenPort().orElseThrow()); + } + + public static class BrokerIdAwareLoadManager extends ExtensibleLoadManagerImpl { + @Override + public CompletableFuture> selectAsync(ServiceUnitId bundle, Set excludeBrokerSet, + LookupOptions options) { + final var clientId = options.getProperties() == null ? null : options.getProperties().get(CLIENT_KEY); + if (clientId == null) { + return super.selectAsync(bundle, excludeBrokerSet, options); + } + return getBrokerRegistry().getAvailableBrokerLookupDataAsync().thenCompose(brokerLookupDataMap -> { + final var optBroker = brokerLookupDataMap.entrySet().stream().filter(entry -> { + final var brokerId = entry.getValue().properties().get(BROKER_KEY); + return brokerId != null && brokerId.equals(clientId); + }).findAny(); + return optBroker.map(Map.Entry::getKey).map(Optional::of).map(CompletableFuture::completedFuture) + .orElseGet(() -> super.selectAsync(bundle, excludeBrokerSet, options)); + }); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java index 77bb36eb68de1..5972c6f724d8c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java @@ -35,6 +35,7 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Properties; @@ -393,4 +394,17 @@ public void testTopicNameCacheConfiguration() throws Exception { assertEquals(conf.getMaxSecondsToClearTopicNameCache(), 2); assertEquals(conf.getTopicNameCacheMaxCapacity(), 100); } + + @Test + public void testLookupProperties() throws Exception { + var confFile = "lookup.key1=value1\nkey=value\nlookup.key2=value2"; + var conf = (ServiceConfiguration) PulsarConfigurationLoader.create( + new ByteArrayInputStream(confFile.getBytes()), ServiceConfiguration.class); + assertEquals(conf.lookupProperties(), Map.of("lookup.key1", "value1", "lookup.key2", "value2")); + + confFile = confFile + "\nlookupPropertyPrefix=lookup.key2"; + conf = PulsarConfigurationLoader.create(new ByteArrayInputStream(confFile.getBytes()), + ServiceConfiguration.class); + assertEquals(conf.lookupProperties(), Map.of("lookup.key2", "value2")); + } } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java index 4adf7d89b0e33..73ad555165c05 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ClientBuilder.java @@ -668,4 +668,16 @@ ClientBuilder authentication(String authPluginClassName, Map aut * @return the client builder instance */ ClientBuilder autoCertRefreshSeconds(int autoCertRefreshSeconds); + + /** + * Set the properties used for topic lookup. + *

+ * When the broker performs topic lookup, these lookup properties will be taken into consideration in a customized + * load manager. + *

+ * Note: The lookup properties are only used in topic lookup when: + * - The protocol is binary protocol, i.e. the service URL starts with "pulsar://" or "pulsar+ssl://" + * - The `loadManagerClassName` config in broker is a class that implements the `ExtensibleLoadManager` interface + */ + ClientBuilder lookupProperties(Map properties); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 6ee6fafde1c25..9dd04acce7ee3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -178,7 +178,8 @@ private CompletableFuture findBroker(InetSocketAddress socket client.getCnxPool().getConnection(socketAddress).thenAccept(clientCnx -> { long requestId = client.newRequestId(); - ByteBuf request = Commands.newLookup(topicName.toString(), listenerName, authoritative, requestId); + ByteBuf request = Commands.newLookup(topicName.toString(), listenerName, authoritative, requestId, + client.getConfiguration().getLookupProperties()); clientCnx.newLookup(request, requestId).whenComplete((r, t) -> { if (t != null) { // lookup failed diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java index d9edc53b50e37..6923218676743 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientBuilderImpl.java @@ -476,4 +476,10 @@ public ClientBuilder description(String description) { conf.setDescription(description); return this; } + + @Override + public ClientBuilder lookupProperties(Map properties) { + conf.setLookupProperties(properties); + return this; + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java index e2713644af641..c1c2e75925502 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java @@ -28,6 +28,7 @@ import java.net.URI; import java.time.Clock; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -412,6 +413,8 @@ public class ClientConfigurationData implements Serializable, Cloneable { ) private String description; + private Map lookupProperties; + private transient OpenTelemetry openTelemetry; /** @@ -477,4 +480,12 @@ public String getSocks5ProxyUsername() { public String getSocks5ProxyPassword() { return Objects.nonNull(socks5ProxyPassword) ? socks5ProxyPassword : System.getProperty("socks5Proxy.password"); } + + public void setLookupProperties(Map lookupProperties) { + this.lookupProperties = Collections.unmodifiableMap(lookupProperties); + } + + public Map getLookupProperties() { + return (lookupProperties == null) ? Collections.emptyMap() : Collections.unmodifiableMap(lookupProperties); + } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java index 983cd21a7a9d8..f691215b04e08 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BinaryProtoLookupServiceTest.java @@ -70,6 +70,8 @@ public void setup() throws Exception { doReturn(cnxPool).when(client).getCnxPool(); doReturn(clientConfig).when(client).getConfiguration(); doReturn(1L).when(client).newRequestId(); + ClientConfigurationData data = new ClientConfigurationData(); + doReturn(data).when(client).getConfiguration(); lookup = spy( new BinaryProtoLookupService(client, "pulsar://localhost:6650", false, mock(ExecutorService.class))); 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 224e093baf112..3fb2fd5ad3d25 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 @@ -936,10 +936,11 @@ public static ByteBuf newPartitionMetadataResponse(int partitions, long requestI } public static ByteBuf newLookup(String topic, boolean authoritative, long requestId) { - return newLookup(topic, null, authoritative, requestId); + return newLookup(topic, null, authoritative, requestId, null); } - public static ByteBuf newLookup(String topic, String listenerName, boolean authoritative, long requestId) { + public static ByteBuf newLookup(String topic, String listenerName, boolean authoritative, long requestId, + Map properties) { BaseCommand cmd = localCmd(Type.LOOKUP); CommandLookupTopic lookup = cmd.setLookupTopic() .setTopic(topic) @@ -948,6 +949,9 @@ public static ByteBuf newLookup(String topic, String listenerName, boolean autho if (StringUtils.isNotBlank(listenerName)) { lookup.setAdvertisedListenerName(listenerName); } + if (properties != null) { + properties.forEach((key, value) -> lookup.addProperty().setKey(key).setValue(value)); + } return serializeWithSize(cmd); } diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto index 5067ed64079c9..19658c5e57ff9 100644 --- a/pulsar-common/src/main/proto/PulsarApi.proto +++ b/pulsar-common/src/main/proto/PulsarApi.proto @@ -446,6 +446,8 @@ message CommandLookupTopic { optional string original_auth_method = 6; // optional string advertised_listener_name = 7; + // The properties used for topic lookup + repeated KeyValue properties = 8; } message CommandLookupTopicResponse { From 325c6a58d53b9e7b4fe31883ec47ae12c5abc71f Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 28 Aug 2024 19:26:45 +0800 Subject: [PATCH 471/580] [fix][broker] Fix thread unsafe access on the bundle range cache for load manager (#23217) --- .../loadbalance/impl/BundleRangeCache.java | 84 +++++++++++++++++++ .../loadbalance/impl/LoadManagerShared.java | 73 ++++------------ .../impl/ModularLoadManagerImpl.java | 40 ++------- .../impl/SimpleLoadManagerImpl.java | 33 ++------ .../AntiAffinityNamespaceGroupTest.java | 33 ++------ .../impl/LoadManagerSharedTest.java | 45 +++------- 6 files changed, 135 insertions(+), 173 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/BundleRangeCache.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/BundleRangeCache.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/BundleRangeCache.java new file mode 100644 index 0000000000000..5cb92682232a5 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/BundleRangeCache.java @@ -0,0 +1,84 @@ +/* + * 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.loadbalance.impl; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Stream; + +/** + * The cache for the bundle ranges. + * The first key is the broker id and the second key is the namespace name, the value is the set of bundle ranges of + * that namespace. When the broker key is accessed if the associated value is not present, an empty map will be created + * as the initial value that will never be removed. + * Therefore, for each broker, there could only be one internal map during the whole lifetime. Then it will be safe + * to apply the synchronized key word on the value for thread safe operations. + */ +public class BundleRangeCache { + + // Map from brokers to namespaces to the bundle ranges in that namespace assigned to that broker. + // Used to distribute bundles within a namespace evenly across brokers. + private final Map>> data = new ConcurrentHashMap<>(); + + public void reloadFromBundles(String broker, Stream bundles) { + final var namespaceToBundleRange = data.computeIfAbsent(broker, __ -> new HashMap<>()); + synchronized (namespaceToBundleRange) { + namespaceToBundleRange.clear(); + bundles.forEach(bundleName -> { + final String namespace = LoadManagerShared.getNamespaceNameFromBundleName(bundleName); + final String bundleRange = LoadManagerShared.getBundleRangeFromBundleName(bundleName); + namespaceToBundleRange.computeIfAbsent(namespace, __ -> new HashSet<>()).add(bundleRange); + }); + } + } + + public void add(String broker, String namespace, String bundleRange) { + final var namespaceToBundleRange = data.computeIfAbsent(broker, __ -> new HashMap<>()); + synchronized (namespaceToBundleRange) { + namespaceToBundleRange.computeIfAbsent(namespace, __ -> new HashSet<>()).add(bundleRange); + } + } + + public int getBundleRangeCount(String broker, String namespace) { + final var namespaceToBundleRange = data.computeIfAbsent(broker, __ -> new HashMap<>()); + synchronized (namespaceToBundleRange) { + final var bundleRangeSet = namespaceToBundleRange.get(namespace); + return bundleRangeSet != null ? bundleRangeSet.size() : 0; + } + } + + /** + * Get the map whose key is the broker and value is the namespace that has at least 1 cached bundle range. + */ + public Map> getBrokerToNamespacesMap() { + final var brokerToNamespaces = new HashMap>(); + for (var entry : data.entrySet()) { + final var broker = entry.getKey(); + final var namespaceToBundleRange = entry.getValue(); + synchronized (namespaceToBundleRange) { + brokerToNamespaces.put(broker, namespaceToBundleRange.keySet().stream().toList()); + } + } + return brokerToNamespaces; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java index 3d627db6cfa9e..7ca2b926db7db 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerShared.java @@ -46,8 +46,6 @@ import org.apache.pulsar.common.policies.data.FailureDomainImpl; import org.apache.pulsar.common.util.DirectMemoryUtils; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.policies.data.loadbalancer.BrokerData; import org.apache.pulsar.policies.data.loadbalancer.ResourceUsage; @@ -282,24 +280,6 @@ public static CompletableFuture> applyNamespacePoliciesAsync( return brokerCandidateCache; }); } - /** - * Using the given bundles, populate the namespace to bundle range map. - * - * @param bundles - * Bundles with which to populate. - * @param target - * Map to fill. - */ - public static void fillNamespaceToBundlesMap(final Set bundles, - final ConcurrentOpenHashMap> target) { - bundles.forEach(bundleName -> { - final String namespaceName = getNamespaceNameFromBundleName(bundleName); - final String bundleRange = getBundleRangeFromBundleName(bundleName); - target.computeIfAbsent(namespaceName, - k -> ConcurrentOpenHashSet.newBuilder().build()) - .add(bundleRange); - }); - } // From a full bundle name, extract the bundle range. public static String getBundleRangeFromBundleName(String bundleName) { @@ -359,8 +339,7 @@ public static boolean isLoadSheddingEnabled(final PulsarService pulsar) { public static void removeMostServicingBrokersForNamespace( final String assignedBundleName, final Set candidates, - final ConcurrentOpenHashMap>> - brokerToNamespaceToBundleRange) { + final BundleRangeCache brokerToNamespaceToBundleRange) { if (candidates.isEmpty()) { return; } @@ -369,13 +348,7 @@ public static void removeMostServicingBrokersForNamespace( int leastBundles = Integer.MAX_VALUE; for (final String broker : candidates) { - int bundles = (int) brokerToNamespaceToBundleRange - .computeIfAbsent(broker, - k -> ConcurrentOpenHashMap.>newBuilder().build()) - .computeIfAbsent(namespaceName, - k -> ConcurrentOpenHashSet.newBuilder().build()) - .size(); + int bundles = brokerToNamespaceToBundleRange.getBundleRangeCount(broker, namespaceName); leastBundles = Math.min(leastBundles, bundles); if (leastBundles == 0) { break; @@ -386,13 +359,8 @@ public static void removeMostServicingBrokersForNamespace( // `leastBundles` may differ from the actual value. final int finalLeastBundles = leastBundles; - candidates.removeIf( - broker -> brokerToNamespaceToBundleRange.computeIfAbsent(broker, - k -> ConcurrentOpenHashMap.>newBuilder().build()) - .computeIfAbsent(namespaceName, - k -> ConcurrentOpenHashSet.newBuilder().build()) - .size() > finalLeastBundles); + candidates.removeIf(broker -> + brokerToNamespaceToBundleRange.getBundleRangeCount(broker, namespaceName) > finalLeastBundles); } /** @@ -426,8 +394,7 @@ public static void removeMostServicingBrokersForNamespace( public static void filterAntiAffinityGroupOwnedBrokers( final PulsarService pulsar, final String assignedBundleName, final Set candidates, - final ConcurrentOpenHashMap>> - brokerToNamespaceToBundleRange, + final BundleRangeCache brokerToNamespaceToBundleRange, Map brokerToDomainMap) { if (candidates.isEmpty()) { return; @@ -572,8 +539,7 @@ private static void filterDomainsNotHavingLeastNumberAntiAffinityNamespaces( */ public static CompletableFuture> getAntiAffinityNamespaceOwnedBrokers( final PulsarService pulsar, final String namespaceName, - final ConcurrentOpenHashMap>> - brokerToNamespaceToBundleRange) { + final BundleRangeCache brokerToNamespaceToBundleRange) { CompletableFuture> antiAffinityNsBrokersResult = new CompletableFuture<>(); getNamespaceAntiAffinityGroupAsync(pulsar, namespaceName) @@ -584,21 +550,16 @@ public static CompletableFuture> getAntiAffinityNamespaceOw } final String antiAffinityGroup = antiAffinityGroupOptional.get(); final Map brokerToAntiAffinityNamespaceCount = new ConcurrentHashMap<>(); - final List> futures = new ArrayList<>(); - brokerToNamespaceToBundleRange.forEach((broker, nsToBundleRange) -> { - nsToBundleRange.forEach((ns, bundleRange) -> { - if (bundleRange.isEmpty()) { - return; - } - - CompletableFuture future = new CompletableFuture<>(); - futures.add(future); - countAntiAffinityNamespaceOwnedBrokers(broker, ns, future, + final var brokerToNamespaces = brokerToNamespaceToBundleRange.getBrokerToNamespacesMap(); + FutureUtil.waitForAll(brokerToNamespaces.entrySet().stream().flatMap(e -> { + final var broker = e.getKey(); + return e.getValue().stream().map(namespace -> { + final var future = new CompletableFuture(); + countAntiAffinityNamespaceOwnedBrokers(broker, namespace, future, pulsar, antiAffinityGroup, brokerToAntiAffinityNamespaceCount); + return future; }); - }); - FutureUtil.waitForAll(futures) - .thenAccept(r -> antiAffinityNsBrokersResult.complete(brokerToAntiAffinityNamespaceCount)); + }).toList()).thenAccept(__ -> antiAffinityNsBrokersResult.complete(brokerToAntiAffinityNamespaceCount)); }).exceptionally(ex -> { // namespace-policies has not been created yet antiAffinityNsBrokersResult.complete(null); @@ -698,7 +659,6 @@ public static Optional getNamespaceAntiAffinityGroup( * by different broker. * * @param namespace - * @param bundle * @param currentBroker * @param pulsar * @param brokerToNamespaceToBundleRange @@ -707,10 +667,9 @@ public static Optional getNamespaceAntiAffinityGroup( * @throws Exception */ public static boolean shouldAntiAffinityNamespaceUnload( - String namespace, String bundle, String currentBroker, + String namespace, String currentBroker, final PulsarService pulsar, - final ConcurrentOpenHashMap>> - brokerToNamespaceToBundleRange, + final BundleRangeCache brokerToNamespaceToBundleRange, Set candidateBrokers) throws Exception { Map brokerNamespaceCount = getAntiAffinityNamespaceOwnedBrokers(pulsar, namespace, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java index 05c984d0349b7..48a6121b9dd13 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImpl.java @@ -43,6 +43,7 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.SystemUtils; @@ -72,8 +73,6 @@ import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.Reflections; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; import org.apache.pulsar.metadata.api.Notification; @@ -116,10 +115,7 @@ public class ModularLoadManagerImpl implements ModularLoadManager { // Broker host usage object used to calculate system resource usage. private BrokerHostUsage brokerHostUsage; - // Map from brokers to namespaces to the bundle ranges in that namespace assigned to that broker. - // Used to distribute bundles within a namespace evenly across brokers. - private final ConcurrentOpenHashMap>> - brokerToNamespaceToBundleRange; + private final BundleRangeCache brokerToNamespaceToBundleRange = new BundleRangeCache(); // Path to the ZNode containing the LocalBrokerData json for this broker. private String brokerZnodePath; @@ -199,10 +195,6 @@ public class ModularLoadManagerImpl implements ModularLoadManager { */ public ModularLoadManagerImpl() { brokerCandidateCache = new HashSet<>(); - brokerToNamespaceToBundleRange = - ConcurrentOpenHashMap.>>newBuilder() - .build(); defaultStats = new NamespaceBundleStats(); filterPipeline = new ArrayList<>(); loadData = new LoadData(); @@ -582,17 +574,9 @@ private void updateBundleData() { TimeAverageBrokerData timeAverageData = new TimeAverageBrokerData(); timeAverageData.reset(statsMap.keySet(), bundleData, defaultStats); brokerData.setTimeAverageData(timeAverageData); - final ConcurrentOpenHashMap> namespaceToBundleRange = - brokerToNamespaceToBundleRange - .computeIfAbsent(broker, k -> - ConcurrentOpenHashMap.>newBuilder() - .build()); - synchronized (namespaceToBundleRange) { - namespaceToBundleRange.clear(); - LoadManagerShared.fillNamespaceToBundlesMap(statsMap.keySet(), namespaceToBundleRange); - LoadManagerShared.fillNamespaceToBundlesMap(preallocatedBundleData.keySet(), namespaceToBundleRange); - } + + brokerToNamespaceToBundleRange.reloadFromBundles(broker, + Stream.of(statsMap.keySet(), preallocatedBundleData.keySet()).flatMap(Collection::stream)); } // Remove not active bundle from loadData @@ -736,7 +720,7 @@ public boolean shouldAntiAffinityNamespaceUnload(String namespace, String bundle .getBundle(namespace, bundle); LoadManagerShared.applyNamespacePolicies(serviceUnit, policies, brokerCandidateCache, getAvailableBrokers(), brokerTopicLoadingPredicate); - return LoadManagerShared.shouldAntiAffinityNamespaceUnload(namespace, bundle, currentBroker, pulsar, + return LoadManagerShared.shouldAntiAffinityNamespaceUnload(namespace, currentBroker, pulsar, brokerToNamespaceToBundleRange, brokerCandidateCache); } @@ -873,17 +857,7 @@ private void preallocateBundle(String bundle, String broker) { final String namespaceName = LoadManagerShared.getNamespaceNameFromBundleName(bundle); final String bundleRange = LoadManagerShared.getBundleRangeFromBundleName(bundle); - final ConcurrentOpenHashMap> namespaceToBundleRange = - brokerToNamespaceToBundleRange - .computeIfAbsent(broker, - k -> ConcurrentOpenHashMap.>newBuilder() - .build()); - synchronized (namespaceToBundleRange) { - namespaceToBundleRange.computeIfAbsent(namespaceName, - k -> ConcurrentOpenHashSet.newBuilder().build()) - .add(bundleRange); - } + brokerToNamespaceToBundleRange.add(broker, namespaceName, bundleRange); } @VisibleForTesting diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java index be0580808cafb..30a7359ce0eb8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/SimpleLoadManagerImpl.java @@ -28,6 +28,7 @@ import com.google.common.collect.TreeMultimap; import java.time.LocalDateTime; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -47,6 +48,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.SystemUtils; import org.apache.pulsar.broker.PulsarServerException; @@ -62,8 +64,6 @@ import org.apache.pulsar.common.policies.data.ResourceQuota; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.Notification; @@ -107,10 +107,7 @@ public class SimpleLoadManagerImpl implements LoadManager, Consumer bundleGainsCache; private final Set bundleLossesCache; - // Map from brokers to namespaces to the bundle ranges in that namespace assigned to that broker. - // Used to distribute bundles within a namespace evenly across brokers. - private final ConcurrentOpenHashMap>> brokerToNamespaceToBundleRange; + private final BundleRangeCache brokerToNamespaceToBundleRange = new BundleRangeCache(); // CPU usage per msg/sec private double realtimeCpuLoadFactor = 0.025; @@ -205,10 +202,6 @@ public SimpleLoadManagerImpl() { bundleLossesCache = new HashSet<>(); brokerCandidateCache = new HashSet<>(); availableBrokersCache = new HashSet<>(); - brokerToNamespaceToBundleRange = - ConcurrentOpenHashMap.>>newBuilder() - .build(); this.brokerTopicLoadingPredicate = new BrokerTopicLoadingPredicate() { @Override public boolean isEnablePersistentTopics(String brokerId) { @@ -853,14 +846,7 @@ private synchronized ResourceUnit findBrokerForPlacement(Multimap ConcurrentOpenHashMap.>newBuilder() - .build()) - .computeIfAbsent(namespaceName, k -> - ConcurrentOpenHashSet.newBuilder().build()) - .add(bundleRange); + brokerToNamespaceToBundleRange.add(selectedRU.getResourceId(), namespaceName, bundleRange); ranking.addPreAllocatedServiceUnit(serviceUnitId, quota); resourceUnitRankings.put(selectedRU, ranking); } @@ -1272,15 +1258,8 @@ private synchronized void updateBrokerToNamespaceToBundle() { final String broker = resourceUnit.getResourceId(); final Set loadedBundles = ranking.getLoadedBundles(); final Set preallocatedBundles = resourceUnitRankings.get(resourceUnit).getPreAllocatedBundles(); - final ConcurrentOpenHashMap> namespaceToBundleRange = - brokerToNamespaceToBundleRange - .computeIfAbsent(broker, - k -> ConcurrentOpenHashMap.>newBuilder() - .build()); - namespaceToBundleRange.clear(); - LoadManagerShared.fillNamespaceToBundlesMap(loadedBundles, namespaceToBundleRange); - LoadManagerShared.fillNamespaceToBundlesMap(preallocatedBundles, namespaceToBundleRange); + brokerToNamespaceToBundleRange.reloadFromBundles(broker, + Stream.of(loadedBundles, preallocatedBundles).flatMap(Collection::stream)); }); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java index fc2fec96294ef..f1e462c4ec784 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AntiAffinityNamespaceGroupTest.java @@ -40,6 +40,7 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; +import org.apache.pulsar.broker.loadbalance.impl.BundleRangeCache; import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; @@ -59,8 +60,6 @@ import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; @@ -167,12 +166,10 @@ protected void createNamespaceIfNotExists(PulsarResources resources, } } - protected Object getBundleOwnershipData(){ - return ConcurrentOpenHashMap.>>newBuilder().build(); + return new BundleRangeCache(); } - protected String getLoadManagerClassName() { return ModularLoadManagerImpl.class.getName(); } @@ -366,17 +363,8 @@ protected void selectBrokerForNamespace( Object ownershipData, String broker, String namespace, String assignedBundleName) { - ConcurrentOpenHashMap>> - brokerToNamespaceToBundleRange = - (ConcurrentOpenHashMap>>) ownershipData; - ConcurrentOpenHashSet bundleSet = - ConcurrentOpenHashSet.newBuilder().build(); - bundleSet.add(assignedBundleName); - ConcurrentOpenHashMap> nsToBundleMap = - ConcurrentOpenHashMap.>newBuilder().build(); - nsToBundleMap.put(namespace, bundleSet); - brokerToNamespaceToBundleRange.put(broker, nsToBundleMap); + final var brokerToNamespaceToBundleRange = (BundleRangeCache) ownershipData; + brokerToNamespaceToBundleRange.add(broker, namespace, assignedBundleName); } /** @@ -562,10 +550,9 @@ private static void filterAntiAffinityGroupOwnedBrokers( if (ownershipData instanceof Set) { LoadManagerShared.filterAntiAffinityGroupOwnedBrokers(pulsar, assignedNamespace, brokers, (Set>) ownershipData, brokerToDomainMap); - } else if (ownershipData instanceof ConcurrentOpenHashMap) { + } else if (ownershipData instanceof BundleRangeCache) { LoadManagerShared.filterAntiAffinityGroupOwnedBrokers(pulsar, assignedNamespace, brokers, - (ConcurrentOpenHashMap>>) - ownershipData, brokerToDomainMap); + (BundleRangeCache) ownershipData, brokerToDomainMap); } else { throw new RuntimeException("Unknown ownershipData class type"); } @@ -582,11 +569,9 @@ private static boolean shouldAntiAffinityNamespaceUnload( if (ownershipData instanceof Set) { return LoadManagerShared.shouldAntiAffinityNamespaceUnload(namespace, bundle, currentBroker, pulsar, (Set>) ownershipData, candidate); - } else if (ownershipData instanceof ConcurrentOpenHashMap) { - return LoadManagerShared.shouldAntiAffinityNamespaceUnload(namespace, bundle, - currentBroker, pulsar, - (ConcurrentOpenHashMap>>) - ownershipData, candidate); + } else if (ownershipData instanceof BundleRangeCache) { + return LoadManagerShared.shouldAntiAffinityNamespaceUnload(namespace, currentBroker, pulsar, + (BundleRangeCache) ownershipData, candidate); } else { throw new RuntimeException("Unknown ownershipData class type"); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerSharedTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerSharedTest.java index 8bc097779b00a..465e8e2d85246 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerSharedTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/LoadManagerSharedTest.java @@ -18,13 +18,9 @@ */ package org.apache.pulsar.broker.loadbalance.impl; +import com.google.common.collect.Sets; import java.util.HashSet; import java.util.Set; - -import com.google.common.collect.Sets; - -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; import org.testng.Assert; import org.testng.annotations.Test; @@ -37,59 +33,44 @@ public void testRemoveMostServicingBrokersForNamespace() { String assignedBundle = namespace + "/0x00000000_0x40000000"; Set candidates = new HashSet<>(); - ConcurrentOpenHashMap>> map = - ConcurrentOpenHashMap.>>newBuilder() - .build(); - LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, map); + final var cache = new BundleRangeCache(); + LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, cache); Assert.assertEquals(candidates.size(), 0); candidates = Sets.newHashSet("broker1"); - LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, map); + LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, cache); Assert.assertEquals(candidates.size(), 1); Assert.assertTrue(candidates.contains("broker1")); candidates = Sets.newHashSet("broker1"); - fillBrokerToNamespaceToBundleMap(map, "broker1", namespace, "0x40000000_0x80000000"); - LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, map); + cache.add("broker1", namespace, "0x40000000_0x80000000"); + LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, cache); Assert.assertEquals(candidates.size(), 1); Assert.assertTrue(candidates.contains("broker1")); candidates = Sets.newHashSet("broker1", "broker2"); - LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, map); + LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, cache); Assert.assertEquals(candidates.size(), 1); Assert.assertTrue(candidates.contains("broker2")); candidates = Sets.newHashSet("broker1", "broker2"); - fillBrokerToNamespaceToBundleMap(map, "broker2", namespace, "0x80000000_0xc0000000"); - LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, map); + cache.add("broker2", namespace, "0x80000000_0xc0000000"); + LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, cache); Assert.assertEquals(candidates.size(), 2); Assert.assertTrue(candidates.contains("broker1")); Assert.assertTrue(candidates.contains("broker2")); candidates = Sets.newHashSet("broker1", "broker2"); - fillBrokerToNamespaceToBundleMap(map, "broker2", namespace, "0xc0000000_0xd0000000"); - LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, map); + cache.add("broker2", namespace, "0xc0000000_0xd0000000"); + LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, cache); Assert.assertEquals(candidates.size(), 1); Assert.assertTrue(candidates.contains("broker1")); candidates = Sets.newHashSet("broker1", "broker2", "broker3"); - fillBrokerToNamespaceToBundleMap(map, "broker3", namespace, "0xd0000000_0xffffffff"); - LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, map); + cache.add("broker3", namespace, "0xd0000000_0xffffffff"); + LoadManagerShared.removeMostServicingBrokersForNamespace(assignedBundle, candidates, cache); Assert.assertEquals(candidates.size(), 2); Assert.assertTrue(candidates.contains("broker1")); Assert.assertTrue(candidates.contains("broker3")); } - - private static void fillBrokerToNamespaceToBundleMap( - ConcurrentOpenHashMap>> map, - String broker, String namespace, String bundle) { - map.computeIfAbsent(broker, - k -> ConcurrentOpenHashMap.>newBuilder().build()) - .computeIfAbsent(namespace, - k -> ConcurrentOpenHashSet.newBuilder().build()) - .add(bundle); - } - } From e2bbb4b7cbc5eb6196e8a11f1d7cdbdad20ce4b4 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Thu, 29 Aug 2024 10:58:38 +0800 Subject: [PATCH 472/580] [improve][txn] Take first snapshot before persisting the first transactional message (#21406) ### Motivation The decision to write a snapshot before the first transaction message instead of before building the producer, is based on the fact that only the act of writing transactional messages signifies the use of the transaction buffer. Furthermore, it is only appropriate to schedule snapshot updates after this point. Otherwise, it will add a lot of unnecessary IO read and write operations and increase the delay of topic load. --- **Scenario** * 1000 topics under namespace1. * Client1 enables transaction and sends transaction messages to topic 1. * Client1 sends normal messages to topic 2~500. * Client2 disables transaction and sends messages to topic 501~1000. **Internal Behavior** * Topic 1~500 will start the 500 task to write snapshots into the same system topic, e.g., system topic 1. * All the topics (1~1000) will read this system topic 1 when topic loading. ### Modifications This Pull Request aims to resolve the unnecessary write operation. Starting to write snapshots when sending first transaction messages instead of building producer. --- .../pulsar/broker/service/ServerCnx.java | 89 ++++---- .../apache/pulsar/broker/service/Topic.java | 9 +- .../nonpersistent/NonPersistentTopic.java | 2 +- .../service/persistent/PersistentTopic.java | 4 +- .../transaction/buffer/TransactionBuffer.java | 11 +- .../buffer/impl/InMemTransactionBuffer.java | 2 +- .../buffer/impl/TopicTransactionBuffer.java | 98 +++++---- .../buffer/impl/TransactionBufferDisable.java | 2 +- .../broker/transaction/TransactionTest.java | 13 +- .../buffer/TopicTransactionBufferTest.java | 193 +++++++++++++++++- .../buffer/TransactionStablePositionTest.java | 9 + .../utils/TransactionBufferTestImpl.java | 54 +++++ .../utils/TransactionBufferTestProvider.java | 33 +++ 13 files changed, 404 insertions(+), 115 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/utils/TransactionBufferTestImpl.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/utils/TransactionBufferTestProvider.java 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 d1fe9776e079d..a5c09d2892342 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 @@ -1616,66 +1616,53 @@ protected void handleProducer(final CommandProducer cmdProducer) { }); schemaVersionFuture.thenAccept(schemaVersion -> { - topic.checkIfTransactionBufferRecoverCompletely(isTxnEnabled).thenAccept(future -> { - CompletionStage createInitSubFuture; - if (!Strings.isNullOrEmpty(initialSubscriptionName) - && topic.isPersistent() - && !topic.getSubscriptions().containsKey(initialSubscriptionName)) { - createInitSubFuture = service.isAllowAutoSubscriptionCreationAsync(topicName) - .thenCompose(isAllowAutoSubscriptionCreation -> { - if (!isAllowAutoSubscriptionCreation) { - return CompletableFuture.failedFuture( - new BrokerServiceException.NotAllowedException( - "Could not create the initial subscription due to" - + " the auto subscription creation is not allowed.")); - } - return topic.createSubscription(initialSubscriptionName, - InitialPosition.Earliest, false, null); - }); - } else { - createInitSubFuture = CompletableFuture.completedFuture(null); - } - - createInitSubFuture.whenComplete((sub, ex) -> { - if (ex != null) { - final Throwable rc = FutureUtil.unwrapCompletionException(ex); - if (rc instanceof BrokerServiceException.NotAllowedException) { - log.warn("[{}] {} initialSubscriptionName: {}, topic: {}", - remoteAddress, rc.getMessage(), initialSubscriptionName, topicName); - if (producerFuture.completeExceptionally(rc)) { - commandSender.sendErrorResponse(requestId, - ServerError.NotAllowedError, rc.getMessage()); + CompletionStage createInitSubFuture; + if (!Strings.isNullOrEmpty(initialSubscriptionName) + && topic.isPersistent() + && !topic.getSubscriptions().containsKey(initialSubscriptionName)) { + createInitSubFuture = service.isAllowAutoSubscriptionCreationAsync(topicName) + .thenCompose(isAllowAutoSubscriptionCreation -> { + if (!isAllowAutoSubscriptionCreation) { + return CompletableFuture.failedFuture( + new BrokerServiceException.NotAllowedException( + "Could not create the initial subscription due to the " + + "auto subscription creation is not allowed.")); } - producers.remove(producerId, producerFuture); - return; - } - String msg = - "Failed to create the initial subscription: " + ex.getCause().getMessage(); + return topic.createSubscription(initialSubscriptionName, + InitialPosition.Earliest, false, null); + }); + } else { + createInitSubFuture = CompletableFuture.completedFuture(null); + } + + createInitSubFuture.whenComplete((sub, ex) -> { + if (ex != null) { + final Throwable rc = FutureUtil.unwrapCompletionException(ex); + if (rc instanceof BrokerServiceException.NotAllowedException) { log.warn("[{}] {} initialSubscriptionName: {}, topic: {}", - remoteAddress, msg, initialSubscriptionName, topicName); - if (producerFuture.completeExceptionally(ex)) { + remoteAddress, rc.getMessage(), initialSubscriptionName, topicName); + if (producerFuture.completeExceptionally(rc)) { commandSender.sendErrorResponse(requestId, - BrokerServiceException.getClientErrorCode(ex), msg); + ServerError.NotAllowedError, rc.getMessage()); } producers.remove(producerId, producerFuture); return; } + String msg = + "Failed to create the initial subscription: " + ex.getCause().getMessage(); + log.warn("[{}] {} initialSubscriptionName: {}, topic: {}", + remoteAddress, msg, initialSubscriptionName, topicName); + if (producerFuture.completeExceptionally(ex)) { + commandSender.sendErrorResponse(requestId, + BrokerServiceException.getClientErrorCode(ex), msg); + } + producers.remove(producerId, producerFuture); + return; + } - buildProducerAndAddTopic(topic, producerId, producerName, requestId, isEncrypted, + buildProducerAndAddTopic(topic, producerId, producerName, requestId, isEncrypted, metadata, schemaVersion, epoch, userProvidedProducerName, topicName, producerAccessMode, topicEpoch, supportsPartialProducer, producerFuture); - }); - }).exceptionally(exception -> { - Throwable cause = exception.getCause(); - log.error("producerId {}, requestId {} : TransactionBuffer recover failed", - producerId, requestId, exception); - if (producerFuture.completeExceptionally(exception)) { - commandSender.sendErrorResponse(requestId, - ServiceUnitNotReadyException.getClientErrorCode(cause), - cause.getMessage()); - } - producers.remove(producerId, producerFuture); - return null; }); }); }); @@ -2249,7 +2236,7 @@ protected void handleGetLastMessageId(CommandGetLastMessageId getLastMessageId) long requestId = getLastMessageId.getRequestId(); Topic topic = consumer.getSubscription().getTopic(); - topic.checkIfTransactionBufferRecoverCompletely(true) + topic.checkIfTransactionBufferRecoverCompletely() .thenCompose(__ -> topic.getLastDispatchablePosition()) .thenApply(lastPosition -> { int partitionIndex = TopicName.getPartitionIndex(topic.getName()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java index 50a28c7979277..3ec09e9bfcd28 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java @@ -146,12 +146,11 @@ default void setEntryTimestamp(long entryTimestamp) { void removeProducer(Producer producer); /** - * Wait TransactionBuffer Recovers completely. - * Take snapshot after TB Recovers completely. - * @param isTxnEnabled isTxnEnabled - * @return a future which has completely if isTxn = false. Or a future return by takeSnapshot. + * Wait TransactionBuffer recovers completely. + * + * @return a future that will be completed after the transaction buffer recover completely. */ - CompletableFuture checkIfTransactionBufferRecoverCompletely(boolean isTxnEnabled); + CompletableFuture checkIfTransactionBufferRecoverCompletely(); /** * record add-latency. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 9456870589191..1b98ee2f8306d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -258,7 +258,7 @@ public boolean isReplicationBacklogExist() { } @Override - public CompletableFuture checkIfTransactionBufferRecoverCompletely(boolean isTxnEnabled) { + public CompletableFuture checkIfTransactionBufferRecoverCompletely() { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 146ac05d695d5..d814e7ce11599 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -836,8 +836,8 @@ public CompletableFuture> addProducer(Producer producer, } @Override - public CompletableFuture checkIfTransactionBufferRecoverCompletely(boolean isTxnEnabled) { - return getTransactionBuffer().checkIfTBRecoverCompletely(isTxnEnabled); + public CompletableFuture checkIfTransactionBufferRecoverCompletely() { + return getTransactionBuffer().checkIfTBRecoverCompletely(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java index b379c4d1db10c..874f4c1c28a02 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java @@ -187,14 +187,11 @@ public interface TransactionBuffer { TransactionBufferStats getStats(boolean lowWaterMarks); /** - * Wait TransactionBuffer Recovers completely. - * Take snapshot after TB Recovers completely. - * @param isTxn - * @return a future which has completely if isTxn = false. Or a future return by takeSnapshot. + * Wait TransactionBuffer recovers completely. + * + * @return a future that will be completed after the transaction buffer recover completely. */ - CompletableFuture checkIfTBRecoverCompletely(boolean isTxn); - - + CompletableFuture checkIfTBRecoverCompletely(); long getOngoingTxnCount(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java index ae755f0715ee2..4da7a48e96c51 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java @@ -411,7 +411,7 @@ public TransactionBufferStats getStats(boolean lowWaterMarks) { @Override - public CompletableFuture checkIfTBRecoverCompletely(boolean isTxn) { + public CompletableFuture checkIfTBRecoverCompletely() { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index 7561457d11f8e..2f90ff8922a81 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -57,6 +57,7 @@ import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.Markers; import org.apache.pulsar.common.util.Codec; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.RecoverTimeRecord; import org.jctools.queues.MessagePassingQueue; import org.jctools.queues.SpscArrayQueue; @@ -89,8 +90,12 @@ public class TopicTransactionBuffer extends TopicTransactionBufferState implemen private final int takeSnapshotIntervalTime; + private final CompletableFuture transactionBufferFuture = new CompletableFuture<>(); + private CompletableFuture publishFuture = getTransactionBufferFuture() + .thenApply(__ -> PositionFactory.EARLIEST); + /** * The map is used to store the lowWaterMarks which key is TC ID and value is lowWaterMark of the TC. */ @@ -138,14 +143,14 @@ public void recoverComplete() { if (!changeToReadyState()) { log.error("[{}]Transaction buffer recover fail, current state: {}", topic.getName(), getState()); - transactionBufferFuture.completeExceptionally + getTransactionBufferFuture().completeExceptionally (new BrokerServiceException.ServiceUnitNotReadyException( "Transaction buffer recover failed to change the status to Ready," + "current state is: " + getState())); } else { timer.newTimeout(TopicTransactionBuffer.this, takeSnapshotIntervalTime, TimeUnit.MILLISECONDS); - transactionBufferFuture.complete(null); + getTransactionBufferFuture().complete(null); recoverTime.setRecoverEndTime(System.currentTimeMillis()); } } @@ -158,7 +163,7 @@ public void noNeedToRecover() { if (!changeToNoSnapshotState()) { log.error("[{}]Transaction buffer recover fail", topic.getName()); } else { - transactionBufferFuture.complete(null); + getTransactionBufferFuture().complete(null); recoverTime.setRecoverEndTime(System.currentTimeMillis()); } } @@ -196,10 +201,10 @@ public void recoverExceptionally(Throwable e) { // if transaction buffer recover fail throw PulsarClientException, // we need to change the PulsarClientException to ServiceUnitNotReadyException, // the tc do op will retry - transactionBufferFuture.completeExceptionally + getTransactionBufferFuture().completeExceptionally (new BrokerServiceException.ServiceUnitNotReadyException(e.getMessage(), e)); } else { - transactionBufferFuture.completeExceptionally(e); + getTransactionBufferFuture().completeExceptionally(e); } recoverTime.setRecoverEndTime(System.currentTimeMillis()); topic.close(true); @@ -212,35 +217,19 @@ public CompletableFuture getTransactionMeta(TxnID txnID) { return CompletableFuture.completedFuture(null); } + @VisibleForTesting + public CompletableFuture getPublishFuture() { + return publishFuture; + } + + @VisibleForTesting + public CompletableFuture getTransactionBufferFuture() { + return transactionBufferFuture; + } + @Override - public CompletableFuture checkIfTBRecoverCompletely(boolean isTxnEnabled) { - if (!isTxnEnabled) { - return CompletableFuture.completedFuture(null); - } else { - CompletableFuture completableFuture = new CompletableFuture<>(); - transactionBufferFuture.thenRun(() -> { - if (checkIfNoSnapshot()) { - snapshotAbortedTxnProcessor.takeAbortedTxnsSnapshot(maxReadPosition).thenRun(() -> { - if (changeToReadyStateFromNoSnapshot()) { - timer.newTimeout(TopicTransactionBuffer.this, - takeSnapshotIntervalTime, TimeUnit.MILLISECONDS); - } - completableFuture.complete(null); - }).exceptionally(exception -> { - log.error("Topic {} failed to take snapshot", this.topic.getName()); - completableFuture.completeExceptionally(exception); - return null; - }); - } else { - completableFuture.complete(null); - } - }).exceptionally(exception -> { - log.error("Topic {}: TransactionBuffer recover failed", this.topic.getName(), exception.getCause()); - completableFuture.completeExceptionally(exception.getCause()); - return null; - }); - return completableFuture; - } + public CompletableFuture checkIfTBRecoverCompletely() { + return getTransactionBufferFuture(); } @Override @@ -260,6 +249,45 @@ public long getCommittedTxnCount() { @Override public CompletableFuture appendBufferToTxn(TxnID txnId, long sequenceId, ByteBuf buffer) { + // Method `takeAbortedTxnsSnapshot` will be executed in the different thread. + // So we need to retain the buffer in this thread. It will be released after message persistent. + buffer.retain(); + CompletableFuture future = getPublishFuture().thenCompose(ignore -> { + if (checkIfNoSnapshot()) { + CompletableFuture completableFuture = new CompletableFuture<>(); + // `publishFuture` will be completed after message persistent, so there will not be two threads + // writing snapshots at the same time. + snapshotAbortedTxnProcessor.takeAbortedTxnsSnapshot(maxReadPosition).thenRun(() -> { + if (changeToReadyStateFromNoSnapshot()) { + timer.newTimeout(TopicTransactionBuffer.this, + takeSnapshotIntervalTime, TimeUnit.MILLISECONDS); + completableFuture.complete(null); + } else { + log.error("[{}]Failed to change state of transaction buffer to Ready from NoSnapshot", + topic.getName()); + completableFuture.completeExceptionally(new BrokerServiceException.ServiceUnitNotReadyException( + "Transaction Buffer take first snapshot failed, the current state is: " + getState())); + } + }).exceptionally(exception -> { + log.error("Topic {} failed to take snapshot", this.topic.getName()); + completableFuture.completeExceptionally(exception); + return null; + }); + return completableFuture.thenCompose(__ -> internalAppendBufferToTxn(txnId, buffer)); + } else if (checkIfReady()) { + return internalAppendBufferToTxn(txnId, buffer); + } else { + // `publishFuture` will be completed after transaction buffer recover completely + // during initializing, so this case should not happen. + return FutureUtil.failedFuture(new BrokerServiceException.ServiceUnitNotReadyException( + "Transaction Buffer recover failed, the current state is: " + getState())); + } + }).whenComplete(((position, throwable) -> buffer.release())); + publishFuture = future; + return future; + } + + private CompletableFuture internalAppendBufferToTxn(TxnID txnId, ByteBuf buffer) { CompletableFuture completableFuture = new CompletableFuture<>(); Long lowWaterMark = lowWaterMarks.get(txnId.getMostSigBits()); if (lowWaterMark != null && lowWaterMark >= txnId.getLeastSigBits()) { @@ -314,7 +342,7 @@ public CompletableFuture commitTxn(TxnID txnID, long lowWaterMark) { } CompletableFuture completableFuture = new CompletableFuture<>(); //Wait TB recover completely. - transactionBufferFuture.thenRun(() -> { + getTransactionBufferFuture().thenRun(() -> { ByteBuf commitMarker = Markers.newTxnCommitMarker(-1L, txnID.getMostSigBits(), txnID.getLeastSigBits()); try { @@ -356,7 +384,7 @@ public CompletableFuture abortTxn(TxnID txnID, long lowWaterMark) { } CompletableFuture completableFuture = new CompletableFuture<>(); //Wait TB recover completely. - transactionBufferFuture.thenRun(() -> { + getTransactionBufferFuture().thenRun(() -> { //no message sent, need not to add abort mark by txn timeout. if (!checkIfReady()) { completableFuture.complete(null); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java index d0efc47c49544..d4fd071fef8a7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java @@ -132,7 +132,7 @@ public TransactionBufferStats getStats(boolean lowWaterMarks) { } @Override - public CompletableFuture checkIfTBRecoverCompletely(boolean isTxn) { + public CompletableFuture checkIfTBRecoverCompletely() { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index 246ab5ef26a8f..3b3eaf7bb2292 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -595,6 +595,11 @@ public void testTakeSnapshotBeforeBuildTxnProducer() throws Exception { .topic(topic).enableBatching(true) .create(); + Transaction transaction = pulsarClient.newTransaction() + .withTransactionTimeout(5, TimeUnit.SECONDS).build().get(); + producer.newMessage(transaction).send(); + transaction.abort().get(); + Awaitility.await().untilAsserted(() -> { Message message1 = reader.readNext(); TransactionBufferSnapshot snapshot1 = message1.getValue(); @@ -608,7 +613,7 @@ public void testTakeSnapshotBeforeBuildTxnProducer() throws Exception { Awaitility.await().untilAsserted(() -> { Message message1 = reader.readNext(); TransactionBufferSnapshot snapshot1 = message1.getValue(); - Assert.assertEquals(snapshot1.getMaxReadPositionEntryId(), 1); + Assert.assertEquals(snapshot1.getMaxReadPositionEntryId(), 3); }); } @@ -716,7 +721,7 @@ public void testMaxReadPositionForNormalPublish() throws Exception { .sendTimeout(0, TimeUnit.SECONDS) .create(); - Awaitility.await().untilAsserted(() -> Assert.assertTrue(topicTransactionBuffer.checkIfReady())); + Awaitility.await().untilAsserted(() -> Assert.assertTrue(topicTransactionBuffer.checkIfNoSnapshot())); //test publishing txn messages will not change maxReadPosition if don`t commit or abort. Transaction transaction = pulsarClient.newTransaction() .withTransactionTimeout(5, TimeUnit.SECONDS).build().get(); @@ -1657,7 +1662,7 @@ public void testTBRecoverChangeStateError() throws InterruptedException, Timeout persistentTopic.set(new PersistentTopic("topic-a", managedLedger, brokerService)); try { // Do check. - persistentTopic.get().checkIfTransactionBufferRecoverCompletely(true).get(5, TimeUnit.SECONDS); + persistentTopic.get().checkIfTransactionBufferRecoverCompletely().get(5, TimeUnit.SECONDS); fail("Expect failure by TB closed, but it is finished."); } catch (ExecutionException executionException){ Throwable t = executionException.getCause(); @@ -1815,8 +1820,6 @@ public void testTBSnapshotWriter() throws Exception { .createAsync(); getTopic("persistent://" + topic + "-partition-0"); Thread.sleep(3000); - // the producer shouldn't be created, because the transaction buffer snapshot writer future didn't finish. - assertFalse(producerFuture.isDone()); // The topic will be closed, because the transaction buffer snapshot writer future is failed, // the failed writer future will be removed, the producer will be reconnected and work well. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java index dea79f391e39a..1ab97eb457a05 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java @@ -18,13 +18,15 @@ */ package org.apache.pulsar.broker.transaction.buffer; +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.fail; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import lombok.Cleanup; import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.Mockito.when; -import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertTrue; -import static org.testng.AssertJUnit.fail; import io.opentelemetry.api.common.Attributes; import java.time.Duration; import java.util.Collections; @@ -35,11 +37,13 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import lombok.Cleanup; + import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.commons.lang3.RandomUtils; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.BrokerService; @@ -50,16 +54,23 @@ import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBufferState; +import org.apache.pulsar.broker.transaction.buffer.utils.TransactionBufferTestImpl; +import org.apache.pulsar.broker.transaction.buffer.utils.TransactionBufferTestProvider; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.TopicMessageId; import org.apache.pulsar.client.api.transaction.Transaction; +import org.apache.pulsar.client.api.transaction.TxnID; +import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.TopicMessageIdImpl; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStore; @@ -228,9 +239,7 @@ public void testGetMaxPositionAfterTBReady() throws Exception { String topic = "persistent://" + NAMESPACE1 + "/testGetMaxReadyPositionAfterTBReady"; // 1.1 Mock component. TransactionBuffer transactionBuffer = Mockito.spy(TransactionBuffer.class); - when(transactionBuffer.checkIfTBRecoverCompletely(anyBoolean())) - // Handle producer will check transaction buffer recover completely. - .thenReturn(CompletableFuture.completedFuture(null)) + when(transactionBuffer.checkIfTBRecoverCompletely()) // If the Transaction buffer failed to recover, we can not get the correct last max read id. .thenReturn(CompletableFuture.failedFuture(new Throwable("Mock fail"))) // If the transaction buffer recover successfully, the max read position can be acquired successfully. @@ -405,4 +414,174 @@ private void assertGetLastMessageId(Consumer consumer, MessageIdImpl expected assertEquals(expected.getLedgerId(), actual.getLedgerId()); } + /** + * This test verifies the state changes of a TransactionBuffer within a topic under different conditions. + * Initially, the TransactionBuffer is in a NoSnapshot state upon topic creation. + * It remains in the NoSnapshot state even after a normal message is sent. + * The state changes to Ready only after a transactional message is sent. + * The test also ensures that the TransactionBuffer can be correctly recovered after the topic is unloaded. + */ + @Test + public void testWriteSnapshotWhenFirstTxnMessageSend() throws Exception { + // 1. Prepare test environment. + String topic = "persistent://" + NAMESPACE1 + "/testWriteSnapshotWhenFirstTxnMessageSend"; + String txnMsg = "transaction message"; + String normalMsg = "normal message"; + admin.topics().createNonPartitionedTopic(topic); + PersistentTopic persistentTopic = (PersistentTopic) pulsarServiceList.get(0).getBrokerService() + .getTopic(topic, false) + .get() + .get(); + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .create(); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("my-sub") + .subscribe(); + // 2. Test the state of transaction buffer after building producer with no new messages. + // The TransactionBuffer should be in NoSnapshot state before transaction message sent. + TopicTransactionBuffer topicTransactionBuffer = (TopicTransactionBuffer) persistentTopic.getTransactionBuffer(); + Awaitility.await().untilAsserted(() -> { + Assert.assertEquals(topicTransactionBuffer.getState(), TopicTransactionBufferState.State.NoSnapshot); + }); + // 3. Test the state of transaction buffer after sending normal messages. + // The TransactionBuffer should still be in NoSnapshot state after a normal message is sent. + producer.newMessage().value(normalMsg).send(); + Assert.assertEquals(topicTransactionBuffer.getState(), TopicTransactionBufferState.State.NoSnapshot); + // 4. Test the state of transaction buffer after sending transaction messages. + // The transaction buffer should be in Ready state at this time. + Transaction transaction = pulsarClient.newTransaction() + .withTransactionTimeout(5, TimeUnit.HOURS) + .build() + .get(); + producer.newMessage(transaction).value(txnMsg).send(); + Assert.assertEquals(topicTransactionBuffer.getState(), TopicTransactionBufferState.State.Ready); + // 5. Test transaction buffer can be recovered correctly. + // There are 4 message sent to this topic, 2 normal message and 2 transaction message |m1|m2-txn1|m3-txn1|m4|. + // Aborting the transaction and unload the topic and then redelivering unacked messages, + // only normal messages can be received. + transaction.abort().get(5, TimeUnit.SECONDS); + producer.newMessage().value(normalMsg).send(); + admin.topics().unload(topic); + PersistentTopic persistentTopic2 = (PersistentTopic) pulsarServiceList.get(0).getBrokerService() + .getTopic(topic, false) + .get() + .get(); + TopicTransactionBuffer topicTransactionBuffer2 = (TopicTransactionBuffer) persistentTopic2 + .getTransactionBuffer(); + Awaitility.await().untilAsserted(() -> { + Assert.assertEquals(topicTransactionBuffer2.getState(), TopicTransactionBufferState.State.Ready); + }); + consumer.redeliverUnacknowledgedMessages(); + for (int i = 0; i < 2; i++) { + Message message = consumer.receive(5, TimeUnit.SECONDS); + Assert.assertEquals(message.getValue(), normalMsg); + } + Message message = consumer.receive(5, TimeUnit.SECONDS); + Assert.assertNull(message); + } + + /** + * Send some messages before transaction buffer ready and then send some messages after transaction buffer ready, + * these messages should be received in order. + */ + @Test + public void testMessagePublishInOrder() throws Exception { + // 1. Prepare test environment. + this.pulsarServiceList.forEach(pulsarService -> { + pulsarService.setTransactionBufferProvider(new TransactionBufferTestProvider()); + }); + String topic = "persistent://" + NAMESPACE1 + "/testMessagePublishInOrder" + RandomUtils.nextLong(); + admin.topics().createNonPartitionedTopic(topic); + PersistentTopic persistentTopic = (PersistentTopic) pulsarServiceList.get(0).getBrokerService() + .getTopic(topic, false) + .get() + .get(); + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .create(); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub") + .subscribe(); + Transaction transaction = pulsarClient.newTransaction() + .withTransactionTimeout(5, TimeUnit.HOURS) + .build().get(); + + // 2. Set a new future in transaction buffer as `transactionBufferFuture` to simulate whether the + // transaction buffer recover completely. + TransactionBufferTestImpl topicTransactionBuffer = (TransactionBufferTestImpl) persistentTopic + .getTransactionBuffer(); + CompletableFuture completableFuture = new CompletableFuture<>(); + CompletableFuture originalFuture = topicTransactionBuffer.getPublishFuture(); + topicTransactionBuffer.setPublishFuture(completableFuture); + topicTransactionBuffer.setState(TopicTransactionBufferState.State.Ready); + // Register this topic to the transaction in advance to avoid the sending request pending here. + ((TransactionImpl) transaction).registerProducedTopic(topic).get(5, TimeUnit.SECONDS); + // 3. Test the messages sent before transaction buffer ready is in order. + for (int i = 0; i < 50; i++) { + producer.newMessage(transaction).value(i).sendAsync(); + } + // 4. Test the messages sent after transaction buffer ready is in order. + completableFuture.complete(originalFuture.get()); + for (int i = 50; i < 100; i++) { + producer.newMessage(transaction).value(i).sendAsync(); + } + transaction.commit().get(); + for (int i = 0; i < 100; i++) { + Message message = consumer.receive(5, TimeUnit.SECONDS); + Assert.assertEquals(message.getValue(), i); + } + } + + /** + * Test `testMessagePublishInOrder` will test the ref count work as expected with no exception. + * And this test is used to test the memory leak due to ref count. + */ + @Test + public void testRefCountWhenAppendBufferToTxn() throws Exception { + // 1. Prepare test resource + this.pulsarServiceList.forEach(pulsarService -> { + pulsarService.setTransactionBufferProvider(new TransactionBufferTestProvider()); + }); + String topic = "persistent://" + NAMESPACE1 + "/testRefCountWhenAppendBufferToTxn"; + admin.topics().createNonPartitionedTopic(topic); + PersistentTopic persistentTopic = (PersistentTopic) pulsarServiceList.get(0).getBrokerService() + .getTopic(topic, false) + .get() + .get(); + TransactionBufferTestImpl topicTransactionBuffer = (TransactionBufferTestImpl) persistentTopic + .getTransactionBuffer(); + // 2. Test reference count does not change in the method `appendBufferToTxn`. + // 2.1 Test sending first transaction message, this will take a snapshot. + ByteBuf byteBuf1 = Unpooled.buffer(); + topicTransactionBuffer.appendBufferToTxn(new TxnID(1, 1), 1L, byteBuf1) + .get(5, TimeUnit.SECONDS); + Awaitility.await().untilAsserted(() -> Assert.assertEquals(byteBuf1.refCnt(), 1)); + // 2.2 Test send the second transaction message, this will not take snapshots. + ByteBuf byteBuf2 = Unpooled.buffer(); + topicTransactionBuffer.appendBufferToTxn(new TxnID(1, 1), 1L, byteBuf1) + .get(5, TimeUnit.SECONDS); + Awaitility.await().untilAsserted(() -> Assert.assertEquals(byteBuf2.refCnt(), 1)); + // 2.3 Test sending message failed. + topicTransactionBuffer.setPublishFuture(FutureUtil.failedFuture(new Exception("fail"))); + ByteBuf byteBuf3 = Unpooled.buffer(); + try { + topicTransactionBuffer.appendBufferToTxn(new TxnID(1, 1), 1L, byteBuf1) + .get(5, TimeUnit.SECONDS); + fail(); + } catch (Exception e) { + assertEquals(e.getCause().getMessage(), "fail"); + } + Awaitility.await().untilAsserted(() -> Assert.assertEquals(byteBuf3.refCnt(), 1)); + // 3. release resource + byteBuf1.release(); + byteBuf2.release(); + byteBuf3.release(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java index eb7b24c7326dc..0b50f91fd403c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java @@ -195,6 +195,15 @@ public void testSyncNormalPositionWhenTBRecover(boolean clientEnableTransaction, .topic(topicName) .create(); + if (clientEnableTransaction) { + Transaction transaction = pulsarClient.newTransaction() + .withTransactionTimeout(5, TimeUnit.HOURS) + .build() + .get(); + producer.newMessage(transaction).send(); + transaction.commit().get(); + } + PersistentTopic persistentTopic = (PersistentTopic) getPulsarServiceList().get(0).getBrokerService() .getTopic(TopicName.get(topicName).toString(), false).get().get(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/utils/TransactionBufferTestImpl.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/utils/TransactionBufferTestImpl.java new file mode 100644 index 0000000000000..7ee14ffc3378e --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/utils/TransactionBufferTestImpl.java @@ -0,0 +1,54 @@ +/* + * 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.transaction.buffer.utils; + +import lombok.Setter; +import org.apache.bookkeeper.mledger.Position; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBuffer; + +import java.util.concurrent.CompletableFuture; + +public class TransactionBufferTestImpl extends TopicTransactionBuffer { + @Setter + public CompletableFuture transactionBufferFuture = null; + @Setter + public State state = null; + @Setter + public CompletableFuture publishFuture = null; + + public TransactionBufferTestImpl(PersistentTopic topic) { + super(topic); + } + + @Override + public CompletableFuture getTransactionBufferFuture() { + return transactionBufferFuture == null ? super.getTransactionBufferFuture() : transactionBufferFuture; + } + + @Override + public State getState() { + return state == null ? super.getState() : state; + } + + @Override + public CompletableFuture getPublishFuture() { + return publishFuture == null ? super.getPublishFuture() : publishFuture; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/utils/TransactionBufferTestProvider.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/utils/TransactionBufferTestProvider.java new file mode 100644 index 0000000000000..7bc93c0e7cf25 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/utils/TransactionBufferTestProvider.java @@ -0,0 +1,33 @@ +/* + * 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.transaction.buffer.utils; + +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; +import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider; + +public class TransactionBufferTestProvider implements TransactionBufferProvider { + + @Override + public TransactionBuffer newTransactionBuffer(Topic originTopic) { + return new TransactionBufferTestImpl((PersistentTopic) originTopic); + } +} + From 587af853fbf976d5007e17dba910a4a14e3e85e8 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 28 Aug 2024 20:50:52 -0700 Subject: [PATCH 473/580] [feat][misc] PIP-264: Add OpenTelemetry authentication and token metrics (#23016) --- .../AuthenticationProviderAthenz.java | 17 +- .../AuthenticationProviderAthenzTest.java | 14 +- .../oidc/AuthenticationProviderOpenID.java | 27 ++- .../broker/authentication/oidc/JwksCache.java | 26 +-- .../oidc/OpenIDProviderMetadataCache.java | 24 +-- ...ticationProviderOpenIDIntegrationTest.java | 20 +-- .../AuthenticationProviderOpenIDTest.java | 111 ++++++------ .../AuthenticationProviderSasl.java | 6 + .../authentication/SaslAuthenticateTest.java | 4 +- pulsar-broker-common/pom.xml | 5 + .../AuthenticationProvider.java | 25 +++ .../AuthenticationProviderBasic.java | 17 +- .../AuthenticationProviderList.java | 44 +++-- .../AuthenticationProviderTls.java | 17 +- .../AuthenticationProviderToken.java | 47 +++-- .../authentication/AuthenticationService.java | 12 +- .../metrics/AuthenticationMetrics.java | 65 ++++++- .../metrics/AuthenticationMetricsToken.java | 109 ++++++++++++ .../AuthenticationProviderBasicTest.java | 8 +- .../AuthenticationProviderListTest.java | 5 +- .../AuthenticationProviderTokenTest.java | 73 ++++---- .../pulsar/broker/service/BrokerService.java | 3 +- .../stats/PulsarBrokerOpenTelemetry.java | 5 + .../broker/stats/MetadataStoreStatsTest.java | 4 +- .../OpenTelemetryAuthenticationStatsTest.java | 161 ++++++++++++++++++ .../broker/stats/PrometheusMetricsTest.java | 42 +++-- 26 files changed, 674 insertions(+), 217 deletions(-) create mode 100644 pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/metrics/AuthenticationMetricsToken.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryAuthenticationStatsTest.java diff --git a/pulsar-broker-auth-athenz/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderAthenz.java b/pulsar-broker-auth-athenz/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderAthenz.java index 652a922b9a5ad..499ebefc8a081 100644 --- a/pulsar-broker-auth-athenz/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderAthenz.java +++ b/pulsar-broker-auth-athenz/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderAthenz.java @@ -43,6 +43,8 @@ public class AuthenticationProviderAthenz implements AuthenticationProvider { private List domainNameList = null; private int allowedOffset = 30; + private AuthenticationMetrics authenticationMetrics; + public enum ErrorCode { UNKNOWN, NO_CLIENT, @@ -54,6 +56,14 @@ public enum ErrorCode { @Override public void initialize(ServiceConfiguration config) throws IOException { + initialize(Context.builder().config(config).build()); + } + + @Override + public void initialize(Context context) throws IOException { + authenticationMetrics = new AuthenticationMetrics(context.getOpenTelemetry(), + getClass().getSimpleName(), getAuthMethodName()); + var config = context.getConfig(); String domainNames; if (config.getProperty(DOMAIN_NAME_LIST) != null) { domainNames = (String) config.getProperty(DOMAIN_NAME_LIST); @@ -86,6 +96,11 @@ public String getAuthMethodName() { return "athenz"; } + @Override + public void incrementFailureMetric(Enum errorCode) { + authenticationMetrics.recordFailure(errorCode); + } + @Override public String authenticate(AuthenticationDataSource authData) throws AuthenticationException { SocketAddress clientAddress; @@ -141,7 +156,7 @@ public String authenticate(AuthenticationDataSource authData) throws Authenticat if (token.validate(ztsPublicKey, allowedOffset, false, null)) { log.debug("Athenz Role Token : {}, Authenticated for Client: {}", roleToken, clientAddress); - AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName()); + authenticationMetrics.recordSuccess(); return token.getPrincipal(); } else { errorCode = ErrorCode.INVALID_TOKEN; diff --git a/pulsar-broker-auth-athenz/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderAthenzTest.java b/pulsar-broker-auth-athenz/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderAthenzTest.java index a5211c2f81455..63dcd09397886 100644 --- a/pulsar-broker-auth-athenz/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderAthenzTest.java +++ b/pulsar-broker-auth-athenz/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderAthenzTest.java @@ -20,10 +20,8 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.fail; - import com.yahoo.athenz.auth.token.RoleToken; import com.yahoo.athenz.zpe.ZpeConsts; - import java.io.IOException; import java.net.InetSocketAddress; import java.nio.file.Files; @@ -31,9 +29,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Properties; - import javax.naming.AuthenticationException; - import org.apache.pulsar.broker.ServiceConfiguration; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -55,7 +51,7 @@ public void setup() throws Exception { // Initialize authentication provider provider = new AuthenticationProviderAthenz(); - provider.initialize(config); + provider.initialize(AuthenticationProvider.Context.builder().config(config).build()); // Specify Athenz configuration file for AuthZpeClient which is used in AuthenticationProviderAthenz System.setProperty(ZpeConsts.ZPE_PROP_ATHENZ_CONF, "./src/test/resources/athenz.conf.test"); @@ -69,7 +65,7 @@ public void testInitilizeFromSystemPropeties() { emptyConf.setProperties(emptyProp); AuthenticationProviderAthenz sysPropProvider1 = new AuthenticationProviderAthenz(); try { - sysPropProvider1.initialize(emptyConf); + sysPropProvider1.initialize(AuthenticationProvider.Context.builder().config(emptyConf).build()); assertEquals(sysPropProvider1.getAllowedOffset(), 30); // default allowed offset is 30 sec } catch (Exception e) { fail("Fail to Read pulsar.athenz.domain.names from System Properties"); @@ -78,7 +74,7 @@ public void testInitilizeFromSystemPropeties() { System.setProperty("pulsar.athenz.role.token_allowed_offset", "0"); AuthenticationProviderAthenz sysPropProvider2 = new AuthenticationProviderAthenz(); try { - sysPropProvider2.initialize(config); + sysPropProvider2.initialize(AuthenticationProvider.Context.builder().config(config).build()); assertEquals(sysPropProvider2.getAllowedOffset(), 0); } catch (Exception e) { fail("Failed to get allowed offset from system property"); @@ -87,7 +83,7 @@ public void testInitilizeFromSystemPropeties() { System.setProperty("pulsar.athenz.role.token_allowed_offset", "invalid"); AuthenticationProviderAthenz sysPropProvider3 = new AuthenticationProviderAthenz(); try { - sysPropProvider3.initialize(config); + sysPropProvider3.initialize(AuthenticationProvider.Context.builder().config(config).build()); fail("Invalid allowed offset should not be specified"); } catch (IOException e) { } @@ -95,7 +91,7 @@ public void testInitilizeFromSystemPropeties() { System.setProperty("pulsar.athenz.role.token_allowed_offset", "-1"); AuthenticationProviderAthenz sysPropProvider4 = new AuthenticationProviderAthenz(); try { - sysPropProvider4.initialize(config); + sysPropProvider4.initialize(AuthenticationProvider.Context.builder().config(config).build()); fail("Negative allowed offset should not be specified"); } catch (IOException e) { } diff --git a/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java b/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java index a9d812c10b06a..38f618091333a 100644 --- a/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java +++ b/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenID.java @@ -88,8 +88,6 @@ public class AuthenticationProviderOpenID implements AuthenticationProvider { private static final Logger log = LoggerFactory.getLogger(AuthenticationProviderOpenID.class); - private static final String SIMPLE_NAME = AuthenticationProviderOpenID.class.getSimpleName(); - // Must match the value used by the OAuth2 Client Plugin. private static final String AUTH_METHOD_NAME = "token"; @@ -148,8 +146,18 @@ public class AuthenticationProviderOpenID implements AuthenticationProvider { private String[] allowedAudiences; private ApiClient k8sApiClient; + private AuthenticationMetrics authenticationMetrics; + @Override public void initialize(ServiceConfiguration config) throws IOException { + initialize(Context.builder().config(config).build()); + } + + @Override + public void initialize(Context context) throws IOException { + authenticationMetrics = new AuthenticationMetrics(context.getOpenTelemetry(), + getClass().getSimpleName(), getAuthMethodName()); + var config = context.getConfig(); this.allowedAudiences = validateAllowedAudiences(getConfigValueAsSet(config, ALLOWED_AUDIENCES)); this.roleClaim = getConfigValueAsString(config, ROLE_CLAIM, ROLE_CLAIM_DEFAULT); this.isRoleClaimNotSubject = !ROLE_CLAIM_DEFAULT.equals(roleClaim); @@ -181,8 +189,8 @@ public void initialize(ServiceConfiguration config) throws IOException { .build(); httpClient = new DefaultAsyncHttpClient(clientConfig); k8sApiClient = fallbackDiscoveryMode != FallbackDiscoveryMode.DISABLED ? Config.defaultClient() : null; - this.openIDProviderMetadataCache = new OpenIDProviderMetadataCache(config, httpClient, k8sApiClient); - this.jwksCache = new JwksCache(config, httpClient, k8sApiClient); + this.openIDProviderMetadataCache = new OpenIDProviderMetadataCache(this, config, httpClient, k8sApiClient); + this.jwksCache = new JwksCache(this, config, httpClient, k8sApiClient); } @Override @@ -190,6 +198,11 @@ public String getAuthMethodName() { return AUTH_METHOD_NAME; } + @Override + public void incrementFailureMetric(Enum errorCode) { + authenticationMetrics.recordFailure(errorCode); + } + /** * Authenticate the parameterized {@link AuthenticationDataSource} by verifying the issuer is an allowed issuer, * then retrieving the JWKS URI from the issuer, then retrieving the Public key from the JWKS URI, and finally @@ -219,7 +232,7 @@ CompletableFuture authenticateTokenAsync(AuthenticationDataSource au return authenticateToken(token) .whenComplete((jwt, e) -> { if (jwt != null) { - AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName()); + authenticationMetrics.recordSuccess(); } // Failure metrics are incremented within methods above }); @@ -463,10 +476,6 @@ DecodedJWT verifyJWT(PublicKey publicKey, } } - static void incrementFailureMetric(AuthenticationExceptionCode code) { - AuthenticationMetrics.authenticateFailure(SIMPLE_NAME, AUTH_METHOD_NAME, code); - } - /** * Validate the configured allow list of allowedIssuers. The allowedIssuers set must be nonempty in order for * the plugin to authenticate any token. Thus, it fails initialization if the configuration is diff --git a/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/JwksCache.java b/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/JwksCache.java index 73934e9c1e05e..c88661c39c6c2 100644 --- a/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/JwksCache.java +++ b/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/JwksCache.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.authentication.oidc; +import static org.apache.pulsar.broker.authentication.oidc.AuthenticationExceptionCode.ERROR_RETRIEVING_PUBLIC_KEY; import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.CACHE_EXPIRATION_SECONDS; import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.CACHE_EXPIRATION_SECONDS_DEFAULT; import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.CACHE_REFRESH_AFTER_WRITE_SECONDS; @@ -26,7 +27,6 @@ import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.CACHE_SIZE_DEFAULT; import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.KEY_ID_CACHE_MISS_REFRESH_SECONDS; import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.KEY_ID_CACHE_MISS_REFRESH_SECONDS_DEFAULT; -import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.incrementFailureMetric; import static org.apache.pulsar.broker.authentication.oidc.ConfigUtils.getConfigValueAsInt; import com.auth0.jwk.Jwk; import com.fasterxml.jackson.databind.ObjectMapper; @@ -49,6 +49,7 @@ import java.util.concurrent.TimeUnit; import javax.naming.AuthenticationException; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.authentication.AuthenticationProvider; import org.asynchttpclient.AsyncHttpClient; public class JwksCache { @@ -60,8 +61,11 @@ public class JwksCache { private final ObjectReader reader = new ObjectMapper().readerFor(HashMap.class); private final AsyncHttpClient httpClient; private final OpenidApi openidApi; + private final AuthenticationProvider authenticationProvider; - JwksCache(ServiceConfiguration config, AsyncHttpClient httpClient, ApiClient apiClient) throws IOException { + JwksCache(AuthenticationProvider authenticationProvider, ServiceConfiguration config, + AsyncHttpClient httpClient, ApiClient apiClient) throws IOException { + this.authenticationProvider = authenticationProvider; // Store the clients this.httpClient = httpClient; this.openidApi = apiClient != null ? new OpenidApi(apiClient) : null; @@ -91,7 +95,7 @@ public class JwksCache { CompletableFuture getJwk(String jwksUri, String keyId) { if (jwksUri == null) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PUBLIC_KEY); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PUBLIC_KEY); return CompletableFuture.failedFuture(new IllegalArgumentException("jwksUri must not be null.")); } return getJwkAndMaybeReload(Optional.of(jwksUri), keyId, false); @@ -139,10 +143,10 @@ private CompletableFuture> getJwksFromJwksUri(String jwksUri) { reader.readValue(result.getResponseBodyAsBytes()); future.complete(convertToJwks(jwksUri, jwks)); } catch (AuthenticationException e) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PUBLIC_KEY); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PUBLIC_KEY); future.completeExceptionally(e); } catch (Exception e) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PUBLIC_KEY); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PUBLIC_KEY); future.completeExceptionally(new AuthenticationException( "Error retrieving public key at " + jwksUri + ": " + e.getMessage())); } @@ -152,7 +156,7 @@ private CompletableFuture> getJwksFromJwksUri(String jwksUri) { CompletableFuture getJwkFromKubernetesApiServer(String keyId) { if (openidApi == null) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PUBLIC_KEY); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PUBLIC_KEY); return CompletableFuture.failedFuture(new AuthenticationException( "Failed to retrieve public key from Kubernetes API server: Kubernetes fallback is not enabled.")); } @@ -165,7 +169,7 @@ private CompletableFuture> getJwksFromKubernetesApiServer() { openidApi.getServiceAccountIssuerOpenIDKeysetAsync(new ApiCallback() { @Override public void onFailure(ApiException e, int statusCode, Map> responseHeaders) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PUBLIC_KEY); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PUBLIC_KEY); // We want the message and responseBody here: https://github.com/kubernetes-client/java/issues/2066. future.completeExceptionally( new AuthenticationException("Failed to retrieve public key from Kubernetes API server. " @@ -178,10 +182,10 @@ public void onSuccess(String result, int statusCode, Map> r HashMap jwks = reader.readValue(result); future.complete(convertToJwks("Kubernetes API server", jwks)); } catch (AuthenticationException e) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PUBLIC_KEY); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PUBLIC_KEY); future.completeExceptionally(e); } catch (Exception e) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PUBLIC_KEY); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PUBLIC_KEY); future.completeExceptionally(new AuthenticationException( "Error retrieving public key at Kubernetes API server: " + e.getMessage())); } @@ -198,7 +202,7 @@ public void onDownloadProgress(long bytesRead, long contentLength, boolean done) } }); } catch (ApiException e) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PUBLIC_KEY); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PUBLIC_KEY); future.completeExceptionally( new AuthenticationException("Failed to retrieve public key from Kubernetes API server: " + e.getMessage())); @@ -212,7 +216,7 @@ private Jwk getJwkForKID(Optional maybeJwksUri, List jwks, String k return jwk; } } - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PUBLIC_KEY); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PUBLIC_KEY); throw new IllegalArgumentException("No JWK found for Key ID " + keyId); } diff --git a/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/OpenIDProviderMetadataCache.java b/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/OpenIDProviderMetadataCache.java index 111399adbd72b..cffa52b00aab9 100644 --- a/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/OpenIDProviderMetadataCache.java +++ b/pulsar-broker-auth-oidc/src/main/java/org/apache/pulsar/broker/authentication/oidc/OpenIDProviderMetadataCache.java @@ -18,13 +18,13 @@ */ package org.apache.pulsar.broker.authentication.oidc; +import static org.apache.pulsar.broker.authentication.oidc.AuthenticationExceptionCode.ERROR_RETRIEVING_PROVIDER_METADATA; import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.CACHE_EXPIRATION_SECONDS; import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.CACHE_EXPIRATION_SECONDS_DEFAULT; import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.CACHE_REFRESH_AFTER_WRITE_SECONDS; import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.CACHE_REFRESH_AFTER_WRITE_SECONDS_DEFAULT; import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.CACHE_SIZE; import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.CACHE_SIZE_DEFAULT; -import static org.apache.pulsar.broker.authentication.oidc.AuthenticationProviderOpenID.incrementFailureMetric; import static org.apache.pulsar.broker.authentication.oidc.ConfigUtils.getConfigValueAsInt; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectReader; @@ -43,6 +43,7 @@ import javax.annotation.Nonnull; import javax.naming.AuthenticationException; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.authentication.AuthenticationProvider; import org.asynchttpclient.AsyncHttpClient; /** @@ -51,13 +52,16 @@ class OpenIDProviderMetadataCache { private final ObjectReader reader = new ObjectMapper().readerFor(OpenIDProviderMetadata.class); + private final AuthenticationProvider authenticationProvider; private final AsyncHttpClient httpClient; private final WellKnownApi wellKnownApi; private final AsyncLoadingCache, OpenIDProviderMetadata> cache; private static final String WELL_KNOWN_OPENID_CONFIG = ".well-known/openid-configuration"; private static final String SLASH_WELL_KNOWN_OPENID_CONFIG = "/" + WELL_KNOWN_OPENID_CONFIG; - OpenIDProviderMetadataCache(ServiceConfiguration config, AsyncHttpClient httpClient, ApiClient apiClient) { + OpenIDProviderMetadataCache(AuthenticationProvider authenticationProvider, ServiceConfiguration config, + AsyncHttpClient httpClient, ApiClient apiClient) { + this.authenticationProvider = authenticationProvider; int maxSize = getConfigValueAsInt(config, CACHE_SIZE, CACHE_SIZE_DEFAULT); int refreshAfterWriteSeconds = getConfigValueAsInt(config, CACHE_REFRESH_AFTER_WRITE_SECONDS, CACHE_REFRESH_AFTER_WRITE_SECONDS_DEFAULT); @@ -124,10 +128,10 @@ private CompletableFuture loadOpenIDProviderMetadataForI verifyIssuer(issuer, openIDProviderMetadata, false); future.complete(openIDProviderMetadata); } catch (AuthenticationException e) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PROVIDER_METADATA); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PROVIDER_METADATA); future.completeExceptionally(e); } catch (Exception e) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PROVIDER_METADATA); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PROVIDER_METADATA); future.completeExceptionally(new AuthenticationException( "Error retrieving OpenID Provider Metadata at " + issuer + ": " + e.getMessage())); } @@ -151,7 +155,7 @@ CompletableFuture getOpenIDProviderMetadataForKubernetes verifyIssuer(issClaim, openIDProviderMetadata, true); future.complete(openIDProviderMetadata); } catch (AuthenticationException e) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PROVIDER_METADATA); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PROVIDER_METADATA); future.completeExceptionally(e); } return future; @@ -164,7 +168,7 @@ private CompletableFuture loadOpenIDProviderMetadataForK wellKnownApi.getServiceAccountIssuerOpenIDConfigurationAsync(new ApiCallback<>() { @Override public void onFailure(ApiException e, int statusCode, Map> responseHeaders) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PROVIDER_METADATA); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PROVIDER_METADATA); // We want the message and responseBody here: https://github.com/kubernetes-client/java/issues/2066. future.completeExceptionally(new AuthenticationException( "Error retrieving OpenID Provider Metadata from Kubernetes API server. Message: " @@ -179,7 +183,7 @@ public void onSuccess(String result, int statusCode, Map> r OpenIDProviderMetadata openIDProviderMetadata = reader.readValue(result); future.complete(openIDProviderMetadata); } catch (Exception e) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PROVIDER_METADATA); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PROVIDER_METADATA); future.completeExceptionally(new AuthenticationException( "Error retrieving OpenID Provider Metadata from Kubernetes API Server: " + e.getMessage())); @@ -197,7 +201,7 @@ public void onDownloadProgress(long bytesRead, long contentLength, boolean done) } }); } catch (ApiException e) { - incrementFailureMetric(AuthenticationExceptionCode.ERROR_RETRIEVING_PROVIDER_METADATA); + authenticationProvider.incrementFailureMetric(ERROR_RETRIEVING_PROVIDER_METADATA); future.completeExceptionally(new AuthenticationException( "Error retrieving OpenID Provider Metadata from Kubernetes API server: " + e.getMessage())); } @@ -221,10 +225,10 @@ private void verifyIssuer(@Nonnull String issuer, OpenIDProviderMetadata metadat boolean isK8s) throws AuthenticationException { if (!issuer.equals(metadata.getIssuer())) { if (isK8s) { - incrementFailureMetric(AuthenticationExceptionCode.UNSUPPORTED_ISSUER); + authenticationProvider.incrementFailureMetric(AuthenticationExceptionCode.UNSUPPORTED_ISSUER); throw new AuthenticationException("Issuer not allowed: " + issuer); } else { - incrementFailureMetric(AuthenticationExceptionCode.ISSUER_MISMATCH); + authenticationProvider.incrementFailureMetric(AuthenticationExceptionCode.ISSUER_MISMATCH); throw new AuthenticationException(String.format("Issuer URL mismatch: [%s] should match [%s]", issuer, metadata.getIssuer())); } diff --git a/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java b/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java index e11fd8395a5bf..f4663a9ee3ce6 100644 --- a/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java +++ b/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java @@ -260,7 +260,7 @@ void beforeClass() throws IOException { Files.write(Path.of(System.getenv("KUBECONFIG")), kubeConfig.getBytes()); provider = new AuthenticationProviderOpenID(); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); } @AfterClass @@ -358,7 +358,7 @@ public void testKidCacheMissWhenRefreshConfigZero() throws Exception { @Cleanup AuthenticationProviderOpenID provider = new AuthenticationProviderOpenID(); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String role = "superuser"; String token = generateToken(validJwk, issuerWithMissingKid, role, "allowed-audience", 0L, 0L, 10000L); @@ -379,7 +379,7 @@ public void testKidCacheMissWhenRefreshConfigLongerThanDelta() throws Exception @Cleanup AuthenticationProviderOpenID provider = new AuthenticationProviderOpenID(); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String role = "superuser"; String token = generateToken(validJwk, issuerWithMissingKid, role, "allowed-audience", 0L, 0L, 10000L); @@ -407,7 +407,7 @@ public void testKubernetesApiServerAsDiscoverTrustedIssuerSuccess() throws Excep @Cleanup AuthenticationProviderOpenID provider = new AuthenticationProviderOpenID(); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String role = "superuser"; // We use the normal issuer on the token because the /k8s endpoint is configured via the kube config file @@ -441,7 +441,7 @@ public void testKubernetesApiServerAsDiscoverTrustedIssuerFailsDueToMismatchedIs @Cleanup AuthenticationProviderOpenID provider = new AuthenticationProviderOpenID(); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String role = "superuser"; String token = generateToken(validJwk, "http://not-the-k8s-issuer", role, "allowed-audience", 0L, 0L, 10000L); @@ -468,7 +468,7 @@ public void testKubernetesApiServerAsDiscoverPublicKeySuccess() throws Exception @Cleanup AuthenticationProviderOpenID provider = new AuthenticationProviderOpenID(); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String role = "superuser"; String token = generateToken(validJwk, issuer, role, "allowed-audience", 0L, 0L, 10000L); @@ -499,7 +499,7 @@ public void testKubernetesApiServerAsDiscoverPublicKeyFailsDueToMismatchedIssuer @Cleanup AuthenticationProviderOpenID provider = new AuthenticationProviderOpenID(); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String role = "superuser"; String token = generateToken(validJwk, "http://not-the-k8s-issuer", role, "allowed-audience", 0L, 0L, 10000L); @@ -562,7 +562,7 @@ public void testAuthenticationStateOpenIDForTokenExpiration() throws Exception { props.setProperty(AuthenticationProviderOpenID.ACCEPTED_TIME_LEEWAY_SECONDS, "10"); @Cleanup AuthenticationProviderOpenID provider = new AuthenticationProviderOpenID(); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String role = "superuser"; String token = generateToken(validJwk, issuer, role, "allowed-audience", 0L, 0L, 0L); @@ -635,7 +635,7 @@ void ensureRoleClaimForNonSubClaimReturnsRole() throws Exception { props.setProperty(AuthenticationProviderOpenID.ISSUER_TRUST_CERTS_FILE_PATH, caCert); ServiceConfiguration config = new ServiceConfiguration(); config.setProperties(props); - provider.initialize(config); + provider.initialize(AuthenticationProvider.Context.builder().config(config).build()); // Build a JWT with a custom claim HashMap claims = new HashMap(); @@ -656,7 +656,7 @@ void ensureRoleClaimForNonSubClaimFailsWhenClaimIsMissing() throws Exception { props.setProperty(AuthenticationProviderOpenID.ISSUER_TRUST_CERTS_FILE_PATH, caCert); ServiceConfiguration config = new ServiceConfiguration(); config.setProperties(props); - provider.initialize(config); + provider.initialize(AuthenticationProvider.Context.builder().config(config).build()); // Build a JWT without the "test" claim, which should cause the authentication to fail String token = generateToken(validJwk, issuer, "not-my-role", "allowed-audience", 0L, diff --git a/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDTest.java b/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDTest.java index f5bb584d16f72..4a12f61528aca 100644 --- a/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDTest.java +++ b/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.authentication.oidc; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.testng.Assert.assertNull; import com.auth0.jwt.JWT; import com.auth0.jwt.interfaces.DecodedJWT; @@ -29,9 +30,9 @@ import java.sql.Date; import java.time.Instant; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Properties; import java.util.Set; @@ -39,6 +40,7 @@ import lombok.Cleanup; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataCommand; +import org.apache.pulsar.broker.authentication.AuthenticationProvider; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -59,16 +61,31 @@ public class AuthenticationProviderOpenIDTest { // https://www.rfc-editor.org/rfc/rfc7518#section-3.1 + private static final Set SUPPORTED_ALGORITHMS = Set.of( + SignatureAlgorithm.RS256, + SignatureAlgorithm.RS384, + SignatureAlgorithm.RS512, + SignatureAlgorithm.ES256, + SignatureAlgorithm.ES384, + SignatureAlgorithm.ES512 + ); + @DataProvider(name = "supportedAlgorithms") public static Object[][] supportedAlgorithms() { - return new Object[][] { - { SignatureAlgorithm.RS256 }, - { SignatureAlgorithm.RS384 }, - { SignatureAlgorithm.RS512 }, - { SignatureAlgorithm.ES256 }, - { SignatureAlgorithm.ES384 }, - { SignatureAlgorithm.ES512 } - }; + return buildDataProvider(SUPPORTED_ALGORITHMS); + } + + @DataProvider(name = "unsupportedAlgorithms") + public static Object[][] unsupportedAlgorithms() { + var unsupportedAlgorithms = Set.of(SignatureAlgorithm.values()) + .stream() + .filter(alg -> !SUPPORTED_ALGORITHMS.contains(alg)) + .toList(); + return buildDataProvider(unsupportedAlgorithms); + } + + private static Object[][] buildDataProvider(Collection collection) { + return collection.stream().map(o -> new Object[] { o }).toArray(Object[][]::new); } // Provider to use in common tests that are not verifying the configuration of the provider itself. @@ -83,7 +100,7 @@ public void setup() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); basicProvider = new AuthenticationProviderOpenID(); - basicProvider.initialize(conf); + basicProvider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); } @AfterClass @@ -100,29 +117,19 @@ public void testNullToken() throws IOException { } @Test - public void testThatNullAlgFails() throws IOException { - @Cleanup - AuthenticationProviderOpenID provider = new AuthenticationProviderOpenID(); - Assert.assertThrows(AuthenticationException.class, - () -> provider.verifyJWT(null, null, null)); + public void testThatNullAlgFails() { + assertThatThrownBy(() -> basicProvider.verifyJWT(null, null, null)) + .isInstanceOf(AuthenticationException.class) + .hasMessage("PublicKey algorithm cannot be null"); } - @Test - public void testThatUnsupportedAlgsThrowExceptions() { - Set unsupportedAlgs = new HashSet<>(Set.of(SignatureAlgorithm.values())); - Arrays.stream(supportedAlgorithms()).map(o -> (SignatureAlgorithm) o[0]).toList() - .forEach(unsupportedAlgs::remove); - unsupportedAlgs.forEach(unsupportedAlg -> { - try { - @Cleanup - AuthenticationProviderOpenID provider = new AuthenticationProviderOpenID(); - // We don't create a public key because it's irrelevant - Assert.assertThrows(AuthenticationException.class, - () -> provider.verifyJWT(null, unsupportedAlg.getValue(), null)); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); + @Test(dataProvider = "unsupportedAlgorithms") + public void testThatUnsupportedAlgsThrowExceptions(SignatureAlgorithm unsupportedAlg) { + var algorithm = unsupportedAlg.getValue(); + // We don't create a public key because it's irrelevant + assertThatThrownBy(() -> basicProvider.verifyJWT(null, algorithm, null)) + .isInstanceOf(AuthenticationException.class) + .hasMessage("Unsupported algorithm: " + algorithm); } @Test(dataProvider = "supportedAlgorithms") @@ -141,29 +148,27 @@ public void testThatSupportedAlgsWork(SignatureAlgorithm alg) throws Authenticat @Test public void testThatSupportedAlgWithMismatchedPublicKeyFromDifferentAlgFamilyFails() throws IOException { KeyPair keyPair = Keys.keyPairFor(SignatureAlgorithm.RS256); - @Cleanup - AuthenticationProviderOpenID provider = new AuthenticationProviderOpenID(); DefaultJwtBuilder defaultJwtBuilder = new DefaultJwtBuilder(); addValidMandatoryClaims(defaultJwtBuilder, basicProviderAudience); defaultJwtBuilder.signWith(keyPair.getPrivate()); DecodedJWT jwt = JWT.decode(defaultJwtBuilder.compact()); // Choose a different algorithm from a different alg family - Assert.assertThrows(AuthenticationException.class, - () -> provider.verifyJWT(keyPair.getPublic(), SignatureAlgorithm.ES512.getValue(), jwt)); + assertThatThrownBy(() -> basicProvider.verifyJWT(keyPair.getPublic(), SignatureAlgorithm.ES512.getValue(), jwt)) + .isInstanceOf(AuthenticationException.class) + .hasMessage("Expected PublicKey alg [ES512] does match actual alg."); } @Test - public void testThatSupportedAlgWithMismatchedPublicKeyFromSameAlgFamilyFails() throws IOException { + public void testThatSupportedAlgWithMismatchedPublicKeyFromSameAlgFamilyFails() { KeyPair keyPair = Keys.keyPairFor(SignatureAlgorithm.RS256); - @Cleanup - AuthenticationProviderOpenID provider = new AuthenticationProviderOpenID(); DefaultJwtBuilder defaultJwtBuilder = new DefaultJwtBuilder(); addValidMandatoryClaims(defaultJwtBuilder, basicProviderAudience); defaultJwtBuilder.signWith(keyPair.getPrivate()); DecodedJWT jwt = JWT.decode(defaultJwtBuilder.compact()); // Choose a different algorithm but within the same alg family as above - Assert.assertThrows(AuthenticationException.class, - () -> provider.verifyJWT(keyPair.getPublic(), SignatureAlgorithm.RS512.getValue(), jwt)); + assertThatThrownBy(() -> basicProvider.verifyJWT(keyPair.getPublic(), SignatureAlgorithm.RS512.getValue(), jwt)) + .isInstanceOf(AuthenticationException.class) + .hasMessageStartingWith("JWT algorithm does not match Public Key algorithm"); } @Test @@ -217,7 +222,7 @@ public void ensureRecentlyExpiredTokenWithinConfiguredLeewaySucceeds() throws Ex props.setProperty(AuthenticationProviderOpenID.ALLOWED_TOKEN_ISSUERS, "https://localhost:8080"); ServiceConfiguration config = new ServiceConfiguration(); config.setProperties(props); - provider.initialize(config); + provider.initialize(AuthenticationProvider.Context.builder().config(config).build()); // Build the JWT with an only recently expired token DefaultJwtBuilder defaultJwtBuilder = new DefaultJwtBuilder(); @@ -244,7 +249,8 @@ public void ensureEmptyIssuersFailsInitialization() throws IOException { props.setProperty(AuthenticationProviderOpenID.ALLOWED_TOKEN_ISSUERS, ""); ServiceConfiguration config = new ServiceConfiguration(); config.setProperties(props); - Assert.assertThrows(IllegalArgumentException.class, () -> provider.initialize(config)); + Assert.assertThrows(IllegalArgumentException.class, + () -> provider.initialize(AuthenticationProvider.Context.builder().config(config).build())); } @Test @@ -256,7 +262,8 @@ public void ensureEmptyIssuersFailsInitializationWithDisabledDiscoveryMode() thr props.setProperty(AuthenticationProviderOpenID.FALLBACK_DISCOVERY_MODE, "DISABLED"); ServiceConfiguration config = new ServiceConfiguration(); config.setProperties(props); - Assert.assertThrows(IllegalArgumentException.class, () -> provider.initialize(config)); + Assert.assertThrows(IllegalArgumentException.class, + () -> provider.initialize(AuthenticationProvider.Context.builder().config(config).build())); } @Test @@ -269,7 +276,7 @@ public void ensureEmptyIssuersWithK8sTrustedIssuerEnabledPassesInitialization() props.setProperty(AuthenticationProviderOpenID.FALLBACK_DISCOVERY_MODE, "KUBERNETES_DISCOVER_TRUSTED_ISSUER"); ServiceConfiguration config = new ServiceConfiguration(); config.setProperties(props); - provider.initialize(config); + provider.initialize(AuthenticationProvider.Context.builder().config(config).build()); } @Test @@ -282,7 +289,7 @@ public void ensureEmptyIssuersWithK8sPublicKeyEnabledPassesInitialization() thro props.setProperty(AuthenticationProviderOpenID.FALLBACK_DISCOVERY_MODE, "KUBERNETES_DISCOVER_PUBLIC_KEYS"); ServiceConfiguration config = new ServiceConfiguration(); config.setProperties(props); - provider.initialize(config); + provider.initialize(AuthenticationProvider.Context.builder().config(config).build()); } @Test @@ -292,7 +299,8 @@ public void ensureNullIssuersFailsInitialization() throws IOException { ServiceConfiguration config = new ServiceConfiguration(); // Make sure this still defaults to null. assertNull(config.getProperties().get(AuthenticationProviderOpenID.ALLOWED_TOKEN_ISSUERS)); - Assert.assertThrows(IllegalArgumentException.class, () -> provider.initialize(config)); + Assert.assertThrows(IllegalArgumentException.class, + () -> provider.initialize(AuthenticationProvider.Context.builder().config(config).build())); } @Test @@ -303,7 +311,8 @@ public void ensureInsecureIssuerFailsInitialization() throws IOException { props.setProperty(AuthenticationProviderOpenID.ALLOWED_TOKEN_ISSUERS, "https://myissuer.com,http://myissuer.com"); ServiceConfiguration config = new ServiceConfiguration(); config.setProperties(props); - Assert.assertThrows(IllegalArgumentException.class, () -> provider.initialize(config)); + Assert.assertThrows(IllegalArgumentException.class, + () -> provider.initialize(AuthenticationProvider.Context.builder().config(config).build())); } @Test void ensureMissingRoleClaimReturnsNull() throws Exception { @@ -325,7 +334,7 @@ public void ensureInsecureIssuerFailsInitialization() throws IOException { props.setProperty(AuthenticationProviderOpenID.ROLE_CLAIM, "sub"); ServiceConfiguration config = new ServiceConfiguration(); config.setProperties(props); - provider.initialize(config); + provider.initialize(AuthenticationProvider.Context.builder().config(config).build()); // Build an empty JWT DefaultJwtBuilder defaultJwtBuilder = new DefaultJwtBuilder(); @@ -345,7 +354,7 @@ public void ensureInsecureIssuerFailsInitialization() throws IOException { props.setProperty(AuthenticationProviderOpenID.ROLE_CLAIM, "roles"); ServiceConfiguration config = new ServiceConfiguration(); config.setProperties(props); - provider.initialize(config); + provider.initialize(AuthenticationProvider.Context.builder().config(config).build()); // Build an empty JWT DefaultJwtBuilder defaultJwtBuilder = new DefaultJwtBuilder(); @@ -367,7 +376,7 @@ public void ensureInsecureIssuerFailsInitialization() throws IOException { props.setProperty(AuthenticationProviderOpenID.ROLE_CLAIM, "roles"); ServiceConfiguration config = new ServiceConfiguration(); config.setProperties(props); - provider.initialize(config); + provider.initialize(AuthenticationProvider.Context.builder().config(config).build()); // Build an empty JWT DefaultJwtBuilder defaultJwtBuilder = new DefaultJwtBuilder(); @@ -389,7 +398,7 @@ public void ensureInsecureIssuerFailsInitialization() throws IOException { props.setProperty(AuthenticationProviderOpenID.ROLE_CLAIM, "roles"); ServiceConfiguration config = new ServiceConfiguration(); config.setProperties(props); - provider.initialize(config); + provider.initialize(AuthenticationProvider.Context.builder().config(config).build()); // Build an empty JWT DefaultJwtBuilder defaultJwtBuilder = new DefaultJwtBuilder(); diff --git a/pulsar-broker-auth-sasl/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderSasl.java b/pulsar-broker-auth-sasl/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderSasl.java index 2616f90c664ed..f8841193ba2d2 100644 --- a/pulsar-broker-auth-sasl/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderSasl.java +++ b/pulsar-broker-auth-sasl/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderSasl.java @@ -78,6 +78,12 @@ public class AuthenticationProviderSasl implements AuthenticationProvider { @Override public void initialize(ServiceConfiguration config) throws IOException { + initialize(Context.builder().config(config).build()); + } + + @Override + public void initialize(Context context) throws IOException { + var config = context.getConfig(); this.configuration = new HashMap<>(); final String allowedIdsPatternRegExp = config.getSaslJaasClientAllowedIds(); configuration.put(JAAS_CLIENT_ALLOWED_IDS, allowedIdsPatternRegExp); diff --git a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/SaslAuthenticateTest.java b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/SaslAuthenticateTest.java index ae282a49dc36c..226ec15d33afe 100644 --- a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/SaslAuthenticateTest.java +++ b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/SaslAuthenticateTest.java @@ -312,7 +312,7 @@ public void testSaslOnlyAuthFirstStage() throws Exception { AuthenticationProviderSasl saslServer = new AuthenticationProviderSasl(); // The cache expiration time is set to 50ms. Residual auth info should be cleaned up conf.setInflightSaslContextExpiryMs(50); - saslServer.initialize(conf); + saslServer.initialize(AuthenticationProvider.Context.builder().config(conf).build()); HttpServletRequest servletRequest = mock(HttpServletRequest.class); doReturn("Init").when(servletRequest).getHeader("State"); @@ -360,7 +360,7 @@ public void testMaxInflightContext() throws Exception { doReturn("Init").when(servletRequest).getHeader("State"); conf.setInflightSaslContextExpiryMs(Integer.MAX_VALUE); conf.setMaxInflightSaslContext(1); - saslServer.initialize(conf); + saslServer.initialize(AuthenticationProvider.Context.builder().config(conf).build()); // add 10 inflight sasl context for (int i = 0; i < 10; i++) { AuthenticationDataProvider dataProvider = authSasl.getAuthData("localhost"); diff --git a/pulsar-broker-common/pom.xml b/pulsar-broker-common/pom.xml index 713ae538d7dee..b04d08c6c8f19 100644 --- a/pulsar-broker-common/pom.xml +++ b/pulsar-broker-common/pom.xml @@ -49,6 +49,11 @@ simpleclient_jetty + + io.opentelemetry + opentelemetry-api + + javax.servlet javax.servlet-api diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProvider.java index 7862a35b5e871..d0a3a487b3478 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProvider.java @@ -20,6 +20,7 @@ import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedDataAttributeName; import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedRoleAttributeName; +import io.opentelemetry.api.OpenTelemetry; import java.io.Closeable; import java.io.IOException; import java.net.SocketAddress; @@ -29,6 +30,8 @@ import javax.net.ssl.SSLSession; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import lombok.Builder; +import lombok.Value; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics; import org.apache.pulsar.common.api.AuthData; @@ -47,8 +50,30 @@ public interface AuthenticationProvider extends Closeable { * @throws IOException * if the initialization fails */ + @Deprecated(since = "3.4.0") void initialize(ServiceConfiguration config) throws IOException; + @Builder + @Value + class Context { + ServiceConfiguration config; + + @Builder.Default + OpenTelemetry openTelemetry = OpenTelemetry.noop(); + } + + /** + * Perform initialization for the authentication provider. + * + * @param context + * the authentication provider context + * @throws IOException + * if the initialization fails + */ + default void initialize(Context context) throws IOException { + initialize(context.getConfig()); + } + /** * @return the authentication method name supported by this provider */ diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderBasic.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderBasic.java index ca5150c9bdb60..91bf56a071c42 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderBasic.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderBasic.java @@ -46,6 +46,8 @@ public class AuthenticationProviderBasic implements AuthenticationProvider { private static final String CONF_PULSAR_PROPERTY_KEY = "basicAuthConf"; private Map users; + private AuthenticationMetrics authenticationMetrics; + private enum ErrorCode { UNKNOWN, EMPTY_AUTH_DATA, @@ -75,6 +77,14 @@ public static byte[] readData(String data) @Override public void initialize(ServiceConfiguration config) throws IOException { + initialize(Context.builder().config(config).build()); + } + + @Override + public void initialize(Context context) throws IOException { + authenticationMetrics = new AuthenticationMetrics(context.getOpenTelemetry(), + getClass().getSimpleName(), getAuthMethodName()); + var config = context.getConfig(); String data = config.getProperties().getProperty(CONF_PULSAR_PROPERTY_KEY); if (StringUtils.isEmpty(data)) { data = System.getProperty(CONF_SYSTEM_PROPERTY_KEY); @@ -106,6 +116,11 @@ public String getAuthMethodName() { return "basic"; } + @Override + public void incrementFailureMetric(Enum errorCode) { + authenticationMetrics.recordFailure(errorCode); + } + @Override public String authenticate(AuthenticationDataSource authData) throws AuthenticationException { AuthParams authParams = new AuthParams(authData); @@ -138,7 +153,7 @@ public String authenticate(AuthenticationDataSource authData) throws Authenticat incrementFailureMetric(errorCode); throw exception; } - AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName()); + authenticationMetrics.recordSuccess(); return userId; } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java index 211f2ea006bc3..0e5559b3c3aab 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderList.java @@ -38,6 +38,8 @@ @Slf4j public class AuthenticationProviderList implements AuthenticationProvider { + private AuthenticationMetrics authenticationMetrics; + private interface AuthProcessor { T apply(W process) throws AuthenticationException; @@ -49,7 +51,8 @@ private enum ErrorCode { AUTH_REQUIRED, } - static T applyAuthProcessor(List processors, AuthProcessor authFunc) + private static T applyAuthProcessor(List processors, AuthenticationMetrics metrics, + AuthProcessor authFunc) throws AuthenticationException { AuthenticationException authenticationException = null; String errorCode = ErrorCode.UNKNOWN.name(); @@ -67,30 +70,29 @@ static T applyAuthProcessor(List processors, AuthProcessor authF } if (null == authenticationException) { - AuthenticationMetrics.authenticateFailure( - AuthenticationProviderList.class.getSimpleName(), + metrics.recordFailure(AuthenticationProviderList.class.getSimpleName(), "authentication-provider-list", ErrorCode.AUTH_REQUIRED); throw new AuthenticationException("Authentication required"); } else { - AuthenticationMetrics.authenticateFailure( - AuthenticationProviderList.class.getSimpleName(), + metrics.recordFailure(AuthenticationProviderList.class.getSimpleName(), "authentication-provider-list", errorCode); throw authenticationException; } - } private static class AuthenticationListState implements AuthenticationState { private final List states; private volatile AuthenticationState authState; + private final AuthenticationMetrics metrics; - AuthenticationListState(List states) { + AuthenticationListState(List states, AuthenticationMetrics metrics) { if (states == null || states.isEmpty()) { throw new IllegalArgumentException("Authentication state requires at least one state"); } this.states = states; this.authState = states.get(0); + this.metrics = metrics; } private AuthenticationState getAuthState() throws AuthenticationException { @@ -135,8 +137,9 @@ private void authenticateRemainingAuthStates(CompletableFuture authCha if (previousException == null) { previousException = new AuthenticationException("Authentication required"); } - AuthenticationMetrics.authenticateFailure(AuthenticationProviderList.class.getSimpleName(), - "authentication-provider-list", ErrorCode.AUTH_REQUIRED); + metrics.recordFailure(AuthenticationProviderList.class.getSimpleName(), + "authentication-provider-list", + ErrorCode.AUTH_REQUIRED); authChallengeFuture.completeExceptionally(previousException); return; } @@ -166,6 +169,7 @@ private void authenticateRemainingAuthStates(CompletableFuture authCha public AuthData authenticate(AuthData authData) throws AuthenticationException { return applyAuthProcessor( states, + metrics, as -> { AuthData ad = as.authenticate(authData); AuthenticationListState.this.authState = as; @@ -216,8 +220,15 @@ public List getProviders() { @Override public void initialize(ServiceConfiguration config) throws IOException { + initialize(Context.builder().config(config).build()); + } + + @Override + public void initialize(Context context) throws IOException { + authenticationMetrics = new AuthenticationMetrics(context.getOpenTelemetry(), + getClass().getSimpleName(), getAuthMethodName()); for (AuthenticationProvider ap : providers) { - ap.initialize(config); + ap.initialize(context); } } @@ -226,6 +237,11 @@ public String getAuthMethodName() { return providers.get(0).getAuthMethodName(); } + @Override + public void incrementFailureMetric(Enum errorCode) { + authenticationMetrics.recordFailure(errorCode); + } + @Override public CompletableFuture authenticateAsync(AuthenticationDataSource authData) { CompletableFuture roleFuture = new CompletableFuture<>(); @@ -241,7 +257,7 @@ private void authenticateRemainingAuthProviders(CompletableFuture roleFu if (previousException == null) { previousException = new AuthenticationException("Authentication required"); } - AuthenticationMetrics.authenticateFailure(AuthenticationProviderList.class.getSimpleName(), + authenticationMetrics.recordFailure(AuthenticationProvider.class.getSimpleName(), "authentication-provider-list", ErrorCode.AUTH_REQUIRED); roleFuture.completeExceptionally(previousException); return; @@ -264,6 +280,7 @@ private void authenticateRemainingAuthProviders(CompletableFuture roleFu public String authenticate(AuthenticationDataSource authData) throws AuthenticationException { return applyAuthProcessor( providers, + authenticationMetrics, provider -> provider.authenticate(authData) ); } @@ -294,7 +311,7 @@ public AuthenticationState newAuthState(AuthData authData, SocketAddress remoteA throw new AuthenticationException("Failed to initialize a new auth state from " + remoteAddress); } } else { - return new AuthenticationListState(states); + return new AuthenticationListState(states, authenticationMetrics); } } @@ -325,7 +342,7 @@ public AuthenticationState newHttpAuthState(HttpServletRequest request) throws A "Failed to initialize a new http auth state from " + request.getRemoteHost()); } } else { - return new AuthenticationListState(states); + return new AuthenticationListState(states, authenticationMetrics); } } @@ -333,6 +350,7 @@ public AuthenticationState newHttpAuthState(HttpServletRequest request) throws A public boolean authenticateHttpRequest(HttpServletRequest request, HttpServletResponse response) throws Exception { Boolean authenticated = applyAuthProcessor( providers, + authenticationMetrics, provider -> { try { return provider.authenticateHttpRequest(request, response); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTls.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTls.java index a4c44121b4b96..f7ff47fe8e61e 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTls.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTls.java @@ -27,6 +27,8 @@ public class AuthenticationProviderTls implements AuthenticationProvider { + private AuthenticationMetrics authenticationMetrics; + private enum ErrorCode { UNKNOWN, INVALID_CERTS, @@ -40,7 +42,13 @@ public void close() throws IOException { @Override public void initialize(ServiceConfiguration config) throws IOException { - // noop + initialize(Context.builder().config(config).build()); + } + + @Override + public void initialize(Context context) throws IOException { + authenticationMetrics = new AuthenticationMetrics(context.getOpenTelemetry(), + getClass().getSimpleName(), getAuthMethodName()); } @Override @@ -48,6 +56,11 @@ public String getAuthMethodName() { return "tls"; } + @Override + public void incrementFailureMetric(Enum errorCode) { + authenticationMetrics.recordFailure(errorCode); + } + @Override public String authenticate(AuthenticationDataSource authData) throws AuthenticationException { String commonName = null; @@ -96,7 +109,7 @@ public String authenticate(AuthenticationDataSource authData) throws Authenticat errorCode = ErrorCode.INVALID_CN; throw new AuthenticationException("Client unable to authenticate with TLS certificate"); } - AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName()); + authenticationMetrics.recordSuccess(); } catch (AuthenticationException exception) { incrementFailureMetric(errorCode); throw exception; diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java index f8992b21ff49f..74bc85ad3ffc3 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationProviderToken.java @@ -21,7 +21,6 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedDataAttributeName; import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedRoleAttributeName; -import com.google.common.annotations.VisibleForTesting; import io.jsonwebtoken.Claims; import io.jsonwebtoken.ExpiredJwtException; import io.jsonwebtoken.Jwt; @@ -31,8 +30,6 @@ import io.jsonwebtoken.RequiredTypeException; import io.jsonwebtoken.SignatureAlgorithm; import io.jsonwebtoken.security.SignatureException; -import io.prometheus.client.Counter; -import io.prometheus.client.Histogram; import java.io.IOException; import java.net.SocketAddress; import java.security.Key; @@ -44,7 +41,7 @@ import javax.servlet.http.HttpServletResponse; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics; +import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetricsToken; import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; import org.apache.pulsar.common.api.AuthData; @@ -79,17 +76,6 @@ public class AuthenticationProviderToken implements AuthenticationProvider { static final String TOKEN = "token"; - private static final Counter expiredTokenMetrics = Counter.build() - .name("pulsar_expired_token_total") - .help("Pulsar expired token") - .register(); - - private static final Histogram expiringTokenMinutesMetrics = Histogram.build() - .name("pulsar_expiring_token_minutes") - .help("The remaining time of expiring token in minutes") - .buckets(5, 10, 60, 240) - .register(); - private Key validationKey; private String roleClaim; private SignatureAlgorithm publicKeyAlg; @@ -106,6 +92,8 @@ public class AuthenticationProviderToken implements AuthenticationProvider { private String confTokenAudienceSettingName; private String confTokenAllowedClockSkewSecondsSettingName; + private AuthenticationMetricsToken authenticationMetricsToken; + public enum ErrorCode { INVALID_AUTH_DATA, INVALID_TOKEN, @@ -117,14 +105,17 @@ public void close() throws IOException { // noop } - @VisibleForTesting - public static void resetMetrics() { - expiredTokenMetrics.clear(); - expiringTokenMinutesMetrics.clear(); + @Override + public void initialize(ServiceConfiguration config) throws IOException { + initialize(Context.builder().config(config).build()); } @Override - public void initialize(ServiceConfiguration config) throws IOException, IllegalArgumentException { + public void initialize(Context context) throws IOException { + authenticationMetricsToken = new AuthenticationMetricsToken(context.getOpenTelemetry(), + getClass().getSimpleName(), getAuthMethodName()); + + var config = context.getConfig(); String prefix = (String) config.getProperty(CONF_TOKEN_SETTING_PREFIX); if (null == prefix) { prefix = ""; @@ -162,6 +153,11 @@ public String getAuthMethodName() { return TOKEN; } + @Override + public void incrementFailureMetric(Enum errorCode) { + authenticationMetricsToken.recordFailure(errorCode); + } + @Override public String authenticate(AuthenticationDataSource authData) throws AuthenticationException { String token; @@ -174,7 +170,7 @@ public String authenticate(AuthenticationDataSource authData) throws Authenticat } // Parse Token by validating String role = getPrincipal(authenticateToken(token)); - AuthenticationMetrics.authenticateSuccess(getClass().getSimpleName(), getAuthMethodName()); + authenticationMetricsToken.recordSuccess(); return role; } @@ -263,14 +259,13 @@ private Jwt authenticateToken(final String token) throws Authenticati } } - if (jwt.getBody().getExpiration() != null) { - expiringTokenMinutesMetrics.observe( - (double) (jwt.getBody().getExpiration().getTime() - new Date().getTime()) / (60 * 1000)); - } + var expiration = jwt.getBody().getExpiration(); + var tokenRemainingDurationMs = expiration != null ? expiration.getTime() - new Date().getTime() : null; + authenticationMetricsToken.recordTokenDuration(tokenRemainingDurationMs); return jwt; } catch (JwtException e) { if (e instanceof ExpiredJwtException) { - expiredTokenMetrics.inc(); + authenticationMetricsToken.recordTokenExpired(); } incrementFailureMetric(ErrorCode.INVALID_TOKEN); throw new AuthenticationException("Failed to authentication token: " + e.getMessage()); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java index 22296b86b4e0c..f6eb785d2e479 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationService.java @@ -20,6 +20,7 @@ import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedDataAttributeName; import static org.apache.pulsar.broker.web.AuthenticationFilter.AuthenticatedRoleAttributeName; +import io.opentelemetry.api.OpenTelemetry; import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; @@ -51,6 +52,11 @@ public class AuthenticationService implements Closeable { private final Map providers = new HashMap<>(); public AuthenticationService(ServiceConfiguration conf) throws PulsarServerException { + this(conf, OpenTelemetry.noop()); + } + + public AuthenticationService(ServiceConfiguration conf, OpenTelemetry openTelemetry) + throws PulsarServerException { anonymousUserRole = conf.getAnonymousUserRole(); if (conf.isAuthenticationEnabled()) { try { @@ -70,6 +76,10 @@ public AuthenticationService(ServiceConfiguration conf) throws PulsarServerExcep providerList.add(provider); } + var authenticationProviderContext = AuthenticationProvider.Context.builder() + .config(conf) + .openTelemetry(openTelemetry) + .build(); for (Map.Entry> entry : providerMap.entrySet()) { AuthenticationProvider provider; if (entry.getValue().size() == 1) { @@ -77,7 +87,7 @@ public AuthenticationService(ServiceConfiguration conf) throws PulsarServerExcep } else { provider = new AuthenticationProviderList(entry.getValue()); } - provider.initialize(conf); + provider.initialize(authenticationProviderContext); providers.put(provider.getAuthMethodName(), provider); LOG.info("[{}] has been loaded.", entry.getValue().stream().map( diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/metrics/AuthenticationMetrics.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/metrics/AuthenticationMetrics.java index 5faaccbe15716..931ad50e11728 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/metrics/AuthenticationMetrics.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/metrics/AuthenticationMetrics.java @@ -18,28 +18,27 @@ */ package org.apache.pulsar.broker.authentication.metrics; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.LongCounter; import io.prometheus.client.Counter; public class AuthenticationMetrics { + @Deprecated private static final Counter authSuccessMetrics = Counter.build() .name("pulsar_authentication_success_total") .help("Pulsar authentication success") .labelNames("provider_name", "auth_method") .register(); + @Deprecated private static final Counter authFailuresMetrics = Counter.build() .name("pulsar_authentication_failures_total") .help("Pulsar authentication failures") .labelNames("provider_name", "auth_method", "reason") .register(); - /** - * Log authenticate success event to the authentication metrics. - * @param providerName The short class name of the provider - * @param authMethod Authentication method name - */ - public static void authenticateSuccess(String providerName, String authMethod) { - authSuccessMetrics.labels(providerName, authMethod).inc(); - } + public static final String INSTRUMENTATION_SCOPE_NAME = "org.apache.pulsar.authentication"; /** * Log authenticate failure event to the authentication metrics. @@ -62,8 +61,58 @@ public static void authenticateFailure(String providerName, String authMethod, S * @param authMethod Authentication method name. * @param errorCode Error code. */ + @Deprecated public static void authenticateFailure(String providerName, String authMethod, Enum errorCode) { authFailuresMetrics.labels(providerName, authMethod, errorCode.name()).inc(); } + public static final String AUTHENTICATION_COUNTER_METRIC_NAME = "pulsar.authentication.operation.count"; + private final LongCounter authenticationCounter; + + public static final AttributeKey PROVIDER_KEY = AttributeKey.stringKey("pulsar.authentication.provider"); + public static final AttributeKey AUTH_METHOD_KEY = AttributeKey.stringKey("pulsar.authentication.method"); + public static final AttributeKey ERROR_CODE_KEY = AttributeKey.stringKey("pulsar.authentication.error"); + public static final AttributeKey AUTH_RESULT_KEY = AttributeKey.stringKey("pulsar.authentication.result"); + public enum AuthenticationResult { + SUCCESS, + FAILURE; + } + + private final String providerName; + private final String authMethod; + + public AuthenticationMetrics(OpenTelemetry openTelemetry, String providerName, String authMethod) { + this.providerName = providerName; + this.authMethod = authMethod; + var meter = openTelemetry.getMeter(INSTRUMENTATION_SCOPE_NAME); + authenticationCounter = meter.counterBuilder(AUTHENTICATION_COUNTER_METRIC_NAME) + .setDescription("The number of authentication operations") + .setUnit("{operation}") + .build(); + } + + public void recordSuccess() { + authSuccessMetrics.labels(providerName, authMethod).inc(); + var attributes = Attributes.of(PROVIDER_KEY, providerName, + AUTH_METHOD_KEY, authMethod, + AUTH_RESULT_KEY, AuthenticationResult.SUCCESS.name().toLowerCase()); + authenticationCounter.add(1, attributes); + } + + public void recordFailure(Enum errorCode) { + recordFailure(providerName, authMethod, errorCode.name()); + } + + public void recordFailure(String providerName, String authMethod, Enum errorCode) { + recordFailure(providerName, authMethod, errorCode.name()); + } + + public void recordFailure(String providerName, String authMethod, String errorCode) { + authenticateFailure(providerName, authMethod, errorCode); + var attributes = Attributes.of(PROVIDER_KEY, providerName, + AUTH_METHOD_KEY, authMethod, + AUTH_RESULT_KEY, AuthenticationResult.FAILURE.name().toLowerCase(), + ERROR_CODE_KEY, errorCode); + authenticationCounter.add(1, attributes); + } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/metrics/AuthenticationMetricsToken.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/metrics/AuthenticationMetricsToken.java new file mode 100644 index 0000000000000..4e9d1d6b16a92 --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/metrics/AuthenticationMetricsToken.java @@ -0,0 +1,109 @@ +/* + * 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.authentication.metrics; + +import com.google.common.annotations.VisibleForTesting; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.metrics.DoubleHistogram; +import io.opentelemetry.api.metrics.LongCounter; +import io.prometheus.client.Counter; +import io.prometheus.client.Histogram; +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.common.stats.MetricsUtil; + +public class AuthenticationMetricsToken extends AuthenticationMetrics { + + @Deprecated + private static final Counter expiredTokenMetrics = Counter.build() + .name("pulsar_expired_token_total") + .help("Pulsar expired token") + .register(); + public static final String EXPIRED_TOKEN_COUNTER_METRIC_NAME = "pulsar.authentication.token.expired.count"; + private LongCounter expiredTokensCounter; + + private static final List TOKEN_DURATION_BUCKET_BOUNDARIES_SECONDS = List.of( + TimeUnit.MINUTES.toSeconds(5), + TimeUnit.MINUTES.toSeconds(10), + TimeUnit.HOURS.toSeconds(1), + TimeUnit.HOURS.toSeconds(4), + TimeUnit.DAYS.toSeconds(1), + TimeUnit.DAYS.toSeconds(7), + TimeUnit.DAYS.toSeconds(14), + TimeUnit.DAYS.toSeconds(30), + TimeUnit.DAYS.toSeconds(90), + TimeUnit.DAYS.toSeconds(180), + TimeUnit.DAYS.toSeconds(270), + TimeUnit.DAYS.toSeconds(365)); + + @Deprecated + private static final Histogram expiringTokenMinutesMetrics = Histogram.build() + .name("pulsar_expiring_token_minutes") + .help("The remaining time of expiring token in minutes") + .buckets(TOKEN_DURATION_BUCKET_BOUNDARIES_SECONDS.stream() + .map(TimeUnit.SECONDS::toMinutes) + .mapToDouble(Double::valueOf) + .toArray()) + .register(); + public static final String EXPIRING_TOKEN_HISTOGRAM_METRIC_NAME = "pulsar.authentication.token.expiry.duration"; + private DoubleHistogram expiringTokenSeconds; + + public AuthenticationMetricsToken(OpenTelemetry openTelemetry, String providerName, + String authMethod) { + super(openTelemetry, providerName, authMethod); + + var meter = openTelemetry.getMeter(AuthenticationMetrics.INSTRUMENTATION_SCOPE_NAME); + expiredTokensCounter = meter.counterBuilder(EXPIRED_TOKEN_COUNTER_METRIC_NAME) + .setDescription("The total number of expired tokens") + .setUnit("{token}") + .build(); + expiringTokenSeconds = meter.histogramBuilder(EXPIRING_TOKEN_HISTOGRAM_METRIC_NAME) + .setExplicitBucketBoundariesAdvice( + TOKEN_DURATION_BUCKET_BOUNDARIES_SECONDS.stream().map(Double::valueOf).toList()) + .setDescription("The remaining time of expiring token in seconds") + .setUnit("s") + .build(); + } + + public void recordTokenDuration(Long durationMs) { + if (durationMs == null) { + // Special case signals a token without expiry. OpenTelemetry supports reporting infinite values. + expiringTokenSeconds.record(Double.POSITIVE_INFINITY); + } else if (durationMs > 0) { + expiringTokenMinutesMetrics.observe(durationMs / 60_000.0d); + expiringTokenSeconds.record(MetricsUtil.convertToSeconds(durationMs, TimeUnit.MILLISECONDS)); + } else { + // Duration can be negative if token expires at processing time. OpenTelemetry does not support negative + // values, so record token expiry instead. + recordTokenExpired(); + } + } + + public void recordTokenExpired() { + expiredTokenMetrics.inc(); + expiredTokensCounter.add(1); + } + + @VisibleForTesting + @Deprecated + public static void reset() { + expiredTokenMetrics.clear(); + expiringTokenMinutesMetrics.clear(); + } +} diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderBasicTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderBasicTest.java index 723fde7083d38..f6e4b8e969ac1 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderBasicTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderBasicTest.java @@ -52,7 +52,7 @@ public void testLoadFileFromPulsarProperties() throws Exception { Properties properties = new Properties(); properties.setProperty("basicAuthConf", basicAuthConf); serviceConfiguration.setProperties(properties); - provider.initialize(serviceConfiguration); + provider.initialize(AuthenticationProvider.Context.builder().config(serviceConfiguration).build()); testAuthenticate(provider); } @@ -64,7 +64,7 @@ public void testLoadBase64FromPulsarProperties() throws Exception { Properties properties = new Properties(); properties.setProperty("basicAuthConf", basicAuthConfBase64); serviceConfiguration.setProperties(properties); - provider.initialize(serviceConfiguration); + provider.initialize(AuthenticationProvider.Context.builder().config(serviceConfiguration).build()); testAuthenticate(provider); } @@ -74,7 +74,7 @@ public void testLoadFileFromSystemProperties() throws Exception { AuthenticationProviderBasic provider = new AuthenticationProviderBasic(); ServiceConfiguration serviceConfiguration = new ServiceConfiguration(); System.setProperty("pulsar.auth.basic.conf", basicAuthConf); - provider.initialize(serviceConfiguration); + provider.initialize(AuthenticationProvider.Context.builder().config(serviceConfiguration).build()); testAuthenticate(provider); } @@ -84,7 +84,7 @@ public void testLoadBase64FromSystemProperties() throws Exception { AuthenticationProviderBasic provider = new AuthenticationProviderBasic(); ServiceConfiguration serviceConfiguration = new ServiceConfiguration(); System.setProperty("pulsar.auth.basic.conf", basicAuthConfBase64); - provider.initialize(serviceConfiguration); + provider.initialize(AuthenticationProvider.Context.builder().config(serviceConfiguration).build()); testAuthenticate(provider); } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderListTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderListTest.java index 7793a5c029f2a..e81198217b5b6 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderListTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderListTest.java @@ -29,7 +29,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; - import io.jsonwebtoken.SignatureAlgorithm; import io.jsonwebtoken.io.Decoders; import io.jsonwebtoken.security.Keys; @@ -90,7 +89,7 @@ public void setUp() throws Exception { ); ServiceConfiguration confA = new ServiceConfiguration(); confA.setProperties(propertiesA); - providerA.initialize(confA); + providerA.initialize(AuthenticationProvider.Context.builder().config(confA).build()); Properties propertiesB = new Properties(); propertiesB.setProperty(AuthenticationProviderToken.CONF_TOKEN_SETTING_PREFIX, "b"); @@ -103,7 +102,7 @@ public void setUp() throws Exception { ); ServiceConfiguration confB = new ServiceConfiguration(); confB.setProperties(propertiesB); - providerB.initialize(confB); + providerB.initialize(AuthenticationProvider.Context.builder().config(confB).build()); this.authProvider = new AuthenticationProviderList(Lists.newArrayList( providerA, providerB diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenTest.java index f50731c7654af..3e1a3e180349e 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authentication/AuthenticationProviderTokenTest.java @@ -55,7 +55,9 @@ import javax.naming.AuthenticationException; import javax.servlet.http.HttpServletRequest; import lombok.Cleanup; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetricsToken; import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; import org.apache.pulsar.common.api.AuthData; import org.mockito.Mockito; @@ -70,7 +72,7 @@ public void testInvalidInitialize() throws Exception { AuthenticationProviderToken provider = new AuthenticationProviderToken(); try { - provider.initialize(new ServiceConfiguration()); + provider.initialize(AuthenticationProvider.Context.builder().config(new ServiceConfiguration()).build()); fail("should have failed"); } catch (IOException e) { // Expected, secret key was not defined @@ -135,7 +137,7 @@ public void testAuthSecretKey() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); try { provider.authenticate(new AuthenticationDataSource() { @@ -249,7 +251,7 @@ public void testTrimAuthSecretKeyFilePath() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); } @Test @@ -268,7 +270,7 @@ public void testAuthSecretKeyFromFile() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String token = AuthTokenUtils.createToken(secretKey, SUBJECT, Optional.empty()); @@ -303,7 +305,7 @@ public void testAuthSecretKeyFromValidFile() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String token = AuthTokenUtils.createToken(secretKey, SUBJECT, Optional.empty()); @@ -335,7 +337,7 @@ public void testAuthSecretKeyFromDataBase64() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String token = AuthTokenUtils.createToken(secretKey, SUBJECT, Optional.empty()); @@ -370,7 +372,7 @@ public void testAuthSecretKeyPair() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); // Use private key to generate token PrivateKey privateKey = AuthTokenUtils.decodePrivateKey(Decoders.BASE64.decode(privateKeyStr), SignatureAlgorithm.RS256); @@ -413,8 +415,7 @@ public void testAuthSecretKeyPairWithCustomClaim() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); - + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); // Use private key to generate token PrivateKey privateKey = AuthTokenUtils.decodePrivateKey(Decoders.BASE64.decode(privateKeyStr), SignatureAlgorithm.RS256); @@ -460,7 +461,7 @@ public void testAuthSecretKeyPairWithECDSA() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); // Use private key to generate token PrivateKey privateKey = AuthTokenUtils.decodePrivateKey(Decoders.BASE64.decode(privateKeyStr), SignatureAlgorithm.ES256); @@ -484,8 +485,11 @@ public String getCommandData() { } @Test(expectedExceptions = AuthenticationException.class) - public void testAuthenticateWhenNoJwtPassed() throws AuthenticationException { + public void testAuthenticateWhenNoJwtPassed() throws Exception { + @Cleanup AuthenticationProviderToken provider = new AuthenticationProviderToken(); + FieldUtils.writeDeclaredField( + provider, "authenticationMetricsToken", mock(AuthenticationMetricsToken.class), true); provider.authenticate(new AuthenticationDataSource() { @Override public boolean hasDataFromCommand() { @@ -500,8 +504,11 @@ public boolean hasDataFromHttp() { } @Test(expectedExceptions = AuthenticationException.class) - public void testAuthenticateWhenAuthorizationHeaderNotExist() throws AuthenticationException { + public void testAuthenticateWhenAuthorizationHeaderNotExist() throws Exception { + @Cleanup AuthenticationProviderToken provider = new AuthenticationProviderToken(); + FieldUtils.writeDeclaredField( + provider, "authenticationMetricsToken", mock(AuthenticationMetricsToken.class), true); provider.authenticate(new AuthenticationDataSource() { @Override public String getHttpHeader(String name) { @@ -516,8 +523,11 @@ public boolean hasDataFromHttp() { } @Test(expectedExceptions = AuthenticationException.class) - public void testAuthenticateWhenAuthHeaderValuePrefixIsInvalid() throws AuthenticationException { + public void testAuthenticateWhenAuthHeaderValuePrefixIsInvalid() throws Exception { + @Cleanup AuthenticationProviderToken provider = new AuthenticationProviderToken(); + FieldUtils.writeDeclaredField( + provider, "authenticationMetricsToken", mock(AuthenticationMetricsToken.class), true); provider.authenticate(new AuthenticationDataSource() { @Override public String getHttpHeader(String name) { @@ -532,8 +542,11 @@ public boolean hasDataFromHttp() { } @Test(expectedExceptions = AuthenticationException.class) - public void testAuthenticateWhenJwtIsBlank() throws AuthenticationException { + public void testAuthenticateWhenJwtIsBlank() throws Exception { + @Cleanup AuthenticationProviderToken provider = new AuthenticationProviderToken(); + FieldUtils.writeDeclaredField( + provider, "authenticationMetricsToken", mock(AuthenticationMetricsToken.class), true); provider.authenticate(new AuthenticationDataSource() { @Override public String getHttpHeader(String name) { @@ -559,7 +572,7 @@ public void testAuthenticateWhenInvalidTokenIsPassed() throws AuthenticationExce conf.setProperties(properties); AuthenticationProviderToken provider = new AuthenticationProviderToken(); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); provider.authenticate(new AuthenticationDataSource() { @Override public String getHttpHeader(String name) { @@ -582,7 +595,7 @@ public void testValidationKeyWhenBlankSecretKeyIsPassed() throws IOException { conf.setProperties(properties); AuthenticationProviderToken provider = new AuthenticationProviderToken(); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); } @Test(expectedExceptions = IOException.class) @@ -594,7 +607,7 @@ public void testValidationKeyWhenBlankPublicKeyIsPassed() throws IOException { conf.setProperties(properties); AuthenticationProviderToken provider = new AuthenticationProviderToken(); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); } @Test(expectedExceptions = IOException.class) @@ -606,7 +619,7 @@ public void testInitializeWhenSecretKeyFilePathIsInvalid() throws IOException { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - new AuthenticationProviderToken().initialize(conf); + new AuthenticationProviderToken().initialize(AuthenticationProvider.Context.builder().config(conf).build()); } @Test(expectedExceptions = IOException.class) @@ -618,7 +631,7 @@ public void testInitializeWhenSecretKeyIsValidPathOrBase64() throws IOException ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - new AuthenticationProviderToken().initialize(conf); + new AuthenticationProviderToken().initialize(AuthenticationProvider.Context.builder().config(conf).build()); } @Test(expectedExceptions = IllegalArgumentException.class) @@ -633,7 +646,7 @@ public void testInitializeWhenSecretKeyFilePathIfNotExist() throws IOException { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - new AuthenticationProviderToken().initialize(conf); + new AuthenticationProviderToken().initialize(AuthenticationProvider.Context.builder().config(conf).build()); } @Test(expectedExceptions = IOException.class) @@ -645,7 +658,7 @@ public void testInitializeWhenPublicKeyFilePathIsInvalid() throws IOException { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - new AuthenticationProviderToken().initialize(conf); + new AuthenticationProviderToken().initialize(AuthenticationProvider.Context.builder().config(conf).build()); } @Test(expectedExceptions = IllegalArgumentException.class) @@ -657,7 +670,7 @@ public void testValidationWhenPublicKeyAlgIsInvalid() throws IOException { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - new AuthenticationProviderToken().initialize(conf); + new AuthenticationProviderToken().initialize(AuthenticationProvider.Context.builder().config(conf).build()); } @@ -676,7 +689,7 @@ public void testExpiringToken() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); // Create a token that will expire in 3 seconds String expiringToken = AuthTokenUtils.createToken(secretKey, SUBJECT, @@ -709,7 +722,7 @@ public void testExpiredTokenFailsOnAuthenticate() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); // Create a token that is already expired String expiringToken = AuthTokenUtils.createToken(secretKey, SUBJECT, @@ -828,7 +841,7 @@ public void testArrayTypeRoleClaim() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); // Use private key to generate token PrivateKey privateKey = AuthTokenUtils.decodePrivateKey(Decoders.BASE64.decode(privateKeyStr), SignatureAlgorithm.RS256); @@ -877,7 +890,7 @@ public void testTokenSettingPrefix() throws Exception { ); Mockito.when(mockConf.getProperty(AuthenticationProviderToken.CONF_TOKEN_SETTING_PREFIX)).thenReturn(prefix); - provider.initialize(mockConf); + provider.initialize(AuthenticationProvider.Context.builder().config(mockConf).build()); // Each property is fetched only once. Prevent multiple fetches. Mockito.verify(mockConf, Mockito.times(1)).getProperty(AuthenticationProviderToken.CONF_TOKEN_SETTING_PREFIX); @@ -908,7 +921,7 @@ public void testTokenFromHttpParams() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String token = AuthTokenUtils.createToken(secretKey, SUBJECT, Optional.empty()); HttpServletRequest servletRequest = mock(HttpServletRequest.class); @@ -934,7 +947,7 @@ public void testTokenFromHttpHeaders() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String token = AuthTokenUtils.createToken(secretKey, SUBJECT, Optional.empty()); HttpServletRequest servletRequest = mock(HttpServletRequest.class); @@ -960,7 +973,7 @@ public void testTokenStateUpdatesAuthenticationDataSource() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); AuthenticationState authState = provider.newAuthState(null,null, null); @@ -1016,7 +1029,7 @@ private static void testTokenAudienceWithDifferentConfig(Properties properties, properties.setProperty(AuthenticationProviderToken.CONF_TOKEN_SECRET_KEY, secretKeyFile.toString()); ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); String token = createTokenWithAudience(secretKey, audienceClaim, audiences); 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 e5248d45d4226..c0e3e7d356be0 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 @@ -398,7 +398,8 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws .name("pulsar-backlog-quota-checker") .numThreads(1) .build(); - this.authenticationService = new AuthenticationService(pulsar.getConfiguration()); + this.authenticationService = new AuthenticationService(pulsar.getConfiguration(), + pulsar.getOpenTelemetry().getOpenTelemetry()); this.blockedDispatchers = ConcurrentOpenHashSet.newBuilder().build(); this.topicFactory = createPersistentTopicFactory(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java index 178da8b84983f..065b03e6454ea 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/PulsarBrokerOpenTelemetry.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.stats; import com.google.common.annotations.VisibleForTesting; +import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.metrics.Meter; import io.opentelemetry.sdk.autoconfigure.AutoConfiguredOpenTelemetrySdkBuilder; import java.io.Closeable; @@ -50,6 +51,10 @@ public PulsarBrokerOpenTelemetry(ServiceConfiguration config, meter = openTelemetryService.getOpenTelemetry().getMeter(Constants.BROKER_INSTRUMENTATION_SCOPE_NAME); } + public OpenTelemetry getOpenTelemetry() { + return openTelemetryService.getOpenTelemetry(); + } + @Override public void close() { openTelemetryService.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java index 726bde3f3d0a9..27bdb2e3004ea 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java @@ -32,7 +32,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; +import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetricsToken; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -52,7 +52,7 @@ public class MetadataStoreStatsTest extends BrokerTestBase { @Override protected void setup() throws Exception { super.baseSetup(); - AuthenticationProviderToken.resetMetrics(); + AuthenticationMetricsToken.reset(); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryAuthenticationStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryAuthenticationStatsTest.java new file mode 100644 index 0000000000000..4cde37b50ffc1 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/OpenTelemetryAuthenticationStatsTest.java @@ -0,0 +1,161 @@ +/* + * 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.stats; + +import static io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions.assertThat; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; +import io.jsonwebtoken.SignatureAlgorithm; +import io.opentelemetry.api.common.Attributes; +import java.time.Duration; +import java.util.Date; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import javax.crypto.SecretKey; +import javax.naming.AuthenticationException; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.authentication.AuthenticationDataSource; +import org.apache.pulsar.broker.authentication.AuthenticationProvider; +import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; +import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetrics; +import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetricsToken; +import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; +import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class OpenTelemetryAuthenticationStatsTest extends BrokerTestBase { + + private static final Duration AUTHENTICATION_TIMEOUT = Duration.ofSeconds(1); + + private SecretKey secretKey; + private AuthenticationProvider provider; + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.baseSetup(); + + secretKey = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256); + provider = new AuthenticationProviderToken(); + registerCloseable(provider); + + var properties = new Properties(); + properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(secretKey)); + + var conf = new ServiceConfiguration(); + conf.setProperties(properties); + + var authenticationProviderContext = AuthenticationProvider.Context.builder() + .config(conf) + .openTelemetry(pulsar.getOpenTelemetry().getOpenTelemetry()) + .build(); + provider.initialize(authenticationProviderContext); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder builder) { + super.customizeMainPulsarTestContextBuilder(builder); + builder.enableOpenTelemetry(true); + } + + @Test + public void testAuthenticationSuccess() { + // Pulsar protocol auth + assertThat(provider.authenticateAsync(new TestAuthenticationDataSource(Optional.empty()))) + .succeedsWithin(AUTHENTICATION_TIMEOUT); + assertMetricLongSumValue(pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(), + AuthenticationMetrics.AUTHENTICATION_COUNTER_METRIC_NAME, + Attributes.of(AuthenticationMetrics.PROVIDER_KEY, "AuthenticationProviderToken", + AuthenticationMetrics.AUTH_RESULT_KEY, "success", + AuthenticationMetrics.AUTH_METHOD_KEY, "token"), + 1); + } + + @Test + public void testTokenDurationHistogram() { + // Token with expiry 15 seconds into the future + var expiryTime = Optional.of(new Date(System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(15))); + assertThat(provider.authenticateAsync(new TestAuthenticationDataSource(expiryTime))) + .succeedsWithin(AUTHENTICATION_TIMEOUT); + // Token without expiry + assertThat(provider.authenticateAsync(new TestAuthenticationDataSource(Optional.empty()))) + .succeedsWithin(AUTHENTICATION_TIMEOUT); + assertThat(pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics()) + .anySatisfy(metric -> assertThat(metric) + .hasName(AuthenticationMetricsToken.EXPIRING_TOKEN_HISTOGRAM_METRIC_NAME) + .hasHistogramSatisfying(histogram -> histogram.hasPointsSatisfying( + histogramPoint -> histogramPoint.hasCount(2).hasMax(Double.POSITIVE_INFINITY)))); + } + + @Test + public void testAuthenticationFailure() { + // Authentication should fail if credentials not passed. + assertThat(provider.authenticateAsync(new AuthenticationDataSource() { })) + .failsWithin(AUTHENTICATION_TIMEOUT) + .withThrowableThat() + .withRootCauseInstanceOf(AuthenticationException.class) + .withMessageContaining("No token credentials passed"); + assertMetricLongSumValue(pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(), + AuthenticationMetrics.AUTHENTICATION_COUNTER_METRIC_NAME, + Attributes.of(AuthenticationMetrics.PROVIDER_KEY, "AuthenticationProviderToken", + AuthenticationMetrics.AUTH_RESULT_KEY, "failure", + AuthenticationMetrics.AUTH_METHOD_KEY, "token", + AuthenticationMetrics.ERROR_CODE_KEY, "INVALID_AUTH_DATA"), + 1); + } + + @Test + public void testTokenExpired() { + var expiredDate = Optional.of(new Date(System.currentTimeMillis() - TimeUnit.HOURS.toMillis(1))); + assertThat(provider.authenticateAsync(new TestAuthenticationDataSource(expiredDate))) + .failsWithin(AUTHENTICATION_TIMEOUT) + .withThrowableThat() + .withRootCauseInstanceOf(AuthenticationException.class) + .withMessageContaining("JWT expired"); + assertMetricLongSumValue(pulsarTestContext.getOpenTelemetryMetricReader().collectAllMetrics(), + AuthenticationMetricsToken.EXPIRED_TOKEN_COUNTER_METRIC_NAME, Attributes.empty(), 1); + } + + private class TestAuthenticationDataSource implements AuthenticationDataSource { + private final String token; + + public TestAuthenticationDataSource(Optional expiryTime) { + token = AuthTokenUtils.createToken(secretKey, "subject", expiryTime); + } + + @Override + public boolean hasDataFromCommand() { + return true; + } + + @Override + public String getCommandData() { + return token; + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index e7f86d542a074..4df2d36a95303 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; @@ -70,7 +71,9 @@ import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; +import org.apache.pulsar.broker.authentication.AuthenticationProvider; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; +import org.apache.pulsar.broker.authentication.metrics.AuthenticationMetricsToken; import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; import org.apache.pulsar.broker.loadbalance.extensions.manager.UnloadManager; @@ -108,7 +111,7 @@ public class PrometheusMetricsTest extends BrokerTestBase { @Override protected void setup() throws Exception { super.baseSetup(); - AuthenticationProviderToken.resetMetrics(); + AuthenticationMetricsToken.reset(); } @Override @@ -1499,7 +1502,7 @@ public void testAuthMetrics() throws IOException, AuthenticationException { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); try { provider.authenticate(new AuthenticationDataSource() { @@ -1563,7 +1566,7 @@ public void testExpiredTokenMetrics() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); Date expiredDate = new Date(System.currentTimeMillis() - TimeUnit.HOURS.toMillis(1)); String expiredToken = AuthTokenUtils.createToken(secretKey, "subject", Optional.of(expiredDate)); @@ -1599,6 +1602,7 @@ public String getCommandData() { public void testExpiringTokenMetrics() throws Exception { SecretKey secretKey = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256); + @Cleanup AuthenticationProviderToken provider = new AuthenticationProviderToken(); Properties properties = new Properties(); @@ -1606,7 +1610,7 @@ public void testExpiringTokenMetrics() throws Exception { ServiceConfiguration conf = new ServiceConfiguration(); conf.setProperties(properties); - provider.initialize(conf); + provider.initialize(AuthenticationProvider.Context.builder().config(conf).build()); int[] tokenRemainTime = new int[]{3, 7, 40, 100, 400}; @@ -1633,27 +1637,19 @@ public String getCommandData() { Metric countMetric = ((List) metrics.get("pulsar_expiring_token_minutes_count")).get(0); assertEquals(countMetric.value, tokenRemainTime.length); List cm = (List) metrics.get("pulsar_expiring_token_minutes_bucket"); - assertEquals(cm.size(), 5); + var buckets = cm.stream().map(m -> m.tags.get("le")).collect(Collectors.toSet()); + assertThat(buckets).isEqualTo(Set.of("5.0", "10.0", "60.0", "240.0", "1440.0", "10080.0", "20160.0", "43200.0", + "129600.0", "259200.0", "388800.0", "525600.0", "+Inf")); cm.forEach((e) -> { - switch (e.tags.get("le")) { - case "5.0": - assertEquals(e.value, 1); - break; - case "10.0": - assertEquals(e.value, 2); - break; - case "60.0": - assertEquals(e.value, 3); - break; - case "240.0": - assertEquals(e.value, 4); - break; - default: - assertEquals(e.value, 5); - break; - } + var expectedValue = switch(e.tags.get("le")) { + case "5.0" -> 1; + case "10.0" -> 2; + case "60.0" -> 3; + case "240.0" -> 4; + default -> 5; + }; + assertEquals(e.value, expectedValue); }); - provider.close(); } @Test From 59424a831b3ab973c607ba17e9ec6dcebd6e9ee5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 29 Aug 2024 07:39:09 +0300 Subject: [PATCH 474/580] [improve][broker] Add msgInReplay subscription stat and metric to improve Key_Shared observability (#23224) --- .../service/persistent/MessageRedeliveryController.java | 9 +++++++++ .../PersistentDispatcherMultipleConsumers.java | 5 ++++- .../service/persistent/PersistentSubscription.java | 1 + .../stats/prometheus/AggregatedNamespaceStats.java | 4 ++++ .../stats/prometheus/AggregatedSubscriptionStats.java | 2 ++ .../stats/prometheus/NamespaceStatsAggregator.java | 3 +++ .../pulsar/broker/stats/prometheus/TopicStats.java | 2 ++ .../pulsar/common/policies/data/SubscriptionStats.java | 3 +++ .../policies/data/stats/SubscriptionStatsImpl.java | 6 ++++++ 9 files changed, 34 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java index 526874a7ae34b..9d29b93ca450d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java @@ -149,4 +149,13 @@ public boolean containsStickyKeyHashes(Set stickyKeyHashes) { public NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { return messagesToRedeliver.items(maxMessagesToRead, PositionFactory::create); } + + /** + * Get the number of messages registered for replay in the redelivery controller. + * + * @return number of messages + */ + public int size() { + return messagesToRedeliver.size(); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 274bdd9947a07..20dbc4925d152 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -135,7 +135,6 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul protected final ExecutorService dispatchMessagesThread; private final SharedConsumerAssignor assignor; - protected enum ReadType { Normal, Replay } @@ -1352,5 +1351,9 @@ public Subscription getSubscription() { return subscription; } + public long getNumberOfMessagesInReplay() { + return redeliveryMessages.size(); + } + private static final Logger log = LoggerFactory.getLogger(PersistentDispatcherMultipleConsumers.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index f59ea18ce8ea7..ea1b7d7602be7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -1276,6 +1276,7 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge subStats.unackedMessages = d.getTotalUnackedMessages(); subStats.blockedSubscriptionOnUnackedMsgs = d.isBlockedDispatcherOnUnackedMsgs(); subStats.msgDelayed = d.getNumberOfDelayedMessages(); + subStats.msgInReplay = d.getNumberOfMessagesInReplay(); } } subStats.msgBacklog = getNumberOfEntriesInBacklog(getStatsOptions.isGetPreciseBacklog()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java index 85ff15c915aa7..aaaea7b493e45 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java @@ -43,6 +43,7 @@ public class AggregatedNamespaceStats { public ManagedLedgerStats managedLedgerStats = new ManagedLedgerStats(); public long msgBacklog; public long msgDelayed; + public long msgInReplay; public long ongoingTxnCount; public long abortedTxnCount; @@ -141,10 +142,12 @@ void updateStats(TopicStats stats) { AggregatedSubscriptionStats subsStats = subscriptionStats.computeIfAbsent(n, k -> new AggregatedSubscriptionStats()); msgDelayed += as.msgDelayed; + msgInReplay += as.msgInReplay; subsStats.blockedSubscriptionOnUnackedMsgs = as.blockedSubscriptionOnUnackedMsgs; subsStats.msgBacklog += as.msgBacklog; subsStats.msgBacklogNoDelayed += as.msgBacklogNoDelayed; subsStats.msgDelayed += as.msgDelayed; + subsStats.msgInReplay += as.msgInReplay; subsStats.msgRateRedeliver += as.msgRateRedeliver; subsStats.unackedMessages += as.unackedMessages; subsStats.filterProcessedMsgCount += as.filterProcessedMsgCount; @@ -200,6 +203,7 @@ public void reset() { msgBacklog = 0; msgDelayed = 0; + msgInReplay = 0; ongoingTxnCount = 0; abortedTxnCount = 0; committedTxnCount = 0; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedSubscriptionStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedSubscriptionStats.java index da0324c55655c..b713146f58bac 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedSubscriptionStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedSubscriptionStats.java @@ -43,6 +43,8 @@ public class AggregatedSubscriptionStats { public long msgDelayed; + public long msgInReplay; + long msgOutCounter; long bytesOutCounter; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java index f0d11167e65fe..25c875778c05c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java @@ -134,6 +134,7 @@ private static void aggregateTopicStats(TopicStats stats, SubscriptionStatsImpl subsStats.msgOutCounter = subscriptionStats.msgOutCounter; subsStats.msgBacklog = subscriptionStats.msgBacklog; subsStats.msgDelayed = subscriptionStats.msgDelayed; + subsStats.msgInReplay = subscriptionStats.msgInReplay; subsStats.msgRateExpired = subscriptionStats.msgRateExpired; subsStats.totalMsgExpired = subscriptionStats.totalMsgExpired; subsStats.msgBacklogNoDelayed = subsStats.msgBacklog - subsStats.msgDelayed; @@ -424,6 +425,8 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat writeMetric(stream, "pulsar_subscription_delayed", stats.msgDelayed, cluster, namespace); + writeMetric(stream, "pulsar_subscription_in_replay", stats.msgInReplay, cluster, namespace); + writeMetric(stream, "pulsar_delayed_message_index_size_bytes", stats.delayedMessageIndexSizeInBytes, cluster, namespace); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java index 013b528731060..e54a3710e1294 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java @@ -310,6 +310,8 @@ public static void printTopicStats(PrometheusMetricStreams stream, TopicStats st subsStats.msgBacklogNoDelayed, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); writeSubscriptionMetric(stream, "pulsar_subscription_delayed", subsStats.msgDelayed, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); + writeSubscriptionMetric(stream, "pulsar_subscription_in_replay", + subsStats.msgInReplay, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); writeSubscriptionMetric(stream, "pulsar_subscription_msg_rate_redeliver", subsStats.msgRateRedeliver, cluster, namespace, topic, sub, splitTopicAndPartitionIndexLabel); writeSubscriptionMetric(stream, "pulsar_subscription_unacked_messages", diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index cabef1ca9602d..e307e41862e74 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -66,6 +66,9 @@ public interface SubscriptionStats { /** Number of delayed messages currently being tracked. */ long getMsgDelayed(); + /** Number of messages registered for replay. */ + long getMsgInReplay(); + /** * Number of unacknowledged messages for the subscription, where an unacknowledged message is one that has been * sent to a consumer but not yet acknowledged. Calculated by summing all {@link ConsumerStats#getUnackedMessages()} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java index ab4d07c7ae486..977ed28e86814 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java @@ -74,6 +74,9 @@ public class SubscriptionStatsImpl implements SubscriptionStats { /** Number of delayed messages currently being tracked. */ public long msgDelayed; + /** Number of messages registered for replay. */ + public long msgInReplay; + /** * Number of unacknowledged messages for the subscription, where an unacknowledged message is one that has been * sent to a consumer but not yet acknowledged. Calculated by summing all {@link ConsumerStatsImpl#unackedMessages} @@ -173,6 +176,8 @@ public void reset() { msgBacklog = 0; backlogSize = 0; msgBacklogNoDelayed = 0; + msgDelayed = 0; + msgInReplay = 0; unackedMessages = 0; type = null; msgRateExpired = 0; @@ -208,6 +213,7 @@ public SubscriptionStatsImpl add(SubscriptionStatsImpl stats) { this.backlogSize += stats.backlogSize; this.msgBacklogNoDelayed += stats.msgBacklogNoDelayed; this.msgDelayed += stats.msgDelayed; + this.msgInReplay += stats.msgInReplay; this.unackedMessages += stats.unackedMessages; this.type = stats.type; this.msgRateExpired += stats.msgRateExpired; From d98e51f7a54463d68d4521189d24566888888514 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 29 Aug 2024 22:27:34 +0300 Subject: [PATCH 475/580] [improve][broker] Reschedule reads with increasing backoff when no messages are dispatched (#23226) --- conf/broker.conf | 10 +++ conf/standalone.conf | 10 +++ .../pulsar/broker/ServiceConfiguration.java | 14 ++++ ...PersistentDispatcherMultipleConsumers.java | 59 +++++++++---- ...tStickyKeyDispatcherMultipleConsumers.java | 3 + .../auth/MockedPulsarServiceBaseTest.java | 3 + ...ckyKeyDispatcherMultipleConsumersTest.java | 84 +++++++++++++++++-- .../transaction/TransactionTestBase.java | 3 + 8 files changed, 163 insertions(+), 23 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index fc32246adea1f..ed59e5c456695 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -467,6 +467,16 @@ dispatcherReadFailureBackoffMaxTimeInMs=60000 # The read failure backoff mandatory stop time in milliseconds. By default it is 0s. dispatcherReadFailureBackoffMandatoryStopTimeInMs=0 +# On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered +# out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff +# delay. This parameter sets the initial backoff delay in milliseconds. +dispatcherRetryBackoffInitialTimeInMs=100 + +# On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered +# out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff +# delay. This parameter sets the maximum backoff delay in milliseconds. +dispatcherRetryBackoffMaxTimeInMs=1000 + # Precise dispatcher flow control according to history message number of each entry preciseDispatcherFlowControl=false diff --git a/conf/standalone.conf b/conf/standalone.conf index ae696410d86bf..d5d79e0383e1f 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -283,6 +283,16 @@ dispatcherReadFailureBackoffMaxTimeInMs=60000 # The read failure backoff mandatory stop time in milliseconds. By default it is 0s. dispatcherReadFailureBackoffMandatoryStopTimeInMs=0 +# On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered +# out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff +# delay. This parameter sets the initial backoff delay in milliseconds. +dispatcherRetryBackoffInitialTimeInMs=100 + +# On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered +# out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff +# delay. This parameter sets the maximum backoff delay in milliseconds. +dispatcherRetryBackoffMaxTimeInMs=1000 + # Precise dispatcher flow control according to history message number of each entry preciseDispatcherFlowControl=false 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 6488ace991e2f..60f37f52b6b8c 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 @@ -1196,6 +1196,20 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece ) private int dispatcherReadFailureBackoffMandatoryStopTimeInMs = 0; + @FieldContext( + category = CATEGORY_POLICIES, + doc = "On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered " + + "out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff " + + "delay. This parameter sets the initial backoff delay in milliseconds.") + private int dispatcherRetryBackoffInitialTimeInMs = 100; + + @FieldContext( + category = CATEGORY_POLICIES, + doc = "On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered " + + "out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff " + + "delay. This parameter sets the maximum backoff delay in milliseconds.") + private int dispatcherRetryBackoffMaxTimeInMs = 1000; + @FieldContext( dynamic = true, category = CATEGORY_SERVER, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 20dbc4925d152..631a728ccce4d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -47,6 +47,7 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.delayed.DelayedDeliveryTracker; import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; @@ -85,7 +86,6 @@ */ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMultipleConsumers implements Dispatcher, ReadEntriesCallback { - protected final PersistentTopic topic; protected final ManagedCursor cursor; protected volatile Range lastIndividualDeletedRangeFromCursorRecovery; @@ -134,7 +134,8 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul private AtomicBoolean isRescheduleReadInProgress = new AtomicBoolean(false); protected final ExecutorService dispatchMessagesThread; private final SharedConsumerAssignor assignor; - + protected int lastNumberOfEntriesDispatched; + private final Backoff retryBackoff; protected enum ReadType { Normal, Replay } @@ -159,10 +160,15 @@ public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCurso this.readBatchSize = serviceConfig.getDispatcherMaxReadBatchSize(); this.initializeDispatchRateLimiterIfNeeded(); this.assignor = new SharedConsumerAssignor(this::getNextConsumer, this::addMessageToReplay); + ServiceConfiguration serviceConfiguration = topic.getBrokerService().pulsar().getConfiguration(); this.readFailureBackoff = new Backoff( - topic.getBrokerService().pulsar().getConfiguration().getDispatcherReadFailureBackoffInitialTimeInMs(), + serviceConfiguration.getDispatcherReadFailureBackoffInitialTimeInMs(), TimeUnit.MILLISECONDS, 1, TimeUnit.MINUTES, 0, TimeUnit.MILLISECONDS); + retryBackoff = new Backoff( + serviceConfiguration.getDispatcherRetryBackoffInitialTimeInMs(), TimeUnit.MILLISECONDS, + serviceConfiguration.getDispatcherRetryBackoffMaxTimeInMs(), TimeUnit.MILLISECONDS, + 0, TimeUnit.MILLISECONDS); } @Override @@ -437,16 +443,20 @@ private boolean shouldPauseOnAckStatePersist(ReadType readType) { @Override protected void reScheduleRead() { + reScheduleReadInMs(MESSAGE_RATE_BACKOFF_MS); + } + + protected void reScheduleReadInMs(long readAfterMs) { if (isRescheduleReadInProgress.compareAndSet(false, true)) { if (log.isDebugEnabled()) { - log.debug("[{}] [{}] Reschedule message read in {} ms", topic.getName(), name, MESSAGE_RATE_BACKOFF_MS); + log.debug("[{}] [{}] Reschedule message read in {} ms", topic.getName(), name, readAfterMs); } topic.getBrokerService().executor().schedule( () -> { isRescheduleReadInProgress.set(false); readMoreEntries(); }, - MESSAGE_RATE_BACKOFF_MS, TimeUnit.MILLISECONDS); + readAfterMs, TimeUnit.MILLISECONDS); } } @@ -659,8 +669,8 @@ public final synchronized void readEntriesComplete(List entries, Object c log.debug("[{}] Distributing {} messages to {} consumers", name, entries.size(), consumerList.size()); } - long size = entries.stream().mapToLong(Entry::getLength).sum(); - updatePendingBytesToDispatch(size); + long totalBytesSize = entries.stream().mapToLong(Entry::getLength).sum(); + updatePendingBytesToDispatch(totalBytesSize); // dispatch messages to a separate thread, but still in order for this subscription // sendMessagesToConsumers is responsible for running broker-side filters @@ -670,19 +680,28 @@ public final synchronized void readEntriesComplete(List entries, Object c // in a separate thread, and we want to prevent more reads acquireSendInProgress(); dispatchMessagesThread.execute(() -> { - if (sendMessagesToConsumers(readType, entries, false)) { - updatePendingBytesToDispatch(-size); - readMoreEntries(); - } else { - updatePendingBytesToDispatch(-size); - } + handleSendingMessagesAndReadingMore(readType, entries, false, totalBytesSize); }); } else { - if (sendMessagesToConsumers(readType, entries, true)) { - updatePendingBytesToDispatch(-size); - readMoreEntriesAsync(); - } else { - updatePendingBytesToDispatch(-size); + handleSendingMessagesAndReadingMore(readType, entries, true, totalBytesSize); + } + } + + private synchronized void handleSendingMessagesAndReadingMore(ReadType readType, List entries, + boolean needAcquireSendInProgress, + long totalBytesSize) { + boolean triggerReadingMore = sendMessagesToConsumers(readType, entries, needAcquireSendInProgress); + int entriesDispatched = lastNumberOfEntriesDispatched; + updatePendingBytesToDispatch(-totalBytesSize); + if (triggerReadingMore) { + if (entriesDispatched > 0) { + // Reset the backoff when we successfully dispatched messages + retryBackoff.reset(); + // Call readMoreEntries in the same thread to trigger the next read + readMoreEntries(); + } else if (entriesDispatched == 0) { + // If no messages were dispatched, we need to reschedule a new read with an increasing backoff delay + reScheduleReadInMs(retryBackoff.next()); } } } @@ -721,6 +740,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis if (needTrimAckedMessages()) { cursor.trimDeletedEntries(entries); } + lastNumberOfEntriesDispatched = 0; int entriesToDispatch = entries.size(); // Trigger read more messages @@ -828,6 +848,8 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); entry.release(); }); + + lastNumberOfEntriesDispatched = entriesToDispatch; } return true; } @@ -890,6 +912,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, totalBytesSent += sendMessageInfo.getTotalBytes(); } + lastNumberOfEntriesDispatched = (int) totalEntries; acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); return numConsumers.get() == 0; // trigger a new readMoreEntries() call diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 91cec1f8e9071..97e6c943b0baa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -201,6 +201,7 @@ protected Map> initialValue() throws Exception { @Override protected synchronized boolean trySendMessagesToConsumers(ReadType readType, List entries) { + lastNumberOfEntriesDispatched = 0; long totalMessagesSent = 0; long totalBytesSent = 0; long totalEntries = 0; @@ -420,6 +421,8 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } } + lastNumberOfEntriesDispatched = (int) totalEntries; + // acquire message-dispatch permits for already delivered messages acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index e155e399e2437..c83888b8022b3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -242,6 +242,9 @@ protected void doInitConf() throws Exception { this.conf.setWebServicePort(Optional.of(0)); this.conf.setNumExecutorThreadPoolSize(5); this.conf.setExposeBundlesMetricsInPrometheus(true); + // Disable the dispatcher retry backoff in tests by default + this.conf.setDispatcherRetryBackoffInitialTimeInMs(0); + this.conf.setDispatcherRetryBackoffMaxTimeInMs(0); } protected final void init() throws Exception { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index 1a205d0f686d5..af99741d09bb6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -35,6 +35,7 @@ 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.assertNull; import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; @@ -53,6 +54,7 @@ import java.util.Queue; import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -60,10 +62,10 @@ import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerService; @@ -107,6 +109,7 @@ public class PersistentStickyKeyDispatcherMultipleConsumersTest { final String topicName = "persistent://public/default/testTopic"; final String subscriptionName = "testSubscription"; + private AtomicInteger consumerMockAvailablePermits; @BeforeMethod public void setup() throws Exception { @@ -117,7 +120,8 @@ public void setup() throws Exception { doReturn(1).when(configMock).getSubscriptionKeySharedConsistentHashingReplicaPoints(); doReturn(true).when(configMock).isDispatcherDispatchMessagesInSubscriptionThread(); doReturn(false).when(configMock).isAllowOverrideEntryFilters(); - + doReturn(10).when(configMock).getDispatcherRetryBackoffInitialTimeInMs(); + doReturn(50).when(configMock).getDispatcherRetryBackoffMaxTimeInMs(); pulsarMock = mock(PulsarService.class); doReturn(configMock).when(pulsarMock).getConfiguration(); @@ -188,7 +192,8 @@ public void setup() throws Exception { consumerMock = mock(Consumer.class); channelMock = mock(ChannelPromise.class); doReturn("consumer1").when(consumerMock).consumerName(); - doReturn(1000).when(consumerMock).getAvailablePermits(); + consumerMockAvailablePermits = new AtomicInteger(1000); + doAnswer(invocation -> consumerMockAvailablePermits.get()).when(consumerMock).getAvailablePermits(); doReturn(true).when(consumerMock).isWritable(); doReturn(channelMock).when(consumerMock).sendMessages( anyList(), @@ -511,8 +516,6 @@ public void testMessageRedelivery() throws Exception { allEntries.forEach(entry -> entry.release()); } - - @DataProvider(name = "initializeLastSentPosition") private Object[][] initialLastSentPositionProvider() { return new Object[][] { { false }, { true } }; @@ -822,6 +825,77 @@ public void testLastSentPositionAndIndividuallySentPositions(final boolean initi assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); } + @DataProvider(name = "dispatchMessagesInSubscriptionThread") + private Object[][] dispatchMessagesInSubscriptionThread() { + return new Object[][] { { false }, { true } }; + } + + @Test(dataProvider = "dispatchMessagesInSubscriptionThread") + public void testBackoffDelayWhenNoMessagesDispatched(boolean dispatchMessagesInSubscriptionThread) + throws Exception { + persistentDispatcher.close(); + + List retryDelays = new CopyOnWriteArrayList<>(); + doReturn(dispatchMessagesInSubscriptionThread).when(configMock).isDispatcherDispatchMessagesInSubscriptionThread(); + persistentDispatcher = new PersistentStickyKeyDispatcherMultipleConsumers( + topicMock, cursorMock, subscriptionMock, configMock, + new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)) { + @Override + protected void reScheduleReadInMs(long readAfterMs) { + retryDelays.add(readAfterMs); + } + }; + + // add a consumer without permits to trigger the retry behavior + consumerMockAvailablePermits.set(0); + persistentDispatcher.addConsumer(consumerMock); + + // call "readEntriesComplete" directly to test the retry behavior + List entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); + persistentDispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + Awaitility.await().untilAsserted(() -> { + assertEquals(retryDelays.size(), 1); + assertEquals(retryDelays.get(0), 10, "Initial retry delay should be 10ms"); + } + ); + // test the second retry delay + entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); + persistentDispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + Awaitility.await().untilAsserted(() -> { + assertEquals(retryDelays.size(), 2); + double delay = retryDelays.get(1); + assertEquals(delay, 20.0, 2.0, "Second retry delay should be 20ms (jitter <-10%)"); + } + ); + // verify the max retry delay + for (int i = 0; i < 100; i++) { + entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); + persistentDispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + } + Awaitility.await().untilAsserted(() -> { + assertEquals(retryDelays.size(), 102); + double delay = retryDelays.get(101); + assertEquals(delay, 50.0, 5.0, "Max delay should be 50ms (jitter <-10%)"); + } + ); + // unblock to check that the retry delay is reset + consumerMockAvailablePermits.set(1000); + entries = List.of(EntryImpl.create(1, 2, createMessage("message2", 1, "key2"))); + persistentDispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + // wait that the possibly async handling has completed + Awaitility.await().untilAsserted(() -> assertFalse(persistentDispatcher.isSendInProgress())); + + // now block again to check the next retry delay so verify it was reset + consumerMockAvailablePermits.set(0); + entries = List.of(EntryImpl.create(1, 3, createMessage("message3", 1, "key3"))); + persistentDispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + Awaitility.await().untilAsserted(() -> { + assertEquals(retryDelays.size(), 103); + assertEquals(retryDelays.get(0), 10, "Resetted retry delay should be 10ms"); + } + ); + } + private ByteBuf createMessage(String message, int sequenceId) { return createMessage(message, sequenceId, "testKey"); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java index 4ab886492a4eb..34af94f2c3185 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java @@ -163,6 +163,9 @@ protected void startBroker() throws Exception { conf.setBrokerDeduplicationEnabled(true); conf.setTransactionBufferSnapshotMaxTransactionCount(2); conf.setTransactionBufferSnapshotMinTimeInMillis(2000); + // Disable the dispatcher retry backoff in tests by default + conf.setDispatcherRetryBackoffInitialTimeInMs(0); + conf.setDispatcherRetryBackoffMaxTimeInMs(0); serviceConfigurationList.add(conf); PulsarTestContext.Builder testContextBuilder = From dccc06bf50bb5ca510b39167908c02d2b4602ca5 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 29 Aug 2024 22:25:58 -0700 Subject: [PATCH 476/580] [fix][broker] support peek-message for compressed and encrypted messages (#23234) --- .../pulsar/broker/admin/impl/PersistentTopicsBase.java | 6 +++++- .../pulsar/client/api/SimpleProducerConsumerTest.java | 5 +++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 40e74f83e986d..b2d455f645daf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.admin.impl; +import static org.apache.pulsar.common.api.proto.CompressionType.NONE; import static org.apache.pulsar.common.naming.SystemTopicNames.isSystemTopic; import static org.apache.pulsar.common.naming.SystemTopicNames.isTransactionCoordinatorAssign; import static org.apache.pulsar.common.naming.SystemTopicNames.isTransactionInternalName; @@ -2999,6 +3000,7 @@ private Response generateResponseWithEntry(Entry entry, PersistentTopic persiste checkNotNull(entry); Position pos = entry.getPosition(); ByteBuf metadataAndPayload = entry.getDataBuffer(); + boolean isEncrypted = false; long totalSize = metadataAndPayload.readableBytes(); BrokerEntryMetadata brokerEntryMetadata = Commands.peekBrokerEntryMetadataIfExist(metadataAndPayload); @@ -3070,6 +3072,7 @@ private Response generateResponseWithEntry(Entry entry, PersistentTopic persiste for (EncryptionKeys encryptionKeys : metadata.getEncryptionKeysList()) { responseBuilder.header("X-Pulsar-Base64-encryption-keys", Base64.getEncoder().encodeToString(encryptionKeys.toByteArray())); + isEncrypted = true; } if (metadata.hasEncryptionParam()) { responseBuilder.header("X-Pulsar-Base64-encryption-param", @@ -3123,7 +3126,8 @@ private Response generateResponseWithEntry(Entry entry, PersistentTopic persiste responseBuilder.header("X-Pulsar-txn-uncommitted", isTxnUncommitted); // Decode if needed - CompressionCodec codec = CompressionCodecProvider.getCompressionCodec(metadata.getCompression()); + CompressionCodec codec = CompressionCodecProvider + .getCompressionCodec(isEncrypted ? NONE : metadata.getCompression()); ByteBuf uncompressedPayload = codec.decode(metadataAndPayload, metadata.getUncompressedSize()); // Copy into a heap buffer for output stream compatibility diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index a9d97b7febdb7..61dd33be64aa2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -2712,12 +2712,17 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe Producer cryptoProducer = pulsarClient.newProducer() .topic(topicName).addEncryptionKey("client-ecdsa.pem") + .compressionType(CompressionType.LZ4) .cryptoKeyReader(new EncKeyReader()).create(); for (int i = 0; i < totalMsg; i++) { String message = "my-message-" + i; cryptoProducer.send(message.getBytes()); } + // admin api should be able to fetch compressed and encrypted message + List> msgs = admin.topics().peekMessages(topicName, "my-subscriber-name", 1); + assertNotNull(msgs); + Message msg; msg = normalConsumer.receive(RECEIVE_TIMEOUT_MEDIUM_MILLIS, TimeUnit.MILLISECONDS); From ed14f21de94c6af2bfae5318a014c56fac8a1a21 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Fri, 30 Aug 2024 10:00:54 -0700 Subject: [PATCH 477/580] [feat][broker] PIP-264: Add replication subscription stats (#23026) --- .../apache/pulsar/broker/PulsarService.java | 3 + .../ReplicatedSubscriptionsController.java | 24 +++++-- ...eplicatedSubscriptionsSnapshotBuilder.java | 14 ++-- ...nTelemetryReplicatedSubscriptionStats.java | 72 +++++++++++++++++++ .../service/ReplicatedSubscriptionTest.java | 19 ++++- .../ReplicatedSubscriptionConfigTest.java | 9 ++- ...catedSubscriptionsSnapshotBuilderTest.java | 35 ++++----- 7 files changed, 146 insertions(+), 30 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryReplicatedSubscriptionStats.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 9e147517ac724..0b994c640a9f5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -114,6 +114,7 @@ import org.apache.pulsar.broker.stats.MetricsGenerator; import org.apache.pulsar.broker.stats.OpenTelemetryConsumerStats; import org.apache.pulsar.broker.stats.OpenTelemetryProducerStats; +import org.apache.pulsar.broker.stats.OpenTelemetryReplicatedSubscriptionStats; import org.apache.pulsar.broker.stats.OpenTelemetryReplicatorStats; import org.apache.pulsar.broker.stats.OpenTelemetryTopicStats; import org.apache.pulsar.broker.stats.OpenTelemetryTransactionCoordinatorStats; @@ -265,6 +266,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private OpenTelemetryConsumerStats openTelemetryConsumerStats; private OpenTelemetryProducerStats openTelemetryProducerStats; private OpenTelemetryReplicatorStats openTelemetryReplicatorStats; + private OpenTelemetryReplicatedSubscriptionStats openTelemetryReplicatedSubscriptionStats; private OpenTelemetryTransactionCoordinatorStats openTelemetryTransactionCoordinatorStats; private OpenTelemetryTransactionPendingAckStoreStats openTelemetryTransactionPendingAckStoreStats; @@ -861,6 +863,7 @@ public void start() throws PulsarServerException { openTelemetryConsumerStats = new OpenTelemetryConsumerStats(this); openTelemetryProducerStats = new OpenTelemetryProducerStats(this); openTelemetryReplicatorStats = new OpenTelemetryReplicatorStats(this); + openTelemetryReplicatedSubscriptionStats = new OpenTelemetryReplicatedSubscriptionStats(this); localMetadataSynchronizer = StringUtils.isNotBlank(config.getMetadataSyncEventTopic()) ? new PulsarMetadataEventSynchronizer(this, config.getMetadataSyncEventTopic()) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java index a8e6885525a19..b873bc93cd1e4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java @@ -39,6 +39,7 @@ import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.pulsar.broker.service.Replicator; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.stats.OpenTelemetryReplicatedSubscriptionStats; import org.apache.pulsar.common.api.proto.ClusterMessageId; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition; @@ -49,6 +50,7 @@ import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshotResponse; import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsUpdate; import org.apache.pulsar.common.protocol.Markers; +import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric; /** * Encapsulate all the logic of replicated subscriptions tracking for a given topic. @@ -70,19 +72,25 @@ public class ReplicatedSubscriptionsController implements AutoCloseable, Topic.P private final ConcurrentMap pendingSnapshots = new ConcurrentHashMap<>(); + @PulsarDeprecatedMetric( + newMetricName = OpenTelemetryReplicatedSubscriptionStats.SNAPSHOT_OPERATION_COUNT_METRIC_NAME) + @Deprecated private static final Gauge pendingSnapshotsMetric = Gauge .build("pulsar_replicated_subscriptions_pending_snapshots", "Counter of currently pending snapshots") .register(); + private final OpenTelemetryReplicatedSubscriptionStats stats; + public ReplicatedSubscriptionsController(PersistentTopic topic, String localCluster) { this.topic = topic; this.localCluster = localCluster; - timer = topic.getBrokerService().pulsar().getExecutor() + var pulsar = topic.getBrokerService().pulsar(); + timer = pulsar.getExecutor() .scheduleAtFixedRate(catchingAndLoggingThrowables(this::startNewSnapshot), 0, - topic.getBrokerService().pulsar().getConfiguration() - .getReplicatedSubscriptionsSnapshotFrequencyMillis(), + pulsar.getConfiguration().getReplicatedSubscriptionsSnapshotFrequencyMillis(), TimeUnit.MILLISECONDS); + stats = pulsar.getOpenTelemetryReplicatedSubscriptionStats(); } public void receivedReplicatedSubscriptionMarker(Position position, int markerType, ByteBuf payload) { @@ -233,11 +241,11 @@ private void startNewSnapshot() { } pendingSnapshotsMetric.inc(); + stats.recordSnapshotStarted(); ReplicatedSubscriptionsSnapshotBuilder builder = new ReplicatedSubscriptionsSnapshotBuilder(this, topic.getReplicators().keys(), topic.getBrokerService().pulsar().getConfiguration(), Clock.systemUTC()); pendingSnapshots.put(builder.getSnapshotId(), builder); builder.start(); - } public Optional getLastCompletedSnapshotId() { @@ -254,6 +262,8 @@ private void cleanupTimedOutSnapshots() { } pendingSnapshotsMetric.dec(); + var latencyMillis = entry.getValue().getDurationMillis(); + stats.recordSnapshotTimedOut(latencyMillis); it.remove(); } } @@ -261,11 +271,15 @@ private void cleanupTimedOutSnapshots() { void snapshotCompleted(String snapshotId) { ReplicatedSubscriptionsSnapshotBuilder snapshot = pendingSnapshots.remove(snapshotId); - pendingSnapshotsMetric.dec(); lastCompletedSnapshotId = snapshotId; if (snapshot != null) { lastCompletedSnapshotStartTime = snapshot.getStartTimeMillis(); + + pendingSnapshotsMetric.dec(); + var latencyMillis = snapshot.getDurationMillis(); + ReplicatedSubscriptionsSnapshotBuilder.SNAPSHOT_METRIC.observe(latencyMillis); + stats.recordSnapshotCompleted(latencyMillis); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java index 4eb20f02907c0..0dacade3eed1c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java @@ -30,9 +30,11 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.stats.OpenTelemetryReplicatedSubscriptionStats; import org.apache.pulsar.common.api.proto.MarkersMessageIdData; import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshotResponse; import org.apache.pulsar.common.protocol.Markers; +import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric; @Slf4j public class ReplicatedSubscriptionsSnapshotBuilder { @@ -52,11 +54,13 @@ public class ReplicatedSubscriptionsSnapshotBuilder { private final Clock clock; - private static final Summary snapshotMetric = Summary.build("pulsar_replicated_subscriptions_snapshot_ms", + @PulsarDeprecatedMetric(newMetricName = OpenTelemetryReplicatedSubscriptionStats.SNAPSHOT_DURATION_METRIC_NAME) + @Deprecated + public static final Summary SNAPSHOT_METRIC = Summary.build("pulsar_replicated_subscriptions_snapshot_ms", "Time taken to create a consistent snapshot across clusters").register(); public ReplicatedSubscriptionsSnapshotBuilder(ReplicatedSubscriptionsController controller, - List remoteClusters, ServiceConfiguration conf, Clock clock) { + List remoteClusters, ServiceConfiguration conf, Clock clock) { this.snapshotId = UUID.randomUUID().toString(); this.controller = controller; this.remoteClusters = remoteClusters; @@ -123,8 +127,6 @@ synchronized void receivedSnapshotResponse(Position position, ReplicatedSubscrip p.getLedgerId(), p.getEntryId(), responses)); controller.snapshotCompleted(snapshotId); - double latencyMillis = clock.millis() - startTimeMillis; - snapshotMetric.observe(latencyMillis); } boolean isTimedOut() { @@ -134,4 +136,8 @@ boolean isTimedOut() { long getStartTimeMillis() { return startTimeMillis; } + + long getDurationMillis() { + return clock.millis() - startTimeMillis; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryReplicatedSubscriptionStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryReplicatedSubscriptionStats.java new file mode 100644 index 0000000000000..55982eba24312 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/OpenTelemetryReplicatedSubscriptionStats.java @@ -0,0 +1,72 @@ +/* + * 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.stats; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.DoubleHistogram; +import io.opentelemetry.api.metrics.LongCounter; +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.common.stats.MetricsUtil; + +public class OpenTelemetryReplicatedSubscriptionStats { + + public static final AttributeKey SNAPSHOT_OPERATION_RESULT = + AttributeKey.stringKey("pulsar.replication.subscription.snapshot.operation.result"); + public enum SnapshotOperationResult { + SUCCESS, + TIMEOUT; + private final Attributes attributes = Attributes.of(SNAPSHOT_OPERATION_RESULT, name().toLowerCase()); + } + + public static final String SNAPSHOT_OPERATION_COUNT_METRIC_NAME = + "pulsar.broker.replication.subscription.snapshot.operation.count"; + private final LongCounter snapshotOperationCounter; + + public static final String SNAPSHOT_DURATION_METRIC_NAME = + "pulsar.broker.replication.subscription.snapshot.operation.duration"; + private final DoubleHistogram snapshotDuration; + + public OpenTelemetryReplicatedSubscriptionStats(PulsarService pulsar) { + var meter = pulsar.getOpenTelemetry().getMeter(); + snapshotOperationCounter = meter.counterBuilder(SNAPSHOT_OPERATION_COUNT_METRIC_NAME) + .setDescription("The number of snapshot operations attempted") + .setUnit("{operation}") + .build(); + snapshotDuration = meter.histogramBuilder(SNAPSHOT_DURATION_METRIC_NAME) + .setDescription("Time taken to complete a consistent snapshot operation across clusters") + .setUnit("s") + .build(); + } + + public void recordSnapshotStarted() { + snapshotOperationCounter.add(1); + } + + public void recordSnapshotTimedOut(long durationMs) { + snapshotDuration.record(MetricsUtil.convertToSeconds(durationMs, TimeUnit.MILLISECONDS), + SnapshotOperationResult.TIMEOUT.attributes); + } + + public void recordSnapshotCompleted(long durationMs) { + snapshotDuration.record(MetricsUtil.convertToSeconds(durationMs, TimeUnit.MILLISECONDS), + SnapshotOperationResult.SUCCESS.attributes); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatedSubscriptionTest.java index e5aad47dc89c7..4273e8bbaeb5b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatedSubscriptionTest.java @@ -18,6 +18,10 @@ */ package org.apache.pulsar.broker.service; +import static org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil.assertMetricLongSumValue; +import static org.apache.pulsar.broker.stats.OpenTelemetryReplicatedSubscriptionStats.SNAPSHOT_DURATION_METRIC_NAME; +import static org.apache.pulsar.broker.stats.OpenTelemetryReplicatedSubscriptionStats.SNAPSHOT_OPERATION_COUNT_METRIC_NAME; +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.assertNotEquals; @@ -26,7 +30,8 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.google.common.collect.Sets; - +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.sdk.testing.assertj.OpenTelemetryAssertions; import java.lang.reflect.Field; import java.lang.reflect.Method; import java.nio.charset.StandardCharsets; @@ -141,7 +146,6 @@ public void testReplicatedSubscriptionAcrossTwoRegions() throws Exception { producer.send(body.getBytes(StandardCharsets.UTF_8)); sentMessages.add(body); } - producer.close(); } Set receivedMessages = new LinkedHashSet<>(); @@ -170,6 +174,17 @@ public void testReplicatedSubscriptionAcrossTwoRegions() throws Exception { // assert that all messages have been received assertEquals(new ArrayList<>(sentMessages), new ArrayList<>(receivedMessages), "Sent and received " + "messages don't match."); + + var metrics1 = metricReader1.collectAllMetrics(); + assertMetricLongSumValue(metrics1, SNAPSHOT_OPERATION_COUNT_METRIC_NAME, + Attributes.empty(),value -> assertThat(value).isPositive()); + assertMetricLongSumValue(metrics1, SNAPSHOT_OPERATION_COUNT_METRIC_NAME, + Attributes.empty(), value -> assertThat(value).isPositive()); + assertThat(metrics1) + .anySatisfy(metric -> OpenTelemetryAssertions.assertThat(metric) + .hasName(SNAPSHOT_DURATION_METRIC_NAME) + .hasHistogramSatisfying(histogram -> histogram.hasPointsSatisfying( + histogramPoint -> histogramPoint.hasSumGreaterThan(0.0)))); } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionConfigTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionConfigTest.java index aa0015742f662..604326203e876 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionConfigTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionConfigTest.java @@ -20,10 +20,9 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; - import lombok.Cleanup; - import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.Schema; @@ -48,6 +47,12 @@ public void cleanup() throws Exception { super.internalCleanup(); } + @Override + protected void customizeMainPulsarTestContextBuilder(PulsarTestContext.Builder pulsarTestContextBuilder) { + super.customizeMainPulsarTestContextBuilder(pulsarTestContextBuilder); + pulsarTestContextBuilder.enableOpenTelemetry(true); + } + @Test public void createReplicatedSubscription() throws Exception { this.conf.setEnableReplicatedSubscriptions(true); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilderTest.java index f5c3bb9d75bbd..562c5eda58109 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilderTest.java @@ -25,11 +25,8 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; - import io.netty.buffer.ByteBuf; - import java.time.Clock; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -71,7 +68,8 @@ public void setup() { Commands.skipMessageMetadata(marker); markers.add(marker); return null; - }).when(controller) + }) + .when(controller) .writeMarker(any(ByteBuf.class)); } @@ -80,7 +78,8 @@ public void testBuildSnapshotWith2Clusters() throws Exception { List remoteClusters = Collections.singletonList("b"); ReplicatedSubscriptionsSnapshotBuilder builder = new ReplicatedSubscriptionsSnapshotBuilder(controller, - remoteClusters, conf, clock); + remoteClusters, + conf, clock); assertTrue(markers.isEmpty()); @@ -93,8 +92,8 @@ public void testBuildSnapshotWith2Clusters() throws Exception { assertEquals(request.getSourceCluster(), localCluster); // Simulate the responses coming back - ReplicatedSubscriptionsSnapshotResponse response = new ReplicatedSubscriptionsSnapshotResponse() - .setSnapshotId("snapshot-1"); + ReplicatedSubscriptionsSnapshotResponse response = new ReplicatedSubscriptionsSnapshotResponse().setSnapshotId( + "snapshot-1"); response.setCluster() .setCluster("b") .setMessageId() @@ -119,7 +118,8 @@ public void testBuildSnapshotWith3Clusters() throws Exception { List remoteClusters = Arrays.asList("b", "c"); ReplicatedSubscriptionsSnapshotBuilder builder = new ReplicatedSubscriptionsSnapshotBuilder(controller, - remoteClusters, conf, clock); + remoteClusters, + conf, clock); assertTrue(markers.isEmpty()); @@ -132,8 +132,8 @@ public void testBuildSnapshotWith3Clusters() throws Exception { assertEquals(request.getSourceCluster(), localCluster); // Simulate the responses coming back - ReplicatedSubscriptionsSnapshotResponse response1 = new ReplicatedSubscriptionsSnapshotResponse() - .setSnapshotId("snapshot-1"); + ReplicatedSubscriptionsSnapshotResponse response1 = new ReplicatedSubscriptionsSnapshotResponse().setSnapshotId( + "snapshot-1"); response1.setCluster() .setCluster("b") .setMessageId() @@ -144,8 +144,8 @@ public void testBuildSnapshotWith3Clusters() throws Exception { // No markers should be sent out assertTrue(markers.isEmpty()); - ReplicatedSubscriptionsSnapshotResponse response2 = new ReplicatedSubscriptionsSnapshotResponse() - .setSnapshotId("snapshot-1"); + ReplicatedSubscriptionsSnapshotResponse response2 = new ReplicatedSubscriptionsSnapshotResponse().setSnapshotId( + "snapshot-1"); response2.setCluster() .setCluster("c") .setMessageId() @@ -159,8 +159,8 @@ public void testBuildSnapshotWith3Clusters() throws Exception { assertEquals(request.getSourceCluster(), localCluster); // Responses coming back - ReplicatedSubscriptionsSnapshotResponse response3 = new ReplicatedSubscriptionsSnapshotResponse() - .setSnapshotId("snapshot-1"); + ReplicatedSubscriptionsSnapshotResponse response3 = new ReplicatedSubscriptionsSnapshotResponse().setSnapshotId( + "snapshot-1"); response3.setCluster() .setCluster("b") .setMessageId() @@ -171,8 +171,8 @@ public void testBuildSnapshotWith3Clusters() throws Exception { // No markers should be sent out assertTrue(markers.isEmpty()); - ReplicatedSubscriptionsSnapshotResponse response4 = new ReplicatedSubscriptionsSnapshotResponse() - .setSnapshotId("snapshot-1"); + ReplicatedSubscriptionsSnapshotResponse response4 = new ReplicatedSubscriptionsSnapshotResponse().setSnapshotId( + "snapshot-1"); response4.setCluster() .setCluster("c") .setMessageId() @@ -201,7 +201,8 @@ public void testBuildTimeout() { List remoteClusters = Collections.singletonList("b"); ReplicatedSubscriptionsSnapshotBuilder builder = new ReplicatedSubscriptionsSnapshotBuilder(controller, - remoteClusters, conf, clock); + remoteClusters, + conf, clock); assertFalse(builder.isTimedOut()); From 3a59e4c391d18637438c70bd2dd9fb030f715bf7 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 30 Aug 2024 11:24:33 -0700 Subject: [PATCH 478/580] [fix][client] Fix client to handle permits for discarded and failed decrypt batch-message (#23068) --- .../api/SimpleProducerConsumerTest.java | 86 +++++++++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 94 +++++++++++-------- 2 files changed, 139 insertions(+), 41 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index 61dd33be64aa2..2e71e8cc28c3e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -48,6 +48,7 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.time.Clock; +import java.time.Duration; import java.time.Instant; import java.time.ZoneId; import java.util.ArrayList; @@ -116,6 +117,8 @@ import org.apache.pulsar.common.compression.CompressionCodec; import org.apache.pulsar.common.compression.CompressionCodecProvider; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats.CursorStats; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.PublisherStats; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.protocol.Commands; @@ -4862,6 +4865,89 @@ public void testBatchReceiveWithMaxBatchSize() throws Exception { assertEquals(consumer.batchReceive().size(), maxBatchSize); } + /** + * + * This test validates that consumer correctly sends permits for batch message that should be discarded. + * @throws Exception + */ + @Test + public void testEncryptionFailureWithBatchPublish() throws Exception { + log.info("-- Starting {} test --", methodName); + String topicName = "persistent://my-property/my-ns/batchFailureTest-" + System.currentTimeMillis(); + + class EncKeyReader implements CryptoKeyReader { + + final EncryptionKeyInfo keyInfo = new EncryptionKeyInfo(); + + @Override + public EncryptionKeyInfo getPublicKey(String keyName, Map keyMeta) { + String CERT_FILE_PATH = "./src/test/resources/certificate/public-key." + keyName; + if (Files.isReadable(Paths.get(CERT_FILE_PATH))) { + try { + keyInfo.setKey(Files.readAllBytes(Paths.get(CERT_FILE_PATH))); + return keyInfo; + } catch (IOException e) { + Assert.fail("Failed to read certificate from " + CERT_FILE_PATH); + } + } else { + Assert.fail("Certificate file " + CERT_FILE_PATH + " is not present or not readable."); + } + return null; + } + + @Override + public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMeta) { + String CERT_FILE_PATH = "./src/test/resources/certificate/private-key." + keyName; + if (Files.isReadable(Paths.get(CERT_FILE_PATH))) { + try { + keyInfo.setKey(Files.readAllBytes(Paths.get(CERT_FILE_PATH))); + return keyInfo; + } catch (IOException e) { + Assert.fail("Failed to read certificate from " + CERT_FILE_PATH); + } + } else { + Assert.fail("Certificate file " + CERT_FILE_PATH + " is not present or not readable."); + } + return null; + } + } + + final int totalMsg = 2000; + + String subName = "without-cryptoreader"; + @Cleanup + Consumer normalConsumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subName) + .messageListener((c, msg) -> { + log.info("Failed to consume message {}", msg.getMessageId()); + c.acknowledgeAsync(msg); + }).cryptoFailureAction(ConsumerCryptoFailureAction.DISCARD).ackTimeout(1, TimeUnit.SECONDS) + .receiverQueueSize(totalMsg / 20).subscribe(); + + @Cleanup + Producer cryptoProducer = pulsarClient.newProducer().topic(topicName) + .addEncryptionKey("client-ecdsa.pem").enableBatching(true).batchingMaxMessages(5) + .batchingMaxPublishDelay(1, TimeUnit.SECONDS).cryptoKeyReader(new EncKeyReader()).create(); + for (int i = 0; i < totalMsg; i++) { + String message = "my-message-" + i; + cryptoProducer.sendAsync(message.getBytes()); + } + cryptoProducer.flush(); + + Awaitility.await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> { + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName); + CursorStats stats = internalStats.cursors.get(subName); + String readPosition = stats.readPosition; + assertEquals(getMessageId(readPosition, 0, 1), (getMessageId(internalStats.lastConfirmedEntry, 0, 0))); + }); + + log.info("-- Exiting {} test --", methodName); + } + + private MessageId getMessageId(String messageId, long subLedgerId, long subEntryId) { + String[] ids = messageId.split(":"); + return new MessageIdImpl(Long.parseLong(ids[0]) - subLedgerId, Long.parseLong(ids[1]) - subEntryId, -1); + } + private int compareMessageIds(MessageIdImpl messageId1, MessageIdImpl messageId2) { if (messageId2.getLedgerId() < messageId1.getLedgerId()) { return -1; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 596e65484d1b2..4f041772af3fb 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1892,30 +1892,10 @@ private ByteBuf decryptPayloadIfNeeded(MessageIdData messageId, int redeliveryCo if (msgMetadata.getEncryptionKeysCount() == 0) { return payload.retain(); } - + int batchSize = msgMetadata.getNumMessagesInBatch(); // If KeyReader is not configured throw exception based on config param if (conf.getCryptoKeyReader() == null) { - switch (conf.getCryptoFailureAction()) { - case CONSUME: - log.debug("[{}][{}][{}] CryptoKeyReader interface is not implemented. Consuming encrypted message.", - topic, subscription, consumerName); - return payload.retain(); - case DISCARD: - log.warn( - "[{}][{}][{}] Skipping decryption since CryptoKeyReader interface is not implemented and" - + " config is set to discard", - topic, subscription, consumerName); - discardMessage(messageId, currentCnx, ValidationError.DecryptionError); - return null; - case FAIL: - MessageId m = new MessageIdImpl(messageId.getLedgerId(), messageId.getEntryId(), partitionIndex); - log.error( - "[{}][{}][{}][{}] Message delivery failed since CryptoKeyReader interface is not" - + " implemented to consume encrypted message", - topic, subscription, consumerName, m); - unAckedMessageTracker.add(m, redeliveryCount); - return null; - } + return handleCryptoFailure(payload, messageId, currentCnx, redeliveryCount, batchSize, true); } @@ -1929,27 +1909,58 @@ private ByteBuf decryptPayloadIfNeeded(MessageIdData messageId, int redeliveryCo decryptedData.release(); + return handleCryptoFailure(payload, messageId, currentCnx, redeliveryCount, batchSize, false); + } + + private ByteBuf handleCryptoFailure(ByteBuf payload, MessageIdData messageId, ClientCnx currentCnx, + int redeliveryCount, int batchSize, boolean cryptoReaderNotExist) { + switch (conf.getCryptoFailureAction()) { - case CONSUME: + case CONSUME: + if (cryptoReaderNotExist) { + log.warn("[{}][{}][{}] CryptoKeyReader interface is not implemented. Consuming encrypted message.", + topic, subscription, consumerName); + } else { // Note, batch message will fail to consume even if config is set to consume log.warn("[{}][{}][{}][{}] Decryption failed. Consuming encrypted message since config is set to" - + " consume.", - topic, subscription, consumerName, messageId); - return payload.retain(); - case DISCARD: - log.warn("[{}][{}][{}][{}] Discarding message since decryption failed and config is set to discard", - topic, subscription, consumerName, messageId); - discardMessage(messageId, currentCnx, ValidationError.DecryptionError); - return null; - case FAIL: - MessageId m = new MessageIdImpl(messageId.getLedgerId(), messageId.getEntryId(), partitionIndex); + + " consume.", topic, subscription, consumerName, messageId); + } + return payload.retain(); + case DISCARD: + if (cryptoReaderNotExist) { + log.warn( + "[{}][{}][{}] Skipping decryption since CryptoKeyReader interface is not implemented and" + + " config is set to discard message with batch size {}", + topic, subscription, consumerName, batchSize); + } else { + log.warn( + "[{}][{}][{}][{}-{}-{}] Discarding message since decryption failed " + + "and config is set to discard", + topic, subscription, consumerName, messageId.getLedgerId(), messageId.getEntryId(), + messageId.getBatchIndex()); + } + discardMessage(messageId, currentCnx, ValidationError.DecryptionError, batchSize); + return null; + case FAIL: + if (cryptoReaderNotExist) { log.error( - "[{}][{}][{}][{}] Message delivery failed since unable to decrypt incoming message", - topic, subscription, consumerName, m); - unAckedMessageTracker.add(m, redeliveryCount); - return null; + "[{}][{}][{}][{}-{}-{}] Message delivery failed since CryptoKeyReader interface is not" + + " implemented to consume encrypted message", + topic, subscription, consumerName, messageId.getLedgerId(), messageId.getEntryId(), + partitionIndex); + } else { + log.error("[{}][{}][{}][{}-{}-{}] Message delivery failed since unable to decrypt incoming message", + topic, subscription, consumerName, messageId.getLedgerId(), messageId.getEntryId(), + partitionIndex); + } + MessageId m = new MessageIdImpl(messageId.getLedgerId(), messageId.getEntryId(), partitionIndex); + unAckedMessageTracker.add(m, redeliveryCount); + return null; + default: + log.warn("[{}][{}][{}] Invalid crypto failure state found, continue message consumption.", topic, + subscription, consumerName); + return payload.retain(); } - return null; } private ByteBuf uncompressPayloadIfNeeded(MessageIdData messageId, MessageMetadata msgMetadata, ByteBuf payload, @@ -2009,14 +2020,15 @@ private void discardCorruptedMessage(MessageIdData messageId, ClientCnx currentC ValidationError validationError) { log.error("[{}][{}] Discarding corrupted message at {}:{}", topic, subscription, messageId.getLedgerId(), messageId.getEntryId()); - discardMessage(messageId, currentCnx, validationError); + discardMessage(messageId, currentCnx, validationError, 1); } - private void discardMessage(MessageIdData messageId, ClientCnx currentCnx, ValidationError validationError) { + private void discardMessage(MessageIdData messageId, ClientCnx currentCnx, ValidationError validationError, + int batchMessages) { ByteBuf cmd = Commands.newAck(consumerId, messageId.getLedgerId(), messageId.getEntryId(), null, AckType.Individual, validationError, Collections.emptyMap(), -1); currentCnx.ctx().writeAndFlush(cmd, currentCnx.ctx().voidPromise()); - increaseAvailablePermits(currentCnx); + increaseAvailablePermits(currentCnx, batchMessages); stats.incrementNumReceiveFailed(); } From 8da3bf8322c536c495541c80926cdf9389612515 Mon Sep 17 00:00:00 2001 From: Girish Sharma Date: Sun, 1 Sep 2024 09:49:45 +0530 Subject: [PATCH 479/580] [improve][admin] PIP-369 Introduce `unload` flag in `ns-isolation-policy set` call (#23120) Co-authored-by: Zixuan Liu --- .../broker/admin/impl/ClustersBase.java | 54 ++++- .../pulsar/broker/admin/AdminApi2Test.java | 208 ++++++++++++++++-- .../policies/data/NamespaceIsolationData.java | 4 + .../NamespaceIsolationPolicyUnloadScope.java | 37 ++++ .../cli/CmdNamespaceIsolationPolicy.java | 17 +- .../policies/NamespaceIsolationPolicy.java | 6 + .../data/NamespaceIsolationDataImpl.java | 17 +- .../impl/NamespaceIsolationPolicyImpl.java | 8 + 8 files changed, 324 insertions(+), 27 deletions(-) create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationPolicyUnloadScope.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java index 4fe8a01e679da..132c99ce16bec 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java @@ -27,11 +27,13 @@ import io.swagger.annotations.ExampleProperty; import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -65,6 +67,7 @@ import org.apache.pulsar.common.policies.data.ClusterPoliciesImpl; import org.apache.pulsar.common.policies.data.FailureDomainImpl; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; +import org.apache.pulsar.common.policies.data.NamespaceIsolationPolicyUnloadScope; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicyImpl; import org.apache.pulsar.common.util.FutureUtil; @@ -721,10 +724,13 @@ public void setNamespaceIsolationPolicy( .setIsolationDataWithCreateAsync(cluster, (p) -> Collections.emptyMap()) .thenApply(__ -> new NamespaceIsolationPolicies())) ).thenCompose(nsIsolationPolicies -> { + NamespaceIsolationDataImpl oldPolicy = nsIsolationPolicies + .getPolicies().getOrDefault(policyName, null); nsIsolationPolicies.setPolicy(policyName, policyData); return namespaceIsolationPolicies() - .setIsolationDataAsync(cluster, old -> nsIsolationPolicies.getPolicies()); - }).thenCompose(__ -> filterAndUnloadMatchedNamespaceAsync(cluster, policyData)) + .setIsolationDataAsync(cluster, old -> nsIsolationPolicies.getPolicies()) + .thenApply(__ -> oldPolicy); + }).thenCompose(oldPolicy -> filterAndUnloadMatchedNamespaceAsync(cluster, policyData, oldPolicy)) .thenAccept(__ -> { log.info("[{}] Successful to update clusters/{}/namespaceIsolationPolicies/{}.", clientAppId(), cluster, policyName); @@ -759,7 +765,13 @@ public void setNamespaceIsolationPolicy( * Get matched namespaces; call unload for each namespaces. */ private CompletableFuture filterAndUnloadMatchedNamespaceAsync(String cluster, - NamespaceIsolationDataImpl policyData) { + NamespaceIsolationDataImpl policyData, + NamespaceIsolationDataImpl oldPolicy) { + // exit early if none of the namespaces need to be unloaded + if (NamespaceIsolationPolicyUnloadScope.none.equals(policyData.getUnloadScope())) { + return CompletableFuture.completedFuture(null); + } + PulsarAdmin adminClient; try { adminClient = pulsar().getAdminClient(); @@ -768,6 +780,7 @@ private CompletableFuture filterAndUnloadMatchedNamespaceAsync(String clus } // compile regex patterns once List namespacePatterns = policyData.getNamespaces().stream().map(Pattern::compile).toList(); + // TODO for 4.x, we should include both old and new namespace regex pattern for unload `all_matching` option return adminClient.tenants().getTenantsAsync().thenCompose(tenants -> { List>> filteredNamespacesForEachTenant = tenants.stream() .map(tenant -> adminClient.namespaces().getNamespacesAsync(tenant).thenCompose(namespaces -> { @@ -793,6 +806,41 @@ private CompletableFuture filterAndUnloadMatchedNamespaceAsync(String clus if (CollectionUtils.isEmpty(shouldUnloadNamespaces)) { return CompletableFuture.completedFuture(null); } + // If unload type is 'changed', we need to figure out a further subset of namespaces whose placement might + // actually have been changed. + + log.debug("Old policy: {} ; new policy: {}", oldPolicy, policyData); + if (oldPolicy != null && NamespaceIsolationPolicyUnloadScope.changed.equals(policyData.getUnloadScope())) { + // We also compare that the previous primary broker list is same as current, in case all namespaces need + // to be placed again anyway. + if (CollectionUtils.isEqualCollection(oldPolicy.getPrimary(), policyData.getPrimary())) { + // list is same, so we continue finding the changed namespaces. + + // We create a union regex list contains old + new regexes + Set combinedNamespaces = new HashSet<>(oldPolicy.getNamespaces()); + combinedNamespaces.addAll(policyData.getNamespaces()); + // We create a intersection of the old and new regexes. These won't need to be unloaded + Set commonNamespaces = new HashSet<>(oldPolicy.getNamespaces()); + commonNamespaces.retainAll(policyData.getNamespaces()); + + log.debug("combined regexes: {}; common regexes:{}", combinedNamespaces, combinedNamespaces); + + // Find the changed regexes (new - new ∩ old). TODO for 4.x, make this (new U old - new ∩ old) + combinedNamespaces.removeAll(commonNamespaces); + + log.debug("changed regexes: {}", commonNamespaces); + + // Now we further filter the filtered namespaces based on this combinedNamespaces set + shouldUnloadNamespaces = shouldUnloadNamespaces.stream() + .filter(name -> combinedNamespaces.stream() + .map(Pattern::compile) + .anyMatch(pattern -> pattern.matcher(name).matches()) + ).toList(); + + } + } + // unload type is either null or not in (changed, none), so we proceed to unload all namespaces + // TODO - default in 4.x should become `changed` List> futures = shouldUnloadNamespaces.stream() .map(namespaceName -> adminClient.namespaces().unloadAsync(namespaceName)) .collect(Collectors.toList()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java index 40e2ca8cce905..155994c814c11 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java @@ -22,6 +22,7 @@ import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; import static org.apache.pulsar.broker.resources.LoadBalanceResources.BUNDLE_DATA_BASE_PATH; +import static org.apache.pulsar.common.policies.data.NamespaceIsolationPolicyUnloadScope.*; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; @@ -53,6 +54,7 @@ import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import javax.ws.rs.NotAcceptableException; @@ -109,27 +111,7 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.AutoFailoverPolicyData; -import org.apache.pulsar.common.policies.data.AutoFailoverPolicyType; -import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; -import org.apache.pulsar.common.policies.data.BacklogQuota; -import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationData; -import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationDataImpl; -import org.apache.pulsar.common.policies.data.BundlesData; -import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.ConsumerStats; -import org.apache.pulsar.common.policies.data.EntryFilters; -import org.apache.pulsar.common.policies.data.FailureDomain; -import org.apache.pulsar.common.policies.data.NamespaceIsolationData; -import org.apache.pulsar.common.policies.data.NonPersistentTopicStats; -import org.apache.pulsar.common.policies.data.PartitionedTopicStats; -import org.apache.pulsar.common.policies.data.PersistencePolicies; -import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; -import org.apache.pulsar.common.policies.data.RetentionPolicies; -import org.apache.pulsar.common.policies.data.SubscriptionStats; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.common.policies.data.TopicStats; -import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.common.policies.data.*; import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; import org.apache.pulsar.common.protocol.schema.SchemaData; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; @@ -3496,4 +3478,188 @@ public void testGetStatsIfPartitionNotExists() throws Exception { // cleanup. admin.topics().deletePartitionedTopic(partitionedTp); } + + private NamespaceIsolationData createPolicyData(NamespaceIsolationPolicyUnloadScope scope, List namespaces, + List primaryBrokers + ) { + // setup ns-isolation-policy in both the clusters. + Map parameters1 = new HashMap<>(); + parameters1.put("min_limit", "1"); + parameters1.put("usage_threshold", "100"); + List nsRegexList = new ArrayList<>(namespaces); + + return NamespaceIsolationData.builder() + // "prop-ig/ns1" is present in test cluster, policy set on test2 should work + .namespaces(nsRegexList) + .primary(primaryBrokers) + .secondary(Collections.singletonList("")) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters1) + .build()) + .unloadScope(scope) + .build(); + } + + private boolean allTopicsUnloaded(List topics) { + for (String topic : topics) { + if (pulsar.getBrokerService().getTopicReference(topic).isPresent()) { + return false; + } + } + return true; + } + + private void loadTopics(List topics) throws PulsarClientException, ExecutionException, InterruptedException { + // create a topic by creating a producer so that the topic is present on the broker + for (String topic : topics) { + Producer producer = pulsarClient.newProducer().topic(topic).create(); + producer.close(); + pulsar.getBrokerService().getTopicIfExists(topic).get(); + } + + // All namespaces are loaded onto broker. Assert that + for (String topic : topics) { + assertTrue(pulsar.getBrokerService().getTopicReference(topic).isPresent()); + } + } + + /** + * Validates that the namespace isolation policy set and update is unloading only the relevant namespaces based on + * the unload scope provided. + * + * @param topicType persistent or non persistent. + * @param policyName policy name. + * @param nsPrefix unique namespace prefix. + * @param totalNamespaces total namespaces to create. Only the end part. Each namespace also gets a topic t1. + * @param initialScope unload scope while creating the policy. + * @param initialNamespaceRegex namespace regex while creating the policy. + * @param initialLoadedNS expected namespaces to be still loaded after the policy create call. Remaining namespaces + * will be asserted to be unloaded within 20 seconds. + * @param updatedScope unload scope while updating the policy. + * @param updatedNamespaceRegex namespace regex while updating the policy. + * @param updatedLoadedNS expected namespaces to be loaded after policy update call. Remaining namespaces will be + * asserted to be unloaded within 20 seconds. + * @throws PulsarAdminException + * @throws PulsarClientException + * @throws ExecutionException + * @throws InterruptedException + */ + private void testIsolationPolicyUnloadsNSWithScope(String topicType, String policyName, String nsPrefix, + List totalNamespaces, + NamespaceIsolationPolicyUnloadScope initialScope, + List initialNamespaceRegex, List initialLoadedNS, + NamespaceIsolationPolicyUnloadScope updatedScope, + List updatedNamespaceRegex, List updatedLoadedNS, + List updatedBrokerRegex) + throws PulsarAdminException, PulsarClientException, ExecutionException, InterruptedException { + + // Create all namespaces + List allTopics = new ArrayList<>(); + for (String namespacePart: totalNamespaces) { + admin.namespaces().createNamespace(nsPrefix + namespacePart, Set.of("test")); + allTopics.add(topicType + "://" + nsPrefix + namespacePart + "/t1"); + } + // Load all topics so that they are present. Assume topic t1 under each namespace + loadTopics(allTopics); + + // Create the policy + NamespaceIsolationData nsPolicyData1 = createPolicyData( + initialScope, initialNamespaceRegex, Collections.singletonList(".*") + ); + admin.clusters().createNamespaceIsolationPolicy("test", policyName, nsPolicyData1); + + List initialLoadedTopics = new ArrayList<>(); + for (String namespacePart: initialLoadedNS) { + initialLoadedTopics.add(topicType + "://" + nsPrefix + namespacePart + "/t1"); + } + + List initialUnloadedTopics = new ArrayList<>(allTopics); + initialUnloadedTopics.removeAll(initialLoadedTopics); + + // Assert that all topics (and thus ns) not under initialLoadedNS namespaces are unloaded + if (initialUnloadedTopics.isEmpty()) { + // Just wait a bit to ensure we don't miss lazy unloading of topics we expect not to unload + TimeUnit.SECONDS.sleep(5); + } else { + Awaitility.await() + .atMost(10, TimeUnit.SECONDS) + .until(() -> allTopicsUnloaded(initialUnloadedTopics)); + } + // Assert that all topics under initialLoadedNS are still present + initialLoadedTopics.forEach(t -> assertTrue(pulsar.getBrokerService().getTopicReference(t).isPresent())); + + // Load the topics again + loadTopics(allTopics); + + // Update policy using updatedScope with updated namespace regex + nsPolicyData1 = createPolicyData(updatedScope, updatedNamespaceRegex, updatedBrokerRegex); + admin.clusters().updateNamespaceIsolationPolicy("test", policyName, nsPolicyData1); + + List updatedLoadedTopics = new ArrayList<>(); + for (String namespacePart : updatedLoadedNS) { + updatedLoadedTopics.add(topicType + "://" + nsPrefix + namespacePart + "/t1"); + } + + List updatedUnloadedTopics = new ArrayList<>(allTopics); + updatedUnloadedTopics.removeAll(updatedLoadedTopics); + + // Assert that all topics (and thus ns) not under updatedLoadedNS namespaces are unloaded + if (updatedUnloadedTopics.isEmpty()) { + // Just wait a bit to ensure we don't miss lazy unloading of topics we expect not to unload + TimeUnit.SECONDS.sleep(5); + } else { + Awaitility.await() + .atMost(10, TimeUnit.SECONDS) + .until(() -> allTopicsUnloaded(updatedUnloadedTopics)); + } + // Assert that all topics under updatedLoadedNS are still present + updatedLoadedTopics.forEach(t -> assertTrue(pulsar.getBrokerService().getTopicReference(t).isPresent())); + + } + + @Test(dataProvider = "topicType") + public void testIsolationPolicyUnloadsNSWithAllScope(final String topicType) throws Exception { + String nsPrefix = newUniqueName(defaultTenant + "/") + "-unload-test-"; + testIsolationPolicyUnloadsNSWithScope( + topicType, "policy-all", nsPrefix, List.of("a1", "a2", "b1", "b2", "c1"), + all_matching, List.of(".*-unload-test-a.*"), List.of("b1", "b2", "c1"), + all_matching, List.of(".*-unload-test-a.*", ".*-unload-test-c.*"), List.of("b1", "b2"), + Collections.singletonList(".*") + ); + } + + @Test(dataProvider = "topicType") + public void testIsolationPolicyUnloadsNSWithChangedScope(final String topicType) throws Exception { + String nsPrefix = newUniqueName(defaultTenant + "/") + "-unload-test-"; + testIsolationPolicyUnloadsNSWithScope( + topicType, "policy-changed", nsPrefix, List.of("a1", "a2", "b1", "b2", "c1"), + all_matching, List.of(".*-unload-test-a.*"), List.of("b1", "b2", "c1"), + changed, List.of(".*-unload-test-a.*", ".*-unload-test-c.*"), List.of("a1", "a2", "b1", "b2"), + Collections.singletonList(".*") + ); + } + + @Test(dataProvider = "topicType") + public void testIsolationPolicyUnloadsNSWithNoneScope(final String topicType) throws Exception { + String nsPrefix = newUniqueName(defaultTenant + "/") + "-unload-test-"; + testIsolationPolicyUnloadsNSWithScope( + topicType, "policy-none", nsPrefix, List.of("a1", "a2", "b1", "b2", "c1"), + all_matching, List.of(".*-unload-test-a.*"), List.of("b1", "b2", "c1"), + none, List.of(".*-unload-test-a.*", ".*-unload-test-c.*"), List.of("a1", "a2", "b1", "b2", "c1"), + Collections.singletonList(".*") + ); + } + + @Test(dataProvider = "topicType") + public void testIsolationPolicyUnloadsNSWithPrimaryChanged(final String topicType) throws Exception { + String nsPrefix = newUniqueName(defaultTenant + "/") + "-unload-test-"; + // As per changed flag, only c1 should unload, but due to primary change, both a* and c* will. + testIsolationPolicyUnloadsNSWithScope( + topicType, "policy-primary-changed", nsPrefix, List.of("a1", "a2", "b1", "b2", "c1"), + all_matching, List.of(".*-unload-test-a.*"), List.of("b1", "b2", "c1"), + changed, List.of(".*-unload-test-a.*", ".*-unload-test-c.*"), List.of("b1", "b2"), + List.of(".*", "broker.*") + ); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationData.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationData.java index aa48e69c14571..4f367f72fda33 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationData.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationData.java @@ -31,6 +31,8 @@ public interface NamespaceIsolationData { AutoFailoverPolicyData getAutoFailoverPolicy(); + NamespaceIsolationPolicyUnloadScope getUnloadScope(); + void validate(); interface Builder { @@ -42,6 +44,8 @@ interface Builder { Builder autoFailoverPolicy(AutoFailoverPolicyData autoFailoverPolicyData); + Builder unloadScope(NamespaceIsolationPolicyUnloadScope unloadScope); + NamespaceIsolationData build(); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationPolicyUnloadScope.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationPolicyUnloadScope.java new file mode 100644 index 0000000000000..2edeac45630f5 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationPolicyUnloadScope.java @@ -0,0 +1,37 @@ +/* + * 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.policies.data; + +/** + * The type of unload to perform while setting the isolation policy. + */ +public enum NamespaceIsolationPolicyUnloadScope { + all_matching, // unloads all matching namespaces as per new regex + none, // unloads no namespaces + changed; // unloads only the namespaces which are newly added or removed from the regex list + + public static NamespaceIsolationPolicyUnloadScope fromString(String unloadScopeString) { + for (NamespaceIsolationPolicyUnloadScope unloadScope : NamespaceIsolationPolicyUnloadScope.values()) { + if (unloadScope.toString().equalsIgnoreCase(unloadScopeString)) { + return unloadScope; + } + } + return null; + } +} diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaceIsolationPolicy.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaceIsolationPolicy.java index e9896decd8c96..0f5f6b211a544 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaceIsolationPolicy.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaceIsolationPolicy.java @@ -32,6 +32,7 @@ import org.apache.pulsar.common.policies.data.BrokerNamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; +import org.apache.pulsar.common.policies.data.NamespaceIsolationPolicyUnloadScope; import picocli.CommandLine.Command; import picocli.CommandLine.Option; import picocli.CommandLine.Parameters; @@ -73,10 +74,19 @@ private class SetPolicy extends CliCommand { required = true, split = ",") private Map autoFailoverPolicyParams; + @Option(names = "--unload-scope", description = "configure the type of unload to do -" + + " ['all_matching', 'none', 'changed'] namespaces. By default, all namespaces matching the namespaces" + + " regex will be unloaded and placed again. You can choose to not unload any namespace while setting" + + " this new policy by choosing `none` or choose to unload only the namespaces whose placement will" + + " actually change. If you chose 'none', you will need to manually unload the namespaces for them to" + + " be placed correctly, or wait till some namespaces get load balanced automatically based on load" + + " shedding configurations.") + private NamespaceIsolationPolicyUnloadScope unloadScope; + void run() throws PulsarAdminException { // validate and create the POJO NamespaceIsolationData namespaceIsolationData = createNamespaceIsolationData(namespaces, primary, secondary, - autoFailoverPolicyTypeName, autoFailoverPolicyParams); + autoFailoverPolicyTypeName, autoFailoverPolicyParams, unloadScope); getAdmin().clusters().createNamespaceIsolationPolicy(clusterName, policyName, namespaceIsolationData); } @@ -167,7 +177,8 @@ private NamespaceIsolationData createNamespaceIsolationData(List namespa List primary, List secondary, String autoFailoverPolicyTypeName, - Map autoFailoverPolicyParams) { + Map autoFailoverPolicyParams, + NamespaceIsolationPolicyUnloadScope unload) { // validate namespaces = validateList(namespaces); @@ -234,6 +245,8 @@ private NamespaceIsolationData createNamespaceIsolationData(List namespa throw new ParameterException("Unknown auto failover policy type specified : " + autoFailoverPolicyTypeName); } + nsIsolationDataBuilder.unloadScope(unload); + return nsIsolationDataBuilder.build(); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/NamespaceIsolationPolicy.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/NamespaceIsolationPolicy.java index bd28d30d4cee9..52480d91eefa4 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/NamespaceIsolationPolicy.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/NamespaceIsolationPolicy.java @@ -23,6 +23,7 @@ import java.util.SortedSet; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.BrokerStatus; +import org.apache.pulsar.common.policies.data.NamespaceIsolationPolicyUnloadScope; /** * Namespace isolation policy. @@ -43,6 +44,11 @@ public interface NamespaceIsolationPolicy { */ List getSecondaryBrokers(); + /** + * Get the unload scope for the policy set call. + */ + NamespaceIsolationPolicyUnloadScope getUnloadScope(); + /** * Get the list of primary brokers for the namespace according to the policy. * diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java index bdb51f63f89ed..1e72f0e50ee05 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java @@ -75,6 +75,15 @@ public class NamespaceIsolationDataImpl implements NamespaceIsolationData { @JsonProperty("auto_failover_policy") private AutoFailoverPolicyData autoFailoverPolicy; + @ApiModelProperty( + name = "unload_scope", + value = "The type of unload to perform while applying the new isolation policy.", + example = "'all_matching' (default) for unloading all matching namespaces. 'none' for not unloading " + + "any namespace. 'changed' for unloading only the namespaces whose placement is actually changing" + ) + @JsonProperty("unload_scope") + private NamespaceIsolationPolicyUnloadScope unloadScope; + public static NamespaceIsolationDataImplBuilder builder() { return new NamespaceIsolationDataImplBuilder(); } @@ -106,6 +115,7 @@ public static class NamespaceIsolationDataImplBuilder implements NamespaceIsolat private List primary = new ArrayList<>(); private List secondary = new ArrayList<>(); private AutoFailoverPolicyData autoFailoverPolicy; + private NamespaceIsolationPolicyUnloadScope unloadScope; public NamespaceIsolationDataImplBuilder namespaces(List namespaces) { this.namespaces = namespaces; @@ -127,8 +137,13 @@ public NamespaceIsolationDataImplBuilder autoFailoverPolicy(AutoFailoverPolicyDa return this; } + public NamespaceIsolationDataImplBuilder unloadScope(NamespaceIsolationPolicyUnloadScope unloadScope) { + this.unloadScope = unloadScope; + return this; + } + public NamespaceIsolationDataImpl build() { - return new NamespaceIsolationDataImpl(namespaces, primary, secondary, autoFailoverPolicy); + return new NamespaceIsolationDataImpl(namespaces, primary, secondary, autoFailoverPolicy, unloadScope); } } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImpl.java index af3663869fa02..440282f29cb36 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/impl/NamespaceIsolationPolicyImpl.java @@ -29,6 +29,7 @@ import org.apache.pulsar.common.policies.NamespaceIsolationPolicy; import org.apache.pulsar.common.policies.data.BrokerStatus; import org.apache.pulsar.common.policies.data.NamespaceIsolationData; +import org.apache.pulsar.common.policies.data.NamespaceIsolationPolicyUnloadScope; /** * Implementation of the namespace isolation policy. @@ -39,6 +40,7 @@ public class NamespaceIsolationPolicyImpl implements NamespaceIsolationPolicy { private List primary; private List secondary; private AutoFailoverPolicy autoFailoverPolicy; + private NamespaceIsolationPolicyUnloadScope unloadScope; private boolean matchNamespaces(String fqnn) { for (String nsRegex : namespaces) { @@ -64,6 +66,7 @@ public NamespaceIsolationPolicyImpl(NamespaceIsolationData policyData) { this.primary = policyData.getPrimary(); this.secondary = policyData.getSecondary(); this.autoFailoverPolicy = AutoFailoverPolicyFactory.create(policyData.getAutoFailoverPolicy()); + this.unloadScope = policyData.getUnloadScope(); } @Override @@ -76,6 +79,11 @@ public List getSecondaryBrokers() { return this.secondary; } + @Override + public NamespaceIsolationPolicyUnloadScope getUnloadScope() { + return this.unloadScope; + } + @Override public List findPrimaryBrokers(List availableBrokers, NamespaceName namespace) { if (!this.matchNamespaces(namespace.toString())) { From 019ae9f0d0ec961389bd093e7350ca6e23c3f496 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Mon, 2 Sep 2024 21:31:07 +0800 Subject: [PATCH 480/580] [improve][admin] Improve BrokerStats.allocatorStats (#23242) --- .../apache/pulsar/broker/stats/AllocatorStatsGenerator.java | 2 ++ .../java/org/apache/pulsar/common/stats/AllocatorStats.java | 2 ++ .../java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java | 4 ++++ 3 files changed, 8 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/AllocatorStatsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/AllocatorStatsGenerator.java index 677b04d8a74ca..d20aef90adc33 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/AllocatorStatsGenerator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/AllocatorStatsGenerator.java @@ -54,6 +54,8 @@ public static AllocatorStats generate(String allocatorName) { stats.numDirectArenas = allocator.metric().numDirectArenas(); stats.numHeapArenas = allocator.metric().numHeapArenas(); stats.numThreadLocalCaches = allocator.metric().numThreadLocalCaches(); + stats.usedHeapMemory = allocator.metric().usedHeapMemory(); + stats.usedDirectMemory = allocator.metric().usedDirectMemory(); stats.normalCacheSize = allocator.metric().normalCacheSize(); stats.smallCacheSize = allocator.metric().smallCacheSize(); return stats; diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/stats/AllocatorStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/stats/AllocatorStats.java index 3dbe831053a4b..aa23e2f755379 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/stats/AllocatorStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/stats/AllocatorStats.java @@ -29,6 +29,8 @@ public class AllocatorStats { public int numThreadLocalCaches; public int normalCacheSize; public int smallCacheSize; + public long usedDirectMemory; + public long usedHeapMemory; public List directArenas; public List heapArenas; diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java index 6e9782a0c2b91..5f8c9f49d65d1 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java @@ -199,6 +199,10 @@ public void brokerStats() throws Exception { doReturn("null").when(mockBrokerStats).getMetrics(); brokerStats.run(split("monitoring-metrics")); verify(mockBrokerStats).getMetrics(); + + doReturn(null).when(mockBrokerStats).getAllocatorStats("default"); + brokerStats.run(split("allocator-stats default")); + verify(mockBrokerStats).getAllocatorStats("default"); } @Test From aee2ee5070d07c683c54877bc1457a58e273440b Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 2 Sep 2024 21:33:41 +0800 Subject: [PATCH 481/580] [fix][broker] Fix brokers still retry start replication after closed the topic (#23237) --- .../service/persistent/PersistentTopic.java | 25 +++++++++++ .../broker/service/OneWayReplicatorTest.java | 44 +++++++++++++++++++ 2 files changed, 69 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index d814e7ce11599..b8cde7619af93 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -1810,6 +1810,28 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { return closeFuture; } + private boolean isClosed() { + if (closeFutures == null) { + return false; + } + if (closeFutures.transferring != null + && closeFutures.transferring.isDone() + && !closeFutures.transferring.isCompletedExceptionally()) { + return true; + } + if (closeFutures.notWaitDisconnectClients != null + && closeFutures.notWaitDisconnectClients.isDone() + && !closeFutures.notWaitDisconnectClients.isCompletedExceptionally()) { + return true; + } + if (closeFutures.waitDisconnectClients != null + && closeFutures.waitDisconnectClients.isDone() + && !closeFutures.waitDisconnectClients.isCompletedExceptionally()) { + return true; + } + return false; + } + private void disposeTopic(CompletableFuture closeFuture) { brokerService.removeTopicFromCache(PersistentTopic.this) .thenRun(() -> { @@ -1832,6 +1854,9 @@ private void disposeTopic(CompletableFuture closeFuture) { @VisibleForTesting CompletableFuture checkReplicationAndRetryOnFailure() { + if (isClosed()) { + return CompletableFuture.completedFuture(null); + } CompletableFuture result = new CompletableFuture(); checkReplication().thenAccept(res -> { result.complete(null); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index 74604dd990c54..440e90da2b694 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -31,6 +31,7 @@ import static org.testng.Assert.fail; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.benmanes.caffeine.cache.AsyncLoadingCache; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import io.netty.util.concurrent.FastThreadLocalThread; @@ -46,6 +47,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -1281,4 +1283,46 @@ public void testReplicationCountMetrics() throws Exception { admin1.topics().delete(topicName, false); admin2.topics().delete(topicName, false); } + + /** + * This test used to confirm the "start replicator retry task" will be skipped after the topic is closed. + */ + @Test + public void testCloseTopicAfterStartReplicationFailed() throws Exception { + Field fieldTopicNameCache = TopicName.class.getDeclaredField("cache"); + fieldTopicNameCache.setAccessible(true); + ConcurrentHashMap topicNameCache = + (ConcurrentHashMap) fieldTopicNameCache.get(null); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); + // 1.Create topic, does not enable replication now. + admin1.topics().createNonPartitionedTopic(topicName); + Producer producer1 = client1.newProducer().topic(topicName).create(); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + + // We inject an error to make "start replicator" to fail. + AsyncLoadingCache existsCache = + WhiteboxImpl.getInternalState(pulsar1.getConfigurationMetadataStore(), "existsCache"); + String path = "/admin/partitioned-topics/" + TopicName.get(topicName).getPersistenceNamingEncoding(); + existsCache.put(path, CompletableFuture.completedFuture(true)); + + // 2.Enable replication and unload topic after failed to start replicator. + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); + Thread.sleep(3000); + producer1.close(); + existsCache.synchronous().invalidate(path); + admin1.topics().unload(topicName); + // Verify: the "start replicator retry task" will be skipped after the topic is closed. + // - Retry delay is "PersistentTopic.POLICY_UPDATE_FAILURE_RETRY_TIME_SECONDS": 60s, so wait for 70s. + // - Since the topic should not be touched anymore, we use "TopicName" to confirm whether it be used by + // Replication again. + Thread.sleep(10 * 1000); + topicNameCache.remove(topicName); + Thread.sleep(60 * 1000); + assertTrue(!topicNameCache.containsKey(topicName)); + + // cleanup. + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); + admin1.topics().delete(topicName, false); + } } From a678e974e0f0a615cb774fb2a89b9a45a6c2bc1e Mon Sep 17 00:00:00 2001 From: hanmz Date: Tue, 3 Sep 2024 01:12:14 +0800 Subject: [PATCH 482/580] [fix][test] Fix flaky UnloadSubscriptionTest.testMultiConsumer (#23243) --- .../org/apache/pulsar/client/api/UnloadSubscriptionTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/UnloadSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/UnloadSubscriptionTest.java index 93d5bf30ec6b1..22f7a5d6a43e4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/UnloadSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/UnloadSubscriptionTest.java @@ -60,6 +60,7 @@ protected void doInitConf() throws Exception { super.doInitConf(); conf.setSystemTopicEnabled(false); conf.setTransactionCoordinatorEnabled(false); + conf.setAcknowledgmentAtBatchIndexLevelEnabled(true); } @AfterClass(alwaysRun = true) @@ -242,6 +243,7 @@ private Consumer createConsumer(String topicName, String subName, Subscr .subscriptionName(subName) .subscriptionType(subType) .isAckReceiptEnabled(true) + .enableBatchIndexAcknowledgment(true) .subscribe(); return consumer; } From 8bb30a1106e8bbe5a76c14932a59805a278b9dd4 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Wed, 4 Sep 2024 19:53:54 +0800 Subject: [PATCH 483/580] [improve][broker] Add retry for start service unit state channel (ExtensibleLoadManagerImpl only) (#23230) --- .../extensions/ExtensibleLoadManagerImpl.java | 59 +++++++++++++++++-- 1 file changed, 53 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 95882cfb21b3c..40efa6390a78a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -99,7 +99,10 @@ import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.stats.Metrics; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.coordination.LeaderElectionState; import org.slf4j.Logger; @@ -122,6 +125,10 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS public static final long COMPACTION_THRESHOLD = 5 * 1024 * 1024; + public static final int STARTUP_TIMEOUT_SECONDS = 30; + + public static final int MAX_RETRY = 5; + private static final String ELECTION_ROOT = "/loadbalance/extension/leader"; public static final Set INTERNAL_TOPICS = @@ -401,10 +408,43 @@ public void start() throws PulsarServerException { this.serviceUnitStateChannel.listen(splitManager); this.leaderElectionService.start(); pulsar.runWhenReadyForIncomingRequests(() -> { - try { - this.serviceUnitStateChannel.start(); - } catch (Exception e) { - failStarting(e); + Backoff backoff = new BackoffBuilder() + .setInitialTime(100, TimeUnit.MILLISECONDS) + .setMax(STARTUP_TIMEOUT_SECONDS, TimeUnit.SECONDS) + .create(); + int retry = 0; + while (!Thread.currentThread().isInterrupted()) { + try { + brokerRegistry.register(); + this.serviceUnitStateChannel.start(); + break; + } catch (Exception e) { + log.warn("The broker:{} failed to start service unit state channel. Retrying {} th ...", + pulsar.getBrokerId(), ++retry, e); + try { + Thread.sleep(backoff.next()); + } catch (InterruptedException ex) { + log.warn("Interrupted while sleeping."); + // preserve thread's interrupt status + Thread.currentThread().interrupt(); + try { + pulsar.close(); + } catch (PulsarServerException exc) { + log.error("Failed to close pulsar service.", exc); + } + return; + } + failStarting(e); + if (retry >= MAX_RETRY) { + log.error("Failed to start the service unit state channel after retry {} th. " + + "Closing pulsar service.", retry, e); + try { + pulsar.close(); + } catch (PulsarServerException ex) { + log.error("Failed to close pulsar service.", ex); + } + } + } } }); this.antiAffinityGroupPolicyHelper = @@ -498,8 +538,15 @@ private void failStarting(Exception ex) { this.brokerRegistry, ex); if (this.brokerRegistry != null) { try { - brokerRegistry.close(); - } catch (PulsarServerException e) { + brokerRegistry.unregister(); + } catch (MetadataStoreException e) { + // ignore + } + } + if (this.serviceUnitStateChannel != null) { + try { + serviceUnitStateChannel.close(); + } catch (IOException e) { // ignore } } From de68e2511e016fa009852625ce21d1bffe47bf21 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Thu, 5 Sep 2024 11:07:40 +0800 Subject: [PATCH 484/580] [improve][broker] Optimize message payload traffic for ShadowReplicator (#23236) --- build/run_unit_group.sh | 2 +- .../org/apache/bookkeeper/mledger/impl/OpAddEntry.java | 3 ++- .../pulsar/broker/service/persistent/ShadowReplicator.java | 6 +++--- .../broker/service/persistent/ShadowReplicatorTest.java | 4 ++-- .../broker/service/persistent/ShadowTopicRealBkTest.java | 3 ++- .../java/org/apache/pulsar/client/impl/MessageImpl.java | 7 +++++++ 6 files changed, 17 insertions(+), 8 deletions(-) diff --git a/build/run_unit_group.sh b/build/run_unit_group.sh index 2694505e0e098..cdaf69e351b6d 100755 --- a/build/run_unit_group.sh +++ b/build/run_unit_group.sh @@ -80,7 +80,7 @@ function test_group_broker_group_1() { } function test_group_broker_group_2() { - mvn_test -pl pulsar-broker -Dgroups='schema,utils,functions-worker,broker-io,broker-discovery,broker-compaction,broker-naming,websocket,other' + mvn_test -pl pulsar-broker -Dgroups='schema,utils,functions-worker,broker-io,broker-discovery,broker-compaction,broker-naming,broker-replication,websocket,other' } function test_group_broker_group_3() { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java index 539b62fe7fe4b..3f0699657b5d4 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java @@ -239,7 +239,8 @@ public void run() { ManagedLedgerImpl.TOTAL_SIZE_UPDATER.addAndGet(ml, dataLength); long ledgerId = ledger != null ? ledger.getId() : ((Position) ctx).getLedgerId(); - if (ml.hasActiveCursors()) { + // Don't insert to the entry cache for the ShadowManagedLedger + if (!(ml instanceof ShadowManagedLedgerImpl) && ml.hasActiveCursors()) { // Avoid caching entries if no cursor has been created EntryImpl entry = EntryImpl.create(ledgerId, entryId, data); // EntryCache.insert: duplicates entry by allocating new entry and data. so, recycle entry after calling diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ShadowReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ShadowReplicator.java index 25591857aa1b5..65bcbfd131f12 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ShadowReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ShadowReplicator.java @@ -67,7 +67,7 @@ protected boolean replicateEntries(List entries) { ByteBuf headersAndPayload = entry.getDataBuffer(); MessageImpl msg; try { - msg = MessageImpl.deserializeSkipBrokerEntryMetaData(headersAndPayload); + msg = MessageImpl.deserializeMetadataWithEmptyPayload(headersAndPayload); } catch (Throwable t) { log.error("[{}] Failed to deserialize message at {} (buffer size: {}): {}", replicatorId, entry.getPosition(), length, t.getMessage(), t); @@ -91,9 +91,9 @@ protected boolean replicateEntries(List entries) { dispatchRateLimiter.ifPresent(rateLimiter -> rateLimiter.consumeDispatchQuota(1, entry.getLength())); - msgOut.recordEvent(headersAndPayload.readableBytes()); + msgOut.recordEvent(msg.getDataBuffer().readableBytes()); stats.incrementMsgOutCounter(); - stats.incrementBytesOutCounter(headersAndPayload.readableBytes()); + stats.incrementBytesOutCounter(msg.getDataBuffer().readableBytes()); msg.setReplicatedFrom(localCluster); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowReplicatorTest.java index 9f1885e034def..511cf87133a0d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowReplicatorTest.java @@ -142,8 +142,8 @@ public void testShadowReplication() throws Exception { Assert.assertEquals(shadowMessage.getBrokerPublishTime(), sourceMessage.getBrokerPublishTime()); Assert.assertEquals(shadowMessage.getIndex(), sourceMessage.getIndex()); - //`replicatedFrom` is set as localClusterName in shadow topic. - Assert.assertNotEquals(shadowMessage.getReplicatedFrom(), sourceMessage.getReplicatedFrom()); + Assert.assertEquals(replicator.stats.getBytesOutCount(), 0); + Assert.assertEquals(shadowMessage.getMessageId(), sourceMessage.getMessageId()); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicRealBkTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicRealBkTest.java index 9d810b06a7c7b..b0e572a826c47 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicRealBkTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicRealBkTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; import java.util.Optional; import java.util.Set; +import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.mledger.impl.ShadowManagedLedgerImpl; import org.apache.pulsar.broker.PulsarService; @@ -74,7 +75,7 @@ public void cleanup() throws Exception { @Test public void testReadFromStorage() throws Exception { - final var sourceTopic = TopicName.get("test-read-from-source").toString(); + final var sourceTopic = TopicName.get("test-read-from-source" + UUID.randomUUID()).toString(); final var shadowTopic = sourceTopic + "-shadow"; admin.topics().createNonPartitionedTopic(sourceTopic); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java index d369d639a73a0..72a5fd54e852b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java @@ -306,6 +306,13 @@ public static MessageImpl deserializeSkipBrokerEntryMetaData( return msg; } + public static MessageImpl deserializeMetadataWithEmptyPayload( + ByteBuf headersAndPayloadWithBrokerEntryMetadata) throws IOException { + MessageImpl msg = deserializeSkipBrokerEntryMetaData(headersAndPayloadWithBrokerEntryMetadata); + msg.payload = Unpooled.EMPTY_BUFFER; + return msg; + } + public void setReplicatedFrom(String cluster) { msgMetadata.setReplicatedFrom(cluster); } From a28c0df9ba3bb7aec8985ef9daa97c7ea38f8a39 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 5 Sep 2024 07:53:07 +0300 Subject: [PATCH 485/580] [improve][misc] Upgrade Netty to 4.1.113 and netty-tcnative to 2.0.66 (#23255) --- .../server/src/assemble/LICENSE.bin.txt | 54 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 52 +++++++++--------- pom.xml | 2 +- 3 files changed, 54 insertions(+), 54 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index d738b4a5027dc..4deba60e812e2 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -292,33 +292,33 @@ The Apache Software License, Version 2.0 - org.apache.commons-commons-lang3-3.11.jar - org.apache.commons-commons-text-1.10.0.jar * Netty - - io.netty-netty-buffer-4.1.111.Final.jar - - io.netty-netty-codec-4.1.111.Final.jar - - io.netty-netty-codec-dns-4.1.111.Final.jar - - io.netty-netty-codec-http-4.1.111.Final.jar - - io.netty-netty-codec-http2-4.1.111.Final.jar - - io.netty-netty-codec-socks-4.1.111.Final.jar - - io.netty-netty-codec-haproxy-4.1.111.Final.jar - - io.netty-netty-common-4.1.111.Final.jar - - io.netty-netty-handler-4.1.111.Final.jar - - io.netty-netty-handler-proxy-4.1.111.Final.jar - - io.netty-netty-resolver-4.1.111.Final.jar - - io.netty-netty-resolver-dns-4.1.111.Final.jar - - io.netty-netty-resolver-dns-classes-macos-4.1.111.Final.jar - - io.netty-netty-resolver-dns-native-macos-4.1.111.Final-osx-aarch_64.jar - - io.netty-netty-resolver-dns-native-macos-4.1.111.Final-osx-x86_64.jar - - io.netty-netty-transport-4.1.111.Final.jar - - io.netty-netty-transport-classes-epoll-4.1.111.Final.jar - - io.netty-netty-transport-native-epoll-4.1.111.Final-linux-aarch_64.jar - - io.netty-netty-transport-native-epoll-4.1.111.Final-linux-x86_64.jar - - io.netty-netty-transport-native-unix-common-4.1.111.Final.jar - - io.netty-netty-tcnative-boringssl-static-2.0.65.Final.jar - - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-osx-aarch_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-osx-x86_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-windows-x86_64.jar - - io.netty-netty-tcnative-classes-2.0.65.Final.jar + - io.netty-netty-buffer-4.1.113.Final.jar + - io.netty-netty-codec-4.1.113.Final.jar + - io.netty-netty-codec-dns-4.1.113.Final.jar + - io.netty-netty-codec-http-4.1.113.Final.jar + - io.netty-netty-codec-http2-4.1.113.Final.jar + - io.netty-netty-codec-socks-4.1.113.Final.jar + - io.netty-netty-codec-haproxy-4.1.113.Final.jar + - io.netty-netty-common-4.1.113.Final.jar + - io.netty-netty-handler-4.1.113.Final.jar + - io.netty-netty-handler-proxy-4.1.113.Final.jar + - io.netty-netty-resolver-4.1.113.Final.jar + - io.netty-netty-resolver-dns-4.1.113.Final.jar + - io.netty-netty-resolver-dns-classes-macos-4.1.113.Final.jar + - io.netty-netty-resolver-dns-native-macos-4.1.113.Final-osx-aarch_64.jar + - io.netty-netty-resolver-dns-native-macos-4.1.113.Final-osx-x86_64.jar + - io.netty-netty-transport-4.1.113.Final.jar + - io.netty-netty-transport-classes-epoll-4.1.113.Final.jar + - io.netty-netty-transport-native-epoll-4.1.113.Final-linux-aarch_64.jar + - io.netty-netty-transport-native-epoll-4.1.113.Final-linux-x86_64.jar + - io.netty-netty-transport-native-unix-common-4.1.113.Final.jar + - io.netty-netty-tcnative-boringssl-static-2.0.66.Final.jar + - io.netty-netty-tcnative-boringssl-static-2.0.66.Final-linux-aarch_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.66.Final-linux-x86_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.66.Final-osx-aarch_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.66.Final-osx-x86_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.66.Final-windows-x86_64.jar + - io.netty-netty-tcnative-classes-2.0.66.Final.jar - io.netty.incubator-netty-incubator-transport-classes-io_uring-0.0.24.Final.jar - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.24.Final-linux-x86_64.jar - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.24.Final-linux-aarch_64.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 944c4901cf1b9..9ab22ae83e42e 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -347,35 +347,35 @@ The Apache Software License, Version 2.0 - commons-text-1.10.0.jar - commons-compress-1.26.0.jar * Netty - - netty-buffer-4.1.111.Final.jar - - netty-codec-4.1.111.Final.jar - - netty-codec-dns-4.1.111.Final.jar - - netty-codec-http-4.1.111.Final.jar - - netty-codec-socks-4.1.111.Final.jar - - netty-codec-haproxy-4.1.111.Final.jar - - netty-common-4.1.111.Final.jar - - netty-handler-4.1.111.Final.jar - - netty-handler-proxy-4.1.111.Final.jar - - netty-resolver-4.1.111.Final.jar - - netty-resolver-dns-4.1.111.Final.jar - - netty-transport-4.1.111.Final.jar - - netty-transport-classes-epoll-4.1.111.Final.jar - - netty-transport-native-epoll-4.1.111.Final-linux-aarch_64.jar - - netty-transport-native-epoll-4.1.111.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.111.Final.jar - - netty-tcnative-boringssl-static-2.0.65.Final.jar - - netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar - - netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar - - netty-tcnative-boringssl-static-2.0.65.Final-osx-aarch_64.jar - - netty-tcnative-boringssl-static-2.0.65.Final-osx-x86_64.jar - - netty-tcnative-boringssl-static-2.0.65.Final-windows-x86_64.jar - - netty-tcnative-classes-2.0.65.Final.jar + - netty-buffer-4.1.113.Final.jar + - netty-codec-4.1.113.Final.jar + - netty-codec-dns-4.1.113.Final.jar + - netty-codec-http-4.1.113.Final.jar + - netty-codec-socks-4.1.113.Final.jar + - netty-codec-haproxy-4.1.113.Final.jar + - netty-common-4.1.113.Final.jar + - netty-handler-4.1.113.Final.jar + - netty-handler-proxy-4.1.113.Final.jar + - netty-resolver-4.1.113.Final.jar + - netty-resolver-dns-4.1.113.Final.jar + - netty-transport-4.1.113.Final.jar + - netty-transport-classes-epoll-4.1.113.Final.jar + - netty-transport-native-epoll-4.1.113.Final-linux-aarch_64.jar + - netty-transport-native-epoll-4.1.113.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.113.Final.jar + - netty-tcnative-boringssl-static-2.0.66.Final.jar + - netty-tcnative-boringssl-static-2.0.66.Final-linux-aarch_64.jar + - netty-tcnative-boringssl-static-2.0.66.Final-linux-x86_64.jar + - netty-tcnative-boringssl-static-2.0.66.Final-osx-aarch_64.jar + - netty-tcnative-boringssl-static-2.0.66.Final-osx-x86_64.jar + - netty-tcnative-boringssl-static-2.0.66.Final-windows-x86_64.jar + - netty-tcnative-classes-2.0.66.Final.jar - netty-incubator-transport-classes-io_uring-0.0.24.Final.jar - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-aarch_64.jar - netty-incubator-transport-native-io_uring-0.0.24.Final-linux-x86_64.jar - - netty-resolver-dns-classes-macos-4.1.111.Final.jar - - netty-resolver-dns-native-macos-4.1.111.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.111.Final-osx-x86_64.jar + - netty-resolver-dns-classes-macos-4.1.113.Final.jar + - netty-resolver-dns-native-macos-4.1.113.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.113.Final-osx-x86_64.jar * Prometheus client - simpleclient-0.16.0.jar - simpleclient_log4j2-0.16.0.jar diff --git a/pom.xml b/pom.xml index 4fa0df897689d..0504f3250c40b 100644 --- a/pom.xml +++ b/pom.xml @@ -148,7 +148,7 @@ flexible messaging model and an intuitive client API. 1.1.10.5 4.1.12.1 5.1.0 - 4.1.111.Final + 4.1.113.Final 0.0.24.Final 9.4.54.v20240208 2.5.2 From 6c300f515dbf245620b14a844d9ff6215426e992 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Wed, 4 Sep 2024 23:33:56 -0700 Subject: [PATCH 486/580] [fix][broker] Add principal name into info log to enhance analysis and troubleshooting (#23257) --- .../java/org/apache/pulsar/broker/service/ServerCnx.java | 6 +++--- 1 file changed, 3 insertions(+), 3 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 a5c09d2892342..0229b9c0f9788 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 @@ -1257,8 +1257,8 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { remoteAddress, getPrincipal()); } - log.info("[{}] Subscribing on topic {} / {}. consumerId: {}", this.toString(), - topicName, subscriptionName, consumerId); + log.info("[{}] Subscribing on topic {} / {}. consumerId: {}, role: {}", this.toString(), topicName, + subscriptionName, consumerId, getPrincipal()); try { Metadata.validateMetadata(metadata, service.getPulsar().getConfiguration().getMaxConsumerMetadataSize()); @@ -1748,7 +1748,7 @@ private void buildProducerAndAddTopic(Topic topic, long producerId, String produ topic.addProducer(producer, producerQueuedFuture).thenAccept(newTopicEpoch -> { if (isActive()) { if (producerFuture.complete(producer)) { - log.info("[{}] Created new producer: {}", remoteAddress, producer); + log.info("[{}] Created new producer: {}, role: {}", remoteAddress, producer, getPrincipal()); commandSender.sendProducerSuccessResponse(requestId, producerName, producer.getLastSequenceId(), producer.getSchemaVersion(), newTopicEpoch, true /* producer is ready now */); From 246647fff6972873c80afaabdb2352f1c8b685ac Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 5 Sep 2024 20:28:59 -0700 Subject: [PATCH 487/580] [fix][client] Fix client handle unknown exception during message-decryption and apply decryption action accordingly (#23256) --- .../apache/pulsar/client/impl/crypto/MessageCryptoBc.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/pulsar-client-messagecrypto-bc/src/main/java/org/apache/pulsar/client/impl/crypto/MessageCryptoBc.java b/pulsar-client-messagecrypto-bc/src/main/java/org/apache/pulsar/client/impl/crypto/MessageCryptoBc.java index aa97421a42fbb..e41ce633c8824 100644 --- a/pulsar-client-messagecrypto-bc/src/main/java/org/apache/pulsar/client/impl/crypto/MessageCryptoBc.java +++ b/pulsar-client-messagecrypto-bc/src/main/java/org/apache/pulsar/client/impl/crypto/MessageCryptoBc.java @@ -521,8 +521,7 @@ private boolean decryptDataKey(String keyName, byte[] encryptedDataKey, List Date: Fri, 6 Sep 2024 12:01:09 +0800 Subject: [PATCH 488/580] [fix][client] Fix concurrent lookup with properties might have different results (#23260) --- .../client/api/LookupPropertiesTest.java | 43 +++++++++++++++++++ .../client/impl/BinaryProtoLookupService.java | 20 +++++---- 2 files changed, 55 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/LookupPropertiesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/LookupPropertiesTest.java index cb8b2d1e526af..768dc29731f49 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/LookupPropertiesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/LookupPropertiesTest.java @@ -19,23 +19,30 @@ package org.apache.pulsar.client.api; import java.net.InetSocketAddress; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; +import java.util.stream.IntStream; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.MultiBrokerBaseTest; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.namespace.LookupOptions; +import org.apache.pulsar.client.impl.LookupTopicResult; import org.apache.pulsar.client.impl.PartitionedProducerImpl; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.ServiceUnitId; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.FutureUtil; import org.testng.Assert; import org.testng.annotations.Test; @@ -72,6 +79,7 @@ private static ServiceConfiguration addCustomConfigs(ServiceConfiguration config @Test public void testLookupProperty() throws Exception { + admin.namespaces().unload("public/default"); final var topic = "test-lookup-property"; admin.topics().createPartitionedTopic(topic, 16); @Cleanup final var client = (PulsarClientImpl) PulsarClient.builder() @@ -89,7 +97,35 @@ public void testLookupProperty() throws Exception { Assert.assertEquals(port, additionalBrokers.get(0).getBrokerListenPort().orElseThrow()); } + @Test + public void testConcurrentLookupProperties() throws Exception { + @Cleanup final var client = (PulsarClientImpl) PulsarClient.builder() + .serviceUrl(pulsar.getBrokerServiceUrl()) + .build(); + final var futures = new ArrayList>(); + BrokerIdAwareLoadManager.clientIdList.clear(); + + final var clientIdList = IntStream.range(0, 10).mapToObj(i -> "key-" + i).toList(); + for (var clientId : clientIdList) { + client.getConfiguration().setLookupProperties(Collections.singletonMap(CLIENT_KEY, clientId)); + futures.add(client.getLookup().getBroker(TopicName.get("test-concurrent-lookup-properties"))); + client.getConfiguration().setLookupProperties(Collections.emptyMap()); + } + FutureUtil.waitForAll(futures).get(); + Assert.assertEquals(clientIdList, BrokerIdAwareLoadManager.clientIdList); + } + public static class BrokerIdAwareLoadManager extends ExtensibleLoadManagerImpl { + + static final List clientIdList = Collections.synchronizedList(new ArrayList<>()); + + @Override + public CompletableFuture> assign(Optional topic, + ServiceUnitId serviceUnit, LookupOptions options) { + getClientId(options).ifPresent(clientIdList::add); + return super.assign(topic, serviceUnit, options); + } + @Override public CompletableFuture> selectAsync(ServiceUnitId bundle, Set excludeBrokerSet, LookupOptions options) { @@ -106,5 +142,12 @@ public CompletableFuture> selectAsync(ServiceUnitId bundle, Set .orElseGet(() -> super.selectAsync(bundle, excludeBrokerSet, options)); }); } + + private static Optional getClientId(LookupOptions options) { + if (options.getProperties() == null) { + return Optional.empty(); + } + return Optional.ofNullable(options.getProperties().get(CLIENT_KEY)); + } } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 9dd04acce7ee3..b45d6e9f6a80a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -24,6 +24,7 @@ import io.opentelemetry.api.common.Attributes; import java.net.InetSocketAddress; import java.net.URI; +import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -32,6 +33,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.lang3.mutable.MutableObject; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SchemaSerializationException; import org.apache.pulsar.client.impl.metrics.LatencyHistogram; @@ -60,7 +62,7 @@ public class BinaryProtoLookupService implements LookupService { private final String listenerName; private final int maxLookupRedirects; - private final ConcurrentHashMap> + private final ConcurrentHashMap>, CompletableFuture> lookupInProgress = new ConcurrentHashMap<>(); private final ConcurrentHashMap> @@ -118,10 +120,12 @@ public void updateServiceUrl(String serviceUrl) throws PulsarClientException { public CompletableFuture getBroker(TopicName topicName) { long startTime = System.nanoTime(); final MutableObject newFutureCreated = new MutableObject<>(); + final Pair> key = Pair.of(topicName, + client.getConfiguration().getLookupProperties()); try { - return lookupInProgress.computeIfAbsent(topicName, tpName -> { - CompletableFuture newFuture = - findBroker(serviceNameResolver.resolveHost(), false, topicName, 0); + return lookupInProgress.computeIfAbsent(key, tpName -> { + CompletableFuture newFuture = findBroker(serviceNameResolver.resolveHost(), false, + topicName, 0, key.getRight()); newFutureCreated.setValue(newFuture); newFuture.thenRun(() -> { @@ -135,7 +139,7 @@ public CompletableFuture getBroker(TopicName topicName) { } finally { if (newFutureCreated.getValue() != null) { newFutureCreated.getValue().whenComplete((v, ex) -> { - lookupInProgress.remove(topicName, newFutureCreated.getValue()); + lookupInProgress.remove(key, newFutureCreated.getValue()); }); } } @@ -167,7 +171,7 @@ public CompletableFuture getPartitionedTopicMetadata( } private CompletableFuture findBroker(InetSocketAddress socketAddress, - boolean authoritative, TopicName topicName, final int redirectCount) { + boolean authoritative, TopicName topicName, final int redirectCount, Map properties) { CompletableFuture addressFuture = new CompletableFuture<>(); if (maxLookupRedirects > 0 && redirectCount > maxLookupRedirects) { @@ -179,7 +183,7 @@ private CompletableFuture findBroker(InetSocketAddress socket client.getCnxPool().getConnection(socketAddress).thenAccept(clientCnx -> { long requestId = client.newRequestId(); ByteBuf request = Commands.newLookup(topicName.toString(), listenerName, authoritative, requestId, - client.getConfiguration().getLookupProperties()); + properties); clientCnx.newLookup(request, requestId).whenComplete((r, t) -> { if (t != null) { // lookup failed @@ -204,7 +208,7 @@ private CompletableFuture findBroker(InetSocketAddress socket // (2) redirect to given address if response is: redirect if (r.redirect) { - findBroker(responseBrokerAddress, r.authoritative, topicName, redirectCount + 1) + findBroker(responseBrokerAddress, r.authoritative, topicName, redirectCount + 1, properties) .thenAccept(addressFuture::complete) .exceptionally((lookupException) -> { Throwable cause = FutureUtil.unwrapCompletionException(lookupException); From ca0fb443ca940c2fa15ecaa9f908d81bed4ebbb1 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Sun, 8 Sep 2024 21:30:42 +0800 Subject: [PATCH 489/580] [fix][broker] Execute the pending callbacks in order before ready for incoming requests (#23266) --- .../apache/pulsar/broker/PulsarService.java | 25 +++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 0b994c640a9f5..b2e67bf4883dd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -298,6 +298,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private final DefaultMonotonicSnapshotClock monotonicSnapshotClock; private String brokerId; private final CompletableFuture readyForIncomingRequestsFuture = new CompletableFuture<>(); + private final List pendingTasksBeforeReadyForIncomingRequests = new ArrayList<>(); public enum State { Init, Started, Closing, Closed @@ -1023,7 +1024,13 @@ public void start() throws PulsarServerException { this.metricsGenerator = new MetricsGenerator(this); // the broker is ready to accept incoming requests by Pulsar binary protocol and http/https - readyForIncomingRequestsFuture.complete(null); + final List runnables; + synchronized (pendingTasksBeforeReadyForIncomingRequests) { + runnables = new ArrayList<>(pendingTasksBeforeReadyForIncomingRequests); + pendingTasksBeforeReadyForIncomingRequests.clear(); + readyForIncomingRequestsFuture.complete(null); + } + runnables.forEach(Runnable::run); // Initialize the message protocol handlers. // start the protocol handlers only after the broker is ready, @@ -1082,7 +1089,21 @@ public void start() throws PulsarServerException { } public void runWhenReadyForIncomingRequests(Runnable runnable) { - readyForIncomingRequestsFuture.thenRun(runnable); + // Here we don't call the thenRun() methods because CompletableFuture maintains a stack for pending callbacks, + // not a queue. Once the future is complete, the pending callbacks will be executed in reverse order of + // when they were added. + final boolean addedToPendingTasks; + synchronized (pendingTasksBeforeReadyForIncomingRequests) { + if (readyForIncomingRequestsFuture.isDone()) { + addedToPendingTasks = false; + } else { + pendingTasksBeforeReadyForIncomingRequests.add(runnable); + addedToPendingTasks = true; + } + } + if (!addedToPendingTasks) { + runnable.run(); + } } public void waitUntilReadyForIncomingRequests() throws ExecutionException, InterruptedException { From 46f99b91145ec8f71e38b8cb9671d39628ed53de Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Mon, 9 Sep 2024 17:22:08 +0800 Subject: [PATCH 490/580] [improve][broker] Make cluster metadata init command support metadata config path (#23269) --- .../pulsar/PulsarClusterMetadataSetup.java | 36 ++++++++++++++++--- .../pulsar/PulsarInitialNamespaceSetup.java | 11 ++++-- ...arTransactionCoordinatorMetadataSetup.java | 11 ++++-- .../zookeeper/ClusterMetadataSetupTest.java | 3 +- .../resources/conf/zk_client_enable_sasl.conf | 20 +++++++++++ 5 files changed, 72 insertions(+), 9 deletions(-) create mode 100644 pulsar-broker/src/test/resources/conf/zk_client_enable_sasl.conf diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java index c818dee124a88..96ea8877c5b61 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java @@ -140,6 +140,15 @@ private static class Arguments { hidden = false) private String configurationMetadataStore; + @Option(names = {"-mscp", + "--metadata-store-config-path"}, description = "Metadata Store config path", hidden = false) + private String metadataStoreConfigPath; + + @Option(names = {"-cmscp", + "--configuration-metadata-store-config-path"}, description = "Configuration Metadata Store config path", + hidden = false) + private String configurationStoreConfigPath; + @Option(names = { "--initial-num-stream-storage-containers" }, description = "Num storage containers of BookKeeper stream storage") @@ -283,9 +292,11 @@ private static void initializeCluster(Arguments arguments, int bundleNumberForDe log.info("Setting up cluster {} with metadata-store={} configuration-metadata-store={}", arguments.cluster, arguments.metadataStoreUrl, arguments.configurationMetadataStore); - MetadataStoreExtended localStore = - initLocalMetadataStore(arguments.metadataStoreUrl, arguments.zkSessionTimeoutMillis); + MetadataStoreExtended localStore = initLocalMetadataStore(arguments.metadataStoreUrl, + arguments.metadataStoreConfigPath, + arguments.zkSessionTimeoutMillis); MetadataStoreExtended configStore = initConfigMetadataStore(arguments.configurationMetadataStore, + arguments.configurationStoreConfigPath, arguments.zkSessionTimeoutMillis); final String metadataStoreUrlNoIdentifer = MetadataStoreFactoryImpl @@ -464,9 +475,17 @@ static void createPartitionedTopic(MetadataStore configStore, TopicName topicNam } } - public static MetadataStoreExtended initLocalMetadataStore(String connection, int sessionTimeout) throws Exception { + public static MetadataStoreExtended initLocalMetadataStore(String connection, + int sessionTimeout) throws Exception { + return initLocalMetadataStore(connection, null, sessionTimeout); + } + + public static MetadataStoreExtended initLocalMetadataStore(String connection, + String configPath, + int sessionTimeout) throws Exception { MetadataStoreExtended store = MetadataStoreExtended.create(connection, MetadataStoreConfig.builder() .sessionTimeoutMillis(sessionTimeout) + .configFilePath(configPath) .metadataStoreName(MetadataStoreConfig.METADATA_STORE) .build()); if (store instanceof MetadataStoreLifecycle) { @@ -475,10 +494,19 @@ public static MetadataStoreExtended initLocalMetadataStore(String connection, in return store; } - public static MetadataStoreExtended initConfigMetadataStore(String connection, int sessionTimeout) + public static MetadataStoreExtended initConfigMetadataStore(String connection, + int sessionTimeout) + throws Exception { + return initConfigMetadataStore(connection, null, sessionTimeout); + } + + public static MetadataStoreExtended initConfigMetadataStore(String connection, + String configPath, + int sessionTimeout) throws Exception { MetadataStoreExtended store = MetadataStoreExtended.create(connection, MetadataStoreConfig.builder() .sessionTimeoutMillis(sessionTimeout) + .configFilePath(configPath) .metadataStoreName(MetadataStoreConfig.CONFIGURATION_METADATA_STORE) .build()); if (store instanceof MetadataStoreLifecycle) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarInitialNamespaceSetup.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarInitialNamespaceSetup.java index 891aa1aa42120..912f43958f469 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarInitialNamespaceSetup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarInitialNamespaceSetup.java @@ -44,6 +44,11 @@ private static class Arguments { "--configuration-store" }, description = "Configuration Store connection string", required = true) private String configurationStore; + @Option(names = {"-cmscp", + "--configuration-metadata-store-config-path"}, description = "Configuration Metadata Store config path", + hidden = false) + private String configurationStoreConfigPath; + @Option(names = { "--zookeeper-session-timeout-ms" }, description = "Local zookeeper session timeout ms") @@ -85,8 +90,10 @@ public static int doMain(String[] args) throws Exception { return 1; } - try (MetadataStore configStore = PulsarClusterMetadataSetup - .initConfigMetadataStore(arguments.configurationStore, arguments.zkSessionTimeoutMillis)) { + try (MetadataStore configStore = PulsarClusterMetadataSetup.initConfigMetadataStore( + arguments.configurationStore, + arguments.configurationStoreConfigPath, + arguments.zkSessionTimeoutMillis)) { PulsarResources pulsarResources = new PulsarResources(null, configStore); for (String namespace : arguments.namespaces) { NamespaceName namespaceName = null; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarTransactionCoordinatorMetadataSetup.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarTransactionCoordinatorMetadataSetup.java index 57b67b011913f..06b68decf36f0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarTransactionCoordinatorMetadataSetup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarTransactionCoordinatorMetadataSetup.java @@ -44,6 +44,11 @@ private static class Arguments { "--configuration-store" }, description = "Configuration Store connection string", required = true) private String configurationStore; + @Option(names = {"-cmscp", + "--configuration-metadata-store-config-path"}, description = "Configuration Metadata Store config path", + hidden = false) + private String configurationStoreConfigPath; + @Option(names = { "--zookeeper-session-timeout-ms" }, description = "Local zookeeper session timeout ms") @@ -92,8 +97,10 @@ public static void main(String[] args) throws Exception { System.exit(1); } - try (MetadataStoreExtended configStore = PulsarClusterMetadataSetup - .initConfigMetadataStore(arguments.configurationStore, arguments.zkSessionTimeoutMillis)) { + try (MetadataStoreExtended configStore = PulsarClusterMetadataSetup.initConfigMetadataStore( + arguments.configurationStore, + arguments.configurationStoreConfigPath, + arguments.zkSessionTimeoutMillis)) { PulsarResources pulsarResources = new PulsarResources(null, configStore); // Create system tenant PulsarClusterMetadataSetup diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataSetupTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataSetupTest.java index 4267c7564fa6f..0c402a83e4227 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataSetupTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ClusterMetadataSetupTest.java @@ -74,10 +74,11 @@ public void testReSetupClusterMetadata() throws Exception { "--cluster", "testReSetupClusterMetadata-cluster", "--zookeeper", "127.0.0.1:" + localZkS.getZookeeperPort(), "--configuration-store", "127.0.0.1:" + localZkS.getZookeeperPort(), + "--configuration-metadata-store-config-path", "src/test/resources/conf/zk_client_enable_sasl.conf", "--web-service-url", "http://127.0.0.1:8080", "--web-service-url-tls", "https://127.0.0.1:8443", "--broker-service-url", "pulsar://127.0.0.1:6650", - "--broker-service-url-tls","pulsar+ssl://127.0.0.1:6651" + "--broker-service-url-tls", "pulsar+ssl://127.0.0.1:6651" }; PulsarClusterMetadataSetup.main(args); SortedMap data1 = localZkS.dumpData(); diff --git a/pulsar-broker/src/test/resources/conf/zk_client_enable_sasl.conf b/pulsar-broker/src/test/resources/conf/zk_client_enable_sasl.conf new file mode 100644 index 0000000000000..c59e093450d39 --- /dev/null +++ b/pulsar-broker/src/test/resources/conf/zk_client_enable_sasl.conf @@ -0,0 +1,20 @@ +# +# 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. +# + +zookeeper.sasl.client=true From 0aaa906cd8c68a212992166221123fd83172ce31 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 9 Sep 2024 23:32:30 +0800 Subject: [PATCH 491/580] [fix][broker] fix pulsar-admin topics stats-internal caused a BK client thread a deadlock (#23258) --- .../service/persistent/PersistentTopic.java | 29 ++++++++++++++----- .../pulsar/compaction/CompactedTopicImpl.java | 8 +++-- 2 files changed, 27 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index b8cde7619af93..f90e10ee64e65 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -48,6 +48,7 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -2819,13 +2820,13 @@ public CompletableFuture getInternalStats(boolean info.entries = -1; info.size = -1; - Optional compactedTopicContext = getCompactedTopicContext(); - if (compactedTopicContext.isPresent()) { - CompactedTopicContext ledgerContext = compactedTopicContext.get(); - info.ledgerId = ledgerContext.getLedger().getId(); - info.entries = ledgerContext.getLedger().getLastAddConfirmed() + 1; - info.size = ledgerContext.getLedger().getLength(); - } + futures.add(getCompactedTopicContextAsync().thenAccept(v -> { + if (v != null) { + info.ledgerId = v.getLedger().getId(); + info.entries = v.getLedger().getLastAddConfirmed() + 1; + info.size = v.getLedger().getLength(); + } + })); stats.compactedLedger = info; @@ -2951,12 +2952,24 @@ public Optional getCompactedTopicContext() { if (topicCompactionService instanceof PulsarTopicCompactionService pulsarCompactedService) { return pulsarCompactedService.getCompactedTopic().getCompactedTopicContext(); } - } catch (ExecutionException | InterruptedException e) { + } catch (ExecutionException | InterruptedException | TimeoutException e) { log.warn("[{}]Fail to get ledger information for compacted topic.", topic); } return Optional.empty(); } + public CompletableFuture getCompactedTopicContextAsync() { + if (topicCompactionService instanceof PulsarTopicCompactionService pulsarCompactedService) { + CompletableFuture res = + pulsarCompactedService.getCompactedTopic().getCompactedTopicContextFuture(); + if (res == null) { + return CompletableFuture.completedFuture(null); + } + return res; + } + return CompletableFuture.completedFuture(null); + } + public long getBacklogSize() { return ledger.getEstimatedBacklogSize(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index baa71ffc645d6..a650189500777 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -32,6 +32,8 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.function.Predicate; import javax.annotation.Nullable; import org.apache.bookkeeper.client.BKException; @@ -304,8 +306,10 @@ static CompletableFuture> readEntries(LedgerHandle lh, long from, lo * Getter for CompactedTopicContext. * @return CompactedTopicContext */ - public Optional getCompactedTopicContext() throws ExecutionException, InterruptedException { - return compactedTopicContext == null ? Optional.empty() : Optional.of(compactedTopicContext.get()); + public Optional getCompactedTopicContext() throws ExecutionException, InterruptedException, + TimeoutException { + return compactedTopicContext == null ? Optional.empty() : + Optional.of(compactedTopicContext.get(30, TimeUnit.SECONDS)); } @Override From 21e256f754b5bb2d7ee8b31a847fd8bf215d6833 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Mon, 9 Sep 2024 10:14:02 -0700 Subject: [PATCH 492/580] [fix][client] Fix broker/Client CPU reaching 100% during retriable connection failure (#23251) --- .../client/api/PulsarClientException.java | 81 ++++--------------- .../pulsar/client/impl/ConsumerImpl.java | 9 +-- .../pulsar/client/impl/ProducerImpl.java | 9 +-- .../pulsar/client/impl/PulsarClientImpl.java | 12 +-- .../pulsar/client/impl/TopicListWatcher.java | 11 ++- .../impl/TransactionMetaStoreHandler.java | 8 +- 6 files changed, 38 insertions(+), 92 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java index 513ee4d7e4edf..9eb6c612a52a2 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClientException.java @@ -19,9 +19,9 @@ package org.apache.pulsar.client.api; import java.io.IOException; -import java.util.Collection; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; import lombok.Getter; import org.apache.pulsar.common.classification.InterfaceAudience; import org.apache.pulsar.common.classification.InterfaceStability; @@ -34,7 +34,7 @@ @SuppressWarnings("serial") public class PulsarClientException extends IOException { private long sequenceId = -1; - private Collection previous; + private AtomicInteger previousExceptionAttempt; /** * Constructs an {@code PulsarClientException} with the specified detail message. @@ -87,47 +87,16 @@ public PulsarClientException(String msg, Throwable t) { super(msg, t); } - /** - * Add a list of previous exception which occurred for the same operation - * and have been retried. - * - * @param previous A collection of throwables that triggered retries - */ - public void setPreviousExceptions(Collection previous) { - this.previous = previous; - } - - /** - * Get the collection of previous exceptions which have caused retries - * for this operation. - * - * @return a collection of exception, ordered as they occurred - */ - public Collection getPreviousExceptions() { - return this.previous; + public void setPreviousExceptionCount(AtomicInteger previousExceptionCount) { + this.previousExceptionAttempt = previousExceptionCount; } @Override public String toString() { - if (previous == null || previous.isEmpty()) { + if (previousExceptionAttempt == null || previousExceptionAttempt.get() == 0) { return super.toString(); } else { - StringBuilder sb = new StringBuilder(super.toString()); - int i = 0; - boolean first = true; - sb.append("{\"previous\":["); - for (Throwable t : previous) { - if (first) { - first = false; - } else { - sb.append(','); - } - sb.append("{\"attempt\":").append(i++) - .append(",\"error\":\"").append(t.toString().replace("\"", "\\\"")) - .append("\"}"); - } - sb.append("]}"); - return sb.toString(); + return super.toString() + ", previous-attempt: " + previousExceptionAttempt; } } /** @@ -1156,39 +1125,9 @@ public static PulsarClientException unwrap(Throwable t) { newException = new PulsarClientException(t); } - Collection previousExceptions = getPreviousExceptions(t); - if (previousExceptions != null) { - newException.setPreviousExceptions(previousExceptions); - } return newException; } - public static Collection getPreviousExceptions(Throwable t) { - Throwable e = t; - for (int maxDepth = 20; maxDepth > 0 && e != null; maxDepth--) { - if (e instanceof PulsarClientException) { - Collection previous = ((PulsarClientException) e).getPreviousExceptions(); - if (previous != null) { - return previous; - } - } - e = t.getCause(); - } - return null; - } - - public static void setPreviousExceptions(Throwable t, Collection previous) { - Throwable e = t; - for (int maxDepth = 20; maxDepth > 0 && e != null; maxDepth--) { - if (e instanceof PulsarClientException) { - ((PulsarClientException) e).setPreviousExceptions(previous); - return; - } - e = t.getCause(); - } - } - - public long getSequenceId() { return sequenceId; } @@ -1222,4 +1161,12 @@ public static boolean isRetriableError(Throwable t) { } return true; } + + public static void setPreviousExceptionCount(Throwable e, AtomicInteger previousExceptionCount) { + if (e instanceof PulsarClientException) { + ((PulsarClientException) e).setPreviousExceptionCount(previousExceptionCount); + return; + } + } + } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 4f041772af3fb..996569704d712 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -54,7 +54,6 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -235,7 +234,7 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private final Counter consumerDlqMessagesCounter; private final AtomicReference clientCnxUsedForConsumerRegistration = new AtomicReference<>(); - private final List previousExceptions = new CopyOnWriteArrayList(); + private final AtomicInteger previousExceptionCount = new AtomicInteger(); private volatile boolean hasSoughtByTimestamp = false; static ConsumerImpl newConsumerImpl(PulsarClientImpl client, @@ -825,7 +824,7 @@ public void negativeAcknowledge(Message message) { @Override public CompletableFuture connectionOpened(final ClientCnx cnx) { - previousExceptions.clear(); + previousExceptionCount.set(0); getConnectionHandler().setMaxMessageSize(cnx.getMaxMessageSize()); final State state = getState(); @@ -1069,7 +1068,7 @@ public void connectionFailed(PulsarClientException exception) { boolean nonRetriableError = !PulsarClientException.isRetriableError(exception); boolean timeout = System.currentTimeMillis() > lookupDeadline; if (nonRetriableError || timeout) { - exception.setPreviousExceptions(previousExceptions); + exception.setPreviousExceptionCount(previousExceptionCount); if (subscribeFuture.completeExceptionally(exception)) { setState(State.Failed); if (nonRetriableError) { @@ -1083,7 +1082,7 @@ public void connectionFailed(PulsarClientException exception) { client.cleanupConsumer(this); } } else { - previousExceptions.add(exception); + previousExceptionCount.incrementAndGet(); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 5c46057ae308d..b686252b58ade 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -54,7 +54,6 @@ import java.util.Optional; import java.util.Queue; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -174,7 +173,7 @@ public class ProducerImpl extends ProducerBase implements TimerTask, Conne private long lastBatchSendNanoTime; private Optional topicEpoch = Optional.empty(); - private final List previousExceptions = new CopyOnWriteArrayList(); + private final AtomicInteger previousExceptionCount = new AtomicInteger(); private boolean errorState; @@ -1749,7 +1748,7 @@ public Iterator iterator() { @Override public CompletableFuture connectionOpened(final ClientCnx cnx) { - previousExceptions.clear(); + previousExceptionCount.set(0); getConnectionHandler().setMaxMessageSize(cnx.getMaxMessageSize()); setChunkMaxMessageSize(); @@ -1955,7 +1954,7 @@ public void connectionFailed(PulsarClientException exception) { boolean nonRetriableError = !PulsarClientException.isRetriableError(exception); boolean timeout = System.currentTimeMillis() > lookupDeadline; if (nonRetriableError || timeout) { - exception.setPreviousExceptions(previousExceptions); + exception.setPreviousExceptionCount(previousExceptionCount); if (producerCreatedFuture.completeExceptionally(exception)) { if (nonRetriableError) { log.info("[{}] Producer creation failed for producer {} with unretriableError = {}", @@ -1968,7 +1967,7 @@ public void connectionFailed(PulsarClientException exception) { client.cleanupProducer(this); } } else { - previousExceptions.add(exception); + previousExceptionCount.incrementAndGet(); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index ae28d835fd22f..e0d4bf35f8a22 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -47,6 +47,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; @@ -1152,7 +1153,8 @@ public CompletableFuture getPartitionedTopicMetadata( .setMandatoryStop(opTimeoutMs.get() * 2, TimeUnit.MILLISECONDS) .setMax(conf.getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) .create(); - getPartitionedTopicMetadata(topicName, backoff, opTimeoutMs, metadataFuture, new ArrayList<>(), + getPartitionedTopicMetadata(topicName, backoff, opTimeoutMs, metadataFuture, + new AtomicInteger(0), metadataAutoCreationEnabled, useFallbackForNonPIP344Brokers); } catch (IllegalArgumentException e) { return FutureUtil.failedFuture(new PulsarClientException.InvalidConfigurationException(e.getMessage())); @@ -1164,7 +1166,7 @@ private void getPartitionedTopicMetadata(TopicName topicName, Backoff backoff, AtomicLong remainingTime, CompletableFuture future, - List previousExceptions, + AtomicInteger previousExceptionCount, boolean metadataAutoCreationEnabled, boolean useFallbackForNonPIP344Brokers) { long startTime = System.nanoTime(); @@ -1179,17 +1181,17 @@ private void getPartitionedTopicMetadata(TopicName topicName, || e.getCause() instanceof PulsarClientException.AuthenticationException || e.getCause() instanceof PulsarClientException.NotFoundException; if (nextDelay <= 0 || isLookupThrottling) { - PulsarClientException.setPreviousExceptions(e, previousExceptions); + PulsarClientException.setPreviousExceptionCount(e, previousExceptionCount); future.completeExceptionally(e); return null; } - previousExceptions.add(e); + previousExceptionCount.getAndIncrement(); ((ScheduledExecutorService) scheduledExecutorProvider.getExecutor()).schedule(() -> { log.warn("[topic: {}] Could not get connection while getPartitionedTopicMetadata -- " + "Will try again in {} ms", topicName, nextDelay); remainingTime.addAndGet(-nextDelay); - getPartitionedTopicMetadata(topicName, backoff, remainingTime, future, previousExceptions, + getPartitionedTopicMetadata(topicName, backoff, remainingTime, future, previousExceptionCount, metadataAutoCreationEnabled, useFallbackForNonPIP344Brokers); }, nextDelay, TimeUnit.MILLISECONDS); return null; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java index 0007f98b253a0..93fa7082f33c6 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java @@ -20,10 +20,9 @@ import com.google.re2j.Pattern; import io.netty.channel.ChannelHandlerContext; -import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.AtomicReference; import org.apache.pulsar.client.api.PulsarClientException; @@ -54,7 +53,7 @@ public class TopicListWatcher extends HandlerState implements ConnectionHandler. private String topicsHash; private final CompletableFuture watcherFuture; - private final List previousExceptions = new CopyOnWriteArrayList<>(); + private final AtomicInteger previousExceptionCount = new AtomicInteger(); private final AtomicReference clientCnxUsedForWatcherRegistration = new AtomicReference<>(); private final Runnable recheckTopicsChangeAfterReconnect; @@ -93,7 +92,7 @@ public TopicListWatcher(PatternConsumerUpdateQueue patternConsumerUpdateQueue, public void connectionFailed(PulsarClientException exception) { boolean nonRetriableError = !PulsarClientException.isRetriableError(exception); if (nonRetriableError) { - exception.setPreviousExceptions(previousExceptions); + exception.setPreviousExceptionCount(previousExceptionCount); if (watcherFuture.completeExceptionally(exception)) { setState(State.Failed); log.info("[{}] Watcher creation failed for {} with non-retriable error {}", @@ -101,13 +100,13 @@ public void connectionFailed(PulsarClientException exception) { deregisterFromClientCnx(); } } else { - previousExceptions.add(exception); + previousExceptionCount.incrementAndGet(); } } @Override public CompletableFuture connectionOpened(ClientCnx cnx) { - previousExceptions.clear(); + previousExceptionCount.set(0); State state = getState(); if (state == State.Closing || state == State.Closed) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java index e45d53971159e..c8c2fa83f94ae 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java @@ -30,10 +30,10 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException; import org.apache.pulsar.client.api.transaction.TxnID; @@ -90,7 +90,7 @@ public RequestTime(long creationTime, long requestId) { private final CompletableFuture connectFuture; private final long lookupDeadline; - private final List previousExceptions = new CopyOnWriteArrayList<>(); + private final AtomicInteger previousExceptionCount = new AtomicInteger(); @@ -126,7 +126,7 @@ public void connectionFailed(PulsarClientException exception) { boolean nonRetriableError = !PulsarClientException.isRetriableError(exception); boolean timeout = System.currentTimeMillis() > lookupDeadline; if (nonRetriableError || timeout) { - exception.setPreviousExceptions(previousExceptions); + exception.setPreviousExceptionCount(previousExceptionCount); if (connectFuture.completeExceptionally(exception)) { if (nonRetriableError) { LOG.error("Transaction meta handler with transaction coordinator id {} connection failed.", @@ -138,7 +138,7 @@ public void connectionFailed(PulsarClientException exception) { setState(State.Failed); } } else { - previousExceptions.add(exception); + previousExceptionCount.getAndIncrement(); } } From 96fd04b22f81abdeafa3f146c871119df7a64e36 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Tue, 10 Sep 2024 13:57:33 +0800 Subject: [PATCH 493/580] [fix] Bump io.grpc from 1.56.0 to 1.56.1 (#23276) --- .../server/src/assemble/LICENSE.bin.txt | 32 +++++++++---------- pom.xml | 2 +- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 4deba60e812e2..d4fdd080dddf9 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -429,22 +429,22 @@ The Apache Software License, Version 2.0 - org.jetbrains.kotlin-kotlin-stdlib-jdk8-1.8.20.jar - org.jetbrains-annotations-13.0.jar * gRPC - - io.grpc-grpc-all-1.56.0.jar - - io.grpc-grpc-auth-1.56.0.jar - - io.grpc-grpc-context-1.56.0.jar - - io.grpc-grpc-core-1.56.0.jar - - io.grpc-grpc-protobuf-1.56.0.jar - - io.grpc-grpc-protobuf-lite-1.56.0.jar - - io.grpc-grpc-stub-1.56.0.jar - - io.grpc-grpc-alts-1.56.0.jar - - io.grpc-grpc-api-1.56.0.jar - - io.grpc-grpc-grpclb-1.56.0.jar - - io.grpc-grpc-netty-shaded-1.56.0.jar - - io.grpc-grpc-services-1.56.0.jar - - io.grpc-grpc-xds-1.56.0.jar - - io.grpc-grpc-rls-1.56.0.jar - - io.grpc-grpc-servlet-1.56.0.jar - - io.grpc-grpc-servlet-jakarta-1.56.0.jar + - io.grpc-grpc-all-1.56.1.jar + - io.grpc-grpc-auth-1.56.1.jar + - io.grpc-grpc-context-1.56.1.jar + - io.grpc-grpc-core-1.56.1.jar + - io.grpc-grpc-protobuf-1.56.1.jar + - io.grpc-grpc-protobuf-lite-1.56.1.jar + - io.grpc-grpc-stub-1.56.1.jar + - io.grpc-grpc-alts-1.56.1.jar + - io.grpc-grpc-api-1.56.1.jar + - io.grpc-grpc-grpclb-1.56.1.jar + - io.grpc-grpc-netty-shaded-1.56.1.jar + - io.grpc-grpc-services-1.56.1.jar + - io.grpc-grpc-xds-1.56.1.jar + - io.grpc-grpc-rls-1.56.1.jar + - io.grpc-grpc-servlet-1.56.1.jar + - io.grpc-grpc-servlet-jakarta-1.56.1.jar - io.grpc-grpc-util-1.60.0.jar * Perfmark - io.perfmark-perfmark-api-0.26.0.jar diff --git a/pom.xml b/pom.xml index 0504f3250c40b..2c6b34d389bce 100644 --- a/pom.xml +++ b/pom.xml @@ -174,7 +174,7 @@ flexible messaging model and an intuitive client API. 1.17 3.22.3 ${protobuf3.version} - 1.56.0 + 1.56.1 1.41.0 0.26.0 ${grpc.version} From a96127a3e0f294730f610b07fc242c2976e286f4 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Tue, 10 Sep 2024 16:37:01 +0800 Subject: [PATCH 494/580] [feat][meta] Bump oxia java version from 0.3.2 to 0.4.5 (#23277) --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index d4fdd080dddf9..579613b0d8f2f 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -480,8 +480,8 @@ The Apache Software License, Version 2.0 * Prometheus - io.prometheus-simpleclient_httpserver-0.16.0.jar * Oxia - - io.streamnative.oxia-oxia-client-api-0.3.2.jar - - io.streamnative.oxia-oxia-client-0.3.2.jar + - io.streamnative.oxia-oxia-client-api-0.4.5.jar + - io.streamnative.oxia-oxia-client-0.4.5.jar * OpenHFT - net.openhft-zero-allocation-hashing-0.16.jar * Java JSON WebTokens diff --git a/pom.xml b/pom.xml index 2c6b34d389bce..4cc982e7bdd64 100644 --- a/pom.xml +++ b/pom.xml @@ -252,7 +252,7 @@ flexible messaging model and an intuitive client API. 4.5.13 4.4.15 0.7.7 - 0.3.2 + 0.4.5 2.0 1.10.12 5.5.0 From d4839fb17499dd39b2601a2d6919586205bb82c8 Mon Sep 17 00:00:00 2001 From: Zike Yang Date: Wed, 11 Sep 2024 10:06:56 +0800 Subject: [PATCH 495/580] [fix][broker] fix the log format error in the AdminResource (#23278) --- .../main/java/org/apache/pulsar/broker/admin/AdminResource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index 497af71955158..d42dff39a8a0d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -613,7 +613,7 @@ protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int n && pulsar().getConfig().isCreateTopicToRemoteClusterForReplication()) { internalCreatePartitionedTopicToReplicatedClustersInBackground(numPartitions); log.info("[{}] Successfully created partitioned for topic {} for the remote clusters", - clientAppId()); + clientAppId(), topicName); } else { log.info("[{}] Skip creating partitioned for topic {} for the remote clusters", clientAppId(), topicName); From a8ae3e4d191c75f291ccb29577c181926a5f4e5d Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 11 Sep 2024 11:20:22 +0800 Subject: [PATCH 496/580] [improve][pip] PIP-376: Make topic policies service pluggable (#23248) --- ...ip-376-Topic-Policies-Service-Pluggable.md | 222 ++++++++++++++++++ 1 file changed, 222 insertions(+) create mode 100644 pip/pip-376-Topic-Policies-Service-Pluggable.md diff --git a/pip/pip-376-Topic-Policies-Service-Pluggable.md b/pip/pip-376-Topic-Policies-Service-Pluggable.md new file mode 100644 index 0000000000000..0659de812af3d --- /dev/null +++ b/pip/pip-376-Topic-Policies-Service-Pluggable.md @@ -0,0 +1,222 @@ +# PIP-376: Make Topic Policies Service Pluggable + +## Background + +### Topic Policies Service and System Topics + +[PIP-39](https://github.com/apache/pulsar/wiki/PIP-39%3A-Namespace-Change-Events) introduces system topics and topic-level policies. Currently, the topic policies service (`TopicPoliciesService`) has only one implementation (`SystemTopicBasedTopicPoliciesService`) that depends on system topics. Therefore, the following configurations are required (though they are enabled by default): + +```properties +systemTopicEnabled=true +topicLevelPoliciesEnabled=true +``` + +However, using system topics to manage topic policies may not always be the best choice. Users might need an alternative approach to manage topic policies. + +### Issues with the Current `TopicPoliciesService` Interface + +The `TopicPoliciesService` interface is poorly designed for third-party implementations due to the following reasons: + +1. **Methods that Should Not Be Exposed**: + - `addOwnedNamespaceBundleAsync` and `removeOwnedNamespaceBundleAsync` are used internally in `SystemTopicBasedTopicPoliciesService`. + - `getTopicPoliciesBypassCacheAsync` is used only in tests to replay the `__change_events` topic and construct the topic policies map. + +2. **Confusing and Inconsistent `getTopicPolicies` Methods**: + - There are two overrides of `getTopicPolicies`: + ```java + TopicPolicies getTopicPolicies(TopicName topicName, boolean isGlobal) throws TopicPoliciesCacheNotInitException; + TopicPolicies getTopicPolicies(TopicName topicName) throws TopicPoliciesCacheNotInitException; + ``` + - The second method is equivalent to `getTopicPolicies(topicName, false)`. + - These methods are asynchronous and start an asynchronous policies initialization, then try to get the policies from the cache. If the initialization hasn't started, they throw `TopicPoliciesCacheNotInitException`. + +These methods are hard to use and are primarily used in tests. The `getTopicPoliciesAsyncWithRetry` method uses a user-provided executor and backoff policy to call `getTopicPolicies` until `TopicPoliciesCacheNotInitException` is not thrown: + +```java +default CompletableFuture> getTopicPoliciesAsyncWithRetry(TopicName topicName, + final Backoff backoff, ScheduledExecutorService scheduledExecutorService, boolean isGlobal) { +``` + +The `getTopicPolicies` methods are confusing for users who want to implement their own topic policies service. They need to look deeply into Pulsar's source code to understand these details. + +[PR #21231](https://github.com/apache/pulsar/pull/21231) adds two asynchronous overrides that are more user-friendly: + +```java +CompletableFuture> getTopicPoliciesAsync(@Nonnull TopicName topicName, boolean isGlobal); +CompletableFuture> getTopicPoliciesAsync(@Nonnull TopicName topicName); +``` + +Now there are five asynchronous `get` methods. Unlike `getTopicPolicies`, `getTopicPoliciesAsync(topic)` is not equivalent to `getTopicPoliciesAsync(topic, false)`. Instead: +- `getTopicPoliciesAsync(topic)` tries getting local policies first, then global policies if absent. +- `getTopicPoliciesAsync(topic, true)` tries getting global policies. +- `getTopicPoliciesAsync(topic, false)` tries getting local policies. + +Since [PR #12517](https://github.com/apache/pulsar/pull/12517), topic policies support global policies across clusters. Therefore, there are local and global policies. + +Currently: +- `getTopicPoliciesAsync(TopicName)` is used in `BrokerService#getTopicPoliciesBypassSystemTopic` for initializing topic policies of `PersistentTopic` objects. +- `getTopicPoliciesAsyncWithRetry` is used in `AdminResource#getTopicPoliciesAsyncWithRetry` for all topic policies admin APIs. +- Other methods are used only in tests. + +There is also a sixth method, `getTopicPoliciesIfExists`, which tries to get local topic policies from the cache: + +```java +TopicPolicies getTopicPoliciesIfExists(TopicName topicName); +``` + +However, this method is called just because there was no `getTopicPoliciesAsync` methods before and `getTopicPolicies` is hard to use. For example, here is an example code snippet in `PersistentTopicsBase#internalUpdatePartitionedTopicAsync`: + +```java +TopicPolicies topicPolicies = + pulsarService.getTopicPoliciesService().getTopicPoliciesIfExists(topicName); +if (topicPolicies != null && topicPolicies.getReplicationClusters() != null) { + replicationClusters = topicPolicies.getReplicationClustersSet(); +} +``` + +With the new `getTopicPoliciesAsync` methods, this code can be replaced with: + +```java +pulsarService.getTopicPoliciesService().getTopicPoliciesAsync(topicName, GetType.LOCAL_ONLY) + .thenAccept(topicPolicies -> { + if (topicPolicies.isPresent() && topicPolicies.get().getReplicationClusters() != null) { + replicationClusters = topicPolicies.get().getReplicationClustersSet(); + } + }); +``` + +## Motivation + +Make `TopicPoliciesService` pluggable so users can customize the topic policies service via another backend metadata store. + +## Goals + +### In Scope + +Redesign a clear and simple `TopicPoliciesService` interface for users to customize. + +## High-Level Design + +Add a `topicPoliciesServiceClassName` configuration to specify the topic policies service class name. If the class name is not the default `SystemTopicBasedTopicPoliciesService`, `systemTopicEnabled` will not be required unless the implementation requires it. + +## Detailed Design + +### Design & Implementation Details + +1. Add a unified method to get topic policies: + ```java + enum GetType { + DEFAULT, // try getting the local topic policies, if not present, then get the global policies + GLOBAL_ONLY, // only get the global policies + LOCAL_ONLY, // only get the local policies + } + CompletableFuture> getTopicPoliciesAsync(TopicName topicName, GetType type); + ``` + + `getTopicPoliciesAsyncWithRetry` will be replaced by `getTopicPoliciesAsync(topicName, LOCAL_ONLY)` or `getTopicPoliciesAsync(topicName, GLOBAL_ONLY)`. The other two original `getTopicPoliciesAsync` methods and `getTopicPoliciesIfExists` will be replaced by `getTopicPoliciesAsync(topicName, DEFAULT)`. + +2. Move `addOwnedNamespaceBundleAsync` and `removeOwnedNamespaceBundleAsync` to private methods of `SystemTopicBasedTopicPoliciesService`. + +3. Add a `TestUtils` class in tests to include `getTopicPolicies` and `getTopicPoliciesBypassCacheAsync` methods. + +4. Remove the generic parameter from `TopicPolicyListener` as the value type should always be `TopicPolicies`. Mark this listener interface as `Stable`. + +5. Add a `PulsarService` parameter to the `start` method so that the implementation can have a constructor with an empty parameter list and get the `PulsarService` instance from the `start` method. + +6. Add a `boolean` return value to `registerListener` since `PersistentTopic#initTopicPolicy` checks if the topic policies are enabled. The return value will indicate if the `TopicPoliciesService` instance is `topicPoliciesServiceClassName.DISABLED`. + +Since the topic policies service is now decoupled from system topics, remove all `isSystemTopicAndTopicLevelPoliciesEnabled()` calls. + +Here is the refactored `TopicPoliciesService` interface: + +```java + /** + * Delete policies for a topic asynchronously. + * + * @param topicName topic name + */ + CompletableFuture deleteTopicPoliciesAsync(TopicName topicName); + + /** + * Update policies for a topic asynchronously. + * + * @param topicName topic name + * @param policies policies for the topic name + */ + CompletableFuture updateTopicPoliciesAsync(TopicName topicName, TopicPolicies policies); + + /** + * It controls the behavior of {@link TopicPoliciesService#getTopicPoliciesAsync}. + */ + enum GetType { + DEFAULT, // try getting the local topic policies, if not present, then get the global policies + GLOBAL_ONLY, // only get the global policies + LOCAL_ONLY, // only get the local policies + } + + /** + * Retrieve the topic policies. + */ + CompletableFuture> getTopicPoliciesAsync(TopicName topicName, GetType type); + + /** + * Start the topic policy service. + */ + default void start(PulsarService pulsar) { + } + + /** + * Close the resources if necessary. + */ + default void close() throws Exception { + } + + /** + * Registers a listener for topic policies updates. + * + *

+ * The listener will receive the latest topic policies when they are updated. If the policies are removed, the + * listener will receive a null value. Note that not every update is guaranteed to trigger the listener. For + * instance, if the policies change from A -> B -> null -> C in quick succession, only the final state (C) is + * guaranteed to be received by the listener. + * In summary, the listener is guaranteed to receive only the latest value. + *

+ * + * @return true if the listener is registered successfully + */ + boolean registerListener(TopicName topicName, TopicPolicyListener listener); + + /** + * Unregister the topic policies listener. + */ + void unregisterListener(TopicName topicName, TopicPolicyListener listener); +``` + +```java +@InterfaceStability.Stable +public interface TopicPolicyListener { + + void onUpdate(TopicPolicies data); +} +``` + +### Configuration + +Add a new configuration `topicPoliciesServiceClassName`. + +## Backward & Forward Compatibility + +If downstream applications need to call APIs from `TopicPoliciesService`, they should modify the code to use the new API. + +## Alternatives + +### Keep the `TopicPoliciesService` Interface Compatible + +The current interface is poorly designed because it has only one implementation. Keeping these methods will burden developers who want to develop a customized interface. They need to understand where these confusing methods are called and handle them carefully. + +## General Notes + +## Links + +* Mailing List discussion thread: https://lists.apache.org/thread/gf6h4n5n1z4n8v6bxdthct1n07onfdxt +* Mailing List voting thread: https://lists.apache.org/thread/potjbkb4w8brcwscgdwzlxnowgdf11gd From 8151639dccbbacee5ec23678cf7b5e0a768902dd Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 11 Sep 2024 14:31:43 +0300 Subject: [PATCH 497/580] [fix][broker] Fix retry backoff for PersistentDispatcherMultipleConsumers (#23284) --- ...PersistentDispatcherMultipleConsumers.java | 19 ++- ...ckyKeyDispatcherMultipleConsumersTest.java | 140 +++++++++++++++--- 2 files changed, 129 insertions(+), 30 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 631a728ccce4d..264bac7cb6aab 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -451,12 +451,15 @@ protected void reScheduleReadInMs(long readAfterMs) { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Reschedule message read in {} ms", topic.getName(), name, readAfterMs); } - topic.getBrokerService().executor().schedule( - () -> { - isRescheduleReadInProgress.set(false); - readMoreEntries(); - }, - readAfterMs, TimeUnit.MILLISECONDS); + Runnable runnable = () -> { + isRescheduleReadInProgress.set(false); + readMoreEntries(); + }; + if (readAfterMs > 0) { + topic.getBrokerService().executor().schedule(runnable, readAfterMs, TimeUnit.MILLISECONDS); + } else { + topic.getBrokerService().executor().execute(runnable); + } } } @@ -836,6 +839,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis totalBytesSent += sendMessageInfo.getTotalBytes(); } + lastNumberOfEntriesDispatched = (int) totalEntries; acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); if (entriesToDispatch > 0) { @@ -848,9 +852,8 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); entry.release(); }); - - lastNumberOfEntriesDispatched = entriesToDispatch; } + return true; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index af99741d09bb6..a7ff9eb9c11f2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -110,6 +110,8 @@ public class PersistentStickyKeyDispatcherMultipleConsumersTest { final String topicName = "persistent://public/default/testTopic"; final String subscriptionName = "testSubscription"; private AtomicInteger consumerMockAvailablePermits; + int retryBackoffInitialTimeInMs = 10; + int retryBackoffMaxTimeInMs = 50; @BeforeMethod public void setup() throws Exception { @@ -120,8 +122,8 @@ public void setup() throws Exception { doReturn(1).when(configMock).getSubscriptionKeySharedConsistentHashingReplicaPoints(); doReturn(true).when(configMock).isDispatcherDispatchMessagesInSubscriptionThread(); doReturn(false).when(configMock).isAllowOverrideEntryFilters(); - doReturn(10).when(configMock).getDispatcherRetryBackoffInitialTimeInMs(); - doReturn(50).when(configMock).getDispatcherRetryBackoffMaxTimeInMs(); + doAnswer(invocation -> retryBackoffInitialTimeInMs).when(configMock).getDispatcherRetryBackoffInitialTimeInMs(); + doAnswer(invocation -> retryBackoffMaxTimeInMs).when(configMock).getDispatcherRetryBackoffMaxTimeInMs(); pulsarMock = mock(PulsarService.class); doReturn(configMock).when(pulsarMock).getConfiguration(); @@ -825,34 +827,45 @@ public void testLastSentPositionAndIndividuallySentPositions(final boolean initi assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); } - @DataProvider(name = "dispatchMessagesInSubscriptionThread") - private Object[][] dispatchMessagesInSubscriptionThread() { - return new Object[][] { { false }, { true } }; + @DataProvider(name = "testBackoffDelayWhenNoMessagesDispatched") + private Object[][] testBackoffDelayWhenNoMessagesDispatchedParams() { + return new Object[][] { { false, true }, { true, true }, { true, false }, { false, false } }; } - @Test(dataProvider = "dispatchMessagesInSubscriptionThread") - public void testBackoffDelayWhenNoMessagesDispatched(boolean dispatchMessagesInSubscriptionThread) + @Test(dataProvider = "testBackoffDelayWhenNoMessagesDispatched") + public void testBackoffDelayWhenNoMessagesDispatched(boolean dispatchMessagesInSubscriptionThread, boolean isKeyShared) throws Exception { persistentDispatcher.close(); List retryDelays = new CopyOnWriteArrayList<>(); doReturn(dispatchMessagesInSubscriptionThread).when(configMock).isDispatcherDispatchMessagesInSubscriptionThread(); - persistentDispatcher = new PersistentStickyKeyDispatcherMultipleConsumers( - topicMock, cursorMock, subscriptionMock, configMock, - new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)) { - @Override - protected void reScheduleReadInMs(long readAfterMs) { - retryDelays.add(readAfterMs); - } - }; + + PersistentDispatcherMultipleConsumers dispatcher; + if (isKeyShared) { + dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers( + topicMock, cursorMock, subscriptionMock, configMock, + new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)) { + @Override + protected void reScheduleReadInMs(long readAfterMs) { + retryDelays.add(readAfterMs); + } + }; + } else { + dispatcher = new PersistentDispatcherMultipleConsumers(topicMock, cursorMock, subscriptionMock) { + @Override + protected void reScheduleReadInMs(long readAfterMs) { + retryDelays.add(readAfterMs); + } + }; + } // add a consumer without permits to trigger the retry behavior consumerMockAvailablePermits.set(0); - persistentDispatcher.addConsumer(consumerMock); + dispatcher.addConsumer(consumerMock); // call "readEntriesComplete" directly to test the retry behavior List entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - persistentDispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 1); assertEquals(retryDelays.get(0), 10, "Initial retry delay should be 10ms"); @@ -860,7 +873,7 @@ protected void reScheduleReadInMs(long readAfterMs) { ); // test the second retry delay entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - persistentDispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 2); double delay = retryDelays.get(1); @@ -870,7 +883,7 @@ protected void reScheduleReadInMs(long readAfterMs) { // verify the max retry delay for (int i = 0; i < 100; i++) { entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - persistentDispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); } Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 102); @@ -881,14 +894,14 @@ protected void reScheduleReadInMs(long readAfterMs) { // unblock to check that the retry delay is reset consumerMockAvailablePermits.set(1000); entries = List.of(EntryImpl.create(1, 2, createMessage("message2", 1, "key2"))); - persistentDispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); // wait that the possibly async handling has completed - Awaitility.await().untilAsserted(() -> assertFalse(persistentDispatcher.isSendInProgress())); + Awaitility.await().untilAsserted(() -> assertFalse(dispatcher.isSendInProgress())); // now block again to check the next retry delay so verify it was reset consumerMockAvailablePermits.set(0); entries = List.of(EntryImpl.create(1, 3, createMessage("message3", 1, "key3"))); - persistentDispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 103); assertEquals(retryDelays.get(0), 10, "Resetted retry delay should be 10ms"); @@ -896,6 +909,89 @@ protected void reScheduleReadInMs(long readAfterMs) { ); } + @Test(dataProvider = "testBackoffDelayWhenNoMessagesDispatched") + public void testBackoffDelayWhenRetryDelayDisabled(boolean dispatchMessagesInSubscriptionThread, boolean isKeyShared) + throws Exception { + persistentDispatcher.close(); + + // it should be possible to disable the retry delay + // by setting retryBackoffInitialTimeInMs and retryBackoffMaxTimeInMs to 0 + retryBackoffInitialTimeInMs=0; + retryBackoffMaxTimeInMs=0; + + List retryDelays = new CopyOnWriteArrayList<>(); + doReturn(dispatchMessagesInSubscriptionThread).when(configMock) + .isDispatcherDispatchMessagesInSubscriptionThread(); + + PersistentDispatcherMultipleConsumers dispatcher; + if (isKeyShared) { + dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers( + topicMock, cursorMock, subscriptionMock, configMock, + new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)) { + @Override + protected void reScheduleReadInMs(long readAfterMs) { + retryDelays.add(readAfterMs); + } + }; + } else { + dispatcher = new PersistentDispatcherMultipleConsumers(topicMock, cursorMock, subscriptionMock) { + @Override + protected void reScheduleReadInMs(long readAfterMs) { + retryDelays.add(readAfterMs); + } + }; + } + + // add a consumer without permits to trigger the retry behavior + consumerMockAvailablePermits.set(0); + dispatcher.addConsumer(consumerMock); + + // call "readEntriesComplete" directly to test the retry behavior + List entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); + dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + Awaitility.await().untilAsserted(() -> { + assertEquals(retryDelays.size(), 1); + assertEquals(retryDelays.get(0), 0, "Initial retry delay should be 0ms"); + } + ); + // test the second retry delay + entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); + dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + Awaitility.await().untilAsserted(() -> { + assertEquals(retryDelays.size(), 2); + double delay = retryDelays.get(1); + assertEquals(delay, 0, 0, "Second retry delay should be 0ms"); + } + ); + // verify the max retry delay + for (int i = 0; i < 100; i++) { + entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); + dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + } + Awaitility.await().untilAsserted(() -> { + assertEquals(retryDelays.size(), 102); + double delay = retryDelays.get(101); + assertEquals(delay, 0, 0, "Max delay should be 0ms"); + } + ); + // unblock to check that the retry delay is reset + consumerMockAvailablePermits.set(1000); + entries = List.of(EntryImpl.create(1, 2, createMessage("message2", 1, "key2"))); + dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + // wait that the possibly async handling has completed + Awaitility.await().untilAsserted(() -> assertFalse(dispatcher.isSendInProgress())); + + // now block again to check the next retry delay so verify it was reset + consumerMockAvailablePermits.set(0); + entries = List.of(EntryImpl.create(1, 3, createMessage("message3", 1, "key3"))); + dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + Awaitility.await().untilAsserted(() -> { + assertEquals(retryDelays.size(), 103); + assertEquals(retryDelays.get(0), 0, "Resetted retry delay should be 0ms"); + } + ); + } + private ByteBuf createMessage(String message, int sequenceId) { return createMessage(message, sequenceId, "testKey"); } From fc0e4e3fe0fa14d9ac1361871edc53957625fe29 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 11 Sep 2024 19:33:47 +0800 Subject: [PATCH 498/580] [fix] [broker] Fix system topic can not be loaded up if it contains data offloaded (#23279) --- .../bookkeeper/mledger/LedgerOffloader.java | 4 + .../mledger/impl/ManagedLedgerImpl.java | 28 +++-- .../impl/NonAppendableLedgerOffloader.java | 74 ++++++++++++ .../mledger/impl/NullLedgerOffloader.java | 5 + .../mledger/impl/ManagedLedgerTest.java | 2 +- .../mledger/impl/OffloadPrefixReadTest.java | 114 ++++++++++++++++-- .../mledger/impl/OffloadPrefixTest.java | 2 +- .../test/MockedBookKeeperTestCase.java | 8 +- .../pulsar/broker/service/BrokerService.java | 41 +++---- .../broker/admin/AdminApiOffloadTest.java | 1 + .../broker/service/BrokerServiceTest.java | 21 +++- 11 files changed, 255 insertions(+), 45 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonAppendableLedgerOffloader.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloader.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloader.java index 868a8e4265365..11148ef1a59f5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloader.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloader.java @@ -230,5 +230,9 @@ default void scanLedgers(OffloadedLedgerMetadataConsumer consumer, Map offloadDriverMetadata) throws ManagedLedgerException { throw ManagedLedgerException.getManagedLedgerException(new UnsupportedOperationException()); } + + default boolean isAppendable() { + return true; + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 92c55e572a49a..8cb5a3ee6acec 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -94,6 +94,7 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.TerminateCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.UpdatePropertiesCallback; import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.LedgerOffloader; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerAttributes; @@ -2451,8 +2452,7 @@ private void scheduleDeferredTrimming(boolean isTruncate, CompletableFuture p } public void maybeOffloadInBackground(CompletableFuture promise) { - if (config.getLedgerOffloader() == null || config.getLedgerOffloader() == NullLedgerOffloader.INSTANCE - || config.getLedgerOffloader().getOffloadPolicies() == null) { + if (getOffloadPoliciesIfAppendable().isEmpty()) { return; } @@ -2468,8 +2468,7 @@ public void maybeOffloadInBackground(CompletableFuture promise) { private void maybeOffload(long offloadThresholdInBytes, long offloadThresholdInSeconds, CompletableFuture finalPromise) { - if (config.getLedgerOffloader() == null || config.getLedgerOffloader() == NullLedgerOffloader.INSTANCE - || config.getLedgerOffloader().getOffloadPolicies() == null) { + if (getOffloadPoliciesIfAppendable().isEmpty()) { String msg = String.format("[%s] Nothing to offload due to offloader or offloadPolicies is NULL", name); finalPromise.completeExceptionally(new IllegalArgumentException(msg)); return; @@ -2572,6 +2571,16 @@ void internalTrimConsumedLedgers(CompletableFuture promise) { internalTrimLedgers(false, promise); } + private Optional getOffloadPoliciesIfAppendable() { + LedgerOffloader ledgerOffloader = config.getLedgerOffloader(); + if (ledgerOffloader == null + || !ledgerOffloader.isAppendable() + || ledgerOffloader.getOffloadPolicies() == null) { + return Optional.empty(); + } + return Optional.ofNullable(ledgerOffloader.getOffloadPolicies()); + } + void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { if (!factory.isMetadataServiceAvailable()) { // Defer trimming of ledger if we cannot connect to metadata service @@ -2587,10 +2596,7 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { List ledgersToDelete = new ArrayList<>(); List offloadedLedgersToDelete = new ArrayList<>(); - Optional optionalOffloadPolicies = Optional.ofNullable(config.getLedgerOffloader() != null - && config.getLedgerOffloader() != NullLedgerOffloader.INSTANCE - ? config.getLedgerOffloader().getOffloadPolicies() - : null); + Optional optionalOffloadPolicies = getOffloadPoliciesIfAppendable(); synchronized (this) { if (log.isDebugEnabled()) { log.debug("[{}] Start TrimConsumedLedgers. ledgers={} totalSize={}", name, ledgers.keySet(), @@ -3117,8 +3123,10 @@ public void offloadFailed(ManagedLedgerException e, Object ctx) { @Override public void asyncOffloadPrefix(Position pos, OffloadCallback callback, Object ctx) { - if (config.getLedgerOffloader() != null && config.getLedgerOffloader() == NullLedgerOffloader.INSTANCE) { - callback.offloadFailed(new ManagedLedgerException("NullLedgerOffloader"), ctx); + LedgerOffloader ledgerOffloader = config.getLedgerOffloader(); + if (ledgerOffloader != null && !ledgerOffloader.isAppendable()) { + String msg = String.format("[%s] does not support offload", ledgerOffloader.getClass().getSimpleName()); + callback.offloadFailed(new ManagedLedgerException(msg), ctx); return; } Position requestOffloadTo = pos; diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonAppendableLedgerOffloader.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonAppendableLedgerOffloader.java new file mode 100644 index 0000000000000..f3001ec8050e2 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonAppendableLedgerOffloader.java @@ -0,0 +1,74 @@ +/* + * 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.bookkeeper.mledger.impl; + +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.mledger.LedgerOffloader; +import org.apache.pulsar.common.policies.data.OffloadPolicies; +import org.apache.pulsar.common.util.FutureUtil; + +public class NonAppendableLedgerOffloader implements LedgerOffloader { + private LedgerOffloader delegate; + + public NonAppendableLedgerOffloader(LedgerOffloader delegate) { + this.delegate = delegate; + } + + @Override + public String getOffloadDriverName() { + return delegate.getOffloadDriverName(); + } + + @Override + public CompletableFuture offload(ReadHandle ledger, + UUID uid, + Map extraMetadata) { + return FutureUtil.failedFuture(new UnsupportedOperationException()); + } + + @Override + public CompletableFuture readOffloaded(long ledgerId, UUID uid, + Map offloadDriverMetadata) { + return delegate.readOffloaded(ledgerId, uid, offloadDriverMetadata); + } + + @Override + public CompletableFuture deleteOffloaded(long ledgerId, UUID uid, + Map offloadDriverMetadata) { + return delegate.deleteOffloaded(ledgerId, uid, offloadDriverMetadata); + } + + @Override + public OffloadPolicies getOffloadPolicies() { + return delegate.getOffloadPolicies(); + } + + @Override + public void close() { + delegate.close(); + } + + @Override + public boolean isAppendable() { + return false; + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NullLedgerOffloader.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NullLedgerOffloader.java index 938ceb0c7dfbc..fe646bc82e55a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NullLedgerOffloader.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NullLedgerOffloader.java @@ -70,4 +70,9 @@ public OffloadPolicies getOffloadPolicies() { public void close() { } + + @Override + public boolean isAppendable() { + return false; + } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index e3b272babb7bb..bb38114ef7117 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -3849,7 +3849,7 @@ public void testDoNotGetOffloadPoliciesMultipleTimesWhenTrimLedgers() throws Exc config.setLedgerOffloader(ledgerOffloader); ledger.internalTrimConsumedLedgers(Futures.NULL_PROMISE); - verify(ledgerOffloader, times(1)).getOffloadPolicies(); + verify(ledgerOffloader, times(1)).isAppendable(); } @Test(timeOut = 30000) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java index 6d8ecba868847..48751417e1714 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java @@ -29,6 +29,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; import io.netty.buffer.ByteBuf; import java.util.ArrayList; @@ -54,6 +55,8 @@ import org.apache.bookkeeper.mledger.LedgerOffloader; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.mledger.util.MockClock; @@ -61,12 +64,34 @@ import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.OffloadedReadPriority; +import org.awaitility.Awaitility; import org.testng.Assert; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; public class OffloadPrefixReadTest extends MockedBookKeeperTestCase { - @Test - public void testOffloadRead() throws Exception { + + private final String offloadTypeAppendable = "NonAppendable"; + + @Override + protected void initManagedLedgerFactoryConfig(ManagedLedgerFactoryConfig config) { + super.initManagedLedgerFactoryConfig(config); + // disable cache. + config.setMaxCacheSize(0); + } + + @DataProvider(name = "offloadAndDeleteTypes") + public Object[][] offloadAndDeleteTypes() { + return new Object[][]{ + {"normal", true}, + {"normal", false}, + {offloadTypeAppendable, true}, + {offloadTypeAppendable, false}, + }; + } + + @Test(dataProvider = "offloadAndDeleteTypes") + public void testOffloadRead(String offloadType, boolean deleteMl) throws Exception { MockLedgerOffloader offloader = spy(MockLedgerOffloader.class); ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(10); @@ -89,6 +114,10 @@ public void testOffloadRead() throws Exception { Assert.assertTrue(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getComplete()); Assert.assertFalse(ledger.getLedgersInfoAsList().get(2).getOffloadContext().getComplete()); + if (offloadTypeAppendable.equals(offloadType)) { + config.setLedgerOffloader(new NonAppendableLedgerOffloader(offloader)); + } + UUID firstLedgerUUID = new UUID(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getUidMsb(), ledger.getLedgersInfoAsList().get(0).getOffloadContext().getUidLsb()); UUID secondLedgerUUID = new UUID(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getUidMsb(), @@ -116,13 +145,30 @@ public void testOffloadRead() throws Exception { verify(offloader, times(2)) .readOffloaded(anyLong(), (UUID) any(), anyMap()); - ledger.close(); - // Ensure that all the read handles had been closed - assertEquals(offloader.openedReadHandles.get(), 0); + if (!deleteMl) { + ledger.close(); + // Ensure that all the read handles had been closed + assertEquals(offloader.openedReadHandles.get(), 0); + } else { + // Verify: the ledger offloaded will be deleted after managed ledger is deleted. + ledger.delete(); + Awaitility.await().untilAsserted(() -> { + assertTrue(offloader.offloads.size() <= 1); + assertTrue(ledger.ledgers.size() <= 1); + }); + } } - @Test - public void testBookkeeperFirstOffloadRead() throws Exception { + @DataProvider(name = "offloadTypes") + public Object[][] offloadTypes() { + return new Object[][]{ + {"normal"}, + {offloadTypeAppendable}, + }; + } + + @Test(dataProvider = "offloadTypes") + public void testBookkeeperFirstOffloadRead(String offloadType) throws Exception { MockLedgerOffloader offloader = spy(MockLedgerOffloader.class); MockClock clock = new MockClock(); offloader.getOffloadPolicies() @@ -187,6 +233,10 @@ public void testBookkeeperFirstOffloadRead() throws Exception { Assert.assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getBookkeeperDeleted()); Assert.assertTrue(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getBookkeeperDeleted()); + if (offloadTypeAppendable.equals(offloadType)) { + config.setLedgerOffloader(new NonAppendableLedgerOffloader(offloader)); + } + for (Entry e : cursor.readEntries(10)) { Assert.assertEquals(new String(e.getData()), "entry-" + i++); } @@ -196,6 +246,56 @@ public void testBookkeeperFirstOffloadRead() throws Exception { .readOffloaded(anyLong(), (UUID) any(), anyMap()); verify(offloader).readOffloaded(anyLong(), eq(secondLedgerUUID), anyMap()); + // Verify: the ledger offloaded will be trimmed after if no backlog. + while (cursor.hasMoreEntries()) { + cursor.readEntries(1); + } + config.setRetentionTime(0, TimeUnit.MILLISECONDS); + config.setRetentionSizeInMB(0); + CompletableFuture trimFuture = new CompletableFuture(); + ledger.trimConsumedLedgersInBackground(trimFuture); + trimFuture.join(); + Awaitility.await().untilAsserted(() -> { + assertTrue(offloader.offloads.size() <= 1); + assertTrue(ledger.ledgers.size() <= 1); + }); + + // cleanup. + ledger.delete(); + } + + + + @Test + public void testSkipOffloadIfReadOnly() throws Exception { + LedgerOffloader ol = new NonAppendableLedgerOffloader(spy(MockLedgerOffloader.class)); + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setMaxEntriesPerLedger(10); + config.setMinimumRolloverTime(0, TimeUnit.SECONDS); + config.setRetentionTime(10, TimeUnit.MINUTES); + config.setRetentionSizeInMB(10); + config.setLedgerOffloader(ol); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", config); + + for (int i = 0; i < 25; i++) { + String content = "entry-" + i; + ledger.addEntry(content.getBytes()); + } + assertEquals(ledger.getLedgersInfoAsList().size(), 3); + + try { + ledger.offloadPrefix(ledger.getLastConfirmedEntry()); + } catch (ManagedLedgerException mle) { + assertTrue(mle.getMessage().contains("does not support offload")); + } + + assertEquals(ledger.getLedgersInfoAsList().size(), 3); + Assert.assertFalse(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); + Assert.assertFalse(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getComplete()); + Assert.assertFalse(ledger.getLedgersInfoAsList().get(2).getOffloadContext().getComplete()); + + // cleanup. + ledger.delete(); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java index 331e7b0317394..3f9f4f8da12f2 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java @@ -95,7 +95,7 @@ public void testNullOffloader() throws Exception { ledger.offloadPrefix(p); fail("Should have thrown an exception"); } catch (ManagedLedgerException e) { - assertEquals(e.getMessage(), "NullLedgerOffloader"); + assertTrue(e.getMessage().contains("does not support offload")); } assertEquals(ledger.getLedgersInfoAsList().size(), 5); assertEquals(ledger.getLedgersInfoAsList().stream() diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java b/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java index 645563eb78c4d..c7685cfaa6594 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java @@ -83,13 +83,17 @@ public final void setUp(Method method) throws Exception { } ManagedLedgerFactoryConfig managedLedgerFactoryConfig = new ManagedLedgerFactoryConfig(); - // increase default cache eviction interval so that caching could be tested with less flakyness - managedLedgerFactoryConfig.setCacheEvictionIntervalMs(200); + initManagedLedgerFactoryConfig(managedLedgerFactoryConfig); factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); setUpTestCase(); } + protected void initManagedLedgerFactoryConfig(ManagedLedgerFactoryConfig config) { + // increase default cache eviction interval so that caching could be tested with less flakyness + config.setCacheEvictionIntervalMs(200); + } + protected void setUpTestCase() throws Exception { } 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 c0e3e7d356be0..17e5288b5f179 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 @@ -94,7 +94,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerNotFoundException; import org.apache.bookkeeper.mledger.ManagedLedgerFactory; -import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; +import org.apache.bookkeeper.mledger.impl.NonAppendableLedgerOffloader; import org.apache.bookkeeper.mledger.util.Futures; import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.StringUtils; @@ -2018,29 +2018,26 @@ private CompletableFuture getManagedLedgerConfig(@Nonnull T topicLevelOffloadPolicies, OffloadPoliciesImpl.oldPoliciesCompatible(nsLevelOffloadPolicies, policies.orElse(null)), getPulsar().getConfig().getProperties()); - if (NamespaceService.isSystemServiceNamespace(namespace.toString()) - || SystemTopicNames.isSystemTopic(topicName)) { - /* - Avoid setting broker internal system topics using off-loader because some of them are the - preconditions of other topics. The slow replying log speed will cause a delay in all the topic - loading.(timeout) - */ - managedLedgerConfig.setLedgerOffloader(NullLedgerOffloader.INSTANCE); - } else { - if (topicLevelOffloadPolicies != null) { - try { - LedgerOffloader topicLevelLedgerOffLoader = - pulsar().createManagedLedgerOffloader(offloadPolicies); - managedLedgerConfig.setLedgerOffloader(topicLevelLedgerOffLoader); - } catch (PulsarServerException e) { - throw new RuntimeException(e); - } - } else { - //If the topic level policy is null, use the namespace level - managedLedgerConfig - .setLedgerOffloader(pulsar.getManagedLedgerOffloader(namespace, offloadPolicies)); + if (topicLevelOffloadPolicies != null) { + try { + LedgerOffloader topicLevelLedgerOffLoader = pulsar().createManagedLedgerOffloader(offloadPolicies); + managedLedgerConfig.setLedgerOffloader(topicLevelLedgerOffLoader); + } catch (PulsarServerException e) { + throw new RuntimeException(e); } + } else { + //If the topic level policy is null, use the namespace level + managedLedgerConfig + .setLedgerOffloader(pulsar.getManagedLedgerOffloader(namespace, offloadPolicies)); } + if (managedLedgerConfig.getLedgerOffloader() != null + && managedLedgerConfig.getLedgerOffloader().isAppendable() + && (NamespaceService.isSystemServiceNamespace(namespace.toString()) + || SystemTopicNames.isSystemTopic(topicName))) { + managedLedgerConfig.setLedgerOffloader( + new NonAppendableLedgerOffloader(managedLedgerConfig.getLedgerOffloader())); + } + managedLedgerConfig.setTriggerOffloadOnTopicLoad(serviceConfig.isTriggerOffloadOnTopicLoad()); managedLedgerConfig.setDeletionAtBatchIndexLevelEnabled( diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java index eac816bd81089..1ea29c9d431bd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java @@ -126,6 +126,7 @@ private void testOffload(String topicName, String mlName) throws Exception { CompletableFuture promise = new CompletableFuture<>(); doReturn(promise).when(offloader).offload(any(), any(), any()); + doReturn(true).when(offloader).isAppendable(); MessageId currentId = MessageId.latest; try (Producer p = pulsarClient.newProducer().topic(topicName).enableBatching(false).create()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index aa236e09da99d..2f27d5917f025 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -76,6 +76,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; +import org.apache.bookkeeper.mledger.impl.NonAppendableLedgerOffloader; import org.apache.http.HttpResponse; import org.apache.http.client.methods.HttpGet; import org.apache.http.impl.client.CloseableHttpClient; @@ -1883,6 +1884,10 @@ public void close() { final String namespace = "prop/" + UUID.randomUUID(); admin.namespaces().createNamespace(namespace); admin.namespaces().setOffloadPolicies(namespace, offloadPolicies); + Awaitility.await().untilAsserted(() -> { + OffloadPolicies policiesGot = admin.namespaces().getOffloadPolicies(namespace); + assertNotNull(policiesGot); + }); // Inject the cache to avoid real load off-loader jar final Map ledgerOffloaderMap = pulsar.getLedgerOffloaderMap(); @@ -1896,8 +1901,20 @@ public void close() { // (2) test system topic for (String eventTopicName : SystemTopicNames.EVENTS_TOPIC_NAMES) { - managedLedgerConfig = brokerService.getManagedLedgerConfig(TopicName.get(eventTopicName)).join(); - Assert.assertEquals(managedLedgerConfig.getLedgerOffloader(), NullLedgerOffloader.INSTANCE); + boolean offloadPoliciesExists = false; + try { + OffloadPolicies policiesGot = + admin.namespaces().getOffloadPolicies(TopicName.get(eventTopicName).getNamespace()); + offloadPoliciesExists = policiesGot != null; + } catch (PulsarAdminException.NotFoundException notFoundException) { + offloadPoliciesExists = false; + } + var managedLedgerConfig2 = brokerService.getManagedLedgerConfig(TopicName.get(eventTopicName)).join(); + if (offloadPoliciesExists) { + Assert.assertTrue(managedLedgerConfig2.getLedgerOffloader() instanceof NonAppendableLedgerOffloader); + } else { + Assert.assertEquals(managedLedgerConfig2.getLedgerOffloader(), NullLedgerOffloader.INSTANCE); + } } } } From 5599699fe41f38c4d156d1954406c081a8c7aff6 Mon Sep 17 00:00:00 2001 From: Vinay Bhat <41332000+touchkey@users.noreply.github.com> Date: Thu, 12 Sep 2024 18:24:13 +0530 Subject: [PATCH 499/580] [improve][pip] PIP-374: Visibility of messages in receiverQueue for the consumers (#23235) Co-authored-by: vbhat6 --- pip/pip-374.md | 71 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 71 insertions(+) create mode 100644 pip/pip-374.md diff --git a/pip/pip-374.md b/pip/pip-374.md new file mode 100644 index 0000000000000..4264617647433 --- /dev/null +++ b/pip/pip-374.md @@ -0,0 +1,71 @@ +# PIP-374: Visibility of messages in receiverQueue for the consumers + +# Background knowledge + +When a consumer connects to the Broker, the broker starts dispatching the messages based on receiverQueueSize configured. +There is no observability for the messages arrived on the consumer side if the user didn't call the receive method. It leads to ambiguities at times as +the consumer application does not know whether the message was actually sent by the broker or is it lost in the network or is it lost in the receiver queue. + +ConsumerInterceptors is a plugin interface that intercept and possibly mutate messages received by the consumer. + + +# Motivation + +* We need to receive queue filling of the event as the particular message is already on particular consumer's receiver queue and waiting for the consumer to pickup and process. It may wait in the recieverQueue longer if the consumer processing takes more time. It's very important to provide the visibility of the messages that are waiting in receiverQueue for processing. + +* Availability of a consumer application w.r.t any messaging system depends on the number of messages dispatched from the server/broker against the number of messages acknowledged from the consumer app. This metric defines the processing rate of a consumer. +Currently, the number of acknowledged messages can be counted by having a counter in onAcknowledge() method of ConsumerInterceptor. But, there is no way to capture the number of messages arrived in Consumer. + + +What does this solve? +* Visibility about the message in receiverQueue for the consumer. +* Stuck consumer state visibility +* Scale the consumers to process the spikes in producer traffic +* Reduce the overhead of processing the redeliveries + + +# Goals + +## In Scope + +The proposal will add a method to the interceptor to allow users to knowthe message has been received by the consumer. + +Add a default abstract method in ConsumerInterceptor called onArrival() and hook this method call in the internal consumer of MultiTopicConsumerImpl and ConsumerImpl. By this way, there will be an observability of message received for the consumer. + + +# High Level Design + +* Add onArrival() abstract method in ConsumerInterceptor interface. +* Hook this method call where the consumer receives the batch messages at once(based on configured receiverQueueSize). + + +# Detailed Design + +## Design & Implementation Details + +* ConsumerInterceptor.java +``` +default Message onArrival()(Consumer consumer, Message message){ + return message; +} + +``` + +* Add hook in ConsumerImpl.messageReceived which calls onArrival method which calculates the the number of message received. +``` +Message interceptMsg = onArrival(consumer,msg); +``` + +# Backward & Forward Compatibility + +## Upgrade + +Since we added a default method onArrival() in interface, one who has provided the implementations for ConsumerInterceptor will not get any compile time error as it has default implementation. If user wants to give implementation from his side, he can override and provide implementation. + +# Links + + +* Mailing List discussion thread: +* Mailing List voting thread: From fc60ec06ae98fa4000473a636bfb06729c210048 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 13 Sep 2024 11:14:08 +0800 Subject: [PATCH 500/580] [fix][broker] Fail fast if the extensible load manager failed to start (#23297) --- .../pulsar/broker/PulsarServerException.java | 17 +++ .../apache/pulsar/broker/PulsarService.java | 2 +- .../extensions/ExtensibleLoadManagerImpl.java | 122 ++++++------------ .../extensions/LoadManagerFailFastTest.java | 120 +++++++++++++++++ 4 files changed, 179 insertions(+), 82 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/LoadManagerFailFastTest.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/PulsarServerException.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/PulsarServerException.java index 2235b9a7128b8..d7c0d0adb3afc 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/PulsarServerException.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/PulsarServerException.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker; import java.io.IOException; +import java.util.concurrent.CompletionException; public class PulsarServerException extends IOException { private static final long serialVersionUID = 1; @@ -44,4 +45,20 @@ public NotFoundException(Throwable t) { super(t); } } + + public static PulsarServerException from(Throwable throwable) { + if (throwable instanceof CompletionException) { + return from(throwable.getCause()); + } + if (throwable instanceof PulsarServerException pulsarServerException) { + return pulsarServerException; + } else { + return new PulsarServerException(throwable); + } + } + + // Wrap this checked exception into a specific unchecked exception + public static CompletionException toUncheckedException(PulsarServerException e) { + return new CompletionException(e); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index b2e67bf4883dd..425e7dafa1bf8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -1080,7 +1080,7 @@ public void start() throws PulsarServerException { state = State.Started; } catch (Exception e) { LOG.error("Failed to start Pulsar service: {}", e.getMessage(), e); - PulsarServerException startException = new PulsarServerException(e); + PulsarServerException startException = PulsarServerException.from(e); readyForIncomingRequestsFuture.completeExceptionally(startException); throw startException; } finally { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 40efa6390a78a..8e34f2f697fb1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -81,7 +81,6 @@ import org.apache.pulsar.broker.loadbalance.extensions.scheduler.SplitScheduler; import org.apache.pulsar.broker.loadbalance.extensions.scheduler.UnloadScheduler; import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStore; -import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreException; import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStoreFactory; import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy; import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategyFactory; @@ -99,10 +98,7 @@ import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.common.util.Backoff; -import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.coordination.LeaderElectionState; import org.slf4j.Logger; @@ -125,10 +121,6 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS public static final long COMPACTION_THRESHOLD = 5 * 1024 * 1024; - public static final int STARTUP_TIMEOUT_SECONDS = 30; - - public static final int MAX_RETRY = 5; - private static final String ELECTION_ROOT = "/loadbalance/extension/leader"; public static final Set INTERNAL_TOPICS = @@ -212,7 +204,7 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS private final ConcurrentHashMap>> lookupRequests = new ConcurrentHashMap<>(); - private final CompletableFuture initWaiter = new CompletableFuture<>(); + private final CompletableFuture initWaiter = new CompletableFuture<>(); /** * Get all the bundles that are owned by this broker. @@ -385,7 +377,7 @@ public void start() throws PulsarServerException { return; } try { - this.brokerRegistry = new BrokerRegistryImpl(pulsar); + this.brokerRegistry = createBrokerRegistry(pulsar); this.leaderElectionService = new LeaderElectionService( pulsar.getCoordinationService(), pulsar.getBrokerId(), pulsar.getSafeWebServiceAddress(), ELECTION_ROOT, @@ -400,53 +392,14 @@ public void start() throws PulsarServerException { }); }); }); - this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar); + this.serviceUnitStateChannel = createServiceUnitStateChannel(pulsar); this.brokerRegistry.start(); this.splitManager = new SplitManager(splitCounter); this.unloadManager = new UnloadManager(unloadCounter, pulsar.getBrokerId()); this.serviceUnitStateChannel.listen(unloadManager); this.serviceUnitStateChannel.listen(splitManager); this.leaderElectionService.start(); - pulsar.runWhenReadyForIncomingRequests(() -> { - Backoff backoff = new BackoffBuilder() - .setInitialTime(100, TimeUnit.MILLISECONDS) - .setMax(STARTUP_TIMEOUT_SECONDS, TimeUnit.SECONDS) - .create(); - int retry = 0; - while (!Thread.currentThread().isInterrupted()) { - try { - brokerRegistry.register(); - this.serviceUnitStateChannel.start(); - break; - } catch (Exception e) { - log.warn("The broker:{} failed to start service unit state channel. Retrying {} th ...", - pulsar.getBrokerId(), ++retry, e); - try { - Thread.sleep(backoff.next()); - } catch (InterruptedException ex) { - log.warn("Interrupted while sleeping."); - // preserve thread's interrupt status - Thread.currentThread().interrupt(); - try { - pulsar.close(); - } catch (PulsarServerException exc) { - log.error("Failed to close pulsar service.", exc); - } - return; - } - failStarting(e); - if (retry >= MAX_RETRY) { - log.error("Failed to start the service unit state channel after retry {} th. " - + "Closing pulsar service.", retry, e); - try { - pulsar.close(); - } catch (PulsarServerException ex) { - log.error("Failed to close pulsar service.", ex); - } - } - } - } - }); + this.antiAffinityGroupPolicyHelper = new AntiAffinityGroupPolicyHelper(pulsar, serviceUnitStateChannel); antiAffinityGroupPolicyHelper.listenFailureDomainUpdate(); @@ -455,15 +408,10 @@ public void start() throws PulsarServerException { SimpleResourceAllocationPolicies policies = new SimpleResourceAllocationPolicies(pulsar); this.isolationPoliciesHelper = new IsolationPoliciesHelper(policies); this.brokerFilterPipeline.add(new BrokerIsolationPoliciesFilter(isolationPoliciesHelper)); - - try { - this.brokerLoadDataStore = LoadDataStoreFactory - .create(pulsar, BROKER_LOAD_DATA_STORE_TOPIC, BrokerLoadData.class); - this.topBundlesLoadDataStore = LoadDataStoreFactory - .create(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TopBundlesLoadData.class); - } catch (LoadDataStoreException e) { - throw new PulsarServerException(e); - } + this.brokerLoadDataStore = LoadDataStoreFactory + .create(pulsar, BROKER_LOAD_DATA_STORE_TOPIC, BrokerLoadData.class); + this.topBundlesLoadDataStore = LoadDataStoreFactory + .create(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TopBundlesLoadData.class); this.context = LoadManagerContextImpl.builder() .configuration(conf) @@ -487,6 +435,7 @@ public void start() throws PulsarServerException { pulsar.runWhenReadyForIncomingRequests(() -> { try { + this.serviceUnitStateChannel.start(); var interval = conf.getLoadBalancerReportUpdateMinIntervalMillis(); this.brokerLoadDataReportTask = this.pulsar.getLoadManagerExecutor() @@ -521,38 +470,33 @@ public void start() throws PulsarServerException { MONITOR_INTERVAL_IN_MILLIS, TimeUnit.MILLISECONDS); this.splitScheduler.start(); - this.initWaiter.complete(null); + this.initWaiter.complete(true); this.started = true; log.info("Started load manager."); - } catch (Exception ex) { - failStarting(ex); + } catch (Throwable e) { + failStarting(e); } }); - } catch (Exception ex) { + } catch (Throwable ex) { failStarting(ex); } } - private void failStarting(Exception ex) { - log.error("Failed to start the extensible load balance and close broker registry {}.", - this.brokerRegistry, ex); + private void failStarting(Throwable throwable) { if (this.brokerRegistry != null) { try { - brokerRegistry.unregister(); - } catch (MetadataStoreException e) { - // ignore - } - } - if (this.serviceUnitStateChannel != null) { - try { - serviceUnitStateChannel.close(); - } catch (IOException e) { - // ignore + brokerRegistry.close(); + } catch (PulsarServerException e) { + // If close failed, this broker might still exist in the metadata store. Then it could be found by other + // brokers as an available broker. Hence, print a warning log for it. + log.warn("Failed to close the broker registry: {}", e.getMessage()); } } - initWaiter.completeExceptionally(ex); + initWaiter.complete(false); // exit the background thread gracefully + throw PulsarServerException.toUncheckedException(PulsarServerException.from(throwable)); } + @Override public void initialize(PulsarService pulsar) { this.pulsar = pulsar; @@ -897,7 +841,9 @@ synchronized void playLeader() { boolean becameFollower = false; while (!Thread.currentThread().isInterrupted()) { try { - initWaiter.get(); + if (!initWaiter.get()) { + return; + } if (!serviceUnitStateChannel.isChannelOwner()) { becameFollower = true; break; @@ -947,7 +893,9 @@ synchronized void playFollower() { boolean becameLeader = false; while (!Thread.currentThread().isInterrupted()) { try { - initWaiter.get(); + if (!initWaiter.get()) { + return; + } if (serviceUnitStateChannel.isChannelOwner()) { becameLeader = true; break; @@ -1018,7 +966,9 @@ private List getIgnoredCommandMetrics(String advertisedBrokerAddress) { @VisibleForTesting protected void monitor() { try { - initWaiter.get(); + if (!initWaiter.get()) { + return; + } // Monitor role // Periodically check the role in case ZK watcher fails. @@ -1073,4 +1023,14 @@ private void closeInternalTopics() { log.warn("Failed to wait for closing internal topics", e); } } + + @VisibleForTesting + protected BrokerRegistry createBrokerRegistry(PulsarService pulsar) { + return new BrokerRegistryImpl(pulsar); + } + + @VisibleForTesting + protected ServiceUnitStateChannel createServiceUnitStateChannel(PulsarService pulsar) { + return new ServiceUnitStateChannelImpl(pulsar); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/LoadManagerFailFastTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/LoadManagerFailFastTest.java new file mode 100644 index 0000000000000..a400bf733e557 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/LoadManagerFailFastTest.java @@ -0,0 +1,120 @@ +/* + * 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.loadbalance.extensions; + +import java.util.Optional; +import lombok.Cleanup; +import org.apache.pulsar.broker.PulsarServerException; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.LoadManager; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; +import org.apache.pulsar.common.util.PortManager; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.awaitility.Awaitility; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class LoadManagerFailFastTest { + + private static final String cluster = "test"; + private final int zkPort = PortManager.nextLockedFreePort(); + private final LocalBookkeeperEnsemble bk = new LocalBookkeeperEnsemble(2, zkPort, PortManager::nextLockedFreePort); + private final ServiceConfiguration config = new ServiceConfiguration(); + + @BeforeClass + protected void setup() throws Exception { + bk.start(); + config.setClusterName(cluster); + config.setAdvertisedAddress("localhost"); + config.setBrokerServicePort(Optional.of(0)); + config.setWebServicePort(Optional.of(0)); + config.setMetadataStoreUrl("zk:localhost:" + zkPort); + } + + @AfterClass + protected void cleanup() throws Exception { + bk.stop(); + } + + @Test(timeOut = 30000) + public void testBrokerRegistryFailure() throws Exception { + config.setLoadManagerClassName(BrokerRegistryLoadManager.class.getName()); + @Cleanup final var pulsar = new PulsarService(config); + try { + pulsar.start(); + Assert.fail(); + } catch (PulsarServerException e) { + Assert.assertNull(e.getCause()); + Assert.assertEquals(e.getMessage(), "Cannot start BrokerRegistry"); + } + Assert.assertTrue(pulsar.getLocalMetadataStore().getChildren(LoadManager.LOADBALANCE_BROKERS_ROOT).get() + .isEmpty()); + } + + @Test(timeOut = 30000) + public void testServiceUnitStateChannelFailure() throws Exception { + config.setLoadManagerClassName(ChannelLoadManager.class.getName()); + @Cleanup final var pulsar = new PulsarService(config); + try { + pulsar.start(); + Assert.fail(); + } catch (PulsarServerException e) { + Assert.assertNull(e.getCause()); + Assert.assertEquals(e.getMessage(), "Cannot start ServiceUnitStateChannel"); + } + Awaitility.await().untilAsserted(() -> Assert.assertTrue(pulsar.getLocalMetadataStore() + .getChildren(LoadManager.LOADBALANCE_BROKERS_ROOT).get().isEmpty())); + } + + private static class BrokerRegistryLoadManager extends ExtensibleLoadManagerImpl { + + @Override + protected BrokerRegistry createBrokerRegistry(PulsarService pulsar) { + final var mockBrokerRegistry = Mockito.mock(BrokerRegistryImpl.class); + try { + Mockito.doThrow(new PulsarServerException("Cannot start BrokerRegistry")).when(mockBrokerRegistry) + .start(); + } catch (PulsarServerException e) { + throw new RuntimeException(e); + } + return mockBrokerRegistry; + } + } + + private static class ChannelLoadManager extends ExtensibleLoadManagerImpl { + + @Override + protected ServiceUnitStateChannel createServiceUnitStateChannel(PulsarService pulsar) { + final var channel = Mockito.mock(ServiceUnitStateChannelImpl.class); + try { + Mockito.doThrow(new PulsarServerException("Cannot start ServiceUnitStateChannel")).when(channel) + .start(); + } catch (PulsarServerException e) { + throw new RuntimeException(e); + } + Mockito.doAnswer(__ -> null).when(channel).listen(Mockito.any()); + return channel; + } + } +} From 13c19b50216ba7e73766e6fa7b57d2700614e3b5 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 13 Sep 2024 19:07:12 +0800 Subject: [PATCH 501/580] [improve][broker] Register the broker to metadata store without version id compare (#23298) --- .../extensions/BrokerRegistryImpl.java | 44 +++++++++---------- .../extensions/BrokerRegistryTest.java | 14 +++--- .../pulsar/metadata/api/MetadataCache.java | 20 +++++++++ .../cache/impl/MetadataCacheImpl.java | 24 ++++++++++ .../pulsar/metadata/MetadataCacheTest.java | 25 +++++++++++ 5 files changed, 96 insertions(+), 31 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java index 5db11d40c33ff..f34d377990b68 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java @@ -21,11 +21,11 @@ import static org.apache.pulsar.broker.loadbalance.LoadManager.LOADBALANCE_BROKERS_ROOT; import com.google.common.annotations.VisibleForTesting; import java.util.ArrayList; +import java.util.EnumSet; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.RejectedExecutionException; @@ -39,11 +39,11 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; -import org.apache.pulsar.metadata.api.coordination.LockManager; -import org.apache.pulsar.metadata.api.coordination.ResourceLock; +import org.apache.pulsar.metadata.api.extended.CreateOption; /** * The broker registry impl, base on the LockManager. @@ -57,16 +57,14 @@ public class BrokerRegistryImpl implements BrokerRegistry { private final BrokerLookupData brokerLookupData; - private final LockManager brokerLookupDataLockManager; + private final MetadataCache brokerLookupDataMetadataCache; - private final String brokerId; + private final String brokerIdKeyPath; private final ScheduledExecutorService scheduler; private final List> listeners; - private volatile ResourceLock brokerLookupDataLock; - protected enum State { Init, Started, @@ -79,10 +77,10 @@ protected enum State { public BrokerRegistryImpl(PulsarService pulsar) { this.pulsar = pulsar; this.conf = pulsar.getConfiguration(); - this.brokerLookupDataLockManager = pulsar.getCoordinationService().getLockManager(BrokerLookupData.class); + this.brokerLookupDataMetadataCache = pulsar.getLocalMetadataStore().getMetadataCache(BrokerLookupData.class); this.scheduler = pulsar.getLoadManagerExecutor(); this.listeners = new ArrayList<>(); - this.brokerId = pulsar.getBrokerId(); + this.brokerIdKeyPath = keyPath(pulsar.getBrokerId()); this.brokerLookupData = new BrokerLookupData( pulsar.getWebServiceAddress(), pulsar.getWebServiceAddressTls(), @@ -122,7 +120,7 @@ public boolean isStarted() { public synchronized void register() throws MetadataStoreException { if (this.state == State.Started) { try { - this.brokerLookupDataLock = brokerLookupDataLockManager.acquireLock(keyPath(brokerId), brokerLookupData) + brokerLookupDataMetadataCache.put(brokerIdKeyPath, brokerLookupData, EnumSet.of(CreateOption.Ephemeral)) .get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); this.state = State.Registered; } catch (InterruptedException | ExecutionException | TimeoutException e) { @@ -135,30 +133,37 @@ public synchronized void register() throws MetadataStoreException { public synchronized void unregister() throws MetadataStoreException { if (this.state == State.Registered) { try { - this.brokerLookupDataLock.release() + brokerLookupDataMetadataCache.delete(brokerIdKeyPath) .get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); - this.state = State.Started; - } catch (CompletionException | InterruptedException | ExecutionException | TimeoutException e) { + } catch (ExecutionException e) { + if (e.getCause() instanceof MetadataStoreException.NotFoundException) { + log.warn("{} has already been unregistered", brokerIdKeyPath); + } else { + throw MetadataStoreException.unwrap(e); + } + } catch (InterruptedException | TimeoutException e) { throw MetadataStoreException.unwrap(e); + } finally { + this.state = State.Started; } } } @Override public String getBrokerId() { - return this.brokerId; + return pulsar.getBrokerId(); } @Override public CompletableFuture> getAvailableBrokersAsync() { this.checkState(); - return brokerLookupDataLockManager.listLocks(LOADBALANCE_BROKERS_ROOT).thenApply(ArrayList::new); + return brokerLookupDataMetadataCache.getChildren(LOADBALANCE_BROKERS_ROOT); } @Override public CompletableFuture> lookupAsync(String broker) { this.checkState(); - return brokerLookupDataLockManager.readLock(keyPath(broker)); + return brokerLookupDataMetadataCache.get(keyPath(broker)); } public CompletableFuture> getAvailableBrokerLookupDataAsync() { @@ -192,13 +197,8 @@ public synchronized void close() throws PulsarServerException { try { this.listeners.clear(); this.unregister(); - this.brokerLookupDataLockManager.close(); } catch (Exception ex) { - if (ex.getCause() instanceof MetadataStoreException.NotFoundException) { - throw new PulsarServerException.NotFoundException(MetadataStoreException.unwrap(ex)); - } else { - throw new PulsarServerException(MetadataStoreException.unwrap(ex)); - } + log.error("Unexpected error when unregistering the broker registry", ex); } finally { this.state = State.Closed; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java index 42600a4203551..91ada90dda690 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java @@ -291,7 +291,7 @@ public void testRegisterAndLookup() throws Exception { } @Test - public void testRegisterFailWithSameBrokerId() throws Exception { + public void testRegisterWithSameBrokerId() throws Exception { PulsarService pulsar1 = createPulsarService(); PulsarService pulsar2 = createPulsarService(); pulsar1.start(); @@ -301,14 +301,10 @@ public void testRegisterFailWithSameBrokerId() throws Exception { BrokerRegistryImpl brokerRegistry1 = createBrokerRegistryImpl(pulsar1); BrokerRegistryImpl brokerRegistry2 = createBrokerRegistryImpl(pulsar2); brokerRegistry1.start(); - try { - brokerRegistry2.start(); - fail(); - } catch (Exception ex) { - log.info("Broker registry start failed.", ex); - assertTrue(ex instanceof PulsarServerException); - assertTrue(ex.getMessage().contains("LockBusyException")); - } + brokerRegistry2.start(); + + pulsar1.close(); + pulsar2.close(); } @Test diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java index 6d558e709716d..8e153b23d3087 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java @@ -18,12 +18,14 @@ */ package org.apache.pulsar.metadata.api; +import java.util.EnumSet; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.function.Function; import org.apache.pulsar.metadata.api.MetadataStoreException.AlreadyExistsException; import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; +import org.apache.pulsar.metadata.api.extended.CreateOption; /** * Represent the caching layer access for a specific type of objects. @@ -128,6 +130,24 @@ public interface MetadataCache { */ CompletableFuture create(String path, T value); + /** + * Create or update the value of the given path in the metadata store without version comparison. + *

+ * This method is equivalent to + * {@link org.apache.pulsar.metadata.api.extended.MetadataStoreExtended#put(String, byte[], Optional, EnumSet)} or + * {@link MetadataStore#put(String, byte[], Optional)} if the metadata store does not support this extended API, + * with `Optional.empty()` as the 3rd argument. It means if the path does not exist, it will be created. If the path + * already exists, the new value will override the old value. + *

+ * @param path the path of the object in the metadata store + * @param value the object to put in the metadata store + * @param options the create options if the path does not in the metadata store + * @return the future that indicates if this operation failed, it could fail with + * {@link java.io.IOException} if the value failed to be serialized + * {@link MetadataStoreException} if the metadata store operation failed + */ + CompletableFuture put(String path, T value, EnumSet options); + /** * Delete an object from the metadata store. *

diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java index b9051a7dc7df4..ee394b0267c88 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java @@ -25,6 +25,7 @@ import com.github.benmanes.caffeine.cache.Caffeine; import com.google.common.annotations.VisibleForTesting; import java.io.IOException; +import java.util.EnumSet; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -47,12 +48,15 @@ import org.apache.pulsar.metadata.api.MetadataStoreException.ContentDeserializationException; import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; import org.apache.pulsar.metadata.api.Notification; +import org.apache.pulsar.metadata.api.extended.CreateOption; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.apache.pulsar.metadata.impl.AbstractMetadataStore; @Slf4j public class MetadataCacheImpl implements MetadataCache, Consumer { @Getter private final MetadataStore store; + private final MetadataStoreExtended storeExtended; private final MetadataSerde serde; private final AsyncLoadingCache>> objCache; @@ -67,6 +71,11 @@ public MetadataCacheImpl(MetadataStore store, JavaType type, MetadataCacheConfig public MetadataCacheImpl(MetadataStore store, MetadataSerde serde, MetadataCacheConfig cacheConfig) { this.store = store; + if (store instanceof MetadataStoreExtended) { + this.storeExtended = (MetadataStoreExtended) store; + } else { + this.storeExtended = null; + } this.serde = serde; Caffeine cacheBuilder = Caffeine.newBuilder(); @@ -243,6 +252,21 @@ public CompletableFuture create(String path, T value) { return future; } + @Override + public CompletableFuture put(String path, T value, EnumSet options) { + final byte[] bytes; + try { + bytes = serde.serialize(path, value); + } catch (IOException e) { + return CompletableFuture.failedFuture(e); + } + if (storeExtended != null) { + return storeExtended.put(path, bytes, Optional.empty(), options).thenAccept(__ -> refresh(path)); + } else { + return store.put(path, bytes, Optional.empty()).thenAccept(__ -> refresh(path)); + } + } + @Override public CompletableFuture delete(String path) { return store.delete(path, Optional.empty()); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java index df59d25bdcc0e..bac5807360453 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java @@ -29,6 +29,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.EnumSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -55,6 +56,7 @@ import org.apache.pulsar.metadata.api.MetadataStoreFactory; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.Stat; +import org.apache.pulsar.metadata.api.extended.CreateOption; import org.apache.pulsar.metadata.cache.impl.MetadataCacheImpl; import org.awaitility.Awaitility; import org.testng.annotations.DataProvider; @@ -597,4 +599,27 @@ public CustomClass deserialize(String path, byte[] content, Stat stat) throws IO assertEquals(res.getValue().b, 2); assertEquals(res.getValue().path, key1); } + + @Test(dataProvider = "distributedImpl") + public void testPut(String provider, Supplier urlSupplier) throws Exception { + @Cleanup final var store1 = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder() + .build()); + final var cache1 = store1.getMetadataCache(Integer.class); + @Cleanup final var store2 = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder() + .build()); + final var cache2 = store2.getMetadataCache(Integer.class); + final var key = "/testPut"; + + cache1.put(key, 1, EnumSet.of(CreateOption.Ephemeral)); // create + Awaitility.await().untilAsserted(() -> { + assertEquals(cache1.get(key).get().orElse(-1), 1); + assertEquals(cache2.get(key).get().orElse(-1), 1); + }); + + cache2.put(key, 2, EnumSet.of(CreateOption.Ephemeral)); // update + Awaitility.await().untilAsserted(() -> { + assertEquals(cache1.get(key).get().orElse(-1), 2); + assertEquals(cache2.get(key).get().orElse(-1), 2); + }); + } } From 4f96146f13b136644a4eb0cf4ec36699e0431929 Mon Sep 17 00:00:00 2001 From: AloysZhang Date: Sat, 14 Sep 2024 19:11:26 +0800 Subject: [PATCH 502/580] [improve][broker] Reducing the parse of MessageMetadata in compaction (#23285) Co-authored-by: Aloys Zhang --- .../pulsar/client/impl/RawBatchConverter.java | 28 +++++++++++++++---- .../compaction/AbstractTwoPhaseCompactor.java | 14 +++++----- .../compaction/EventTimeOrderCompactor.java | 24 ++++++++-------- .../compaction/PublishingOrderCompactor.java | 10 +++---- 4 files changed, 46 insertions(+), 30 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java index f41a7aedd59b2..d8c491dab2906 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java @@ -52,12 +52,16 @@ public static boolean isReadableBatch(MessageMetadata metadata) { return metadata.hasNumMessagesInBatch() && metadata.getEncryptionKeysCount() == 0; } - public static List extractMessageCompactionData(RawMessage msg) + public static List extractMessageCompactionData(RawMessage msg, MessageMetadata metadata) throws IOException { checkArgument(msg.getMessageIdData().getBatchIndex() == -1); ByteBuf payload = msg.getHeadersAndPayload(); - MessageMetadata metadata = Commands.parseMessageMetadata(payload); + if (metadata == null) { + metadata = Commands.parseMessageMetadata(payload); + } else { + Commands.skipMessageMetadata(payload); + } int batchSize = metadata.getNumMessagesInBatch(); CompressionType compressionType = metadata.getCompression(); @@ -91,7 +95,16 @@ public static List> extractIdsAndKey RawMessage msg) throws IOException { List> idsAndKeysAndSize = new ArrayList<>(); - for (MessageCompactionData mcd : extractMessageCompactionData(msg)) { + for (MessageCompactionData mcd : extractMessageCompactionData(msg, null)) { + idsAndKeysAndSize.add(ImmutableTriple.of(mcd.messageId(), mcd.key(), mcd.payloadSize())); + } + return idsAndKeysAndSize; + } + + public static List> extractIdsAndKeysAndSize( + RawMessage msg, MessageMetadata metadata) throws IOException { + List> idsAndKeysAndSize = new ArrayList<>(); + for (MessageCompactionData mcd : extractMessageCompactionData(msg, metadata)) { idsAndKeysAndSize.add(ImmutableTriple.of(mcd.messageId(), mcd.key(), mcd.payloadSize())); } return idsAndKeysAndSize; @@ -99,7 +112,7 @@ public static List> extractIdsAndKey public static Optional rebatchMessage(RawMessage msg, BiPredicate filter) throws IOException { - return rebatchMessage(msg, filter, true); + return rebatchMessage(msg, null, filter, true); } /** @@ -109,6 +122,7 @@ public static Optional rebatchMessage(RawMessage msg, * NOTE: this message does not alter the reference count of the RawMessage argument. */ public static Optional rebatchMessage(RawMessage msg, + MessageMetadata metadata, BiPredicate filter, boolean retainNullKey) throws IOException { @@ -123,7 +137,11 @@ public static Optional rebatchMessage(RawMessage msg, payload.readerIndex(readerIndex); brokerMeta = payload.readSlice(brokerEntryMetadataSize + Short.BYTES + Integer.BYTES); } - MessageMetadata metadata = Commands.parseMessageMetadata(payload); + if (metadata == null) { + metadata = Commands.parseMessageMetadata(payload); + } else { + Commands.skipMessageMetadata(payload); + } ByteBuf batchBuffer = PulsarByteBufAllocator.DEFAULT.buffer(payload.capacity()); CompressionType compressionType = metadata.getCompression(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/AbstractTwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/AbstractTwoPhaseCompactor.java index 5b03f270251a0..ddfe8825a8888 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/AbstractTwoPhaseCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/AbstractTwoPhaseCompactor.java @@ -77,7 +77,7 @@ public AbstractTwoPhaseCompactor(ServiceConfiguration conf, protected abstract Map toLatestMessageIdForKey(Map latestForKey); protected abstract boolean compactMessage(String topic, Map latestForKey, - RawMessage m, MessageId id); + RawMessage m, MessageMetadata metadata, MessageId id); protected abstract boolean compactBatchMessage(String topic, Map latestForKey, @@ -147,7 +147,7 @@ private void phaseOneLoop(RawReader reader, } else if (RawBatchConverter.isReadableBatch(metadata)) { deletedMessage = compactBatchMessage(reader.getTopic(), latestForKey, m, metadata, id); } else { - deletedMessage = compactMessage(reader.getTopic(), latestForKey, m, id); + deletedMessage = compactMessage(reader.getTopic(), latestForKey, m, metadata, id); } MessageId first = firstMessageId.orElse(deletedMessage ? null : id); MessageId to = deletedMessage ? toMessageId.orElse(null) : id; @@ -239,7 +239,7 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map } else if (RawBatchConverter.isReadableBatch(metadata)) { try { messageToAdd = rebatchMessage(reader.getTopic(), - m, (key, subid) -> subid.equals(latestForKey.get(key)), + m, metadata, (key, subid) -> subid.equals(latestForKey.get(key)), topicCompactionRetainNullKey); } catch (IOException ioe) { log.info("Error decoding batch for message {}. Whole batch will be included in output", @@ -247,7 +247,7 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map messageToAdd = Optional.of(m); } } else { - Pair keyAndSize = extractKeyAndSize(m); + Pair keyAndSize = extractKeyAndSize(m, metadata); MessageId msg; if (keyAndSize == null) { messageToAdd = topicCompactionRetainNullKey ? Optional.of(m) : Optional.empty(); @@ -392,9 +392,8 @@ private CompletableFuture addToCompactedLedger(LedgerHandle lh, RawMessage return bkf; } - protected Pair extractKeyAndSize(RawMessage m) { + protected Pair extractKeyAndSize(RawMessage m, MessageMetadata msgMetadata) { ByteBuf headersAndPayload = m.getHeadersAndPayload(); - MessageMetadata msgMetadata = Commands.parseMessageMetadata(headersAndPayload); if (msgMetadata.hasPartitionKey()) { int size = headersAndPayload.readableBytes(); if (msgMetadata.hasUncompressedSize()) { @@ -408,13 +407,14 @@ protected Pair extractKeyAndSize(RawMessage m) { protected Optional rebatchMessage(String topic, RawMessage msg, + MessageMetadata metadata, BiPredicate filter, boolean retainNullKey) throws IOException { if (log.isDebugEnabled()) { log.debug("Rebatching message {} for topic {}", msg.getMessageId(), topic); } - return RawBatchConverter.rebatchMessage(msg, filter, retainNullKey); + return RawBatchConverter.rebatchMessage(msg, metadata, filter, retainNullKey); } protected static class PhaseOneResult { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/EventTimeOrderCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/EventTimeOrderCompactor.java index 2cd19ba15d608..db129b54533a8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/EventTimeOrderCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/EventTimeOrderCompactor.java @@ -34,7 +34,6 @@ import org.apache.pulsar.client.api.RawMessage; import org.apache.pulsar.client.impl.RawBatchConverter; import org.apache.pulsar.common.api.proto.MessageMetadata; -import org.apache.pulsar.common.protocol.Commands; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,10 +60,10 @@ protected Map toLatestMessageIdForKey( @Override protected boolean compactMessage(String topic, Map> latestForKey, - RawMessage m, MessageId id) { + RawMessage m, MessageMetadata metadata, MessageId id) { boolean deletedMessage = false; boolean replaceMessage = false; - MessageCompactionData mcd = extractMessageCompactionData(m); + MessageCompactionData mcd = extractMessageCompactionData(m, metadata); if (mcd != null) { boolean newer = Optional.ofNullable(latestForKey.get(mcd.key())) @@ -100,7 +99,7 @@ protected boolean compactBatchMessage(String topic, Map extractMessageCompactionDataFromBatch(RawMessage msg) + private List extractMessageCompactionDataFromBatch(RawMessage msg, MessageMetadata metadata) throws IOException { - return RawBatchConverter.extractMessageCompactionData(msg); + return RawBatchConverter.extractMessageCompactionData(msg, metadata); } } \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PublishingOrderCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PublishingOrderCompactor.java index a825c0782fbf9..223e8c421a5ef 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PublishingOrderCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/PublishingOrderCompactor.java @@ -53,10 +53,10 @@ protected Map toLatestMessageIdForKey(Map @Override protected boolean compactMessage(String topic, Map latestForKey, - RawMessage m, MessageId id) { + RawMessage m, MessageMetadata metadata, MessageId id) { boolean deletedMessage = false; boolean replaceMessage = false; - Pair keyAndSize = extractKeyAndSize(m); + Pair keyAndSize = extractKeyAndSize(m, metadata); if (keyAndSize != null) { if (keyAndSize.getRight() > 0) { MessageId old = latestForKey.put(keyAndSize.getLeft(), id); @@ -84,7 +84,7 @@ protected boolean compactBatchMessage(String topic, Map lates int numMessagesInBatch = metadata.getNumMessagesInBatch(); int deleteCnt = 0; for (ImmutableTriple e : extractIdsAndKeysAndSizeFromBatch( - m)) { + m, metadata)) { if (e != null) { if (e.getMiddle() == null) { if (!topicCompactionRetainNullKey) { @@ -119,9 +119,9 @@ protected boolean compactBatchMessage(String topic, Map lates } protected List> extractIdsAndKeysAndSizeFromBatch( - RawMessage msg) + RawMessage msg, MessageMetadata metadata) throws IOException { - return RawBatchConverter.extractIdsAndKeysAndSize(msg); + return RawBatchConverter.extractIdsAndKeysAndSize(msg, metadata); } } \ No newline at end of file From 9ebd97941d8ebe074423349d03307683e7fa3de2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 17 Sep 2024 13:11:32 +0300 Subject: [PATCH 503/580] [improve][broker] Decouple ManagedLedger interfaces from the current implementation (#23311) Co-authored-by: Yan Zhao Co-authored-by: Yong Zhang Co-authored-by: Yunze Xu --- .../bookkeeper/mledger/AsyncCallbacks.java | 3 +- .../bookkeeper/mledger/ManagedCursor.java | 15 + .../bookkeeper/mledger/ManagedLedger.java | 30 ++ .../mledger/ManagedLedgerFactory.java | 12 + .../bookkeeper/mledger/PositionBound.java | 24 + .../mledger/ReadOnlyManagedLedger.java | 33 ++ .../ReadOnlyManagedLedgerImplWrapper.java | 63 +++ .../mledger/impl/ManagedCursorImpl.java | 33 +- .../impl/ManagedLedgerFactoryImpl.java | 427 +++++++++++++++++- .../mledger/impl/ManagedLedgerImpl.java | 150 ++++-- .../impl/ManagedLedgerOfflineBacklog.java | 380 +--------------- .../bookkeeper/mledger/impl/OpAddEntry.java | 6 +- .../bookkeeper/mledger/impl/OpFindNewest.java | 4 +- .../bookkeeper/mledger/impl/OpScan.java | 3 +- .../impl/OpenTelemetryManagedCursorStats.java | 3 +- .../impl/OpenTelemetryManagedLedgerStats.java | 5 +- .../mledger/impl/ReadOnlyCursorImpl.java | 2 +- .../impl/ReadOnlyManagedLedgerImpl.java | 8 +- .../mledger/impl/ShadowManagedLedgerImpl.java | 3 +- .../intercept/ManagedLedgerInterceptor.java | 57 ++- .../impl/ManagedCursorContainerTest.java | 32 ++ .../impl/ManagedCursorPropertiesTest.java | 2 +- .../mledger/impl/ManagedLedgerTest.java | 17 +- .../impl/ReadOnlyManagedLedgerImplTest.java | 5 +- .../apache/pulsar/broker/PulsarService.java | 3 + .../admin/impl/PersistentTopicsBase.java | 12 +- .../bucket/BucketDelayedDeliveryTracker.java | 2 +- .../ManagedLedgerInterceptorImpl.java | 62 +-- .../pulsar/broker/service/AbstractTopic.java | 3 + .../broker/service/BacklogQuotaManager.java | 29 +- .../pulsar/broker/service/BrokerService.java | 3 + .../pulsar/broker/service/ServerCnx.java | 8 +- .../PersistentMessageExpiryMonitor.java | 46 +- ...tStickyKeyDispatcherMultipleConsumers.java | 6 +- .../persistent/PersistentSubscription.java | 55 ++- .../service/persistent/PersistentTopic.java | 395 +++++++--------- .../broker/stats/metrics/AbstractMetrics.java | 13 +- .../stats/metrics/ManagedCursorMetrics.java | 13 +- .../stats/metrics/ManagedLedgerMetrics.java | 21 +- ...SingleSnapshotAbortedTxnProcessorImpl.java | 5 +- ...napshotSegmentAbortedTxnProcessorImpl.java | 19 +- .../buffer/impl/TopicTransactionBuffer.java | 5 +- .../pendingack/impl/MLPendingAckStore.java | 6 +- .../impl/MLPendingAckStoreProvider.java | 7 +- .../pendingack/impl/PendingAckHandleImpl.java | 13 +- .../pulsar/compaction/CompactedTopicImpl.java | 6 +- .../compaction/CompactedTopicUtils.java | 6 +- .../broker/delayed/MockManagedCursor.java | 31 ++ .../MangedLedgerInterceptorImplTest.java | 6 +- .../CurrentLedgerRolloverIfFullTest.java | 4 +- .../persistent/BucketDelayedDeliveryTest.java | 19 +- .../stats/ManagedLedgerMetricsTest.java | 6 +- .../TopicTransactionBufferRecoverTest.java | 4 +- .../broker/transaction/TransactionTest.java | 3 +- .../pendingack/PendingAckMetadataTest.java | 24 +- .../data/ManagedLedgerInternalStats.java | 5 + .../MLTransactionSequenceIdGenerator.java | 58 +-- .../jcloud/impl/MockManagedLedger.java | 53 +++ 58 files changed, 1359 insertions(+), 909 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/PositionBound.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyManagedLedger.java create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyManagedLedgerImplWrapper.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/AsyncCallbacks.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/AsyncCallbacks.java index dcf2c225e8b35..70db427afce4f 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/AsyncCallbacks.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/AsyncCallbacks.java @@ -24,7 +24,6 @@ import java.util.Optional; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; -import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; /** * Definition of all the callbacks used for the ManagedLedger asynchronous API. @@ -48,7 +47,7 @@ interface OpenReadOnlyCursorCallback { } interface OpenReadOnlyManagedLedgerCallback { - void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl managedLedger, Object ctx); + void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedger managedLedger, Object ctx); void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java index f6345e7b9ec5b..042e03998696c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java @@ -34,6 +34,7 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.SkipEntriesCallback; +import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; /** * A ManagedCursor is a persisted cursor inside a ManagedLedger. @@ -45,6 +46,8 @@ @InterfaceStability.Stable public interface ManagedCursor { + String CURSOR_INTERNAL_PROPERTY_PREFIX = "#pulsar.internal."; + @SuppressWarnings("checkstyle:javadoctype") enum FindPositionConstraint { SearchActiveEntries, SearchAllAvailableEntries @@ -885,4 +888,16 @@ default boolean periodicRollover() { default ManagedCursorAttributes getManagedCursorAttributes() { return new ManagedCursorAttributes(this); } + + ManagedLedgerInternalStats.CursorStats getCursorStats(); + + boolean isMessageDeleted(Position position); + + ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) throws ManagedLedgerException; + + long[] getBatchPositionAckSet(Position position); + + int applyMaxSizeCap(int maxEntries, long maxSizeBytes); + + void updateReadStats(int readEntriesCount, long readEntriesSize); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java index a9242d5cc65b4..de69d97bb79aa 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java @@ -18,8 +18,10 @@ */ package org.apache.bookkeeper.mledger; +import com.google.common.collect.Range; import io.netty.buffer.ByteBuf; import java.util.Map; +import java.util.NavigableMap; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.function.Predicate; @@ -374,6 +376,8 @@ void asyncOpenCursor(String name, InitialPosition initialPosition, Map range); + /** * Get the total number of active entries for this managed ledger. * @@ -703,4 +707,30 @@ default void skipNonRecoverableLedger(long ledgerId){} default ManagedLedgerAttributes getManagedLedgerAttributes() { return new ManagedLedgerAttributes(this); } + + void asyncReadEntry(Position position, AsyncCallbacks.ReadEntryCallback callback, Object ctx); + + /** + * Get all the managed ledgers. + */ + NavigableMap getLedgersInfo(); + + Position getNextValidPosition(Position position); + + Position getPreviousPosition(Position position); + + long getEstimatedBacklogSize(Position position); + + Position getPositionAfterN(Position startPosition, long n, PositionBound startRange); + + int getPendingAddEntriesCount(); + + long getCacheSize(); + + default CompletableFuture getLastDispatchablePosition(final Predicate predicate, + final Position startPosition) { + return CompletableFuture.completedFuture(PositionFactory.EARLIEST); + } + + Position getFirstPosition(); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java index e09fd84ea55f2..d9c887fac468e 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java @@ -28,6 +28,8 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenLedgerCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenReadOnlyCursorCallback; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; /** * A factory to open/create managed ledgers and delete them. @@ -233,4 +235,14 @@ void asyncDelete(String name, CompletableFuture mlConfigFut * @return properties of this managedLedger. */ CompletableFuture> getManagedLedgerPropertiesAsync(String name); + + Map getManagedLedgers(); + + ManagedLedgerFactoryMXBean getCacheStats(); + + + void estimateUnloadedTopicBacklog(PersistentOfflineTopicStats offlineTopicStats, TopicName topicName, + boolean accurate, Object ctx) throws Exception; + + ManagedLedgerFactoryConfig getConfig(); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/PositionBound.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/PositionBound.java new file mode 100644 index 0000000000000..a7ab4a48a9b02 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/PositionBound.java @@ -0,0 +1,24 @@ +/* + * 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.bookkeeper.mledger; + +public enum PositionBound { + // define boundaries for position based seeks and searches + startIncluded, startExcluded +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyManagedLedger.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyManagedLedger.java new file mode 100644 index 0000000000000..91b8f92eb637e --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyManagedLedger.java @@ -0,0 +1,33 @@ +/* + * 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.bookkeeper.mledger; + +import java.util.Map; + +public interface ReadOnlyManagedLedger { + + void asyncReadEntry(Position position, AsyncCallbacks.ReadEntryCallback callback, Object ctx); + + long getNumberOfEntries(); + + ReadOnlyCursor createReadOnlyCursor(Position position); + + Map getProperties(); + +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyManagedLedgerImplWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyManagedLedgerImplWrapper.java new file mode 100644 index 0000000000000..5bc94c04beefd --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadOnlyManagedLedgerImplWrapper.java @@ -0,0 +1,63 @@ +/* + * 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.bookkeeper.mledger; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; +import org.apache.bookkeeper.mledger.impl.MetaStore; +import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; + +public class ReadOnlyManagedLedgerImplWrapper implements ReadOnlyManagedLedger { + + private final ReadOnlyManagedLedgerImpl readOnlyManagedLedger; + + public ReadOnlyManagedLedgerImplWrapper(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, + ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, + String name) { + this.readOnlyManagedLedger = + new ReadOnlyManagedLedgerImpl(factory, bookKeeper, store, config, scheduledExecutor, name); + } + + public CompletableFuture initialize() { + return readOnlyManagedLedger.initialize(); + } + + @Override + public void asyncReadEntry(Position position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { + readOnlyManagedLedger.asyncReadEntry(position, callback, ctx); + } + + @Override + public long getNumberOfEntries() { + return readOnlyManagedLedger.getNumberOfEntries(); + } + + @Override + public ReadOnlyCursor createReadOnlyCursor(Position position) { + return readOnlyManagedLedger.createReadOnlyCursor(position); + } + + @Override + public Map getProperties() { + return readOnlyManagedLedger.getProperties(); + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index e808c31bc89f1..e27814eadd0b5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -86,9 +86,9 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.MetaStoreException; import org.apache.bookkeeper.mledger.ManagedLedgerException.NoMoreEntriesToReadException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionBound; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ScanOutcome; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongProperty; @@ -98,6 +98,8 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.StringProperty; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; +import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.common.util.collections.LongPairRangeSet; @@ -124,8 +126,6 @@ public class ManagedCursorImpl implements ManagedCursor { protected final ManagedLedgerImpl ledger; private final String name; - public static final String CURSOR_INTERNAL_PROPERTY_PREFIX = "#pulsar.internal."; - private volatile Map cursorProperties; private final BookKeeper.DigestType digestType; @@ -1769,7 +1769,7 @@ public void asyncSkipEntries(int numEntriesToSkip, IndividualDeletedEntries dele } asyncMarkDelete(ledger.getPositionAfterN(markDeletePosition, numEntriesToSkip + numDeletedMessages, - ManagedLedgerImpl.PositionBound.startExcluded), new MarkDeleteCallback() { + PositionBound.startExcluded), new MarkDeleteCallback() { @Override public void markDeleteComplete(Object ctx) { callback.skipEntriesComplete(ctx); @@ -3485,6 +3485,7 @@ public Position processIndividuallyDeletedMessagesAndGetMarkDeletedPosition( return mdp; } + @Override public boolean isMessageDeleted(Position position) { lock.readLock().lock(); try { @@ -3496,6 +3497,7 @@ public boolean isMessageDeleted(Position position) { } //this method will return a copy of the position's ack set + @Override public long[] getBatchPositionAckSet(Position position) { if (batchDeletedIndexes != null) { BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.get(position); @@ -3618,6 +3620,7 @@ public ManagedCursorMXBean getStats() { return this.mbean; } + @Override public void updateReadStats(int readEntriesCount, long readEntriesSize) { this.entriesReadCount += readEntriesCount; this.entriesReadSize += readEntriesSize; @@ -3650,6 +3653,7 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { }, null); } + @Override public int applyMaxSizeCap(int maxEntries, long maxSizeBytes) { if (maxSizeBytes == NO_MAX_SIZE_LIMIT) { return maxEntries; @@ -3712,6 +3716,7 @@ public ManagedLedgerConfig getConfig() { /*** * Create a non-durable cursor and copy the ack stats. */ + @Override public ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) throws ManagedLedgerException { NonDurableCursorImpl newNonDurableCursor = (NonDurableCursorImpl) ledger.newNonDurableCursor(getMarkDeletedPosition(), nonDurableCursorName); @@ -3746,4 +3751,24 @@ public ManagedCursorAttributes getManagedCursorAttributes() { } return ATTRIBUTES_UPDATER.updateAndGet(this, old -> old != null ? old : new ManagedCursorAttributes(this)); } + + @Override + public ManagedLedgerInternalStats.CursorStats getCursorStats() { + ManagedLedgerInternalStats.CursorStats cs = new ManagedLedgerInternalStats.CursorStats(); + cs.markDeletePosition = getMarkDeletedPosition().toString(); + cs.readPosition = getReadPosition().toString(); + cs.waitingReadOp = hasPendingReadRequest(); + cs.pendingReadOps = getPendingReadOpsCount(); + cs.messagesConsumedCounter = getMessagesConsumedCounter(); + cs.cursorLedger = getCursorLedger(); + cs.cursorLedgerLastEntry = getCursorLedgerLastEntry(); + cs.individuallyDeletedMessages = getIndividuallyDeletedMessages(); + cs.lastLedgerSwitchTimestamp = DateFormatter.format(getLastLedgerSwitchTimestamp()); + cs.state = getState(); + cs.active = isActive(); + cs.numberOfEntriesSinceFirstNotAckedMessage = getNumberOfEntriesSinceFirstNotAckedMessage(); + cs.totalNonContiguousDeletedMessagesRange = getTotalNonContiguousDeletedMessagesRange(); + cs.properties = getProperties(); + return cs; + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 8ba800ff88130..586beb412d297 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -22,14 +22,19 @@ import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.NULL_OFFLOAD_PROMISE; import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; import com.google.common.base.Predicates; +import com.google.common.collect.BoundType; import com.google.common.collect.Maps; +import com.google.common.collect.Range; +import com.google.protobuf.InvalidProtocolBufferException; import io.netty.util.concurrent.DefaultThreadFactory; import io.opentelemetry.api.OpenTelemetry; import java.io.IOException; import java.util.ArrayList; +import java.util.Enumeration; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.NavigableMap; import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -45,8 +50,12 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import lombok.Getter; +import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.LedgerEntry; +import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.mledger.AsyncCallbacks; @@ -70,7 +79,10 @@ import org.apache.bookkeeper.mledger.MetadataCompressionConfig; import org.apache.bookkeeper.mledger.OpenTelemetryManagedLedgerCacheStats; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ReadOnlyCursor; +import org.apache.bookkeeper.mledger.ReadOnlyManagedLedger; +import org.apache.bookkeeper.mledger.ReadOnlyManagedLedgerImplWrapper; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.ManagedLedgerInitializeLedgerCallback; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.State; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; @@ -81,14 +93,18 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongProperty; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange; +import org.apache.bookkeeper.mledger.util.Errors; import org.apache.bookkeeper.mledger.util.Futures; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.EnsemblePlacementPolicyConfig; +import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.Runnables; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.Stat; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @@ -301,7 +317,8 @@ private synchronized void doCacheEviction() { * * @return */ - public Map getManagedLedgers() { + @Override + public Map getManagedLedgers() { // Return a view of already created ledger by filtering futures not yet completed return Maps.filterValues(Maps.transformValues(ledgers, future -> future.getNow(null)), Predicates.notNull()); } @@ -468,7 +485,7 @@ public void asyncOpenReadOnlyManagedLedger(String managedLedgerName, .get(new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), config.getBookKeeperEnsemblePlacementPolicyProperties())) .thenCompose(bk -> { - ReadOnlyManagedLedgerImpl roManagedLedger = new ReadOnlyManagedLedgerImpl(this, bk, + ReadOnlyManagedLedgerImplWrapper roManagedLedger = new ReadOnlyManagedLedgerImplWrapper(this, bk, store, config, scheduledExecutor, managedLedgerName); return roManagedLedger.initialize().thenApply(v -> roManagedLedger); }).thenAccept(roManagedLedger -> { @@ -524,7 +541,7 @@ public void asyncOpenReadOnlyCursor(String managedLedgerName, Position startPosi AsyncCallbacks.OpenReadOnlyManagedLedgerCallback openReadOnlyManagedLedgerCallback = new AsyncCallbacks.OpenReadOnlyManagedLedgerCallback() { @Override - public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl readOnlyManagedLedger, Object ctx) { + public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedger readOnlyManagedLedger, Object ctx) { callback.openReadOnlyCursorComplete(readOnlyManagedLedger. createReadOnlyCursor(startPosition), ctx); } @@ -1041,6 +1058,7 @@ public MetaStore getMetaStore() { return store; } + @Override public ManagedLedgerFactoryConfig getConfig() { return config; } @@ -1060,6 +1078,7 @@ public long getCacheEvictionTimeThreshold(){ return cacheEvictionTimeThresholdNanos; } + @Override public ManagedLedgerFactoryMXBean getCacheStats() { return this.mbean; } @@ -1068,6 +1087,408 @@ public CompletableFuture getBookKeeper() { return bookkeeperFactory.get(); } + @Override + public void estimateUnloadedTopicBacklog(PersistentOfflineTopicStats offlineTopicStats, + TopicName topicName, boolean accurate, Object ctx) + throws Exception { + String managedLedgerName = topicName.getPersistenceNamingEncoding(); + long numberOfEntries = 0; + long totalSize = 0; + BookKeeper.DigestType digestType = (BookKeeper.DigestType) ((List) ctx).get(0); + byte[] password = (byte[]) ((List) ctx).get(1); + NavigableMap ledgers = + getManagedLedgersInfo(topicName, accurate, digestType, password); + for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : ledgers.values()) { + numberOfEntries += ls.getEntries(); + totalSize += ls.getSize(); + if (accurate) { + offlineTopicStats.addLedgerDetails(ls.getEntries(), ls.getTimestamp(), ls.getSize(), ls.getLedgerId()); + } + } + offlineTopicStats.totalMessages = numberOfEntries; + offlineTopicStats.storageSize = totalSize; + if (log.isDebugEnabled()) { + log.debug("[{}] Total number of entries - {} and size - {}", managedLedgerName, numberOfEntries, totalSize); + } + + // calculate per cursor message backlog + calculateCursorBacklogs(topicName, ledgers, offlineTopicStats, accurate, digestType, password); + offlineTopicStats.statGeneratedAt.setTime(System.currentTimeMillis()); + } + + private NavigableMap getManagedLedgersInfo( + final TopicName topicName, boolean accurate, BookKeeper.DigestType digestType, byte[] password) + throws Exception { + final NavigableMap ledgers = new ConcurrentSkipListMap<>(); + + String managedLedgerName = topicName.getPersistenceNamingEncoding(); + MetaStore store = getMetaStore(); + + final CountDownLatch mlMetaCounter = new CountDownLatch(1); + store.getManagedLedgerInfo(managedLedgerName, false /* createIfMissing */, + new MetaStore.MetaStoreCallback() { + @Override + public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) { + for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : mlInfo.getLedgerInfoList()) { + ledgers.put(ls.getLedgerId(), ls); + } + + // find no of entries in last ledger + if (!ledgers.isEmpty()) { + final long id = ledgers.lastKey(); + AsyncCallback.OpenCallback opencb = (rc, lh, ctx1) -> { + if (log.isDebugEnabled()) { + log.debug("[{}] Opened ledger {}: {}", managedLedgerName, id, + BKException.getMessage(rc)); + } + if (rc == BKException.Code.OK) { + MLDataFormats.ManagedLedgerInfo.LedgerInfo info = + MLDataFormats.ManagedLedgerInfo.LedgerInfo + .newBuilder().setLedgerId(id) + .setEntries(lh.getLastAddConfirmed() + 1) + .setSize(lh.getLength()).setTimestamp(System.currentTimeMillis()) + .build(); + ledgers.put(id, info); + mlMetaCounter.countDown(); + } else if (Errors.isNoSuchLedgerExistsException(rc)) { + log.warn("[{}] Ledger not found: {}", managedLedgerName, ledgers.lastKey()); + ledgers.remove(ledgers.lastKey()); + mlMetaCounter.countDown(); + } else { + log.error("[{}] Failed to open ledger {}: {}", managedLedgerName, id, + BKException.getMessage(rc)); + mlMetaCounter.countDown(); + } + }; + + if (log.isDebugEnabled()) { + log.debug("[{}] Opening ledger {}", managedLedgerName, id); + } + getBookKeeper() + .thenAccept(bk -> { + bk.asyncOpenLedgerNoRecovery(id, digestType, password, opencb, null); + }).exceptionally(ex -> { + log.warn("[{}] Failed to open ledger {}: {}", managedLedgerName, id, ex); + opencb.openComplete(-1, null, null); + mlMetaCounter.countDown(); + return null; + }); + } else { + log.warn("[{}] Ledger list empty", managedLedgerName); + mlMetaCounter.countDown(); + } + } + + @Override + public void operationFailed(ManagedLedgerException.MetaStoreException e) { + log.warn("[{}] Unable to obtain managed ledger metadata - {}", managedLedgerName, e); + mlMetaCounter.countDown(); + } + }); + + if (accurate) { + // block until however long it takes for operation to complete + mlMetaCounter.await(); + } else { + mlMetaCounter.await(META_READ_TIMEOUT_SECONDS, TimeUnit.SECONDS); + } + return ledgers; + } + + public void calculateCursorBacklogs(final TopicName topicName, + final NavigableMap ledgers, + final PersistentOfflineTopicStats offlineTopicStats, boolean accurate, + BookKeeper.DigestType digestType, byte[] password) throws Exception { + if (ledgers.isEmpty()) { + return; + } + String managedLedgerName = topicName.getPersistenceNamingEncoding(); + MetaStore store = getMetaStore(); + BookKeeper bk = getBookKeeper().get(); + final CountDownLatch allCursorsCounter = new CountDownLatch(1); + final long errorInReadingCursor = -1; + ConcurrentOpenHashMap ledgerRetryMap = + ConcurrentOpenHashMap.newBuilder().build(); + + final MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo = ledgers.lastEntry().getValue(); + final Position lastLedgerPosition = + PositionFactory.create(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1); + if (log.isDebugEnabled()) { + log.debug("[{}] Last ledger position {}", managedLedgerName, lastLedgerPosition); + } + + store.getCursors(managedLedgerName, new MetaStore.MetaStoreCallback>() { + @Override + public void operationComplete(List cursors, Stat v) { + // Load existing cursors + if (log.isDebugEnabled()) { + log.debug("[{}] Found {} cursors", managedLedgerName, cursors.size()); + } + + if (cursors.isEmpty()) { + allCursorsCounter.countDown(); + return; + } + + final CountDownLatch cursorCounter = new CountDownLatch(cursors.size()); + + for (final String cursorName : cursors) { + // determine subscription position from cursor ledger + if (log.isDebugEnabled()) { + log.debug("[{}] Loading cursor {}", managedLedgerName, cursorName); + } + + AsyncCallback.OpenCallback cursorLedgerOpenCb = (rc, lh, ctx1) -> { + long ledgerId = lh.getId(); + if (log.isDebugEnabled()) { + log.debug("[{}] Opened cursor ledger {} for cursor {}. rc={}", managedLedgerName, ledgerId, + cursorName, rc); + } + if (rc != BKException.Code.OK) { + log.warn("[{}] Error opening metadata ledger {} for cursor {}: {}", managedLedgerName, + ledgerId, cursorName, BKException.getMessage(rc)); + cursorCounter.countDown(); + return; + } + long lac = lh.getLastAddConfirmed(); + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor {} LAC {} read from ledger {}", managedLedgerName, cursorName, lac, + ledgerId); + } + + if (lac == LedgerHandle.INVALID_ENTRY_ID) { + // save the ledger id and cursor to retry outside of this call back + // since we are trying to read the same cursor ledger, we will block until + // this current callback completes, since an attempt to read the entry + // will block behind this current operation to complete + ledgerRetryMap.put(cursorName, ledgerId); + log.info("[{}] Cursor {} LAC {} read from ledger {}", managedLedgerName, cursorName, lac, + ledgerId); + cursorCounter.countDown(); + return; + } + final long entryId = lac; + // read last acked message position for subscription + lh.asyncReadEntries(entryId, entryId, new AsyncCallback.ReadCallback() { + @Override + public void readComplete(int rc, LedgerHandle lh, Enumeration seq, + Object ctx) { + try { + if (log.isDebugEnabled()) { + log.debug("readComplete rc={} entryId={}", rc, entryId); + } + if (rc != BKException.Code.OK) { + log.warn("[{}] Error reading from metadata ledger {} for cursor {}: {}", + managedLedgerName, ledgerId, cursorName, BKException.getMessage(rc)); + // indicate that this cursor should be excluded + offlineTopicStats.addCursorDetails(cursorName, errorInReadingCursor, + lh.getId()); + } else { + LedgerEntry entry = seq.nextElement(); + MLDataFormats.PositionInfo positionInfo; + try { + positionInfo = MLDataFormats.PositionInfo.parseFrom(entry.getEntry()); + } catch (InvalidProtocolBufferException e) { + log.warn( + "[{}] Error reading position from metadata ledger {} for cursor " + + "{}: {}", managedLedgerName, ledgerId, cursorName, e); + offlineTopicStats.addCursorDetails(cursorName, errorInReadingCursor, + lh.getId()); + return; + } + final Position lastAckedMessagePosition = + PositionFactory.create(positionInfo.getLedgerId(), + positionInfo.getEntryId()); + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor {} MD {} read last ledger position {}", + managedLedgerName, cursorName, lastAckedMessagePosition, + lastLedgerPosition); + } + // calculate cursor backlog + Range range = Range.openClosed(lastAckedMessagePosition, + lastLedgerPosition); + if (log.isDebugEnabled()) { + log.debug("[{}] Calculating backlog for cursor {} using range {}", + managedLedgerName, cursorName, range); + } + long cursorBacklog = getNumberOfEntries(range, ledgers); + offlineTopicStats.messageBacklog += cursorBacklog; + offlineTopicStats.addCursorDetails(cursorName, cursorBacklog, lh.getId()); + } + } finally { + cursorCounter.countDown(); + } + } + }, null); + + }; // end of cursor meta read callback + + store.asyncGetCursorInfo(managedLedgerName, cursorName, + new MetaStore.MetaStoreCallback() { + @Override + public void operationComplete(MLDataFormats.ManagedCursorInfo info, + Stat stat) { + long cursorLedgerId = info.getCursorsLedgerId(); + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor {} meta-data read ledger id {}", managedLedgerName, + cursorName, cursorLedgerId); + } + if (cursorLedgerId != -1) { + bk.asyncOpenLedgerNoRecovery(cursorLedgerId, digestType, password, + cursorLedgerOpenCb, null); + } else { + Position lastAckedMessagePosition = PositionFactory.create( + info.getMarkDeleteLedgerId(), info.getMarkDeleteEntryId()); + Range range = Range.openClosed(lastAckedMessagePosition, + lastLedgerPosition); + if (log.isDebugEnabled()) { + log.debug("[{}] Calculating backlog for cursor {} using range {}", + managedLedgerName, cursorName, range); + } + long cursorBacklog = getNumberOfEntries(range, ledgers); + offlineTopicStats.messageBacklog += cursorBacklog; + offlineTopicStats.addCursorDetails(cursorName, cursorBacklog, cursorLedgerId); + cursorCounter.countDown(); + } + + } + + @Override + public void operationFailed(ManagedLedgerException.MetaStoreException e) { + log.warn("[{}] Unable to obtain cursor ledger for cursor {}: {}", managedLedgerName, + cursorName, e); + cursorCounter.countDown(); + } + }); + } // for every cursor find backlog + try { + if (accurate) { + cursorCounter.await(); + } else { + cursorCounter.await(META_READ_TIMEOUT_SECONDS, TimeUnit.SECONDS); + } + } catch (Exception e) { + log.warn("[{}] Error reading subscription positions{}", managedLedgerName, e); + } finally { + allCursorsCounter.countDown(); + } + } + + @Override + public void operationFailed(ManagedLedgerException.MetaStoreException e) { + log.warn("[{}] Failed to get the cursors list", managedLedgerName, e); + allCursorsCounter.countDown(); + } + }); + if (accurate) { + allCursorsCounter.await(); + } else { + allCursorsCounter.await(META_READ_TIMEOUT_SECONDS, TimeUnit.SECONDS); + } + + // go through ledgers where LAC was -1 + if (accurate && ledgerRetryMap.size() > 0) { + ledgerRetryMap.forEach((cursorName, ledgerId) -> { + if (log.isDebugEnabled()) { + log.debug("Cursor {} Ledger {} Trying to obtain MD from BkAdmin", cursorName, ledgerId); + } + Position lastAckedMessagePosition = tryGetMDPosition(bk, ledgerId, cursorName); + if (lastAckedMessagePosition == null) { + log.warn("[{}] Cursor {} read from ledger {}. Unable to determine cursor position", + managedLedgerName, cursorName, ledgerId); + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor {} read from ledger using bk admin {}. position {}", managedLedgerName, + cursorName, ledgerId, lastAckedMessagePosition); + } + // calculate cursor backlog + Range range = Range.openClosed(lastAckedMessagePosition, lastLedgerPosition); + if (log.isDebugEnabled()) { + log.debug("[{}] Calculating backlog for cursor {} using range {}", managedLedgerName, + cursorName, range); + } + long cursorBacklog = getNumberOfEntries(range, ledgers); + offlineTopicStats.messageBacklog += cursorBacklog; + offlineTopicStats.addCursorDetails(cursorName, cursorBacklog, ledgerId); + } + }); + } + } + + // need a better way than to duplicate the functionality below from ML + private long getNumberOfEntries(Range range, + NavigableMap ledgers) { + Position fromPosition = range.lowerEndpoint(); + boolean fromIncluded = range.lowerBoundType() == BoundType.CLOSED; + Position toPosition = range.upperEndpoint(); + boolean toIncluded = range.upperBoundType() == BoundType.CLOSED; + + if (fromPosition.getLedgerId() == toPosition.getLedgerId()) { + // If the 2 positions are in the same ledger + long count = toPosition.getEntryId() - fromPosition.getEntryId() - 1; + count += fromIncluded ? 1 : 0; + count += toIncluded ? 1 : 0; + return count; + } else { + long count = 0; + // If the from & to are pointing to different ledgers, then we need to : + // 1. Add the entries in the ledger pointed by toPosition + count += toPosition.getEntryId(); + count += toIncluded ? 1 : 0; + + // 2. Add the entries in the ledger pointed by fromPosition + MLDataFormats.ManagedLedgerInfo.LedgerInfo li = ledgers.get(fromPosition.getLedgerId()); + if (li != null) { + count += li.getEntries() - (fromPosition.getEntryId() + 1); + count += fromIncluded ? 1 : 0; + } + + // 3. Add the whole ledgers entries in between + for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : ledgers + .subMap(fromPosition.getLedgerId(), false, toPosition.getLedgerId(), false).values()) { + count += ls.getEntries(); + } + + return count; + } + } + + + private Position tryGetMDPosition(BookKeeper bookKeeper, long ledgerId, String cursorName) { + BookKeeperAdmin bookKeeperAdmin = null; + long lastEntry = LedgerHandle.INVALID_ENTRY_ID; + Position lastAckedMessagePosition = null; + try { + bookKeeperAdmin = new BookKeeperAdmin(bookKeeper); + for (LedgerEntry ledgerEntry : bookKeeperAdmin.readEntries(ledgerId, 0, lastEntry)) { + lastEntry = ledgerEntry.getEntryId(); + if (log.isDebugEnabled()) { + log.debug(" Read entry {} from ledger {} for cursor {}", lastEntry, ledgerId, cursorName); + } + MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(ledgerEntry.getEntry()); + lastAckedMessagePosition = + PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId()); + if (log.isDebugEnabled()) { + log.debug("Cursor {} read position {}", cursorName, lastAckedMessagePosition); + } + } + } catch (Exception e) { + log.warn("Unable to determine LAC for ledgerId {} for cursor {}: {}", ledgerId, cursorName, e); + } finally { + if (bookKeeperAdmin != null) { + try { + bookKeeperAdmin.close(); + } catch (Exception e) { + log.warn("Unable to close bk admin for ledgerId {} for cursor {}", ledgerId, cursorName, e); + } + } + + } + return lastAckedMessagePosition; + } + + private static final int META_READ_TIMEOUT_SECONDS = 60; + /** * Factory to create Bookkeeper-client for a given ensemblePlacementPolicy. * diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 8cb5a3ee6acec..cb19bd94bce01 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -34,6 +34,7 @@ import java.io.IOException; import java.time.Clock; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -77,10 +78,13 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.client.api.LedgerEntry; +import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.common.util.Backoff; import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.common.util.Retries; +import org.apache.bookkeeper.discover.RegistrationClient; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; @@ -115,6 +119,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.ManagedLedgerMXBean; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionBound; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.WaitingEntryCallBack; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.VoidCallback; @@ -129,6 +134,7 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.OffloadContext; import org.apache.bookkeeper.mledger.util.CallbackMutex; import org.apache.bookkeeper.mledger.util.Futures; +import org.apache.bookkeeper.mledger.util.ManagedLedgerImplUtils; import org.apache.bookkeeper.net.BookieId; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition; @@ -286,10 +292,7 @@ public boolean isFenced() { } } - // define boundaries for position based seeks and searches - public enum PositionBound { - startIncluded, startExcluded - } + protected static final AtomicReferenceFieldUpdater STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(ManagedLedgerImpl.class, State.class, "state"); @@ -429,13 +432,14 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { .setTimestamp(clock.millis()).build(); ledgers.put(id, info); if (managedLedgerInterceptor != null) { - managedLedgerInterceptor.onManagedLedgerLastLedgerInitialize(name, lh) - .thenRun(() -> initializeBookKeeper(callback)) - .exceptionally(ex -> { - callback.initializeFailed( - new ManagedLedgerInterceptException(ex.getCause())); - return null; - }); + managedLedgerInterceptor + .onManagedLedgerLastLedgerInitialize(name, createLastEntryHandle(lh)) + .thenRun(() -> initializeBookKeeper(callback)) + .exceptionally(ex -> { + callback.initializeFailed( + new ManagedLedgerInterceptException(ex.getCause())); + return null; + }); } else { initializeBookKeeper(callback); } @@ -475,6 +479,42 @@ public void operationFailed(MetaStoreException e) { scheduleTimeoutTask(); } + protected ManagedLedgerInterceptor.LastEntryHandle createLastEntryHandle(LedgerHandle lh) { + return () -> { + CompletableFuture> promise = new CompletableFuture<>(); + if (lh.getLastAddConfirmed() >= 0) { + lh.readAsync(lh.getLastAddConfirmed(), lh.getLastAddConfirmed()) + .whenComplete((entries, ex) -> { + if (ex != null) { + promise.completeExceptionally(ex); + } else { + if (entries != null) { + try { + LedgerEntry ledgerEntry = + entries.getEntry(lh.getLastAddConfirmed()); + if (ledgerEntry != null) { + promise.complete( + Optional.of(EntryImpl.create(ledgerEntry))); + } else { + promise.complete(Optional.empty()); + } + entries.close(); + } catch (Exception e) { + entries.close(); + promise.completeExceptionally(e); + } + } else { + promise.complete(Optional.empty()); + } + } + }); + } else { + promise.complete(Optional.empty()); + } + return promise; + }; + } + protected synchronized void initializeBookKeeper(final ManagedLedgerInitializeLedgerCallback callback) { if (log.isDebugEnabled()) { log.debug("[{}] initializing bookkeeper; ledgers {}", name, ledgers); @@ -1207,7 +1247,7 @@ public CompletableFuture getEarliestMessagePublishTimeInBacklog() { return getEarliestMessagePublishTimeOfPos(pos); } - public CompletableFuture getEarliestMessagePublishTimeOfPos(Position pos) { + private CompletableFuture getEarliestMessagePublishTimeOfPos(Position pos) { CompletableFuture future = new CompletableFuture<>(); if (pos == null) { future.complete(0L); @@ -1881,12 +1921,12 @@ void asyncReadEntries(OpReadEntry opReadEntry) { } } - public CompletableFuture getLedgerMetadata(long ledgerId) { + public CompletableFuture getLedgerMetadata(long ledgerId) { LedgerHandle currentLedger = this.currentLedger; if (currentLedger != null && ledgerId == currentLedger.getId()) { - return CompletableFuture.completedFuture(currentLedger.getLedgerMetadata().toSafeString()); + return CompletableFuture.completedFuture(currentLedger.getLedgerMetadata()); } else { - return getLedgerHandle(ledgerId).thenApply(rh -> rh.getLedgerMetadata().toSafeString()); + return getLedgerHandle(ledgerId).thenApply(rh -> rh.getLedgerMetadata()); } } @@ -1992,6 +2032,7 @@ public void invalidateLedgerHandle(ReadHandle ledgerHandle) { } } + @Override public void asyncReadEntry(Position position, ReadEntryCallback callback, Object ctx) { LedgerHandle currentLedger = this.currentLedger; if (log.isDebugEnabled()) { @@ -3511,6 +3552,7 @@ public long getNumberOfEntries(Range range) { * specifies whether to include the start position in calculating the distance * @return the new position that is n entries ahead */ + @Override public Position getPositionAfterN(final Position startPosition, long n, PositionBound startRange) { long entriesToSkip = n; long currentLedgerId; @@ -3609,6 +3651,7 @@ private boolean isNoMessagesAfterPosForSpecifiedLac(Position specifiedLac, Posit * the current position * @return the previous position */ + @Override public Position getPreviousPosition(Position position) { if (position.getEntryId() > 0) { return PositionFactory.create(position.getLedgerId(), position.getEntryId() - 1); @@ -3687,9 +3730,11 @@ public Long getNextValidLedger(long ledgerId) { return ledgers.ceilingKey(ledgerId + 1); } + @Override public Position getNextValidPosition(final Position position) { return getValidPositionAfterSkippedEntries(position, 1); } + public Position getValidPositionAfterSkippedEntries(final Position position, int skippedEntryNum) { Position skippedPosition = position.getPositionAfterEntries(skippedEntryNum); while (!isValidPosition(skippedPosition)) { @@ -3839,6 +3884,7 @@ public List getLedgersInfoAsList() { return Lists.newArrayList(ledgers.values()); } + @Override public NavigableMap getLedgersInfo() { return ledgers; } @@ -3952,6 +3998,7 @@ public int getWaitingCursorsCount() { return waitingCursors.size(); } + @Override public int getPendingAddEntriesCount() { return pendingAddEntries.size(); } @@ -3965,6 +4012,7 @@ public State getState() { return STATE_UPDATER.get(this); } + @Override public long getCacheSize() { return entryCache.getSize(); } @@ -4382,7 +4430,7 @@ public CompletableFuture getManagedLedgerInternalSta List ledgersInfos = new ArrayList<>(this.getLedgersInfo().values()); // add asynchronous metadata retrieval operations to a hashmap - Map> ledgerMetadataFutures = new HashMap(); + Map> ledgerMetadataFutures = new HashMap(); if (includeLedgerMetadata) { ledgersInfos.forEach(li -> { long ledgerId = li.getLedgerId(); @@ -4393,24 +4441,55 @@ public CompletableFuture getManagedLedgerInternalSta }); } + + CompletableFuture> bookiesFuture; + if (includeLedgerMetadata) { + RegistrationClient registrationClient = bookKeeper.getMetadataClientDriver().getRegistrationClient(); + bookiesFuture = registrationClient.getReadOnlyBookies() + .thenCombine(registrationClient.getWritableBookies(), (readOnlyBookies, writableBookies) -> { + Set bookies = new HashSet<>(); + bookies.addAll(readOnlyBookies.getValue()); + bookies.addAll(writableBookies.getValue()); + return bookies; + }); + } else { + bookiesFuture = CompletableFuture.completedFuture(null); + } + // wait until metadata has been retrieved - FutureUtil.waitForAll(ledgerMetadataFutures.values()).thenAccept(__ -> { - stats.ledgers = new ArrayList(); - ledgersInfos.forEach(li -> { - ManagedLedgerInternalStats.LedgerInfo info = new ManagedLedgerInternalStats.LedgerInfo(); - info.ledgerId = li.getLedgerId(); - info.entries = li.getEntries(); - info.size = li.getSize(); - info.offloaded = li.hasOffloadContext() && li.getOffloadContext().getComplete(); - if (includeLedgerMetadata) { - // lookup metadata from the hashmap which contains completed async operations - info.metadata = ledgerMetadataFutures.get(li.getLedgerId()).getNow(null); - } - stats.ledgers.add(info); + bookiesFuture.thenCompose(bookies -> + FutureUtil.waitForAll(ledgerMetadataFutures.values()).thenAccept(__ -> { + stats.ledgers = new ArrayList<>(); + ledgersInfos.forEach(li -> { + ManagedLedgerInternalStats.LedgerInfo info = new ManagedLedgerInternalStats.LedgerInfo(); + info.ledgerId = li.getLedgerId(); + info.entries = li.getEntries(); + info.size = li.getSize(); + info.offloaded = li.hasOffloadContext() && li.getOffloadContext().getComplete(); + if (includeLedgerMetadata) { + // lookup metadata from the hashmap which contains completed async operations + LedgerMetadata lm = ledgerMetadataFutures.get(li.getLedgerId()).getNow(null); + if (lm == null) { + info.metadata = null; + info.underReplicated = false; + } else { + info.metadata = lm.toSafeString(); + Set ensemble = lm.getAllEnsembles().values().stream() + .flatMap(Collection::stream) + .collect(Collectors.toSet()); + if (bookies != null) { + info.underReplicated = !bookies.contains(ensemble); + } + } + } + stats.ledgers.add(info); + }); + statFuture.complete(stats); + })) + .exceptionally(e -> { + statFuture.completeExceptionally(e); + return null; }); - statFuture.complete(stats); - }); - return statFuture; } @@ -4540,4 +4619,11 @@ public Position getTheSlowestNonDurationReadPosition() { } return theSlowestNonDurableReadPosition; } + + @Override + public CompletableFuture getLastDispatchablePosition(final Predicate predicate, + final Position startPosition) { + return ManagedLedgerImplUtils + .asyncGetLastValidPosition(this, predicate, startPosition); + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java index 60e24e8df0498..9a1753c715eff 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java @@ -20,29 +20,16 @@ import com.google.common.collect.BoundType; import com.google.common.collect.Range; -import com.google.protobuf.InvalidProtocolBufferException; -import java.util.Enumeration; +import java.util.ArrayList; import java.util.List; import java.util.NavigableMap; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import org.apache.bookkeeper.client.AsyncCallback; -import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; -import org.apache.bookkeeper.client.BookKeeperAdmin; -import org.apache.bookkeeper.client.LedgerEntry; -import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.api.DigestType; -import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.proto.MLDataFormats; -import org.apache.bookkeeper.mledger.util.Errors; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; -import org.apache.pulsar.metadata.api.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -102,369 +89,28 @@ private long getNumberOfEntries(Range range, } } - public PersistentOfflineTopicStats getEstimatedUnloadedTopicBacklog(ManagedLedgerFactoryImpl factory, + public PersistentOfflineTopicStats getEstimatedUnloadedTopicBacklog(ManagedLedgerFactory factory, String managedLedgerName) throws Exception { return estimateUnloadedTopicBacklog(factory, TopicName.get("persistent://" + managedLedgerName)); } - public PersistentOfflineTopicStats estimateUnloadedTopicBacklog(ManagedLedgerFactoryImpl factory, - TopicName topicName) throws Exception { + public PersistentOfflineTopicStats estimateUnloadedTopicBacklog(ManagedLedgerFactory factory, + TopicName topicName) throws Exception { String managedLedgerName = topicName.getPersistenceNamingEncoding(); - long numberOfEntries = 0; - long totalSize = 0; - final NavigableMap ledgers = new ConcurrentSkipListMap<>(); final PersistentOfflineTopicStats offlineTopicStats = new PersistentOfflineTopicStats(managedLedgerName, brokerName); - - // calculate total managed ledger size and number of entries without loading the topic - readLedgerMeta(factory, topicName, ledgers); - for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : ledgers.values()) { - numberOfEntries += ls.getEntries(); - totalSize += ls.getSize(); - if (accurate) { - offlineTopicStats.addLedgerDetails(ls.getEntries(), ls.getTimestamp(), ls.getSize(), ls.getLedgerId()); - } - } - offlineTopicStats.totalMessages = numberOfEntries; - offlineTopicStats.storageSize = totalSize; - if (log.isDebugEnabled()) { - log.debug("[{}] Total number of entries - {} and size - {}", managedLedgerName, numberOfEntries, totalSize); - } - - // calculate per cursor message backlog - calculateCursorBacklogs(factory, topicName, ledgers, offlineTopicStats); - offlineTopicStats.statGeneratedAt.setTime(System.currentTimeMillis()); - - return offlineTopicStats; - } - - private void readLedgerMeta(final ManagedLedgerFactoryImpl factory, final TopicName topicName, - final NavigableMap ledgers) throws Exception { - String managedLedgerName = topicName.getPersistenceNamingEncoding(); - MetaStore store = factory.getMetaStore(); - - final CountDownLatch mlMetaCounter = new CountDownLatch(1); - - store.getManagedLedgerInfo(managedLedgerName, false /* createIfMissing */, - new MetaStore.MetaStoreCallback() { - @Override - public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) { - for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : mlInfo.getLedgerInfoList()) { - ledgers.put(ls.getLedgerId(), ls); - } - - // find no of entries in last ledger - if (!ledgers.isEmpty()) { - final long id = ledgers.lastKey(); - AsyncCallback.OpenCallback opencb = (rc, lh, ctx1) -> { - if (log.isDebugEnabled()) { - log.debug("[{}] Opened ledger {}: {}", managedLedgerName, id, - BKException.getMessage(rc)); - } - if (rc == BKException.Code.OK) { - MLDataFormats.ManagedLedgerInfo.LedgerInfo info = - MLDataFormats.ManagedLedgerInfo.LedgerInfo - .newBuilder().setLedgerId(id).setEntries(lh.getLastAddConfirmed() + 1) - .setSize(lh.getLength()).setTimestamp(System.currentTimeMillis()).build(); - ledgers.put(id, info); - mlMetaCounter.countDown(); - } else if (Errors.isNoSuchLedgerExistsException(rc)) { - log.warn("[{}] Ledger not found: {}", managedLedgerName, ledgers.lastKey()); - ledgers.remove(ledgers.lastKey()); - mlMetaCounter.countDown(); - } else { - log.error("[{}] Failed to open ledger {}: {}", managedLedgerName, id, - BKException.getMessage(rc)); - mlMetaCounter.countDown(); - } - }; - - if (log.isDebugEnabled()) { - log.debug("[{}] Opening ledger {}", managedLedgerName, id); - } - - factory.getBookKeeper() - .thenAccept(bk -> { - bk.asyncOpenLedgerNoRecovery(id, digestType, password, opencb, null); - }).exceptionally(ex -> { - log.warn("[{}] Failed to open ledger {}: {}", managedLedgerName, id, ex); - opencb.openComplete(-1, null, null); - mlMetaCounter.countDown(); - return null; - }); - } else { - log.warn("[{}] Ledger list empty", managedLedgerName); - mlMetaCounter.countDown(); - } - } - - @Override - public void operationFailed(ManagedLedgerException.MetaStoreException e) { - log.warn("[{}] Unable to obtain managed ledger metadata - {}", managedLedgerName, e); - mlMetaCounter.countDown(); - } - }); - - if (accurate) { - // block until however long it takes for operation to complete - mlMetaCounter.await(); + if (factory instanceof ManagedLedgerFactoryImpl) { + List ctx = new ArrayList<>(); + ctx.add(digestType); + ctx.add(password); + factory.estimateUnloadedTopicBacklog(offlineTopicStats, topicName, accurate, ctx); } else { - mlMetaCounter.await(META_READ_TIMEOUT_SECONDS, TimeUnit.SECONDS); - + Object ctx = null; + factory.estimateUnloadedTopicBacklog(offlineTopicStats, topicName, accurate, ctx); } - } - - private void calculateCursorBacklogs(final ManagedLedgerFactoryImpl factory, final TopicName topicName, - final NavigableMap ledgers, - final PersistentOfflineTopicStats offlineTopicStats) throws Exception { - - if (ledgers.isEmpty()) { - return; - } - String managedLedgerName = topicName.getPersistenceNamingEncoding(); - MetaStore store = factory.getMetaStore(); - BookKeeper bk = factory.getBookKeeper().get(); - final CountDownLatch allCursorsCounter = new CountDownLatch(1); - final long errorInReadingCursor = -1; - ConcurrentOpenHashMap ledgerRetryMap = - ConcurrentOpenHashMap.newBuilder().build(); - final MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo = ledgers.lastEntry().getValue(); - final Position lastLedgerPosition = - PositionFactory.create(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1); - if (log.isDebugEnabled()) { - log.debug("[{}] Last ledger position {}", managedLedgerName, lastLedgerPosition); - } - store.getCursors(managedLedgerName, new MetaStore.MetaStoreCallback>() { - @Override - public void operationComplete(List cursors, Stat v) { - // Load existing cursors - if (log.isDebugEnabled()) { - log.debug("[{}] Found {} cursors", managedLedgerName, cursors.size()); - } - - if (cursors.isEmpty()) { - allCursorsCounter.countDown(); - return; - } - - final CountDownLatch cursorCounter = new CountDownLatch(cursors.size()); - - for (final String cursorName : cursors) { - // determine subscription position from cursor ledger - if (log.isDebugEnabled()) { - log.debug("[{}] Loading cursor {}", managedLedgerName, cursorName); - } - - AsyncCallback.OpenCallback cursorLedgerOpenCb = (rc, lh, ctx1) -> { - long ledgerId = lh.getId(); - if (log.isDebugEnabled()) { - log.debug("[{}] Opened cursor ledger {} for cursor {}. rc={}", managedLedgerName, ledgerId, - cursorName, rc); - } - if (rc != BKException.Code.OK) { - log.warn("[{}] Error opening metadata ledger {} for cursor {}: {}", managedLedgerName, - ledgerId, cursorName, BKException.getMessage(rc)); - cursorCounter.countDown(); - return; - } - long lac = lh.getLastAddConfirmed(); - if (log.isDebugEnabled()) { - log.debug("[{}] Cursor {} LAC {} read from ledger {}", managedLedgerName, cursorName, lac, - ledgerId); - } - - if (lac == LedgerHandle.INVALID_ENTRY_ID) { - // save the ledger id and cursor to retry outside of this call back - // since we are trying to read the same cursor ledger, we will block until - // this current callback completes, since an attempt to read the entry - // will block behind this current operation to complete - ledgerRetryMap.put(cursorName, ledgerId); - log.info("[{}] Cursor {} LAC {} read from ledger {}", managedLedgerName, cursorName, lac, - ledgerId); - cursorCounter.countDown(); - return; - } - final long entryId = lac; - // read last acked message position for subscription - lh.asyncReadEntries(entryId, entryId, new AsyncCallback.ReadCallback() { - @Override - public void readComplete(int rc, LedgerHandle lh, Enumeration seq, - Object ctx) { - try { - if (log.isDebugEnabled()) { - log.debug("readComplete rc={} entryId={}", rc, entryId); - } - if (rc != BKException.Code.OK) { - log.warn("[{}] Error reading from metadata ledger {} for cursor {}: {}", - managedLedgerName, ledgerId, cursorName, BKException.getMessage(rc)); - // indicate that this cursor should be excluded - offlineTopicStats.addCursorDetails(cursorName, errorInReadingCursor, - lh.getId()); - } else { - LedgerEntry entry = seq.nextElement(); - MLDataFormats.PositionInfo positionInfo; - try { - positionInfo = MLDataFormats.PositionInfo.parseFrom(entry.getEntry()); - } catch (InvalidProtocolBufferException e) { - log.warn( - "[{}] Error reading position from metadata ledger {} for cursor {}: {}", - managedLedgerName, ledgerId, cursorName, e); - offlineTopicStats.addCursorDetails(cursorName, errorInReadingCursor, - lh.getId()); - return; - } - final Position lastAckedMessagePosition = - PositionFactory.create(positionInfo.getLedgerId(), - positionInfo.getEntryId()); - if (log.isDebugEnabled()) { - log.debug("[{}] Cursor {} MD {} read last ledger position {}", - managedLedgerName, cursorName, lastAckedMessagePosition, - lastLedgerPosition); - } - // calculate cursor backlog - Range range = Range.openClosed(lastAckedMessagePosition, - lastLedgerPosition); - if (log.isDebugEnabled()) { - log.debug("[{}] Calculating backlog for cursor {} using range {}", - managedLedgerName, cursorName, range); - } - long cursorBacklog = getNumberOfEntries(range, ledgers); - offlineTopicStats.messageBacklog += cursorBacklog; - offlineTopicStats.addCursorDetails(cursorName, cursorBacklog, lh.getId()); - } - } finally { - cursorCounter.countDown(); - } - } - }, null); - - }; // end of cursor meta read callback - - store.asyncGetCursorInfo(managedLedgerName, cursorName, - new MetaStore.MetaStoreCallback() { - @Override - public void operationComplete(MLDataFormats.ManagedCursorInfo info, - Stat stat) { - long cursorLedgerId = info.getCursorsLedgerId(); - if (log.isDebugEnabled()) { - log.debug("[{}] Cursor {} meta-data read ledger id {}", managedLedgerName, - cursorName, cursorLedgerId); - } - if (cursorLedgerId != -1) { - bk.asyncOpenLedgerNoRecovery(cursorLedgerId, digestType, password, - cursorLedgerOpenCb, null); - } else { - Position lastAckedMessagePosition = PositionFactory.create( - info.getMarkDeleteLedgerId(), info.getMarkDeleteEntryId()); - Range range = Range.openClosed(lastAckedMessagePosition, - lastLedgerPosition); - if (log.isDebugEnabled()) { - log.debug("[{}] Calculating backlog for cursor {} using range {}", - managedLedgerName, cursorName, range); - } - long cursorBacklog = getNumberOfEntries(range, ledgers); - offlineTopicStats.messageBacklog += cursorBacklog; - offlineTopicStats.addCursorDetails(cursorName, cursorBacklog, cursorLedgerId); - cursorCounter.countDown(); - } - - } - - @Override - public void operationFailed(ManagedLedgerException.MetaStoreException e) { - log.warn("[{}] Unable to obtain cursor ledger for cursor {}: {}", managedLedgerName, - cursorName, e); - cursorCounter.countDown(); - } - }); - } // for every cursor find backlog - try { - if (accurate) { - cursorCounter.await(); - } else { - cursorCounter.await(META_READ_TIMEOUT_SECONDS, TimeUnit.SECONDS); - } - } catch (Exception e) { - log.warn("[{}] Error reading subscription positions{}", managedLedgerName, e); - } finally { - allCursorsCounter.countDown(); - } - } - - @Override - public void operationFailed(ManagedLedgerException.MetaStoreException e) { - log.warn("[{}] Failed to get the cursors list", managedLedgerName, e); - allCursorsCounter.countDown(); - } - }); - if (accurate) { - allCursorsCounter.await(); - } else { - allCursorsCounter.await(META_READ_TIMEOUT_SECONDS, TimeUnit.SECONDS); - } - - // go through ledgers where LAC was -1 - if (accurate && ledgerRetryMap.size() > 0) { - ledgerRetryMap.forEach((cursorName, ledgerId) -> { - if (log.isDebugEnabled()) { - log.debug("Cursor {} Ledger {} Trying to obtain MD from BkAdmin", cursorName, ledgerId); - } - Position lastAckedMessagePosition = tryGetMDPosition(bk, ledgerId, cursorName); - if (lastAckedMessagePosition == null) { - log.warn("[{}] Cursor {} read from ledger {}. Unable to determine cursor position", - managedLedgerName, cursorName, ledgerId); - } else { - if (log.isDebugEnabled()) { - log.debug("[{}] Cursor {} read from ledger using bk admin {}. position {}", managedLedgerName, - cursorName, ledgerId, lastAckedMessagePosition); - } - // calculate cursor backlog - Range range = Range.openClosed(lastAckedMessagePosition, lastLedgerPosition); - if (log.isDebugEnabled()) { - log.debug("[{}] Calculating backlog for cursor {} using range {}", managedLedgerName, - cursorName, range); - } - long cursorBacklog = getNumberOfEntries(range, ledgers); - offlineTopicStats.messageBacklog += cursorBacklog; - offlineTopicStats.addCursorDetails(cursorName, cursorBacklog, ledgerId); - } - }); - } - } - - private Position tryGetMDPosition(BookKeeper bookKeeper, long ledgerId, String cursorName) { - BookKeeperAdmin bookKeeperAdmin = null; - long lastEntry = LedgerHandle.INVALID_ENTRY_ID; - Position lastAckedMessagePosition = null; - try { - bookKeeperAdmin = new BookKeeperAdmin(bookKeeper); - for (LedgerEntry ledgerEntry : bookKeeperAdmin.readEntries(ledgerId, 0, lastEntry)) { - lastEntry = ledgerEntry.getEntryId(); - if (log.isDebugEnabled()) { - log.debug(" Read entry {} from ledger {} for cursor {}", lastEntry, ledgerId, cursorName); - } - MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(ledgerEntry.getEntry()); - lastAckedMessagePosition = - PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId()); - if (log.isDebugEnabled()) { - log.debug("Cursor {} read position {}", cursorName, lastAckedMessagePosition); - } - } - } catch (Exception e) { - log.warn("Unable to determine LAC for ledgerId {} for cursor {}: {}", ledgerId, cursorName, e); - } finally { - if (bookKeeperAdmin != null) { - try { - bookKeeperAdmin.close(); - } catch (Exception e) { - log.warn("Unable to close bk admin for ledgerId {} for cursor {}", ledgerId, cursorName, e); - } - } - - } - return lastAckedMessagePosition; + return offlineTopicStats; } private static final Logger log = LoggerFactory.getLogger(ManagedLedgerOfflineBacklog.class); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java index 3f0699657b5d4..036ce9223e89d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java @@ -45,7 +45,7 @@ * */ @Slf4j -public class OpAddEntry implements AddCallback, CloseCallback, Runnable { +public class OpAddEntry implements AddCallback, CloseCallback, Runnable, ManagedLedgerInterceptor.AddEntryOperation { protected ManagedLedgerImpl ml; LedgerHandle ledger; long entryId; @@ -139,8 +139,8 @@ public void initiate() { lastInitTime = System.nanoTime(); if (ml.getManagedLedgerInterceptor() != null) { long originalDataLen = data.readableBytes(); - payloadProcessorHandle = ml.getManagedLedgerInterceptor().processPayloadBeforeLedgerWrite(this, - duplicateBuffer); + payloadProcessorHandle = ml.getManagedLedgerInterceptor() + .processPayloadBeforeLedgerWrite(this.getCtx(), duplicateBuffer); if (payloadProcessorHandle != null) { duplicateBuffer = payloadProcessorHandle.getProcessedPayload(); // If data len of entry changes, correct "dataLength" and "currentLedgerSize". diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpFindNewest.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpFindNewest.java index 707cb389eba1a..26d5e8d3f661d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpFindNewest.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpFindNewest.java @@ -26,7 +26,7 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound; +import org.apache.bookkeeper.mledger.PositionBound; @Slf4j class OpFindNewest implements ReadEntryCallback { @@ -97,7 +97,7 @@ public void readEntryComplete(Entry entry, Object ctx) { searchPosition = ledger.getPositionAfterN(searchPosition, max, PositionBound.startExcluded); Position lastPosition = ledger.getLastPosition(); searchPosition = - ledger.getPositionAfterN(searchPosition, max, ManagedLedgerImpl.PositionBound.startExcluded); + ledger.getPositionAfterN(searchPosition, max, PositionBound.startExcluded); if (lastPosition.compareTo(searchPosition) < 0) { if (log.isDebugEnabled()) { log.debug("first position {} matches, last should be {}, but moving to lastPos {}", position, diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpScan.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpScan.java index e4f6fd04ff4da..72d05ede3a0f5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpScan.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpScan.java @@ -29,6 +29,7 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionBound; import org.apache.bookkeeper.mledger.ScanOutcome; @Slf4j @@ -88,7 +89,7 @@ public void readEntriesComplete(List entries, Object ctx) { } } searchPosition = ledger.getPositionAfterN(lastPositionForBatch, 1, - ManagedLedgerImpl.PositionBound.startExcluded); + PositionBound.startExcluded); if (log.isDebugEnabled()) { log.debug("readEntryComplete {} at {} next is {}", lastPositionForBatch, searchPosition); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedCursorStats.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedCursorStats.java index 93a749d4aef51..ec73c9d5e5eb2 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedCursorStats.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedCursorStats.java @@ -23,6 +23,7 @@ import io.opentelemetry.api.metrics.BatchCallback; import io.opentelemetry.api.metrics.ObservableLongMeasurement; import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.pulsar.opentelemetry.Constants; public class OpenTelemetryManagedCursorStats implements AutoCloseable { @@ -98,7 +99,7 @@ public OpenTelemetryManagedCursorStats(OpenTelemetry openTelemetry, ManagedLedge batchCallback = meter.batchCallback(() -> factory.getManagedLedgers() .values() .stream() - .map(ManagedLedgerImpl::getCursors) + .map(ManagedLedger::getCursors) .flatMap(Streams::stream) .forEach(this::recordMetrics), persistOperationCounter, diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedLedgerStats.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedLedgerStats.java index f7b9d91dff6ad..26c4b62cf7694 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedLedgerStats.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpenTelemetryManagedLedgerStats.java @@ -21,6 +21,7 @@ import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.metrics.BatchCallback; import io.opentelemetry.api.metrics.ObservableLongMeasurement; +import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.pulsar.opentelemetry.Constants; public class OpenTelemetryManagedLedgerStats implements AutoCloseable { @@ -130,8 +131,8 @@ public void close() { batchCallback.close(); } - private void recordMetrics(ManagedLedgerImpl ml) { - var stats = ml.getMbean(); + private void recordMetrics(ManagedLedger ml) { + var stats = ml.getStats(); var ledgerAttributeSet = ml.getManagedLedgerAttributes(); var attributes = ledgerAttributeSet.getAttributes(); var attributesSucceed = ledgerAttributeSet.getAttributesOperationSucceed(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java index bd3e461d94e5c..00ed5a0c5b9d9 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java @@ -23,9 +23,9 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionBound; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ReadOnlyCursor; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound; import org.apache.bookkeeper.mledger.proto.MLDataFormats; @Slf4j diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java index e64941c3201cb..1fb2aa3629092 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java @@ -43,12 +43,12 @@ public class ReadOnlyManagedLedgerImpl extends ManagedLedgerImpl { public ReadOnlyManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, - ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, - String name) { + ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, + String name) { super(factory, bookKeeper, store, config, scheduledExecutor, name); } - CompletableFuture initialize() { + public CompletableFuture initialize() { CompletableFuture future = new CompletableFuture<>(); // Fetch the list of existing ledgers in the managed ledger @@ -128,7 +128,7 @@ public void operationFailed(MetaStoreException e) { return future; } - ReadOnlyCursor createReadOnlyCursor(Position startPosition) { + public ReadOnlyCursor createReadOnlyCursor(Position startPosition) { if (ledgers.isEmpty()) { lastConfirmedEntry = PositionFactory.EARLIEST; } else if (ledgers.lastEntry().getValue().getEntries() > 0) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java index 546733f909e21..4b03cad8e0a1d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java @@ -131,7 +131,8 @@ public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) currentLedger = lh; if (managedLedgerInterceptor != null) { - managedLedgerInterceptor.onManagedLedgerLastLedgerInitialize(name, lh) + managedLedgerInterceptor + .onManagedLedgerLastLedgerInitialize(name, createLastEntryHandle(lh)) .thenRun(() -> ShadowManagedLedgerImpl.super.initialize(callback, ctx)) .exceptionally(ex -> { callback.initializeFailed( diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/intercept/ManagedLedgerInterceptor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/intercept/ManagedLedgerInterceptor.java index d26a5e15735aa..0ca6fa9dd866c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/intercept/ManagedLedgerInterceptor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/intercept/ManagedLedgerInterceptor.java @@ -20,11 +20,11 @@ import io.netty.buffer.ByteBuf; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; -import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; -import org.apache.bookkeeper.mledger.impl.OpAddEntry; +import org.apache.bookkeeper.mledger.Entry; /** * Interceptor for ManagedLedger. @@ -32,14 +32,34 @@ @InterfaceAudience.LimitedPrivate @InterfaceStability.Stable public interface ManagedLedgerInterceptor { + /** + * An operation to add an entry to a ledger. + */ + interface AddEntryOperation { + /** + * Get the data to be written to the ledger. + * @return data to be written to the ledger + */ + ByteBuf getData(); + /** + * Set the data to be written to the ledger. + * @param data data to be written to the ledger + */ + void setData(ByteBuf data); + /** + * Get the operation context object. + * @return context the context object + */ + Object getCtx(); + } /** - * Intercept an OpAddEntry and return an OpAddEntry. - * @param op an OpAddEntry to be intercepted. + * Intercept adding an entry to a ledger. + * + * @param op an operation to be intercepted. * @param numberOfMessages - * @return an OpAddEntry. */ - OpAddEntry beforeAddEntry(OpAddEntry op, int numberOfMessages); + void beforeAddEntry(AddEntryOperation op, int numberOfMessages); /** * Intercept When add entry failed. @@ -55,12 +75,25 @@ default void afterFailedAddEntry(int numberOfMessages){ */ void onManagedLedgerPropertiesInitialize(Map propertiesMap); + /** + * A handle for reading the last ledger entry. + */ + interface LastEntryHandle { + /** + * Read the last entry from the ledger. + * The caller is responsible for releasing the entry. + * @return the last entry from the ledger, if any + */ + CompletableFuture> readLastEntryAsync(); + } + /** * Intercept when ManagedLedger is initialized. - * @param name name of ManagedLedger - * @param ledgerHandle a LedgerHandle. + * + * @param name name of ManagedLedger + * @param lastEntryHandle a LedgerHandle. */ - CompletableFuture onManagedLedgerLastLedgerInitialize(String name, LedgerHandle ledgerHandle); + CompletableFuture onManagedLedgerLastLedgerInitialize(String name, LastEntryHandle lastEntryHandle); /** * @param propertiesMap map of properties. @@ -93,12 +126,12 @@ default PayloadProcessorHandle processPayloadBeforeEntryCache(ByteBuf dataReadFr /** * Intercept before payload gets written to ledger. - * @param ledgerWriteOp OpAddEntry used to trigger ledger write. + * @param ctx the operation context object * @param dataToBeStoredInLedger data to be stored in ledger * @return handle to the processor */ - default PayloadProcessorHandle processPayloadBeforeLedgerWrite(OpAddEntry ledgerWriteOp, - ByteBuf dataToBeStoredInLedger){ + default PayloadProcessorHandle processPayloadBeforeLedgerWrite(Object ctx, + ByteBuf dataToBeStoredInLedger) { return null; } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java index a387331f3c047..2afbcef0926e7 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java @@ -47,8 +47,10 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursorMXBean; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; import org.testng.annotations.Test; public class ManagedCursorContainerTest { @@ -409,6 +411,36 @@ public boolean checkAndUpdateReadPositionChanged() { public boolean isClosed() { return false; } + + @Override + public ManagedLedgerInternalStats.CursorStats getCursorStats() { + return null; + } + + @Override + public boolean isMessageDeleted(Position position) { + return false; + } + + @Override + public ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) throws ManagedLedgerException { + return null; + } + + @Override + public long[] getBatchPositionAckSet(Position position) { + return new long[0]; + } + + @Override + public int applyMaxSizeCap(int maxEntries, long maxSizeBytes) { + return 0; + } + + @Override + public void updateReadStats(int readEntriesCount, long readEntriesSize) { + + } } @Test diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorPropertiesTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorPropertiesTest.java index 500de5dd13879..990c298604e59 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorPropertiesTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorPropertiesTest.java @@ -18,7 +18,7 @@ */ package org.apache.bookkeeper.mledger.impl; -import static org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.CURSOR_INTERNAL_PROPERTY_PREFIX; +import static org.apache.bookkeeper.mledger.ManagedCursor.CURSOR_INTERNAL_PROPERTY_PREFIX; import static org.apache.bookkeeper.mledger.util.Futures.executeWithRetry; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index bb38114ef7117..83a6c771513a9 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -123,6 +123,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryMXBean; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionBound; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.VoidCallback; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; @@ -2178,7 +2179,9 @@ public void testNoRolloverIfNoMetadataSession() throws Exception { ledger.addEntry("data".getBytes()); // After the re-establishment, we'll be creating new ledgers - assertEquals(ledger.getLedgersInfoAsList().size(), 3); + Awaitility.await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> { + assertEquals(ledger.getLedgersInfoAsList().size(), 4); + }); } @Test @@ -2595,11 +2598,11 @@ public void testGetPositionAfterN() throws Exception { Position startPosition = PositionFactory.create(firstLedger, 0); - Position targetPosition = managedLedger.getPositionAfterN(startPosition, 1, ManagedLedgerImpl.PositionBound.startExcluded); + Position targetPosition = managedLedger.getPositionAfterN(startPosition, 1, PositionBound.startExcluded); assertEquals(targetPosition.getLedgerId(), firstLedger); assertEquals(targetPosition.getEntryId(), 1); - targetPosition = managedLedger.getPositionAfterN(startPosition, 4, ManagedLedgerImpl.PositionBound.startExcluded); + targetPosition = managedLedger.getPositionAfterN(startPosition, 4, PositionBound.startExcluded); assertEquals(targetPosition.getLedgerId(), firstLedger); assertEquals(targetPosition.getEntryId(), 4); @@ -2607,25 +2610,25 @@ public void testGetPositionAfterN() throws Exception { Position searchPosition = managedLedger.getNextValidPosition(managedCursor.getMarkDeletedPosition()); long length = managedCursor.getNumberOfEntriesInStorage(); // return the last confirm entry position if searchPosition is exceed the last confirm entry - targetPosition = managedLedger.getPositionAfterN(searchPosition, length, ManagedLedgerImpl.PositionBound.startExcluded); + targetPosition = managedLedger.getPositionAfterN(searchPosition, length, PositionBound.startExcluded); log.info("Target position is {}", targetPosition); assertEquals(targetPosition.getLedgerId(), secondLedger); assertEquals(targetPosition.getEntryId(), 4); // test for n > NumberOfEntriesInStorage searchPosition = PositionFactory.create(secondLedger, 0); - targetPosition = managedLedger.getPositionAfterN(searchPosition, 100, ManagedLedgerImpl.PositionBound.startIncluded); + targetPosition = managedLedger.getPositionAfterN(searchPosition, 100, PositionBound.startIncluded); assertEquals(targetPosition.getLedgerId(), secondLedger); assertEquals(targetPosition.getEntryId(), 4); // test for startPosition > current ledger searchPosition = PositionFactory.create(999, 0); - targetPosition = managedLedger.getPositionAfterN(searchPosition, 0, ManagedLedgerImpl.PositionBound.startIncluded); + targetPosition = managedLedger.getPositionAfterN(searchPosition, 0, PositionBound.startIncluded); assertEquals(targetPosition.getLedgerId(), secondLedger); assertEquals(targetPosition.getEntryId(), 4); searchPosition = PositionFactory.create(999, 0); - targetPosition = managedLedger.getPositionAfterN(searchPosition, 10, ManagedLedgerImpl.PositionBound.startExcluded); + targetPosition = managedLedger.getPositionAfterN(searchPosition, 10, PositionBound.startExcluded); assertEquals(targetPosition.getLedgerId(), secondLedger); assertEquals(targetPosition.getEntryId(), 4); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImplTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImplTest.java index 028ecad407276..61056d0b4b602 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImplTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImplTest.java @@ -31,6 +31,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ReadOnlyManagedLedger; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.testng.annotations.Test; @@ -56,7 +57,7 @@ public void testReadOnlyManagedLedgerImplAttachProperties() factory.asyncOpenReadOnlyManagedLedger(MANAGED_LEDGER_NAME_ATTACHED_PROPERTIES, new AsyncCallbacks.OpenReadOnlyManagedLedgerCallback() { @Override - public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl managedLedger, + public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedger managedLedger, Object ctx) { managedLedger.getProperties().forEach((key, value) -> { assertEquals(key, propertiesKey); @@ -85,7 +86,7 @@ public void testReadOnlyManagedLedgerImplNoProperties() factory.asyncOpenReadOnlyManagedLedger(MANAGED_LEDGER_NAME_NON_PROPERTIES, new AsyncCallbacks.OpenReadOnlyManagedLedgerCallback() { @Override - public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl managedLedger, + public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedger managedLedger, Object ctx) { assertEquals(managedLedger.getProperties().size(), 0); future.complete(null); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 425e7dafa1bf8..87196d3f3a9a6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -37,6 +37,7 @@ import java.lang.reflect.Constructor; import java.net.InetSocketAddress; import java.net.MalformedURLException; +import java.time.Clock; import java.time.Duration; import java.util.ArrayList; import java.util.Collection; @@ -220,6 +221,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private StrategicTwoPhaseCompactor strategicCompactor; private ResourceUsageTransportManager resourceUsageTransportManager; private ResourceGroupService resourceGroupServiceManager; + private final Clock clock; private final ScheduledExecutorService executor; @@ -340,6 +342,7 @@ public PulsarService(ServiceConfiguration config, PulsarConfigurationLoader.isComplete(config); TransactionBatchedWriteValidator.validate(config); this.config = config; + this.clock = Clock.systemUTC(); this.openTelemetry = new PulsarBrokerOpenTelemetry(config, openTelemetrySdkBuilderCustomizer); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index b2d455f645daf..4d04dfeda7a74 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -66,8 +66,6 @@ import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ScanOutcome; import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerOfflineBacklog; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.MapUtils; @@ -2622,7 +2620,7 @@ private void getEntryBatchSize(CompletableFuture batchSizeFuture, Persi MessageIdImpl messageId, int batchIndex) { if (batchIndex >= 0) { try { - ManagedLedgerImpl ledger = (ManagedLedgerImpl) topic.getManagedLedger(); + ManagedLedger ledger = topic.getManagedLedger(); ledger.asyncReadEntry(PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()), new AsyncCallbacks.ReadEntryCallback() { @Override @@ -2733,8 +2731,7 @@ protected CompletableFuture internalGetMessageById(long ledgerId, long .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenCompose(topic -> { CompletableFuture results = new CompletableFuture<>(); - ManagedLedgerImpl ledger = - (ManagedLedgerImpl) ((PersistentTopic) topic).getManagedLedger(); + ManagedLedger ledger = ((PersistentTopic) topic).getManagedLedger(); ledger.asyncReadEntry(PositionFactory.create(ledgerId, entryId), new AsyncCallbacks.ReadEntryCallback() { @Override @@ -3173,7 +3170,7 @@ protected CompletableFuture internalGetBacklogAsync try { PersistentOfflineTopicStats estimateOfflineTopicStats = offlineTopicBacklog.estimateUnloadedTopicBacklog( - (ManagedLedgerFactoryImpl) pulsar().getManagedLedgerFactory(), + pulsar().getManagedLedgerFactory(), topicName); pulsar().getBrokerService() .cacheOfflineTopicStats(topicName, estimateOfflineTopicStats); @@ -3248,8 +3245,7 @@ protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, getTopicNotFoundErrorMessage(topicName.toString()))); return; } - ManagedLedgerImpl managedLedger = - (ManagedLedgerImpl) topic.getManagedLedger(); + ManagedLedger managedLedger = topic.getManagedLedger(); if (messageId.getLedgerId() == -1) { asyncResponse.resume(managedLedger.getTotalSize()); } else { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index 5a6df389aeddb..47c78fa9ee2ec 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -19,7 +19,7 @@ package org.apache.pulsar.broker.delayed.bucket; import static com.google.common.base.Preconditions.checkArgument; -import static org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.CURSOR_INTERNAL_PROPERTY_PREFIX; +import static org.apache.bookkeeper.mledger.ManagedCursor.CURSOR_INTERNAL_PROPERTY_PREFIX; import static org.apache.pulsar.broker.delayed.bucket.Bucket.DELIMITER; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.HashBasedTable; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImpl.java index 02c6c575fd919..db138989a8eee 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/intercept/ManagedLedgerInterceptorImpl.java @@ -23,9 +23,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; -import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.api.LedgerEntry; -import org.apache.bookkeeper.mledger.impl.OpAddEntry; +import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.pulsar.common.api.proto.BrokerEntryMetadata; @@ -85,12 +83,11 @@ public long getIndex() { } @Override - public OpAddEntry beforeAddEntry(OpAddEntry op, int numberOfMessages) { + public void beforeAddEntry(AddEntryOperation op, int numberOfMessages) { if (op == null || numberOfMessages <= 0) { - return op; + return; } op.setData(Commands.addBrokerEntryMetadata(op.getData(), brokerEntryMetadataInterceptors, numberOfMessages)); - return op; } @Override @@ -115,43 +112,22 @@ public void onManagedLedgerPropertiesInitialize(Map propertiesMa } @Override - public CompletableFuture onManagedLedgerLastLedgerInitialize(String name, LedgerHandle lh) { - CompletableFuture promise = new CompletableFuture<>(); - boolean hasAppendIndexMetadataInterceptor = appendIndexMetadataInterceptor != null; - if (hasAppendIndexMetadataInterceptor && lh.getLastAddConfirmed() >= 0) { - lh.readAsync(lh.getLastAddConfirmed(), lh.getLastAddConfirmed()).whenComplete((entries, ex) -> { - if (ex != null) { - log.error("[{}] Read last entry error.", name, ex); - promise.completeExceptionally(ex); - } else { - if (entries != null) { - try { - LedgerEntry ledgerEntry = entries.getEntry(lh.getLastAddConfirmed()); - if (ledgerEntry != null) { - BrokerEntryMetadata brokerEntryMetadata = - Commands.parseBrokerEntryMetadataIfExist(ledgerEntry.getEntryBuffer()); - if (brokerEntryMetadata != null && brokerEntryMetadata.hasIndex()) { - appendIndexMetadataInterceptor.recoveryIndexGenerator( - brokerEntryMetadata.getIndex()); - } - } - entries.close(); - promise.complete(null); - } catch (Exception e) { - entries.close(); - log.error("[{}] Failed to recover the index generator from the last add confirmed entry.", - name, e); - promise.completeExceptionally(e); - } - } else { - promise.complete(null); + public CompletableFuture onManagedLedgerLastLedgerInitialize(String name, LastEntryHandle lh) { + return lh.readLastEntryAsync().thenAccept(lastEntryOptional -> { + if (lastEntryOptional.isPresent()) { + Entry lastEntry = lastEntryOptional.get(); + try { + BrokerEntryMetadata brokerEntryMetadata = + Commands.parseBrokerEntryMetadataIfExist(lastEntry.getDataBuffer()); + if (brokerEntryMetadata != null && brokerEntryMetadata.hasIndex()) { + appendIndexMetadataInterceptor.recoveryIndexGenerator( + brokerEntryMetadata.getIndex()); } + } finally { + lastEntry.release(); } - }); - } else { - promise.complete(null); - } - return promise; + } + }); } @Override @@ -189,11 +165,11 @@ public void release() { }; } @Override - public PayloadProcessorHandle processPayloadBeforeLedgerWrite(OpAddEntry op, ByteBuf ledgerData) { + public PayloadProcessorHandle processPayloadBeforeLedgerWrite(Object ctx, ByteBuf ledgerData) { if (this.inputProcessors == null || this.inputProcessors.size() == 0) { return null; } - return processPayload(this.inputProcessors, op.getCtx(), ledgerData); + return processPayload(this.inputProcessors, ctx, ledgerData); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index f25dfef966bfd..9e5d6ef7191d1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -23,6 +23,7 @@ import static org.apache.bookkeeper.mledger.impl.ManagedLedgerMBeanImpl.ENTRY_LATENCY_BUCKETS_USEC; import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; import com.google.common.base.MoreObjects; +import java.time.Clock; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -164,11 +165,13 @@ public abstract class AbstractTopic implements Topic, TopicPolicyListener> entryFilters; protected volatile boolean transferring = false; private volatile List activeRateLimiters; + protected final Clock clock; protected Set additionalSystemCursorNames = new TreeSet<>(); public AbstractTopic(String topic, BrokerService brokerService) { this.topic = topic; + this.clock = brokerService.getClock(); this.brokerService = brokerService; this.producers = new ConcurrentHashMap<>(); this.isFenced = false; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java index 012cbcad1e26d..689e8514078c2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java @@ -27,10 +27,10 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursor.IndividualDeletedEntries; +import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -132,7 +132,7 @@ private void dropBacklogForSizeLimit(PersistentTopic persistentTopic, BacklogQuo // Get estimated unconsumed size for the managed ledger associated with this topic. Estimated size is more // useful than the actual storage size. Actual storage size gets updated only when managed ledger is trimmed. - ManagedLedgerImpl mLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedLedger mLedger = persistentTopic.getManagedLedger(); long backlogSize = mLedger.getEstimatedBacklogSize(); if (log.isDebugEnabled()) { @@ -214,29 +214,30 @@ private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuo ); } else { // If disabled precise time based backlog quota check, will try to remove whole ledger from cursor's backlog - long currentMillis = ((ManagedLedgerImpl) persistentTopic.getManagedLedger()).getClock().millis(); - ManagedLedgerImpl mLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + long currentMillis = persistentTopic.getManagedLedger().getConfig().getClock().millis(); + ManagedLedger mLedger = persistentTopic.getManagedLedger(); try { for (; ; ) { ManagedCursor slowestConsumer = mLedger.getSlowestConsumer(); Position oldestPosition = slowestConsumer.getMarkDeletedPosition(); if (log.isDebugEnabled()) { log.debug("[{}] slowest consumer mark delete position is [{}], read position is [{}]", - slowestConsumer.getName(), oldestPosition, slowestConsumer.getReadPosition()); + slowestConsumer.getName(), oldestPosition, slowestConsumer.getReadPosition()); } - ManagedLedgerInfo.LedgerInfo ledgerInfo = mLedger.getLedgerInfo(oldestPosition.getLedgerId()).get(); + MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo = + mLedger.getLedgerInfo(oldestPosition.getLedgerId()).get(); if (ledgerInfo == null) { - Position nextPosition = - PositionFactory.create(mLedger.getNextValidLedger(oldestPosition.getLedgerId()), -1); + long ledgerId = mLedger.getLedgersInfo().ceilingKey(oldestPosition.getLedgerId() + 1); + Position nextPosition = PositionFactory.create(ledgerId, -1); slowestConsumer.markDelete(nextPosition); continue; } // Timestamp only > 0 if ledger has been closed if (ledgerInfo.getTimestamp() > 0 - && currentMillis - ledgerInfo.getTimestamp() > SECONDS.toMillis(quota.getLimitTime())) { + && currentMillis - ledgerInfo.getTimestamp() > SECONDS.toMillis(quota.getLimitTime())) { // skip whole ledger for the slowest cursor - Position nextPosition = - PositionFactory.create(mLedger.getNextValidLedger(ledgerInfo.getLedgerId()), -1); + long ledgerId = mLedger.getLedgersInfo().ceilingKey(oldestPosition.getLedgerId() + 1); + Position nextPosition = PositionFactory.create(ledgerId, -1); if (!nextPosition.equals(oldestPosition)) { slowestConsumer.markDelete(nextPosition); continue; @@ -246,7 +247,7 @@ private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuo } } catch (Exception e) { log.error("[{}] Error resetting cursor for slowest consumer [{}]", persistentTopic.getName(), - mLedger.getSlowestConsumer().getName(), e); + mLedger.getSlowestConsumer().getName(), e); } } } @@ -285,7 +286,7 @@ private void disconnectProducers(PersistentTopic persistentTopic) { */ private boolean advanceSlowestSystemCursor(PersistentTopic persistentTopic) { - ManagedLedgerImpl mLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedLedger mLedger = persistentTopic.getManagedLedger(); ManagedCursor slowestConsumer = mLedger.getSlowestConsumer(); if (slowestConsumer == null) { return false; 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 17e5288b5f179..cb5e0853d53f3 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 @@ -49,6 +49,7 @@ import java.lang.reflect.Field; import java.net.InetSocketAddress; import java.net.SocketAddress; +import java.time.Clock; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -288,6 +289,7 @@ public class BrokerService implements Closeable { private final int keepAliveIntervalSeconds; private final PulsarStats pulsarStats; private final AuthenticationService authenticationService; + private final Clock clock; public static final String MANAGED_LEDGER_PATH_ZNODE = "/managed-ledgers"; @@ -327,6 +329,7 @@ public class BrokerService implements Closeable { public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws Exception { this.pulsar = pulsar; + this.clock = pulsar.getClock(); this.dynamicConfigurationMap = prepareDynamicConfigurationMap(); this.brokerPublishRateLimiter = new PublishRateLimiterImpl(pulsar.getMonotonicSnapshotClock()); this.preciseTopicPublishRateLimitingEnable = 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 0229b9c0f9788..7d196ad30235b 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 @@ -65,11 +65,11 @@ import javax.net.ssl.SSLSession; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.pulsar.broker.PulsarServerException; @@ -2277,7 +2277,7 @@ private void getLargestBatchIndexWhenPossible( boolean readCompacted) { PersistentTopic persistentTopic = (PersistentTopic) topic; - ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedLedger ml = persistentTopic.getManagedLedger(); // If it's not pointing to a valid entry, respond messageId of the current position. // If the compaction cursor reach the end of the topic, respond messageId from compacted ledger @@ -2292,7 +2292,8 @@ private void getLargestBatchIndexWhenPossible( return; } - if (lastPosition.getEntryId() == -1 || !ml.ledgerExists(lastPosition.getLedgerId())) { + + if (lastPosition.getEntryId() == -1 || !ml.getLedgersInfo().containsKey(lastPosition.getLedgerId())) { // there is no entry in the original topic if (compactionHorizon != null) { // if readCompacted is true, we need to read the last entry from compacted topic @@ -2367,6 +2368,7 @@ public String toString() { }); }); } + private void handleLastMessageIdFromCompactionService(PersistentTopic persistentTopic, long requestId, int partitionIndex, Position markDeletePosition) { persistentTopic.getTopicCompactionService().readLastCompactedEntry().thenAccept(entry -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java index 9a8a39c8e9a12..3b4bc9d8bceb1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java @@ -28,13 +28,12 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.FindEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback; import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerException.LedgerNotExistException; import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.pulsar.broker.service.MessageExpirer; import org.apache.pulsar.client.impl.MessageImpl; @@ -113,27 +112,25 @@ private void checkExpiryByLedgerClosureTime(ManagedCursor cursor, int messageTTL if (messageTTLInSeconds <= 0) { return; } - if (cursor instanceof ManagedCursorImpl managedCursor) { - ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) managedCursor.getManagedLedger(); - Position deletedPosition = managedCursor.getMarkDeletedPosition(); - SortedMap ledgerInfoSortedMap = - managedLedger.getLedgersInfo().subMap(deletedPosition.getLedgerId(), true, - managedLedger.getLedgersInfo().lastKey(), true); - MLDataFormats.ManagedLedgerInfo.LedgerInfo info = null; - for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo : ledgerInfoSortedMap.values()) { - if (!ledgerInfo.hasTimestamp() || ledgerInfo.getTimestamp() == 0L - || !MessageImpl.isEntryExpired(messageTTLInSeconds, ledgerInfo.getTimestamp())) { - break; - } - info = ledgerInfo; + ManagedLedger managedLedger = cursor.getManagedLedger(); + Position deletedPosition = cursor.getMarkDeletedPosition(); + SortedMap ledgerInfoSortedMap = + managedLedger.getLedgersInfo().subMap(deletedPosition.getLedgerId(), true, + managedLedger.getLedgersInfo().lastKey(), true); + MLDataFormats.ManagedLedgerInfo.LedgerInfo info = null; + for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo : ledgerInfoSortedMap.values()) { + if (!ledgerInfo.hasTimestamp() || ledgerInfo.getTimestamp() == 0L + || !MessageImpl.isEntryExpired(messageTTLInSeconds, ledgerInfo.getTimestamp())) { + break; } - if (info != null && info.getLedgerId() > -1) { - Position position = PositionFactory.create(info.getLedgerId(), info.getEntries() - 1); - if (managedLedger.getLastConfirmedEntry().compareTo(position) < 0) { - findEntryComplete(managedLedger.getLastConfirmedEntry(), null); - } else { - findEntryComplete(position, null); - } + info = ledgerInfo; + } + if (info != null && info.getLedgerId() > -1) { + Position position = PositionFactory.create(info.getLedgerId(), info.getEntries() - 1); + if (managedLedger.getLastConfirmedEntry().compareTo(position) < 0) { + findEntryComplete(managedLedger.getLastConfirmedEntry(), null); + } else { + findEntryComplete(position, null); } } } @@ -240,11 +237,12 @@ public void findEntryFailed(ManagedLedgerException exception, Optional exception.getMessage()); if (exception instanceof LedgerNotExistException) { long failedLedgerId = failedReadPosition.get().getLedgerId(); - ManagedLedgerImpl ledger = ((ManagedLedgerImpl) cursor.getManagedLedger()); + ManagedLedger ledger = cursor.getManagedLedger(); Position lastPositionInLedger = ledger.getOptionalLedgerInfo(failedLedgerId) .map(ledgerInfo -> PositionFactory.create(failedLedgerId, ledgerInfo.getEntries() - 1)) .orElseGet(() -> { - Long nextExistingLedger = ledger.getNextValidLedger(failedReadPosition.get().getLedgerId()); + Long nextExistingLedger = + ledger.getLedgersInfo().ceilingKey(failedReadPosition.get().getLedgerId() + 1); if (nextExistingLedger == null) { log.info("[{}] [{}] Couldn't find next next valid ledger for expiry monitor when find " + "entry failed {}", ledger.getName(), ledger.getName(), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 97e6c943b0baa..e8e4919a9be52 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -37,10 +37,10 @@ import javax.annotation.Nullable; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.collections4.MapUtils; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -323,7 +323,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } if (messagesForC > 0) { - final ManagedLedgerImpl managedLedger = ((ManagedLedgerImpl) cursor.getManagedLedger()); + final ManagedLedger managedLedger = cursor.getManagedLedger(); for (int i = 0; i < messagesForC; i++) { final Entry entry = entriesWithSameKey.get(i); // remove positions first from replay list first : sendMessages recycles entries @@ -368,7 +368,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis // Update the last sent position and remove ranges from individuallySentPositions if necessary if (!allowOutOfOrderDelivery && lastSentPosition != null) { - final ManagedLedgerImpl managedLedger = ((ManagedLedgerImpl) cursor.getManagedLedger()); + final ManagedLedger managedLedger = cursor.getManagedLedger(); com.google.common.collect.Range range = individuallySentPositions.firstRange(); // If the upper bound is before the last sent position, we need to move ahead as these diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index ea1b7d7602be7..9a0545e6f0ab2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -23,6 +23,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import io.netty.buffer.ByteBuf; +import java.io.IOException; import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; @@ -51,8 +52,6 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.InvalidCursorPositionException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.ScanOutcome; -import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.ServiceConfiguration; @@ -405,7 +404,7 @@ public void acknowledgeMessage(List positions, AckType ackType, Map { - if (((ManagedCursorImpl) cursor).isMessageDeleted(position)) { + if ((cursor.isMessageDeleted(position))) { pendingAckHandle.clearIndividualPosition(position); } }); @@ -552,7 +551,7 @@ public CompletableFuture analyzeBacklog(Optional final String newNonDurableCursorName = "analyze-backlog-" + UUID.randomUUID(); ManagedCursor newNonDurableCursor; try { - newNonDurableCursor = ((ManagedCursorImpl) cursor).duplicateNonDurableCursor(newNonDurableCursorName); + newNonDurableCursor = cursor.duplicateNonDurableCursor(newNonDurableCursorName); } catch (ManagedLedgerException e) { return CompletableFuture.failedFuture(e); } @@ -1281,7 +1280,7 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge } subStats.msgBacklog = getNumberOfEntriesInBacklog(getStatsOptions.isGetPreciseBacklog()); if (getStatsOptions.isSubscriptionBacklogSize()) { - subStats.backlogSize = ((ManagedLedgerImpl) topic.getManagedLedger()) + subStats.backlogSize = topic.getManagedLedger() .getEstimatedBacklogSize(cursor.getMarkDeletedPosition()); } else { subStats.backlogSize = -1; @@ -1331,9 +1330,9 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge return CompletableFuture.completedFuture(subStats); } if (subStats.msgBacklog > 0) { - ManagedLedgerImpl managedLedger = ((ManagedLedgerImpl) cursor.getManagedLedger()); + ManagedLedger managedLedger = cursor.getManagedLedger(); Position markDeletedPosition = cursor.getMarkDeletedPosition(); - return managedLedger.getEarliestMessagePublishTimeOfPos(markDeletedPosition).thenApply(v -> { + return getEarliestMessagePublishTimeOfPos(managedLedger, markDeletedPosition).thenApply(v -> { subStats.earliestMsgPublishTimeInBacklog = v; return subStats; }); @@ -1343,6 +1342,48 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge } } + private CompletableFuture getEarliestMessagePublishTimeOfPos(ManagedLedger ml, Position pos) { + CompletableFuture future = new CompletableFuture<>(); + if (pos == null) { + future.complete(0L); + return future; + } + Position nextPos = ml.getNextValidPosition(pos); + + if (nextPos.compareTo(ml.getLastConfirmedEntry()) > 0) { + return CompletableFuture.completedFuture(-1L); + } + + ml.asyncReadEntry(nextPos, new ReadEntryCallback() { + @Override + public void readEntryComplete(Entry entry, Object ctx) { + try { + long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); + future.complete(entryTimestamp); + } catch (IOException e) { + log.error("Error deserializing message for message position {}", nextPos, e); + future.completeExceptionally(e); + } finally { + entry.release(); + } + } + + @Override + public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + log.error("Error read entry for position {}", nextPos, exception); + future.completeExceptionally(exception); + } + + @Override + public String toString() { + return String.format("ML [%s] get earliest message publish time of pos", + ml.getName()); + } + }, null); + + return future; + } + @Override public void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpoch) { Dispatcher dispatcher = getDispatcher(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index f90e10ee64e65..fc47889c60aac 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -54,7 +54,6 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.function.BiFunction; -import java.util.stream.Collectors; import javax.annotation.Nonnull; import lombok.Getter; import lombok.Value; @@ -80,14 +79,11 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.MetadataNotFoundException; import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionBound; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer.CursorInfo; -import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.util.Futures; -import org.apache.bookkeeper.mledger.util.ManagedLedgerImplUtils; -import org.apache.bookkeeper.net.BookieId; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.mutable.MutableInt; @@ -189,7 +185,6 @@ import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.topics.TopicCompactionStrategy; import org.apache.pulsar.common.util.Codec; -import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.compaction.CompactedTopicContext; @@ -689,32 +684,15 @@ private void asyncAddEntry(ByteBuf headersAndPayload, PublishContext publishCont } public void asyncReadEntry(Position position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { - if (ledger instanceof ManagedLedgerImpl) { - ((ManagedLedgerImpl) ledger).asyncReadEntry(position, callback, ctx); - } else { - callback.readEntryFailed(new ManagedLedgerException( - "Unexpected managedledger implementation, doesn't support " - + "direct read entry operation."), ctx); - } + ledger.asyncReadEntry(position, callback, ctx); } public Position getPositionAfterN(Position startPosition, long n) throws ManagedLedgerException { - if (ledger instanceof ManagedLedgerImpl) { - return ((ManagedLedgerImpl) ledger).getPositionAfterN(startPosition, n, - ManagedLedgerImpl.PositionBound.startExcluded); - } else { - throw new ManagedLedgerException("Unexpected managedledger implementation, doesn't support " - + "getPositionAfterN operation."); - } + return ledger.getPositionAfterN(startPosition, n, PositionBound.startExcluded); } public Position getFirstPosition() throws ManagedLedgerException { - if (ledger instanceof ManagedLedgerImpl) { - return ((ManagedLedgerImpl) ledger).getFirstPosition(); - } else { - throw new ManagedLedgerException("Unexpected managedledger implementation, doesn't support " - + "getFirstPosition operation."); - } + return ledger.getFirstPosition(); } public long getNumberOfEntries() { @@ -2545,7 +2523,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats topicStatsStream.writePair("msgThroughputOut", topicStatsHelper.aggMsgThroughputOut); topicStatsStream.writePair("storageSize", ledger.getTotalSize()); topicStatsStream.writePair("backlogSize", ledger.getEstimatedBacklogSize()); - topicStatsStream.writePair("pendingAddEntriesCount", ((ManagedLedgerImpl) ledger).getPendingAddEntriesCount()); + topicStatsStream.writePair("pendingAddEntriesCount", ledger.getPendingAddEntriesCount()); topicStatsStream.writePair("filteredEntriesCount", getFilteredEntriesCount()); nsStats.msgRateIn += topicStatsHelper.aggMsgRateIn; @@ -2558,7 +2536,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats bundleStats.msgRateOut += topicStatsHelper.aggMsgRateOut; bundleStats.msgThroughputIn += topicStatsHelper.aggMsgThroughputIn; bundleStats.msgThroughputOut += topicStatsHelper.aggMsgThroughputOut; - bundleStats.cacheSize += ((ManagedLedgerImpl) ledger).getCacheSize(); + bundleStats.cacheSize += ledger.getCacheSize(); // Close topic object topicStatsStream.endObject(); @@ -2756,194 +2734,158 @@ public CompletableFuture deleteSchema() { public CompletableFuture getInternalStats(boolean includeLedgerMetadata) { CompletableFuture statFuture = new CompletableFuture<>(); - PersistentTopicInternalStats stats = new PersistentTopicInternalStats(); - - ManagedLedgerImpl ml = (ManagedLedgerImpl) ledger; - stats.entriesAddedCounter = ml.getEntriesAddedCounter(); - stats.numberOfEntries = ml.getNumberOfEntries(); - stats.totalSize = ml.getTotalSize(); - stats.currentLedgerEntries = ml.getCurrentLedgerEntries(); - stats.currentLedgerSize = ml.getCurrentLedgerSize(); - stats.lastLedgerCreatedTimestamp = DateFormatter.format(ml.getLastLedgerCreatedTimestamp()); - if (ml.getLastLedgerCreationFailureTimestamp() != 0) { - stats.lastLedgerCreationFailureTimestamp = DateFormatter.format(ml.getLastLedgerCreationFailureTimestamp()); - } - - stats.waitingCursorsCount = ml.getWaitingCursorsCount(); - stats.pendingAddEntriesCount = ml.getPendingAddEntriesCount(); - - stats.lastConfirmedEntry = ml.getLastConfirmedEntry().toString(); - stats.state = ml.getState().toString(); - - stats.ledgers = new ArrayList<>(); - Set> futures = Sets.newConcurrentHashSet(); - CompletableFuture> availableBookiesFuture = - brokerService.pulsar().getPulsarResources().getBookieResources().listAvailableBookiesAsync(); - futures.add( - availableBookiesFuture - .whenComplete((bookies, e) -> { - if (e != null) { - log.error("[{}] Failed to fetch available bookies.", topic, e); - statFuture.completeExceptionally(e); - } else { - ml.getLedgersInfo().forEach((id, li) -> { - LedgerInfo info = new LedgerInfo(); - info.ledgerId = li.getLedgerId(); - info.entries = li.getEntries(); - info.size = li.getSize(); - info.offloaded = li.hasOffloadContext() && li.getOffloadContext().getComplete(); - stats.ledgers.add(info); - if (includeLedgerMetadata) { - futures.add(ml.getLedgerMetadata(li.getLedgerId()).handle((lMetadata, ex) -> { - if (ex == null) { - info.metadata = lMetadata; - } - return null; - })); - futures.add(ml.getEnsemblesAsync(li.getLedgerId()).handle((ensembles, ex) -> { - if (ex == null) { - info.underReplicated = - !bookies.containsAll(ensembles.stream().map(BookieId::toString) - .collect(Collectors.toList())); - } - return null; - })); - } - }); + + ledger.getManagedLedgerInternalStats(includeLedgerMetadata) + .thenCombine(getCompactedTopicContextAsync(), (ledgerInternalStats, compactedTopicContext) -> { + PersistentTopicInternalStats stats = new PersistentTopicInternalStats(); + stats.entriesAddedCounter = ledgerInternalStats.getEntriesAddedCounter(); + stats.numberOfEntries = ledgerInternalStats.getNumberOfEntries(); + stats.totalSize = ledgerInternalStats.getTotalSize(); + stats.currentLedgerEntries = ledgerInternalStats.getCurrentLedgerEntries(); + stats.currentLedgerSize = ledgerInternalStats.getCurrentLedgerSize(); + stats.lastLedgerCreatedTimestamp = ledgerInternalStats.getLastLedgerCreatedTimestamp(); + stats.lastLedgerCreationFailureTimestamp = ledgerInternalStats.getLastLedgerCreationFailureTimestamp(); + stats.waitingCursorsCount = ledgerInternalStats.getWaitingCursorsCount(); + stats.pendingAddEntriesCount = ledgerInternalStats.getPendingAddEntriesCount(); + stats.lastConfirmedEntry = ledgerInternalStats.getLastConfirmedEntry(); + stats.state = ledgerInternalStats.getState(); + stats.ledgers = ledgerInternalStats.ledgers; + + // Add ledger info for compacted topic ledger if exist. + LedgerInfo info = new LedgerInfo(); + info.ledgerId = -1; + info.entries = -1; + info.size = -1; + if (compactedTopicContext != null) { + info.ledgerId = compactedTopicContext.getLedger().getId(); + info.entries = compactedTopicContext.getLedger().getLastAddConfirmed() + 1; + info.size = compactedTopicContext.getLedger().getLength(); + } + + stats.compactedLedger = info; + + stats.cursors = new HashMap<>(); + ledger.getCursors().forEach(c -> { + CursorStats cs = new CursorStats(); + + CursorStats cursorInternalStats = c.getCursorStats(); + cs.markDeletePosition = cursorInternalStats.getMarkDeletePosition(); + cs.readPosition = cursorInternalStats.getReadPosition(); + cs.waitingReadOp = cursorInternalStats.isWaitingReadOp(); + cs.pendingReadOps = cursorInternalStats.getPendingReadOps(); + cs.messagesConsumedCounter = cursorInternalStats.getMessagesConsumedCounter(); + cs.cursorLedger = cursorInternalStats.getCursorLedger(); + cs.cursorLedgerLastEntry = cursorInternalStats.getCursorLedgerLastEntry(); + cs.individuallyDeletedMessages = cursorInternalStats.getIndividuallyDeletedMessages(); + cs.lastLedgerSwitchTimestamp = cursorInternalStats.getLastLedgerSwitchTimestamp(); + cs.state = cursorInternalStats.getState(); + cs.active = cursorInternalStats.isActive(); + cs.numberOfEntriesSinceFirstNotAckedMessage = + cursorInternalStats.getNumberOfEntriesSinceFirstNotAckedMessage(); + cs.totalNonContiguousDeletedMessagesRange = + cursorInternalStats.getTotalNonContiguousDeletedMessagesRange(); + cs.properties = cursorInternalStats.getProperties(); + // subscription metrics + PersistentSubscription sub = subscriptions.get(Codec.decode(c.getName())); + if (sub != null) { + if (sub.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) { + PersistentDispatcherMultipleConsumers dispatcher = + (PersistentDispatcherMultipleConsumers) sub.getDispatcher(); + cs.subscriptionHavePendingRead = dispatcher.havePendingRead; + cs.subscriptionHavePendingReplayRead = dispatcher.havePendingReplayRead; + } else if (sub.getDispatcher() instanceof PersistentDispatcherSingleActiveConsumer) { + PersistentDispatcherSingleActiveConsumer dispatcher = + (PersistentDispatcherSingleActiveConsumer) sub.getDispatcher(); + cs.subscriptionHavePendingRead = dispatcher.havePendingRead; + } } - }) - ); - - // Add ledger info for compacted topic ledger if exist. - LedgerInfo info = new LedgerInfo(); - info.ledgerId = -1; - info.entries = -1; - info.size = -1; - - futures.add(getCompactedTopicContextAsync().thenAccept(v -> { - if (v != null) { - info.ledgerId = v.getLedger().getId(); - info.entries = v.getLedger().getLastAddConfirmed() + 1; - info.size = v.getLedger().getLength(); - } - })); + stats.cursors.put(c.getName(), cs); + }); - stats.compactedLedger = info; - - stats.cursors = new HashMap<>(); - ml.getCursors().forEach(c -> { - ManagedCursorImpl cursor = (ManagedCursorImpl) c; - CursorStats cs = new CursorStats(); - cs.markDeletePosition = cursor.getMarkDeletedPosition().toString(); - cs.readPosition = cursor.getReadPosition().toString(); - cs.waitingReadOp = cursor.hasPendingReadRequest(); - cs.pendingReadOps = cursor.getPendingReadOpsCount(); - cs.messagesConsumedCounter = cursor.getMessagesConsumedCounter(); - cs.cursorLedger = cursor.getCursorLedger(); - cs.cursorLedgerLastEntry = cursor.getCursorLedgerLastEntry(); - cs.individuallyDeletedMessages = cursor.getIndividuallyDeletedMessages(); - cs.lastLedgerSwitchTimestamp = DateFormatter.format(cursor.getLastLedgerSwitchTimestamp()); - cs.state = cursor.getState(); - cs.active = cursor.isActive(); - cs.numberOfEntriesSinceFirstNotAckedMessage = cursor.getNumberOfEntriesSinceFirstNotAckedMessage(); - cs.totalNonContiguousDeletedMessagesRange = cursor.getTotalNonContiguousDeletedMessagesRange(); - cs.properties = cursor.getProperties(); - // subscription metrics - PersistentSubscription sub = subscriptions.get(Codec.decode(c.getName())); - if (sub != null) { - if (sub.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers) sub - .getDispatcher(); - cs.subscriptionHavePendingRead = dispatcher.havePendingRead; - cs.subscriptionHavePendingReplayRead = dispatcher.havePendingReplayRead; - } else if (sub.getDispatcher() instanceof PersistentDispatcherSingleActiveConsumer) { - PersistentDispatcherSingleActiveConsumer dispatcher = (PersistentDispatcherSingleActiveConsumer) sub - .getDispatcher(); - cs.subscriptionHavePendingRead = dispatcher.havePendingRead; + //Schema store ledgers + String schemaId; + try { + schemaId = TopicName.get(topic).getSchemaName(); + } catch (Throwable t) { + statFuture.completeExceptionally(t); + return null; } - } - stats.cursors.put(cursor.getName(), cs); - }); - //Schema store ledgers - String schemaId; - try { - schemaId = TopicName.get(topic).getSchemaName(); - } catch (Throwable t) { - statFuture.completeExceptionally(t); - return statFuture; - } - - - CompletableFuture schemaStoreLedgersFuture = new CompletableFuture<>(); - stats.schemaLedgers = Collections.synchronizedList(new ArrayList<>()); - if (brokerService.getPulsar().getSchemaStorage() != null - && brokerService.getPulsar().getSchemaStorage() instanceof BookkeeperSchemaStorage) { - ((BookkeeperSchemaStorage) brokerService.getPulsar().getSchemaStorage()) - .getStoreLedgerIdsBySchemaId(schemaId) - .thenAccept(ledgers -> { - List> getLedgerMetadataFutures = new ArrayList<>(); - ledgers.forEach(ledgerId -> { - CompletableFuture completableFuture = new CompletableFuture<>(); - getLedgerMetadataFutures.add(completableFuture); - CompletableFuture metadataFuture = null; - try { - metadataFuture = brokerService.getPulsar().getBookKeeperClient() - .getLedgerMetadata(ledgerId); - } catch (NullPointerException e) { - // related to bookkeeper issue https://github.com/apache/bookkeeper/issues/2741 - if (log.isDebugEnabled()) { - log.debug("{{}} Failed to get ledger metadata for the schema ledger {}", + + CompletableFuture schemaStoreLedgersFuture = new CompletableFuture<>(); + stats.schemaLedgers = Collections.synchronizedList(new ArrayList<>()); + if (brokerService.getPulsar().getSchemaStorage() != null + && brokerService.getPulsar().getSchemaStorage() instanceof BookkeeperSchemaStorage) { + ((BookkeeperSchemaStorage) brokerService.getPulsar().getSchemaStorage()) + .getStoreLedgerIdsBySchemaId(schemaId) + .thenAccept(ledgers -> { + List> getLedgerMetadataFutures = new ArrayList<>(); + ledgers.forEach(ledgerId -> { + CompletableFuture completableFuture = new CompletableFuture<>(); + getLedgerMetadataFutures.add(completableFuture); + CompletableFuture metadataFuture = null; + try { + metadataFuture = brokerService.getPulsar().getBookKeeperClient() + .getLedgerMetadata(ledgerId); + } catch (NullPointerException e) { + // related to bookkeeper issue https://github.com/apache/bookkeeper/issues/2741 + if (log.isDebugEnabled()) { + log.debug("{{}} Failed to get ledger metadata for the schema ledger {}", topic, ledgerId, e); - } - } - if (metadataFuture != null) { - metadataFuture.thenAccept(metadata -> { - LedgerInfo schemaLedgerInfo = new LedgerInfo(); - schemaLedgerInfo.ledgerId = metadata.getLedgerId(); - schemaLedgerInfo.entries = metadata.getLastEntryId() + 1; - schemaLedgerInfo.size = metadata.getLength(); - if (includeLedgerMetadata) { - info.metadata = metadata.toSafeString(); } - stats.schemaLedgers.add(schemaLedgerInfo); - completableFuture.complete(null); - }).exceptionally(e -> { - log.error("[{}] Failed to get ledger metadata for the schema ledger {}", + } + if (metadataFuture != null) { + metadataFuture.thenAccept(metadata -> { + LedgerInfo schemaLedgerInfo = new LedgerInfo(); + schemaLedgerInfo.ledgerId = metadata.getLedgerId(); + schemaLedgerInfo.entries = metadata.getLastEntryId() + 1; + schemaLedgerInfo.size = metadata.getLength(); + if (includeLedgerMetadata) { + info.metadata = metadata.toSafeString(); + } + stats.schemaLedgers.add(schemaLedgerInfo); + completableFuture.complete(null); + }).exceptionally(e -> { + log.error("[{}] Failed to get ledger metadata for the schema ledger {}", topic, ledgerId, e); - if ((e.getCause() instanceof BKNoSuchLedgerExistsOnMetadataServerException) + if ((e.getCause() instanceof BKNoSuchLedgerExistsOnMetadataServerException) || (e.getCause() instanceof BKNoSuchLedgerExistsException)) { - completableFuture.complete(null); + completableFuture.complete(null); + return null; + } + completableFuture.completeExceptionally(e); return null; - } - completableFuture.completeExceptionally(e); - return null; - }); - } else { - completableFuture.complete(null); - } - }); - FutureUtil.waitForAll(getLedgerMetadataFutures).thenRun(() -> { - schemaStoreLedgersFuture.complete(null); + }); + } else { + completableFuture.complete(null); + } + }); + FutureUtil.waitForAll(getLedgerMetadataFutures).thenRun(() -> { + schemaStoreLedgersFuture.complete(null); + }).exceptionally(e -> { + schemaStoreLedgersFuture.completeExceptionally(e); + return null; + }); }).exceptionally(e -> { schemaStoreLedgersFuture.completeExceptionally(e); return null; }); - }).exceptionally(e -> { - schemaStoreLedgersFuture.completeExceptionally(e); + } else { + schemaStoreLedgersFuture.complete(null); + } + schemaStoreLedgersFuture.whenComplete((r, ex) -> { + if (ex != null) { + statFuture.completeExceptionally(ex); + } else { + statFuture.complete(stats); + } + }); return null; - }); - } else { - schemaStoreLedgersFuture.complete(null); - } - schemaStoreLedgersFuture.thenRun(() -> - FutureUtil.waitForAll(futures).handle((res, ex) -> { - statFuture.complete(stats); + }) + .exceptionally(ex -> { + statFuture.completeExceptionally(ex); return null; - })).exceptionally(e -> { - statFuture.completeExceptionally(e); - return null; - }); + }); return statFuture; } @@ -3638,8 +3580,8 @@ public CompletableFuture checkTimeBacklogExceeded() { CompletableFuture future = new CompletableFuture<>(); // Check if first unconsumed message(first message after mark delete position) // for slowest cursor's has expired. - Position position = ((ManagedLedgerImpl) ledger).getNextValidPosition(oldestMarkDeletePosition); - ((ManagedLedgerImpl) ledger).asyncReadEntry(position, + Position position = ledger.getNextValidPosition(oldestMarkDeletePosition); + ledger.asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { @Override public void readEntryComplete(Entry entry, Object ctx) { @@ -3714,11 +3656,10 @@ private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaC Position markDeletePosition) throws ExecutionException, InterruptedException { int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); - ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) ledger; // The ledger timestamp is only known when ledger is closed, hence when the mark-delete // is at active ledger (open) we can't estimate it. - if (managedLedger.getLedgersInfo().lastKey().equals(markDeletePosition.getLedgerId())) { + if (ledger.getLedgersInfo().lastKey().equals(markDeletePosition.getLedgerId())) { return new EstimateTimeBasedBacklogQuotaCheckResult(false, null); } @@ -3731,14 +3672,14 @@ private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaC // if the mark-delete position is the last entry it means all entries for // that ledger are acknowledged if (markDeletePosition.getEntryId() == markDeletePositionLedgerInfo.getEntries() - 1) { - Position positionToCheck = managedLedger.getNextValidPosition(markDeletePosition); + Position positionToCheck = ledger.getNextValidPosition(markDeletePosition); positionToCheckLedgerInfo = ledger.getLedgerInfo(positionToCheck.getLedgerId()).get(); } if (positionToCheckLedgerInfo != null && positionToCheckLedgerInfo.hasTimestamp() && positionToCheckLedgerInfo.getTimestamp() > 0) { - long estimateMsgAgeMs = managedLedger.getClock().millis() - positionToCheckLedgerInfo.getTimestamp(); + long estimateMsgAgeMs = clock.millis() - positionToCheckLedgerInfo.getTimestamp(); boolean shouldTruncateBacklog = estimateMsgAgeMs > SECONDS.toMillis(backlogQuotaLimitInSecond); if (log.isDebugEnabled()) { log.debug("Time based backlog quota exceeded, quota {}[ms], age of ledger " @@ -3890,24 +3831,22 @@ public CompletableFuture getLastDispatchablePosition() { if (lastDispatchablePosition != null) { return CompletableFuture.completedFuture(lastDispatchablePosition); } - return ManagedLedgerImplUtils - .asyncGetLastValidPosition((ManagedLedgerImpl) ledger, entry -> { - MessageMetadata md = Commands.parseMessageMetadata(entry.getDataBuffer()); - // If a messages has marker will filter by AbstractBaseDispatcher.filterEntriesForConsumer - if (Markers.isServerOnlyMarker(md)) { - return false; - } else if (md.hasTxnidMostBits() && md.hasTxnidLeastBits()) { - // Filter-out transaction aborted messages. - TxnID txnID = new TxnID(md.getTxnidMostBits(), md.getTxnidLeastBits()); - return !isTxnAborted(txnID, entry.getPosition()); - } - return true; - }, getMaxReadPosition()) - .thenApply(position -> { - // Update lastDispatchablePosition to the given position - updateLastDispatchablePosition(position); - return position; - }); + return ledger.getLastDispatchablePosition(entry -> { + MessageMetadata md = Commands.parseMessageMetadata(entry.getDataBuffer()); + // If a messages has marker will filter by AbstractBaseDispatcher.filterEntriesForConsumer + if (Markers.isServerOnlyMarker(md)) { + return false; + } else if (md.hasTxnidMostBits() && md.hasTxnidLeastBits()) { + // Filter-out transaction aborted messages. + TxnID txnID = new TxnID(md.getTxnidMostBits(), md.getTxnidLeastBits()); + return !isTxnAborted(txnID, entry.getPosition()); + } + return true; + }, getMaxReadPosition()).thenApply(position -> { + // Update lastDispatchablePosition to the given position + updateLastDispatchablePosition(position); + return position; + }); } /** @@ -3952,13 +3891,13 @@ public CompletableFuture getLastMessageId() { .complete(new MessageIdImpl(position.getLedgerId(), position.getEntryId(), partitionIndex)); return completableFuture; } - ManagedLedgerImpl ledgerImpl = (ManagedLedgerImpl) ledger; - if (!ledgerImpl.ledgerExists(position.getLedgerId())) { + + if (!ledger.getLedgersInfo().containsKey(position.getLedgerId())) { completableFuture .complete(MessageId.earliest); return completableFuture; } - ledgerImpl.asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { + ledger.asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { @Override public void readEntryComplete(Entry entry, Object ctx) { try { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java index f87abcf495308..489d37dd0a307 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java @@ -24,9 +24,8 @@ import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryMXBean; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerMBeanImpl; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.common.policies.data.TopicStats; @@ -132,7 +131,7 @@ protected Metrics createMetrics(Map dimensionMap) { * @return */ protected ManagedLedgerFactoryMXBean getManagedLedgerCacheStats() { - return ((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getCacheStats(); + return pulsar.getManagedLedgerFactory().getCacheStats(); } /** @@ -140,8 +139,8 @@ protected ManagedLedgerFactoryMXBean getManagedLedgerCacheStats() { * * @return */ - protected Map getManagedLedgers() { - return ((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getManagedLedgers(); + protected Map getManagedLedgers() { + return pulsar.getManagedLedgerFactory().getManagedLedgers(); } protected String getLocalClusterName() { @@ -235,8 +234,8 @@ protected void populateMaxMap(Map map, String mkey, long value) { * @param metrics * @param ledger */ - protected void populateDimensionMap(Map> ledgersByDimensionMap, Metrics metrics, - ManagedLedgerImpl ledger) { + protected void populateDimensionMap(Map> ledgersByDimensionMap, Metrics metrics, + ManagedLedger ledger) { ledgersByDimensionMap.computeIfAbsent(metrics, __ -> new ArrayList<>()).add(ledger); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedCursorMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedCursorMetrics.java index 424a7cb2f81ac..639f51ead6cee 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedCursorMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedCursorMetrics.java @@ -25,9 +25,7 @@ import java.util.Map; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursorMXBean; -import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; -import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.common.stats.Metrics; @@ -55,16 +53,15 @@ public synchronized List generate() { */ private List aggregate() { metricsCollection.clear(); - for (Map.Entry e : getManagedLedgers().entrySet()) { + for (Map.Entry e : getManagedLedgers().entrySet()) { String ledgerName = e.getKey(); - ManagedLedgerImpl ledger = e.getValue(); + ManagedLedger ledger = e.getValue(); String namespace = parseNamespaceFromLedgerName(ledgerName); - ManagedCursorContainer cursorContainer = ledger.getCursors(); - Iterator cursorIterator = cursorContainer.iterator(); + Iterator cursorIterator = ledger.getCursors().iterator(); while (cursorIterator.hasNext()) { - ManagedCursorImpl cursor = (ManagedCursorImpl) cursorIterator.next(); + ManagedCursor cursor = cursorIterator.next(); ManagedCursorMXBean cStats = cursor.getStats(); dimensionMap.clear(); dimensionMap.put("namespace", namespace); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java index 36004bc1281bb..52c69265c2f1f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java @@ -23,16 +23,15 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerMXBean; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.common.stats.Metrics; public class ManagedLedgerMetrics extends AbstractMetrics { private List metricsCollection; - private Map> ledgersByDimensionMap; + private Map> ledgersByDimensionMap; // temp map to prepare aggregation metrics private Map tempAggregatedMetricsMap; private static final Buckets @@ -53,7 +52,7 @@ public ManagedLedgerMetrics(PulsarService pulsar) { this.metricsCollection = new ArrayList<>(); this.ledgersByDimensionMap = new HashMap<>(); this.tempAggregatedMetricsMap = new HashMap<>(); - this.statsPeriodSeconds = ((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()) + this.statsPeriodSeconds = pulsar.getManagedLedgerFactory() .getConfig().getStatsPeriodSeconds(); } @@ -71,20 +70,20 @@ public synchronized List generate() { * @param ledgersByDimension * @return */ - private List aggregate(Map> ledgersByDimension) { + private List aggregate(Map> ledgersByDimension) { metricsCollection.clear(); - for (Entry> e : ledgersByDimension.entrySet()) { + for (Entry> e : ledgersByDimension.entrySet()) { Metrics metrics = e.getKey(); - List ledgers = e.getValue(); + List ledgers = e.getValue(); // prepare aggregation map tempAggregatedMetricsMap.clear(); // generate the collections by each metrics and then apply the aggregation - for (ManagedLedgerImpl ledger : ledgers) { + for (ManagedLedger ledger : ledgers) { ManagedLedgerMXBean lStats = ledger.getStats(); populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_ml_AddEntryBytesRate", @@ -151,17 +150,17 @@ private List aggregate(Map> ledgersByD * * @return */ - private Map> groupLedgersByDimension() { + private Map> groupLedgersByDimension() { ledgersByDimensionMap.clear(); // get the current topics statistics from StatsBrokerFilter // Map : topic-name->dest-stat - for (Entry e : getManagedLedgers().entrySet()) { + for (Entry e : getManagedLedgers().entrySet()) { String ledgerName = e.getKey(); - ManagedLedgerImpl ledger = e.getValue(); + ManagedLedger ledger = e.getValue(); // we want to aggregate by NS dimension diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java index 1649349e3e6f6..a0ffa121b8999 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java @@ -24,7 +24,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService.ReferenceCountedWriter; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -69,8 +68,8 @@ public void putAbortedTxnAndPosition(TxnID abortedTxnId, Position abortedMarkerP //In this implementation we clear the invalid aborted txn ID one by one. @Override public void trimExpiredAbortedTxns() { - while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger()) - .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) { + while (!aborts.isEmpty() && !topic.getManagedLedger().getLedgersInfo() + .containsKey(aborts.get(aborts.firstKey()).getLedgerId())) { if (log.isDebugEnabled()) { log.debug("[{}] Topic transaction buffer clear aborted transaction, TxnId : {}, Position : {}", topic.getName(), aborts.firstKey(), aborts.get(aborts.firstKey())); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java index 4ca27f77a87f5..88a3968b7b430 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java @@ -38,8 +38,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; +import org.apache.bookkeeper.mledger.ReadOnlyManagedLedger; import org.apache.commons.collections4.map.LinkedMap; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -189,8 +188,8 @@ public boolean checkAbortedTransaction(TxnID txnID) { public void trimExpiredAbortedTxns() { //Checking whether there are some segment expired. List positionsNeedToDelete = new ArrayList<>(); - while (!segmentIndex.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger()) - .ledgerExists(segmentIndex.firstKey().getLedgerId())) { + while (!segmentIndex.isEmpty() && !topic.getManagedLedger().getLedgersInfo() + .containsKey(segmentIndex.firstKey().getLedgerId())) { if (log.isDebugEnabled()) { log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}", topic.getName(), segmentIndex.firstKey()); @@ -275,8 +274,8 @@ private void readSegmentEntries(TopicName topicName, TransactionBufferSnapshotIn entry.release(); } } catch (Throwable throwable) { - if (((ManagedLedgerImpl) topic.getManagedLedger()) - .ledgerExists(index.getAbortedMarkLedgerID())) { + if (topic.getManagedLedger().getLedgersInfo() + .containsKey(index.getAbortedMarkLedgerID())) { log.error("[{}] Failed to read snapshot segment [{}:{}]", topic.getName(), index.segmentLedgerID, index.segmentEntryID, throwable); @@ -293,11 +292,11 @@ private void readSegmentEntries(TopicName topicName, TransactionBufferSnapshotIn } } - private ReadOnlyManagedLedgerImpl openReadOnlyManagedLedger(TopicName topicName) throws Exception { - final var future = new CompletableFuture(); + private ReadOnlyManagedLedger openReadOnlyManagedLedger(TopicName topicName) throws Exception { + final var future = new CompletableFuture(); final var callback = new AsyncCallbacks.OpenReadOnlyManagedLedgerCallback() { @Override - public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl managedLedger, Object ctx) { + public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedger managedLedger, Object ctx) { future.complete(managedLedger); } @@ -317,7 +316,7 @@ public String toString() { return wait(future, "open read only ml for " + topicName); } - private Entry readEntry(ReadOnlyManagedLedgerImpl managedLedger, Position position) throws Exception { + private Entry readEntry(ReadOnlyManagedLedger managedLedger, Position position) throws Exception { final var future = new CompletableFuture(); managedLedger.asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index 2f90ff8922a81..41977e6b61d88 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -38,7 +38,6 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceException; @@ -320,7 +319,7 @@ private void handleTransactionMessage(TxnID txnId, Position position) { ongoingTxns.put(txnId, position); Position firstPosition = ongoingTxns.get(ongoingTxns.firstKey()); // max read position is less than first ongoing transaction message position - updateMaxReadPosition(((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(firstPosition), + updateMaxReadPosition(topic.getManagedLedger().getPreviousPosition(firstPosition), false); } } @@ -488,7 +487,7 @@ void removeTxnAndUpdateMaxReadPosition(TxnID txnID) { ongoingTxns.remove(txnID); if (!ongoingTxns.isEmpty()) { Position position = ongoingTxns.get(ongoingTxns.firstKey()); - updateMaxReadPosition(((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(position), false); + updateMaxReadPosition(topic.getManagedLedger().getPreviousPosition(position), false); } else { updateMaxReadPosition(topic.getManagedLedger().getLastConfirmedEntry(), false); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java index f8143cfc4c125..25c7727259db3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java @@ -32,6 +32,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Predicate; @@ -44,7 +45,6 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceException; import org.apache.pulsar.broker.transaction.pendingack.PendingAckReplyCallBack; @@ -121,7 +121,7 @@ public class MLPendingAckStore implements PendingAckStore { public MLPendingAckStore(ManagedLedger managedLedger, ManagedCursor cursor, ManagedCursor subManagedCursor, long transactionPendingAckLogIndexMinLag, TxnLogBufferedWriterConfig bufferedWriterConfig, - Timer timer, TxnLogBufferedWriterMetricsStats bufferedWriterMetrics) { + Timer timer, TxnLogBufferedWriterMetricsStats bufferedWriterMetrics, Executor executor) { this.managedLedger = managedLedger; this.cursor = cursor; this.currentLoadPosition = this.cursor.getMarkDeletedPosition(); @@ -131,7 +131,7 @@ public MLPendingAckStore(ManagedLedger managedLedger, ManagedCursor cursor, this.subManagedCursor = subManagedCursor; this.logIndexBackoff = new LogIndexLagBackoff(transactionPendingAckLogIndexMinLag, Long.MAX_VALUE, 1); this.maxIndexLag = logIndexBackoff.next(0); - this.bufferedWriter = new TxnLogBufferedWriter(managedLedger, ((ManagedLedgerImpl) managedLedger).getExecutor(), + this.bufferedWriter = new TxnLogBufferedWriter(managedLedger, executor, timer, PendingAckLogSerializer.INSTANCE, bufferedWriterConfig.getBatchedWriteMaxRecords(), bufferedWriterConfig.getBatchedWriteMaxSize(), bufferedWriterConfig.getBatchedWriteMaxDelayInMillis(), bufferedWriterConfig.isBatchEnabled(), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreProvider.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreProvider.java index 5308648b80c1d..6fc61d423ce85 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreProvider.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreProvider.java @@ -134,7 +134,12 @@ public void openCursorComplete(ManagedCursor cursor, Object ctx) { .getConfiguration() .getTransactionPendingAckLogIndexMinLag(), txnLogBufferedWriterConfig, - brokerClientSharedTimer, bufferedWriterMetrics)); + brokerClientSharedTimer, bufferedWriterMetrics, + originPersistentTopic + .getBrokerService() + .getPulsar() + .getOrderedExecutor() + .chooseThread())); if (log.isDebugEnabled()) { log.debug("{},{} open MLPendingAckStore cursor success", originPersistentTopic.getName(), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java index 27408854b0198..591842927f35b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java @@ -47,7 +47,6 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.AckSetState; import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; -import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.commons.lang3.tuple.Pair; @@ -235,8 +234,7 @@ public void internalIndividualAcknowledgeMessage(TxnID txnID, List findStartPoint(cursorPosition, context.ledger.getLastAddConfirmed(), context.cache) @@ -143,7 +141,7 @@ public void asyncReadEntriesOrWait(ManagedCursor cursor, for (Entry entry : entries) { entriesSize += entry.getLength(); } - managedCursor.updateReadStats(entries.size(), entriesSize); + cursor.updateReadStats(entries.size(), entriesSize); Entry lastEntry = entries.get(entries.size() - 1); // The compaction task depends on the last snapshot and the incremental diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java index 5023180e0b979..a7a5fd4ef1113 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java @@ -31,7 +31,6 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.commons.collections4.CollectionUtils; import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; @@ -75,8 +74,7 @@ public static void asyncReadCompactedEntries(TopicCompactionService topicCompact return CompletableFuture.completedFuture(null); } - ManagedCursorImpl managedCursor = (ManagedCursorImpl) cursor; - int numberOfEntriesToRead = managedCursor.applyMaxSizeCap(maxEntries, bytesToRead); + int numberOfEntriesToRead = cursor.applyMaxSizeCap(maxEntries, bytesToRead); return topicCompactionService.readCompactedEntries(readPosition, numberOfEntriesToRead) .thenAccept(entries -> { @@ -94,7 +92,7 @@ public static void asyncReadCompactedEntries(TopicCompactionService topicCompact for (Entry entry : entries) { entriesSize += entry.getLength(); } - managedCursor.updateReadStats(entries.size(), entriesSize); + cursor.updateReadStats(entries.size(), entriesSize); Entry lastEntry = entries.get(entries.size() - 1); cursor.seek(lastEntry.getPosition().getNext(), true); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/MockManagedCursor.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/MockManagedCursor.java index e0e679b113f33..cf0b3c45d7023 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/MockManagedCursor.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/MockManagedCursor.java @@ -32,6 +32,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; public class MockManagedCursor implements ManagedCursor { @@ -414,4 +415,34 @@ public boolean checkAndUpdateReadPositionChanged() { public boolean isClosed() { return false; } + + @Override + public ManagedLedgerInternalStats.CursorStats getCursorStats() { + return null; + } + + @Override + public boolean isMessageDeleted(Position position) { + return false; + } + + @Override + public ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) throws ManagedLedgerException { + return null; + } + + @Override + public long[] getBatchPositionAckSet(Position position) { + return new long[0]; + } + + @Override + public int applyMaxSizeCap(int maxEntries, long maxSizeBytes) { + return 0; + } + + @Override + public void updateReadStats(int readEntriesCount, long readEntriesSize) { + + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java index 16953d76ade45..74a88382b0e0e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/MangedLedgerInterceptorImplTest.java @@ -41,7 +41,6 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.OpAddEntry; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.pulsar.common.api.proto.BrokerEntryMetadata; @@ -391,16 +390,15 @@ public MockManagedLedgerInterceptorImpl( } @Override - public OpAddEntry beforeAddEntry(OpAddEntry op, int numberOfMessages) { + public void beforeAddEntry(AddEntryOperation op, int numberOfMessages) { if (op == null || numberOfMessages <= 0) { - return op; + return; } op.setData(Commands.addBrokerEntryMetadata(op.getData(), brokerEntryMetadataInterceptors, numberOfMessages)); if (op != null) { throw new RuntimeException("throw exception before add entry for test"); } - return op; } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CurrentLedgerRolloverIfFullTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CurrentLedgerRolloverIfFullTest.java index 375fe41f143cd..4f83d25a29210 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CurrentLedgerRolloverIfFullTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/CurrentLedgerRolloverIfFullTest.java @@ -81,7 +81,9 @@ public void testCurrentLedgerRolloverIfFull() throws Exception { } ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); - Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), msgNum / 2); + Awaitility.await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> { + Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), msgNum / 2 + 1); + }); for (int i = 0; i < msgNum; i++) { Message msg = consumer.receive(2, TimeUnit.SECONDS); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index ff8e418c024a0..20ea33fb3e1ed 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.broker.service.persistent; -import static org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.CURSOR_INTERNAL_PROPERTY_PREFIX; +import static org.apache.bookkeeper.mledger.ManagedCursor.CURSOR_INTERNAL_PROPERTY_PREFIX; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; @@ -38,7 +38,6 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.mledger.ManagedCursor; -import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.BrokerTestUtil; @@ -104,7 +103,7 @@ public void testBucketDelayedDeliveryWithAllConsumersDisconnecting() throws Exce Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher.getNumberOfDelayedMessages(), 1000)); List bucketKeys = ((PersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties().keySet().stream() - .filter(x -> x.startsWith(ManagedCursorImpl.CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); + .filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); c1.close(); @@ -119,7 +118,7 @@ public void testBucketDelayedDeliveryWithAllConsumersDisconnecting() throws Exce Dispatcher dispatcher2 = pulsar.getBrokerService().getTopicReference(topic).get().getSubscription("sub").getDispatcher(); List bucketKeys2 = ((PersistentDispatcherMultipleConsumers) dispatcher2).getCursor().getCursorProperties().keySet().stream() - .filter(x -> x.startsWith(ManagedCursorImpl.CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); + .filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher2.getNumberOfDelayedMessages(), 1000)); Assert.assertEquals(bucketKeys, bucketKeys2); @@ -384,7 +383,7 @@ public void testDeleteTopicIfCursorPropsEmpty(SubscriptionType subscriptionType) admin.topics().createSubscription(topic, subscriptionName, MessageId.earliest); pulsarClient.newConsumer().topic(topic).subscriptionName(subscriptionName) .subscriptionType(subscriptionType).subscribe().close(); - ManagedCursorImpl cursor = findCursor(topic, subscriptionName); + ManagedCursor cursor = findCursor(topic, subscriptionName); assertNotNull(cursor); assertTrue(cursor.getCursorProperties() == null || cursor.getCursorProperties().isEmpty()); // Test topic deletion is successful. @@ -404,7 +403,7 @@ public void testDeletePartitionedTopicIfCursorPropsEmpty(SubscriptionType subscr admin.topics().createSubscription(topic, subscriptionName, MessageId.earliest); pulsarClient.newConsumer().topic(topic).subscriptionName(subscriptionName) .subscriptionType(subscriptionType).subscribe().close(); - ManagedCursorImpl cursor = findCursor(topic + "-partition-0", subscriptionName); + ManagedCursor cursor = findCursor(topic + "-partition-0", subscriptionName); assertNotNull(cursor); assertTrue(cursor.getCursorProperties() == null || cursor.getCursorProperties().isEmpty()); // Test topic deletion is successful. @@ -424,7 +423,7 @@ public void testDeleteTopicIfCursorPropsNotEmpty(SubscriptionType subscriptionTy admin.topics().createSubscription(topic, subscriptionName, MessageId.earliest); pulsarClient.newConsumer().topic(topic).subscriptionName(subscriptionName) .subscriptionType(subscriptionType).subscribe().close(); - ManagedCursorImpl cursor = findCursor(topic, subscriptionName); + ManagedCursor cursor = findCursor(topic, subscriptionName); assertNotNull(cursor); assertTrue(cursor.getCursorProperties() == null || cursor.getCursorProperties().isEmpty()); // Put a subscription prop. @@ -451,7 +450,7 @@ public void testDeletePartitionedTopicIfCursorPropsNotEmpty(SubscriptionType sub pulsarClient.newConsumer().topic(topic).subscriptionName(subscriptionName) .subscriptionType(subscriptionType).subscribe().close(); - ManagedCursorImpl cursor = findCursor(topic + "-partition-0", subscriptionName); + ManagedCursor cursor = findCursor(topic + "-partition-0", subscriptionName); assertNotNull(cursor); assertTrue(cursor.getCursorProperties() == null || cursor.getCursorProperties().isEmpty()); // Put a subscription prop. @@ -464,7 +463,7 @@ public void testDeletePartitionedTopicIfCursorPropsNotEmpty(SubscriptionType sub } - private ManagedCursorImpl findCursor(String topic, String subscriptionName) { + private ManagedCursor findCursor(String topic, String subscriptionName) { PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); Iterator cursorIterator = persistentTopic.getManagedLedger().getCursors().iterator(); @@ -473,7 +472,7 @@ private ManagedCursorImpl findCursor(String topic, String subscriptionName) { if (managedCursor == null || !managedCursor.getName().equals(subscriptionName)) { continue; } - return (ManagedCursorImpl) managedCursor; + return managedCursor; } return null; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java index b9c0ab08e4ea1..d0fd384ba78fb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java @@ -29,9 +29,9 @@ import java.util.Map.Entry; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; -import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerMBeanImpl; import org.apache.bookkeeper.mledger.impl.OpenTelemetryManagedLedgerStats; import org.apache.pulsar.broker.BrokerTestUtil; @@ -106,7 +106,7 @@ public void testManagedLedgerMetrics() throws Exception { } var managedLedgerFactory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory(); - for (Entry ledger : managedLedgerFactory.getManagedLedgers().entrySet()) { + for (Entry ledger : managedLedgerFactory.getManagedLedgers().entrySet()) { ManagedLedgerMBeanImpl stats = (ManagedLedgerMBeanImpl) ledger.getValue().getStats(); stats.refreshStats(1, TimeUnit.SECONDS); } @@ -118,7 +118,7 @@ public void testManagedLedgerMetrics() throws Exception { String message = "my-message-" + i; producer.send(message.getBytes()); } - for (Entry ledger : managedLedgerFactory.getManagedLedgers().entrySet()) { + for (Entry ledger : managedLedgerFactory.getManagedLedgers().entrySet()) { ManagedLedgerMBeanImpl stats = (ManagedLedgerMBeanImpl) ledger.getValue().getStats(); stats.refreshStats(1, TimeUnit.SECONDS); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 8ab9d58f57076..3924281c094b1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -49,8 +49,8 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.ReadOnlyManagedLedger; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.commons.collections4.map.LinkedMap; import org.apache.commons.lang3.RandomUtils; @@ -796,7 +796,7 @@ public void testTransactionBufferSegmentSystemTopic() throws Exception { AsyncCallbacks.OpenReadOnlyManagedLedgerCallback callback = new AsyncCallbacks .OpenReadOnlyManagedLedgerCallback() { @Override - public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl readOnlyManagedLedger, Object ctx) { + public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedger readOnlyManagedLedger, Object ctx) { readOnlyManagedLedger.asyncReadEntry( PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()), new AsyncCallbacks.ReadEntryCallback() { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index 3b3eaf7bb2292..d3e0391443f0f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -864,7 +864,8 @@ public void testEndTPRecoveringWhenManagerLedgerDisReadable() throws Exception{ doReturn(CompletableFuture.completedFuture( new MLPendingAckStore(persistentTopic.getManagedLedger(), managedCursor, null, 500, bufferedWriterConfig, transactionTimer, - DISABLED_BUFFERED_WRITER_METRICS))) + DISABLED_BUFFERED_WRITER_METRICS, persistentTopic.getBrokerService().getPulsar() + .getOrderedExecutor().chooseThread()))) .when(pendingAckStoreProvider).newPendingAckStore(any()); doReturn(CompletableFuture.completedFuture(true)).when(pendingAckStoreProvider).checkInitializedBefore(any()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckMetadataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckMetadataTest.java index efe83cebc3661..f01ea7ac67ff9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckMetadataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckMetadataTest.java @@ -18,9 +18,19 @@ */ package org.apache.pulsar.broker.transaction.pendingack; +import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.State.WriteFailed; +import static org.apache.pulsar.transaction.coordinator.impl.DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS; +import static org.testng.Assert.assertTrue; +import static org.testng.AssertJUnit.fail; import io.netty.util.HashedWheelTimer; import io.netty.util.concurrent.DefaultThreadFactory; +import java.lang.reflect.Field; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import lombok.Cleanup; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -36,14 +46,6 @@ import org.apache.pulsar.common.api.proto.CommandAck; import org.apache.pulsar.transaction.coordinator.impl.TxnLogBufferedWriterConfig; import org.testng.annotations.Test; -import java.lang.reflect.Field; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; -import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.State.WriteFailed; -import static org.apache.pulsar.transaction.coordinator.impl.DisabledTxnLogBufferedWriterMetricsStats.DISABLED_BUFFERED_WRITER_METRICS; -import static org.testng.Assert.assertTrue; -import static org.testng.AssertJUnit.fail; public class PendingAckMetadataTest extends MockedBookKeeperTestCase { @@ -80,9 +82,13 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { ManagedCursor cursor = completableFuture.get().openCursor("test"); ManagedCursor subCursor = completableFuture.get().openCursor("test"); + + @Cleanup("shutdownNow") + ExecutorService executorService = Executors.newSingleThreadExecutor(); + MLPendingAckStore pendingAckStore = new MLPendingAckStore(completableFuture.get(), cursor, subCursor, 500, - bufferedWriterConfig, transactionTimer, DISABLED_BUFFERED_WRITER_METRICS); + bufferedWriterConfig, transactionTimer, DISABLED_BUFFERED_WRITER_METRICS, executorService); Field field = MLPendingAckStore.class.getDeclaredField("managedLedger"); field.setAccessible(true); diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ManagedLedgerInternalStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ManagedLedgerInternalStats.java index 95a45d37d9556..b68b6308c8f3b 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ManagedLedgerInternalStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ManagedLedgerInternalStats.java @@ -20,10 +20,14 @@ import java.util.List; import java.util.Map; +import lombok.AccessLevel; +import lombok.Getter; + /** * ManagedLedger internal statistics. */ +@Getter(AccessLevel.PUBLIC) public class ManagedLedgerInternalStats { /** Messages published since this broker loaded this managedLedger. */ @@ -82,6 +86,7 @@ public static class LedgerInfo { /** * Pulsar cursor statistics. */ + @Getter(AccessLevel.PUBLIC) public static class CursorStats { public String markDeletePosition; public String readPosition; diff --git a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionSequenceIdGenerator.java b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionSequenceIdGenerator.java index 204555a1cfc67..a6605046eeff6 100644 --- a/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionSequenceIdGenerator.java +++ b/pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionSequenceIdGenerator.java @@ -22,9 +22,7 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; -import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.client.api.LedgerEntry; -import org.apache.bookkeeper.mledger.impl.OpAddEntry; +import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; import org.apache.commons.collections4.CollectionUtils; import org.apache.pulsar.transaction.coordinator.proto.TransactionMetadataEntry; @@ -42,8 +40,8 @@ public class MLTransactionSequenceIdGenerator implements ManagedLedgerIntercepto private final AtomicLong sequenceId = new AtomicLong(TC_ID_NOT_USED); @Override - public OpAddEntry beforeAddEntry(OpAddEntry op, int numberOfMessages) { - return op; + public void beforeAddEntry(AddEntryOperation op, int numberOfMessages) { + // do nothing } // When all of ledger have been deleted, we will generate sequenceId from managedLedger properties @@ -60,43 +58,23 @@ public void onManagedLedgerPropertiesInitialize(Map propertiesMa // When we don't roll over ledger, we can init sequenceId from the getLastAddConfirmed transaction metadata entry @Override - public CompletableFuture onManagedLedgerLastLedgerInitialize(String name, LedgerHandle lh) { - CompletableFuture promise = new CompletableFuture<>(); - if (lh.getLastAddConfirmed() >= 0) { - lh.readAsync(lh.getLastAddConfirmed(), lh.getLastAddConfirmed()).whenComplete((entries, ex) -> { - if (ex != null) { - log.error("[{}] Read last entry error.", name, ex); - promise.completeExceptionally(ex); - } else { - if (entries != null) { - try { - LedgerEntry ledgerEntry = entries.getEntry(lh.getLastAddConfirmed()); - if (ledgerEntry != null) { - List transactionLogs = - MLTransactionLogImpl.deserializeEntry(ledgerEntry.getEntryBuffer()); - if (!CollectionUtils.isEmpty(transactionLogs)){ - TransactionMetadataEntry lastConfirmEntry = - transactionLogs.get(transactionLogs.size() - 1); - this.sequenceId.set(lastConfirmEntry.getMaxLocalTxnId()); - } - } - entries.close(); - promise.complete(null); - } catch (Exception e) { - entries.close(); - log.error("[{}] Failed to recover the tc sequenceId from the last add confirmed entry.", - name, e); - promise.completeExceptionally(e); - } - } else { - promise.complete(null); + public CompletableFuture onManagedLedgerLastLedgerInitialize(String name, LastEntryHandle lh) { + return lh.readLastEntryAsync().thenAccept(lastEntryOptional -> { + if (lastEntryOptional.isPresent()) { + Entry lastEntry = lastEntryOptional.get(); + try { + List transactionLogs = + MLTransactionLogImpl.deserializeEntry(lastEntry.getDataBuffer()); + if (!CollectionUtils.isEmpty(transactionLogs)) { + TransactionMetadataEntry lastConfirmEntry = + transactionLogs.get(transactionLogs.size() - 1); + this.sequenceId.set(lastConfirmEntry.getMaxLocalTxnId()); } + } finally { + lastEntry.release(); } - }); - } else { - promise.complete(null); - } - return promise; + } + }); } // roll over ledger will update sequenceId to managedLedger properties diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java index 8f52d20c5ee83..60dcbb8b3acd8 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java @@ -18,8 +18,10 @@ */ package org.apache.bookkeeper.mledger.offload.jcloud.impl; +import com.google.common.collect.Range; import io.netty.buffer.ByteBuf; import java.util.Map; +import java.util.NavigableMap; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.function.Predicate; @@ -32,6 +34,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerMXBean; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionBound; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.pulsar.common.api.proto.CommandSubscribe; @@ -177,6 +180,11 @@ public long getNumberOfEntries() { return 0; } + @Override + public long getNumberOfEntries(Range range) { + return 0; + } + @Override public long getNumberOfActiveEntries() { return 0; @@ -381,6 +389,51 @@ public void checkCursorsToCacheEntries() { // no-op } + @Override + public void asyncReadEntry(Position position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { + + } + + @Override + public NavigableMap getLedgersInfo() { + return null; + } + + @Override + public Position getNextValidPosition(Position position) { + return null; + } + + @Override + public Position getPreviousPosition(Position position) { + return null; + } + + @Override + public long getEstimatedBacklogSize(Position position) { + return 0; + } + + @Override + public Position getPositionAfterN(Position startPosition, long n, PositionBound startRange) { + return null; + } + + @Override + public int getPendingAddEntriesCount() { + return 0; + } + + @Override + public long getCacheSize() { + return 0; + } + + @Override + public Position getFirstPosition() { + return null; + } + @Override public CompletableFuture asyncMigrate() { // no-op From 2e987369fa4444dc303ba2674e94b7a712710d64 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Tue, 17 Sep 2024 20:46:09 +0800 Subject: [PATCH 504/580] [feat][client] PIP-374: Visibility of messages in receiverQueue for the consumers (#23303) --- pip/pip-374.md | 4 +- .../pulsar/client/api/InterceptorsTest.java | 96 +++++++++++++++++++ .../client/api/ConsumerInterceptor.java | 38 ++++++++ .../pulsar/client/impl/ConsumerBase.java | 8 ++ .../pulsar/client/impl/ConsumerImpl.java | 5 +- .../client/impl/ConsumerInterceptors.java | 32 +++++++ .../client/impl/MultiTopicsConsumerImpl.java | 5 + 7 files changed, 184 insertions(+), 4 deletions(-) diff --git a/pip/pip-374.md b/pip/pip-374.md index 4264617647433..49fe337159628 100644 --- a/pip/pip-374.md +++ b/pip/pip-374.md @@ -67,5 +67,5 @@ Since we added a default method onArrival() in interface, one who has provided t -* Mailing List discussion thread: -* Mailing List voting thread: +* Mailing List discussion thread: https://lists.apache.org/thread/hcfpm4j6hpwxb2olfrro8g4dls35q8rx +* Mailing List voting thread: https://lists.apache.org/thread/wrr02s4cdzqmo1vonp92w6229qo0rv0z diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java index afb17a186477c..8115f34121d3c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InterceptorsTest.java @@ -33,6 +33,7 @@ import lombok.Cleanup; import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.RandomUtils; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.TopicMessageImpl; import org.apache.pulsar.common.api.proto.KeyValue; @@ -870,6 +871,101 @@ public void onPartitionsChange(String topicName, int partitions) { Assert.assertNull(reader.readNext(3, TimeUnit.SECONDS)); } + @Test(dataProvider = "topicPartition") + public void testConsumerInterceptorForOnArrive(int topicPartition) throws PulsarClientException, + InterruptedException, PulsarAdminException { + String topicName = "persistent://my-property/my-ns/on-arrive"; + if (topicPartition > 0) { + admin.topics().createPartitionedTopic(topicName, topicPartition); + } + + final int receiveQueueSize = 100; + final int totalNumOfMessages = receiveQueueSize * 2; + + // The onArrival method is called for half of the receiveQueueSize messages before beforeConsume is called for all messages. + CountDownLatch latch = new CountDownLatch(receiveQueueSize / 2); + final AtomicInteger onArrivalCount = new AtomicInteger(0); + ConsumerInterceptor interceptor = new ConsumerInterceptor() { + @Override + public void close() {} + + @Override + public Message onArrival(Consumer consumer, Message message) { + MessageImpl msg = (MessageImpl) message; + msg.getMessageBuilder().addProperty().setKey("onArrival").setValue("1"); + latch.countDown(); + onArrivalCount.incrementAndGet(); + return msg; + } + + @Override + public Message beforeConsume(Consumer consumer, Message message) { + return message; + } + + @Override + public void onAcknowledge(Consumer consumer, MessageId messageId, Throwable cause) { + + } + + @Override + public void onAcknowledgeCumulative(Consumer consumer, MessageId messageId, Throwable cause) { + + } + + @Override + public void onNegativeAcksSend(Consumer consumer, Set messageIds) { + } + + @Override + public void onAckTimeoutSend(Consumer consumer, Set messageIds) { + + } + }; + + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topicName) + .create(); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName("test-arrive") + .intercept(interceptor) + .receiverQueueSize(receiveQueueSize) + .subscribe(); + + for (int i = 0; i < totalNumOfMessages; i++) { + producer.send("Mock message"); + } + + // Not call receive message, just wait for onArrival interceptor. + latch.await(); + Assert.assertEquals(latch.getCount(), 0); + + for (int i = 0; i < totalNumOfMessages; i++) { + Message message = consumer.receive(); + MessageImpl msgImpl; + if (message instanceof MessageImpl) { + msgImpl = (MessageImpl) message; + } else if (message instanceof TopicMessageImpl) { + msgImpl = (MessageImpl) ((TopicMessageImpl) message).getMessage(); + } else { + throw new ClassCastException("Message type is not expected"); + } + boolean haveKey = false; + for (KeyValue keyValue : msgImpl.getMessageBuilder().getPropertiesList()) { + if ("onArrival".equals(keyValue.getKey())) { + haveKey = true; + } + } + Assert.assertTrue(haveKey); + } + Assert.assertEquals(totalNumOfMessages, onArrivalCount.get()); + + producer.close(); + consumer.close(); + } + private void produceAndConsume(int msgCount, Producer producer, Reader reader) throws PulsarClientException { for (int i = 0; i < msgCount; i++) { diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerInterceptor.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerInterceptor.java index be2f9b0f10826..1beea3adba239 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerInterceptor.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerInterceptor.java @@ -41,6 +41,44 @@ public interface ConsumerInterceptor extends AutoCloseable { */ void close(); + /** + * This method is called when a message arrives in the consumer. + * + *

This method provides visibility into the messages that have been received + * by the consumer but have not yet been processed. This can be useful for + * monitoring the state of the consumer's receiver queue and understanding + * the consumer's processing rate. + * + *

The method is allowed to modify the message, in which case the modified + * message will be returned. + * + *

Any exception thrown by this method will be caught by the caller, logged, + * but not propagated to the client. + * + *

Since the consumer may run multiple interceptors, a particular + * interceptor's onArrival callback will be called in the order + * specified by {@link ConsumerBuilder#intercept(ConsumerInterceptor[])}. The + * first interceptor in the list gets the consumed message, the following + * interceptor will be passed the message returned by the previous interceptor, + * and so on. Since interceptors are allowed to modify the message, interceptors + * may potentially get the messages already modified by other interceptors. + * However, building a pipeline of mutable interceptors that depend on the output + * of the previous interceptor is discouraged, because of potential side-effects + * caused by interceptors potentially failing to modify the message and throwing + * an exception. If one of the interceptors in the list throws an exception from + * onArrival, the exception is caught, logged, and the next interceptor + * is called with the message returned by the last successful interceptor in the + * list, or otherwise the original consumed message. + * + * @param consumer the consumer which contains the interceptor + * @param message the message that has arrived in the receiver queue + * @return the message that is either modified by the interceptor or the same + * message passed into the method + */ + default Message onArrival(Consumer consumer, Message message) { + return message; + } + /** * This is called just before the message is returned by * {@link Consumer#receive()}, {@link MessageListener#received(Consumer, diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 9748a42f0cb2b..03256a3e139b6 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -852,6 +852,14 @@ public String toString() { + '}'; } + protected Message onArrival(Message message) { + if (interceptors != null) { + return interceptors.onArrival(this, message); + } else { + return message; + } + } + protected Message beforeConsume(Message message) { if (interceptors != null) { return interceptors.beforeConsume(this, message); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 996569704d712..60b9d145c4897 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1301,9 +1301,10 @@ private void executeNotifyCallback(final MessageImpl message) { increaseAvailablePermits(cnx()); return; } + Message interceptMsg = onArrival(message); if (hasNextPendingReceive()) { - notifyPendingReceivedCallback(message, null); - } else if (enqueueMessageAndCheckBatchReceive(message) && hasPendingBatchReceive()) { + notifyPendingReceivedCallback(interceptMsg, null); + } else if (enqueueMessageAndCheckBatchReceive(interceptMsg) && hasPendingBatchReceive()) { notifyPendingBatchReceivedCallBack(); } }); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerInterceptors.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerInterceptors.java index 832dc0bacaee9..dd1e2cec3b3ef 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerInterceptors.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerInterceptors.java @@ -44,6 +44,38 @@ public ConsumerInterceptors(List> interceptors) { this.interceptors = interceptors; } + + /** + * This method is called when a message arrives in the consumer. + *

+ * This method calls {@link ConsumerInterceptor#onArrival(Consumer, Message) method for each + * interceptor. + *

+ * This method does not throw exceptions. If any of the interceptors in the chain throws an exception, it gets + * caught and logged, and next interceptor in int the chain is called with 'messages' returned by the previous + * successful interceptor beforeConsume call. + * + * @param consumer the consumer which contains the interceptors + * @param message message to be consume by the client. + * @return messages that are either modified by interceptors or same as messages passed to this method. + */ + public Message onArrival(Consumer consumer, Message message) { + Message interceptorMessage = message; + for (int i = 0, interceptorsSize = interceptors.size(); i < interceptorsSize; i++) { + try { + interceptorMessage = interceptors.get(i).onArrival(consumer, interceptorMessage); + } catch (Throwable e) { + if (consumer != null) { + log.warn("Error executing interceptor beforeConsume callback topic: {} consumerName: {}", + consumer.getTopic(), consumer.getConsumerName(), e); + } else { + log.warn("Error executing interceptor beforeConsume callback", e); + } + } + } + return interceptorMessage; + } + /** * This is called just before the message is returned by {@link Consumer#receive()}, * {@link MessageListener#received(Consumer, Message)} or the {@link java.util.concurrent.CompletableFuture} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index bf8bd6cc95117..513c0101ac6ac 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -1608,6 +1608,11 @@ private CompletableFuture> getExistsPartitions(String topic) { private ConsumerInterceptors getInternalConsumerInterceptors(ConsumerInterceptors multiTopicInterceptors) { return new ConsumerInterceptors(new ArrayList<>()) { + @Override + public Message onArrival(Consumer consumer, Message message) { + return multiTopicInterceptors.onArrival(consumer, message); + } + @Override public Message beforeConsume(Consumer consumer, Message message) { return message; From 590e1331d9a4a2b62297f155131c1641037ca3b1 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Tue, 17 Sep 2024 12:27:11 -0700 Subject: [PATCH 505/580] [improve][pip] PIP-378 Add ServiceUnitStateTableView abstraction (ExtensibleLoadMangerImpl only) (#23300) --- pip/pip-378.md | 321 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 321 insertions(+) create mode 100644 pip/pip-378.md diff --git a/pip/pip-378.md b/pip/pip-378.md new file mode 100644 index 0000000000000..352c7fa560d1c --- /dev/null +++ b/pip/pip-378.md @@ -0,0 +1,321 @@ +# PIP-378: Add ServiceUnitStateTableView abstraction (ExtensibleLoadMangerImpl only) + +## Background + +### ExtensibleLoadMangerImpl uses system topics to event-source bundle ownerships + +PIP-192 introduces a new broker load balancer using a persistent system topic to event-source bundle ownerships among brokers. + +PIP-307 introduces graceful ownership change protocol over the system topic (from PIP-192). + +However, using system topics to manage bundle ownerships may not always be the best choice. Users might need an alternative approach to event-source bundle ownerships. + + +## Motivation + +Add `ServiceUnitStateTableView` abstraction and make it pluggable, so users can customize `ServiceUnitStateTableView` implementations and event-source bundles ownerships using other stores. + +## Goals + +### In Scope + +- Add `ServiceUnitStateTableView` interface +- Add `ServiceUnitStateTableViewImpl` implementation that uses Pulsar System topic (compatible with existing behavior) +- Add `ServiceUnitStateMetadataStoreTableViewImpl` implementation that uses Pulsar Metadata Store (new behavior) +- Refactor related code and test code + +## High-Level Design + +- Refactor `ServiceUnitStateChannelImpl` to accept `ServiceUnitStateTableView` interface and `ServiceUnitStateTableViewImpl` system topic implementation. +- Introduce `MetadataStoreTableView` interface to support `ServiceUnitStateMetadataStoreTableViewImpl` implementation. +- `MetadataStoreTableViewImpl` will use shadow hashmap to maintain the metadata tableview. It will initially fill the local tableview by scanning all existing items in the metadata store path. Also, new items will be updated to the tableview via metadata watch notifications. +- Add `BiConsumer>> asyncReloadConsumer` in MetadataCacheConfig to listen the automatic cache async reload. This can be useful to re-sync the the shadow hashmap in MetadataStoreTableViewImpl in case it is out-dated in the worst case(e.g. network or metadata issues). +- Introduce `ServiceUnitStateTableViewSyncer` to sync system topic and metadata store table views to migrate to one from the other. This syncer can be enabled by a dynamic config, `loadBalancerServiceUnitTableViewSyncerEnabled`. + +## Detailed Design + +### Design & Implementation Details +```java +/** + * Given that the ServiceUnitStateChannel event-sources service unit (bundle) ownership states via a persistent store + * and reacts to ownership changes, the ServiceUnitStateTableView provides an interface to the + * ServiceUnitStateChannel's persistent store and its locally replicated ownership view (tableview) with listener + * registration. It initially populates its local table view by scanning existing items in the remote store. The + * ServiceUnitStateTableView receives notifications whenever ownership states are updated in the remote store, and + * upon notification, it applies the updates to its local tableview with the listener logic. + */ +public interface ServiceUnitStateTableView extends Closeable { + + /** + * Starts the tableview. + * It initially populates its local table view by scanning existing items in the remote store, and it starts + * listening to service unit ownership changes from the remote store. + * @param pulsar pulsar service reference + * @param tailItemListener listener to listen tail(newly updated) items + * @param existingItemListener listener to listen existing items + * @throws IOException if it fails to init the tableview. + */ + void start(PulsarService pulsar, + BiConsumer tailItemListener, + BiConsumer existingItemListener) throws IOException; + + + /** + * Closes the tableview. + * @throws IOException if it fails to close the tableview. + */ + void close() throws IOException; + + /** + * Gets one item from the local tableview. + * @param key the key to get + * @return value if exists. Otherwise, null. + */ + ServiceUnitStateData get(String key); + + /** + * Tries to put the item in the persistent store. + * If it completes, all peer tableviews (including the local one) will be notified and be eventually consistent + * with this put value. + * + * It ignores put operation if the input value conflicts with the existing one in the persistent store. + * + * @param key the key to put + * @param value the value to put + * @return a future to track the completion of the operation + */ + CompletableFuture put(String key, ServiceUnitStateData value); + + /** + * Tries to delete the item from the persistent store. + * All peer tableviews (including the local one) will be notified and be eventually consistent with this deletion. + * + * It ignores delete operation if the key is not present in the persistent store. + * + * @param key the key to delete + * @return a future to track the completion of the operation + */ + CompletableFuture delete(String key); + + /** + * Returns the entry set of the items in the local tableview. + * @return entry set + */ + Set> entrySet(); + + /** + * Returns service units (namespace bundles) owned by this broker. + * @return a set of owned service units (namespace bundles) + */ + Set ownedServiceUnits(); + + /** + * Tries to flush any batched or buffered updates. + * @param waitDurationInMillis time to wait until complete. + * @throws ExecutionException + * @throws InterruptedException + * @throws TimeoutException + */ + void flush(long waitDurationInMillis) throws ExecutionException, InterruptedException, TimeoutException; +} +``` + +```java +/** + * Defines metadata store tableview. + * MetadataStoreTableView initially fills existing items to its local tableview and eventually + * synchronize remote updates to its local tableview from the remote metadata store. + * This abstraction can help replicate metadata in memory from metadata store. + */ +public interface MetadataStoreTableView { + + class ConflictException extends RuntimeException { + public ConflictException(String msg) { + super(msg); + } + } + + /** + * Starts the tableview by filling existing items to its local tableview from the remote metadata store. + */ + void start() throws MetadataStoreException; + + /** + * Reads whether a specific key exists in the local tableview. + * + * @param key the key to check + * @return true if exists. Otherwise, false. + */ + boolean exists(String key); + + /** + * Gets one item from the local tableview. + *

+ * If the key is not found, return null. + * + * @param key the key to check + * @return value if exists. Otherwise, null. + */ + T get(String key); + + /** + * Tries to put the item in the persistent store. + * All peer tableviews (including the local one) will be notified and be eventually consistent with this put value. + *

+ * This operation can fail if the input value conflicts with the existing one. + * + * @param key the key to check on the tableview + * @return a future to track the completion of the operation + * @throws MetadataStoreTableView.ConflictException + * if the input value conflicts with the existing one. + */ + CompletableFuture put(String key, T value); + + /** + * Tries to delete the item from the persistent store. + * All peer tableviews (including the local one) will be notified and be eventually consistent with this deletion. + *

+ * This can fail if the item is not present in the metadata store. + * + * @param key the key to check on the tableview + * @return a future to track the completion of the operation + * @throws MetadataStoreException.NotFoundException + * if the key is not present in the metadata store. + */ + CompletableFuture delete(String key); + + /** + * Returns the size of the items in the local tableview. + * @return size + */ + int size(); + + /** + * Reads whether the local tableview is empty or not. + * @return true if empty. Otherwise, false + */ + boolean isEmpty(); + + /** + * Returns the entry set of the items in the local tableview. + * @return entry set + */ + Set> entrySet(); + + /** + * Returns the key set of the items in the local tableview. + * @return key set + */ + Set keySet(); + + /** + * Returns the values of the items in the local tableview. + * @return values + */ + Collection values(); + + /** + * Runs the action for each item in the local tableview. + */ + void forEach(BiConsumer action); +} +``` + +```java +public class MetadataCacheConfig { + private static final long DEFAULT_CACHE_REFRESH_TIME_MILLIS = TimeUnit.MINUTES.toMillis(5); + + ... + + /** + * Specifies cache reload consumer behavior when the cache is refreshed automatically at refreshAfterWriteMillis + * frequency. + */ + @Builder.Default + private final BiConsumer>> asyncReloadConsumer = null; +``` + +```java + +/** + * ServiceUnitStateTableViewSyncer can be used to sync system topic and metadata store table views to migrate to one + * from the other. + */ +@Slf4j +public class ServiceUnitStateTableViewSyncer implements Cloneable { + private static final int SYNC_TIMEOUT_IN_SECS = 30; + private volatile ServiceUnitStateTableView systemTopicTableView; + private volatile ServiceUnitStateTableView metadataStoreTableView; + + + public void start(PulsarService pulsar) throws IOException { + if (!pulsar.getConfiguration().isLoadBalancerServiceUnitTableViewSyncerEnabled()) { + return; + } + try { + if (systemTopicTableView == null) { + systemTopicTableView = new ServiceUnitStateTableViewImpl(); + systemTopicTableView.start( + pulsar, + this::syncToMetadataStore, + this::syncToMetadataStore); + log.info("Successfully started ServiceUnitStateTableViewSyncer::systemTopicTableView"); + } + + if (metadataStoreTableView == null) { + metadataStoreTableView = new ServiceUnitStateMetadataStoreTableViewImpl(); + metadataStoreTableView.start( + pulsar, + this::syncToSystemTopic, + this::syncToSystemTopic); + log.info("Successfully started ServiceUnitStateTableViewSyncer::metadataStoreTableView"); + } + + } catch (Throwable e) { + log.error("Failed to start ServiceUnitStateTableViewSyncer", e); + throw e; + } + } + + private void syncToSystemTopic(String key, ServiceUnitStateData data) { + try { + systemTopicTableView.put(key, data).get(SYNC_TIMEOUT_IN_SECS, TimeUnit.SECONDS); + } catch (Throwable e) { + log.error("SystemTopicTableView failed to sync key:{}, data:{}", key, data, e); + throw new IllegalStateException(e); + } + } + + private void syncToMetadataStore(String key, ServiceUnitStateData data) { + try { + metadataStoreTableView.put(key, data).get(SYNC_TIMEOUT_IN_SECS, TimeUnit.SECONDS); + } catch (Throwable e) { + log.error("metadataStoreTableView failed to sync key:{}, data:{}", key, data, e); + throw new IllegalStateException(e); + } + } +... +} + + +``` + +### Configuration + +- Add a `loadManagerServiceUnitStateTableViewClassName` configuration to specify `ServiceUnitStateTableView` implementation class name. +- Add a `loadBalancerServiceUnitTableViewSyncerEnabled` configuration to to enable ServiceUnitTableViewSyncer to sync metadata store and system topic ServiceUnitStateTableView during migration. + +## Backward & Forward Compatibility + +It will ba Backward & Forward compatible as `loadManagerServiceUnitStateTableViewClassName` will be `ServiceUnitStateTableViewImpl`(system topic implementation) by default. + +We will introduce `ServiceUnitStateTableViewSyncer` to sync system topic and metadata store table views when migrating to ServiceUnitStateMetadataStoreTableViewImpl from ServiceUnitStateTableViewImpl and vice versa. This syncer can be enabled/disabled by a dynamic config, `loadBalancerServiceUnitTableViewSyncerEnabled`. The admin could enable this syncer before migration and disable it after it is finished. + +## Alternatives + +## General Notes + +## Links + +* Mailing List discussion thread: https://lists.apache.org/thread/v7sod21r56hkt2cjxl9pp348r4jxo6o8 +* Mailing List voting thread: https://lists.apache.org/thread/j453xp0vty8zy2y0ljssjgyvwb47royc From 4f002590450f756a26375fb551136bb11dd72666 Mon Sep 17 00:00:00 2001 From: Girish Sharma Date: Wed, 18 Sep 2024 14:14:49 +0530 Subject: [PATCH 506/580] [improve][admin] PIP-369 Change default value of `unload-scope` in `ns-isolation-policy set` (#23253) Co-authored-by: Zixuan Liu --- .../broker/admin/impl/ClustersBase.java | 65 +++++++++---------- .../pulsar/broker/admin/AdminApi2Test.java | 38 +++++++++-- .../cli/CmdNamespaceIsolationPolicy.java | 12 ++-- .../data/NamespaceIsolationDataImpl.java | 4 +- 4 files changed, 72 insertions(+), 47 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java index 132c99ce16bec..b261033ca52c9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java @@ -778,15 +778,16 @@ private CompletableFuture filterAndUnloadMatchedNamespaceAsync(String clus } catch (PulsarServerException e) { return FutureUtil.failedFuture(e); } - // compile regex patterns once - List namespacePatterns = policyData.getNamespaces().stream().map(Pattern::compile).toList(); - // TODO for 4.x, we should include both old and new namespace regex pattern for unload `all_matching` option + Set combinedNamespaces = new HashSet<>(policyData.getNamespaces()); + final List oldNamespaces = new ArrayList<>(); + if (oldPolicy != null) { + oldNamespaces.addAll(oldPolicy.getNamespaces()); + combinedNamespaces.addAll(oldNamespaces); + } return adminClient.tenants().getTenantsAsync().thenCompose(tenants -> { List>> filteredNamespacesForEachTenant = tenants.stream() .map(tenant -> adminClient.namespaces().getNamespacesAsync(tenant).thenCompose(namespaces -> { List> namespaceNamesInCluster = namespaces.stream() - .filter(namespaceName -> namespacePatterns.stream() - .anyMatch(pattern -> pattern.matcher(namespaceName).matches())) .map(namespaceName -> adminClient.namespaces().getPoliciesAsync(namespaceName) .thenApply(policies -> policies.replication_clusters.contains(cluster) ? namespaceName : null)) @@ -802,46 +803,44 @@ private CompletableFuture filterAndUnloadMatchedNamespaceAsync(String clus .map(CompletableFuture::join) .flatMap(List::stream) .collect(Collectors.toList())); - }).thenCompose(shouldUnloadNamespaces -> { - if (CollectionUtils.isEmpty(shouldUnloadNamespaces)) { + }).thenCompose(clusterLocalNamespaces -> { + if (CollectionUtils.isEmpty(clusterLocalNamespaces)) { return CompletableFuture.completedFuture(null); } // If unload type is 'changed', we need to figure out a further subset of namespaces whose placement might // actually have been changed. log.debug("Old policy: {} ; new policy: {}", oldPolicy, policyData); - if (oldPolicy != null && NamespaceIsolationPolicyUnloadScope.changed.equals(policyData.getUnloadScope())) { - // We also compare that the previous primary broker list is same as current, in case all namespaces need - // to be placed again anyway. - if (CollectionUtils.isEqualCollection(oldPolicy.getPrimary(), policyData.getPrimary())) { - // list is same, so we continue finding the changed namespaces. - // We create a union regex list contains old + new regexes - Set combinedNamespaces = new HashSet<>(oldPolicy.getNamespaces()); - combinedNamespaces.addAll(policyData.getNamespaces()); - // We create a intersection of the old and new regexes. These won't need to be unloaded - Set commonNamespaces = new HashSet<>(oldPolicy.getNamespaces()); - commonNamespaces.retainAll(policyData.getNamespaces()); + boolean unloadAllNamespaces = false; + // We also compare that the previous primary broker list is same as current, in case all namespaces need + // to be placed again anyway. + if (NamespaceIsolationPolicyUnloadScope.all_matching.equals(policyData.getUnloadScope()) + || (oldPolicy != null + && !CollectionUtils.isEqualCollection(oldPolicy.getPrimary(), policyData.getPrimary()))) { + unloadAllNamespaces = true; + } + // list is same, so we continue finding the changed namespaces. - log.debug("combined regexes: {}; common regexes:{}", combinedNamespaces, combinedNamespaces); + // We create a intersection of the old and new regexes. These won't need to be unloaded. + Set commonNamespaces = new HashSet<>(policyData.getNamespaces()); + commonNamespaces.retainAll(oldNamespaces); - // Find the changed regexes (new - new ∩ old). TODO for 4.x, make this (new U old - new ∩ old) - combinedNamespaces.removeAll(commonNamespaces); + log.debug("combined regexes: {}; common regexes:{}", combinedNamespaces, commonNamespaces); - log.debug("changed regexes: {}", commonNamespaces); + if (!unloadAllNamespaces) { + // Find the changed regexes ((new U old) - (new ∩ old)). + combinedNamespaces.removeAll(commonNamespaces); + log.debug("changed regexes: {}", commonNamespaces); + } - // Now we further filter the filtered namespaces based on this combinedNamespaces set - shouldUnloadNamespaces = shouldUnloadNamespaces.stream() - .filter(name -> combinedNamespaces.stream() - .map(Pattern::compile) - .anyMatch(pattern -> pattern.matcher(name).matches()) - ).toList(); + // Now we further filter the filtered namespaces based on this combinedNamespaces set + List namespacePatterns = combinedNamespaces.stream().map(Pattern::compile).toList(); + clusterLocalNamespaces = clusterLocalNamespaces.stream() + .filter(name -> namespacePatterns.stream().anyMatch(pattern -> pattern.matcher(name).matches())) + .toList(); - } - } - // unload type is either null or not in (changed, none), so we proceed to unload all namespaces - // TODO - default in 4.x should become `changed` - List> futures = shouldUnloadNamespaces.stream() + List> futures = clusterLocalNamespaces.stream() .map(namespaceName -> adminClient.namespaces().unloadAsync(namespaceName)) .collect(Collectors.toList()); return FutureUtil.waitForAll(futures).thenAccept(__ -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java index 155994c814c11..df9862691d6d5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java @@ -3488,7 +3488,7 @@ private NamespaceIsolationData createPolicyData(NamespaceIsolationPolicyUnloadSc parameters1.put("usage_threshold", "100"); List nsRegexList = new ArrayList<>(namespaces); - return NamespaceIsolationData.builder() + NamespaceIsolationData.Builder build = NamespaceIsolationData.builder() // "prop-ig/ns1" is present in test cluster, policy set on test2 should work .namespaces(nsRegexList) .primary(primaryBrokers) @@ -3496,9 +3496,11 @@ private NamespaceIsolationData createPolicyData(NamespaceIsolationPolicyUnloadSc .autoFailoverPolicy(AutoFailoverPolicyData.builder() .policyType(AutoFailoverPolicyType.min_available) .parameters(parameters1) - .build()) - .unloadScope(scope) - .build(); + .build()); + if (scope != null) { + build.unloadScope(scope); + } + return build.build(); } private boolean allTopicsUnloaded(List topics) { @@ -3624,18 +3626,42 @@ public void testIsolationPolicyUnloadsNSWithAllScope(final String topicType) thr testIsolationPolicyUnloadsNSWithScope( topicType, "policy-all", nsPrefix, List.of("a1", "a2", "b1", "b2", "c1"), all_matching, List.of(".*-unload-test-a.*"), List.of("b1", "b2", "c1"), + all_matching, List.of(".*-unload-test-c.*"), List.of("b1", "b2"), + Collections.singletonList(".*") + ); + } + + @Test(dataProvider = "topicType") + public void testIsolationPolicyUnloadsNSWithChangedScope1(final String topicType) throws Exception { + String nsPrefix1 = newUniqueName(defaultTenant + "/") + "-unload-test-"; + // Addition case + testIsolationPolicyUnloadsNSWithScope( + topicType, "policy-changed1", nsPrefix1, List.of("a1", "a2", "b1", "b2", "c1"), + all_matching, List.of(".*-unload-test-a.*"), List.of("b1", "b2", "c1"), + changed, List.of(".*-unload-test-a.*", ".*-unload-test-c.*"), List.of("a1", "a2", "b1", "b2"), + Collections.singletonList(".*") + ); + } + + @Test(dataProvider = "topicType") + public void testIsolationPolicyUnloadsNSWithChangedScope2(final String topicType) throws Exception { + String nsPrefix2 = newUniqueName(defaultTenant + "/") + "-unload-test-"; + // removal case + testIsolationPolicyUnloadsNSWithScope( + topicType, "policy-changed2", nsPrefix2, List.of("a1", "a2", "b1", "b2", "c1"), all_matching, List.of(".*-unload-test-a.*", ".*-unload-test-c.*"), List.of("b1", "b2"), + changed, List.of(".*-unload-test-c.*"), List.of("b1", "b2", "c1"), Collections.singletonList(".*") ); } @Test(dataProvider = "topicType") - public void testIsolationPolicyUnloadsNSWithChangedScope(final String topicType) throws Exception { + public void testIsolationPolicyUnloadsNSWithScopeMissing(final String topicType) throws Exception { String nsPrefix = newUniqueName(defaultTenant + "/") + "-unload-test-"; testIsolationPolicyUnloadsNSWithScope( topicType, "policy-changed", nsPrefix, List.of("a1", "a2", "b1", "b2", "c1"), all_matching, List.of(".*-unload-test-a.*"), List.of("b1", "b2", "c1"), - changed, List.of(".*-unload-test-a.*", ".*-unload-test-c.*"), List.of("a1", "a2", "b1", "b2"), + null, List.of(".*-unload-test-a.*", ".*-unload-test-c.*"), List.of("a1", "a2", "b1", "b2"), Collections.singletonList(".*") ); } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaceIsolationPolicy.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaceIsolationPolicy.java index 0f5f6b211a544..ef36eb417136d 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaceIsolationPolicy.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaceIsolationPolicy.java @@ -75,12 +75,12 @@ private class SetPolicy extends CliCommand { private Map autoFailoverPolicyParams; @Option(names = "--unload-scope", description = "configure the type of unload to do -" - + " ['all_matching', 'none', 'changed'] namespaces. By default, all namespaces matching the namespaces" - + " regex will be unloaded and placed again. You can choose to not unload any namespace while setting" - + " this new policy by choosing `none` or choose to unload only the namespaces whose placement will" - + " actually change. If you chose 'none', you will need to manually unload the namespaces for them to" - + " be placed correctly, or wait till some namespaces get load balanced automatically based on load" - + " shedding configurations.") + + " ['all_matching', 'none', 'changed'] namespaces. By default, only namespaces whose placement will" + + " actually change would be unloaded and placed again. You can choose to not unload any namespace" + + " while setting this new policy by choosing `none` or choose to unload all namespaces matching" + + " old (if any) and new namespace regex. If you chose 'none', you will need to manually unload the" + + " namespaces for them to be placed correctly, or wait till some namespaces get load balanced" + + " automatically based on load shedding configurations.") private NamespaceIsolationPolicyUnloadScope unloadScope; void run() throws PulsarAdminException { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java index 1e72f0e50ee05..85be8090f52a1 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java @@ -78,8 +78,8 @@ public class NamespaceIsolationDataImpl implements NamespaceIsolationData { @ApiModelProperty( name = "unload_scope", value = "The type of unload to perform while applying the new isolation policy.", - example = "'all_matching' (default) for unloading all matching namespaces. 'none' for not unloading " - + "any namespace. 'changed' for unloading only the namespaces whose placement is actually changing" + example = "'changed' (default) for unloading only the namespaces whose placement is actually changing. " + + "'all_matching' for unloading all matching namespaces. 'none' for not unloading any namespaces." ) @JsonProperty("unload_scope") private NamespaceIsolationPolicyUnloadScope unloadScope; From bf53164c1b04866e1c3647dd92c9d44a3fe059e6 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov <8622884+dlg99@users.noreply.github.com> Date: Wed, 18 Sep 2024 04:22:18 -0700 Subject: [PATCH 507/580] [improve] Upgrade opensearch sink to client 2.16 and tests to use server 2.16.0 (#23312) --- pom.xml | 2 +- .../pulsar/io/elasticsearch/ElasticSearchConfig.java | 4 +++- .../opensearch/OpenSearchHighLevelRestClient.java | 10 +++------- .../io/elasticsearch/ElasticSearchConfigTests.java | 4 ---- .../pulsar/io/elasticsearch/ElasticSearchTestBase.java | 3 ++- .../opensearch/OpenSearchClientSslTests.java | 6 +++--- .../integration/io/sinks/OpenSearchSinkTester.java | 3 ++- 7 files changed, 14 insertions(+), 18 deletions(-) diff --git a/pom.xml b/pom.xml index 4cc982e7bdd64..dda7f316acc6f 100644 --- a/pom.xml +++ b/pom.xml @@ -199,7 +199,7 @@ flexible messaging model and an intuitive client API. 0.4.4-hotfix1 3.3.5 2.4.10 - 1.2.4 + 2.16.0 8.12.1 1.9.7.Final 42.5.5 diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchConfig.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchConfig.java index 9f42dbda7be1b..33c2d34a1c992 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchConfig.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchConfig.java @@ -59,10 +59,12 @@ public class ElasticSearchConfig implements Serializable { ) private String indexName; + @Deprecated @FieldDoc( required = false, defaultValue = "_doc", - help = "The type name that the connector writes messages to, with the default value set to _doc." + help = "No longer in use in OpenSearch 2+. " + + "The type name that the connector writes messages to, with the default value set to _doc." + " This value should be set explicitly to a valid type name other than _doc for Elasticsearch version before 6.2," + " and left to the default value otherwise." ) diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java index bb92047f17a31..87c4913529f04 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java @@ -49,12 +49,12 @@ import org.opensearch.client.indices.CreateIndexRequest; import org.opensearch.client.indices.CreateIndexResponse; import org.opensearch.client.indices.GetIndexRequest; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; -import org.opensearch.common.unit.ByteSizeUnit; -import org.opensearch.common.unit.ByteSizeValue; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; +import org.opensearch.core.common.unit.ByteSizeUnit; +import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.index.query.QueryBuilder; import org.opensearch.index.query.QueryBuilders; import org.opensearch.search.builder.SearchSourceBuilder; @@ -229,7 +229,6 @@ public boolean indexDocument(String index, String documentId, String documentSou if (!Strings.isNullOrEmpty(documentId)) { indexRequest.id(documentId); } - indexRequest.type(config.getTypeName()); indexRequest.source(documentSource, XContentType.JSON); IndexResponse indexResponse = client.index(indexRequest, RequestOptions.DEFAULT); @@ -245,7 +244,6 @@ public boolean indexDocument(String index, String documentId, String documentSou public boolean deleteDocument(String index, String documentId) throws IOException { DeleteRequest deleteRequest = Requests.deleteRequest(index); deleteRequest.id(documentId); - deleteRequest.type(config.getTypeName()); DeleteResponse deleteResponse = client.delete(deleteRequest, RequestOptions.DEFAULT); if (log.isDebugEnabled()) { log.debug("delete result {}", deleteResponse.getResult()); @@ -301,7 +299,6 @@ public void appendIndexRequest(BulkProcessor.BulkIndexRequest request) throws IO if (!Strings.isNullOrEmpty(request.getDocumentId())) { indexRequest.id(request.getDocumentId()); } - indexRequest.type(config.getTypeName()); indexRequest.source(request.getDocumentSource(), XContentType.JSON); internalBulkProcessor.add(indexRequest); } @@ -310,7 +307,6 @@ public void appendIndexRequest(BulkProcessor.BulkIndexRequest request) throws IO public void appendDeleteRequest(BulkProcessor.BulkDeleteRequest request) throws IOException { DeleteRequest deleteRequest = new DeleteRequestWithPulsarRecord(request.getIndex(), request.getRecord()); deleteRequest.id(request.getDocumentId()); - deleteRequest.type(config.getTypeName()); internalBulkProcessor.add(deleteRequest); } diff --git a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchConfigTests.java b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchConfigTests.java index 85e30e766f030..506df31923378 100644 --- a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchConfigTests.java +++ b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchConfigTests.java @@ -44,7 +44,6 @@ public final void loadFromYamlFileTest() throws IOException { assertNotNull(config); assertEquals(config.getElasticSearchUrl(), "http://localhost:90902"); assertEquals(config.getIndexName(), "myIndex"); - assertEquals(config.getTypeName(), "doc"); assertEquals(config.getUsername(), "scooby"); assertEquals(config.getPassword(), "doobie"); assertEquals(config.getPrimaryFields(), "id,a"); @@ -64,7 +63,6 @@ public final void loadFromMapTest() throws IOException { assertNotNull(config); assertEquals(config.getElasticSearchUrl(), "http://localhost:90902"); assertEquals(config.getIndexName(), "myIndex"); - assertEquals(config.getTypeName(), "doc"); assertEquals(config.getUsername(), "racerX"); assertEquals(config.getPassword(), "go-speedie-go"); assertEquals(config.getPrimaryFields(), "x"); @@ -75,7 +73,6 @@ public final void defaultValueTest() throws IOException { Map requiredConfig = Map.of("elasticSearchUrl", "http://localhost:90902"); ElasticSearchConfig config = ElasticSearchConfig.load(requiredConfig, mockContext); assertNull(config.getIndexName()); - assertEquals(config.getTypeName(), "_doc"); assertNull(config.getUsername()); assertNull(config.getPassword()); assertNull(config.getToken()); @@ -336,7 +333,6 @@ public final void loadConfigFromSecretsTest() throws IOException { assertNotNull(config); assertEquals(config.getElasticSearchUrl(), "http://localhost:90902"); assertEquals(config.getIndexName(), "myIndex"); - assertEquals(config.getTypeName(), "doc"); assertEquals(config.getPrimaryFields(), "x"); assertEquals(config.getUsername(), "secretUser"); assertEquals(config.getPassword(), "$ecret123"); diff --git a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchTestBase.java b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchTestBase.java index 0f5a42051c7d1..8c5868f27689b 100644 --- a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchTestBase.java +++ b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchTestBase.java @@ -46,7 +46,7 @@ public abstract class ElasticSearchTestBase { .orElse("docker.elastic.co/elasticsearch/elasticsearch:7.17.7"); public static final String OPENSEARCH = Optional.ofNullable(System.getenv("OPENSEARCH_IMAGE")) - .orElse("opensearchproject/opensearch:1.2.4"); + .orElse("opensearchproject/opensearch:2.16.0"); protected final String elasticImageName; @@ -59,6 +59,7 @@ protected ElasticsearchContainer createElasticsearchContainer() { if (elasticImageName.equals(OPENSEARCH)) { DockerImageName dockerImageName = DockerImageName.parse(OPENSEARCH).asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"); elasticsearchContainer = new ElasticsearchContainer(dockerImageName) + .withEnv("OPENSEARCH_INITIAL_ADMIN_PASSWORD", "0pEn7earch!") .withEnv("OPENSEARCH_JAVA_OPTS", "-Xms128m -Xmx256m") .withEnv("bootstrap.memory_lock", "true") .withEnv("plugins.security.disabled", "true"); diff --git a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/opensearch/OpenSearchClientSslTests.java b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/opensearch/OpenSearchClientSslTests.java index 72bebfe2bbf8d..0b78506491657 100644 --- a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/opensearch/OpenSearchClientSslTests.java +++ b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/opensearch/OpenSearchClientSslTests.java @@ -78,7 +78,7 @@ public void testSslBasic() throws IOException { .setElasticSearchUrl("https://" + container.getHttpHostAddress()) .setIndexName(INDEX) .setUsername("admin") - .setPassword("admin") + .setPassword("0pEn7earch!") .setSsl(new ElasticSearchSslConfig() .setEnabled(true) .setTruststorePath(sslResourceDir + "/truststore.jks") @@ -102,7 +102,7 @@ public void testSslWithHostnameVerification() throws IOException { .setElasticSearchUrl("https://" + container.getHttpHostAddress()) .setIndexName(INDEX) .setUsername("admin") - .setPassword("admin") + .setPassword("0pEn7earch!") .setSsl(new ElasticSearchSslConfig() .setEnabled(true) .setProtocols("TLSv1.2") @@ -127,7 +127,7 @@ public void testSslWithClientAuth() throws IOException { .setElasticSearchUrl("https://" + container.getHttpHostAddress()) .setIndexName(INDEX) .setUsername("admin") - .setPassword("admin") + .setPassword("0pEn7earch!") .setSsl(new ElasticSearchSslConfig() .setEnabled(true) .setHostnameVerification(true) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/OpenSearchSinkTester.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/OpenSearchSinkTester.java index 75f0fdac6f90c..8daed8d5c04d5 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/OpenSearchSinkTester.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/OpenSearchSinkTester.java @@ -35,7 +35,7 @@ public class OpenSearchSinkTester extends ElasticSearchSinkTester { public static final String OPENSEARCH = Optional.ofNullable(System.getenv("OPENSEARCH_IMAGE")) - .orElse("opensearchproject/opensearch:1.2.4"); + .orElse("opensearchproject/opensearch:2.16.0"); private RestHighLevelClient elasticClient; @@ -49,6 +49,7 @@ protected ElasticsearchContainer createElasticContainer() { DockerImageName dockerImageName = DockerImageName.parse(OPENSEARCH) .asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"); return new ElasticsearchContainer(dockerImageName) + .withEnv("OPENSEARCH_INITIAL_ADMIN_PASSWORD", "0pEn7earch!") .withEnv("OPENSEARCH_JAVA_OPTS", "-Xms128m -Xmx256m") .withEnv("bootstrap.memory_lock", "true") .withEnv("plugins.security.disabled", "true"); From 77570d5db5f2809c23ee40fc7aae743a1750f5c3 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 18 Sep 2024 22:45:12 +0300 Subject: [PATCH 508/580] [improve][broker] Add limits for Key_Shared Subscription look ahead in dispatching (#23231) Co-authored-by: Matteo Merli Co-authored-by: Yuri Mizushima --- conf/broker.conf | 19 + conf/standalone.conf | 19 + .../pulsar/broker/ServiceConfiguration.java | 30 + .../MessageRedeliveryController.java | 31 +- ...PersistentDispatcherMultipleConsumers.java | 160 +++-- ...tStickyKeyDispatcherMultipleConsumers.java | 582 +++++++++++------- .../ConcurrentBitmapSortedLongPairSet.java | 36 +- .../apache/pulsar/broker/BrokerTestUtil.java | 157 +++++ .../admin/v3/AdminApiTransactionTest.java | 79 +-- .../auth/MockedPulsarServiceBaseTest.java | 5 + .../KeySharedLookAheadConfigTest.java | 63 ++ .../MessageRedeliveryControllerTest.java | 4 +- ...ckyKeyDispatcherMultipleConsumersTest.java | 66 +- .../client/api/KeySharedSubscriptionTest.java | 151 ++++- .../client/api/ProducerConsumerBase.java | 44 +- .../impl/KeySharedSubscriptionTest.java | 6 +- 16 files changed, 1068 insertions(+), 384 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/KeySharedLookAheadConfigTest.java diff --git a/conf/broker.conf b/conf/broker.conf index ed59e5c456695..74130d709cdd2 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -355,6 +355,25 @@ maxUnackedMessagesPerBroker=0 # limit/2 messages maxUnackedMessagesPerSubscriptionOnBrokerBlocked=0.16 +# For Key_Shared subscriptions, if messages cannot be dispatched to consumers due to a slow consumer +# or a blocked key hash (because of ordering constraints), the broker will continue reading more +# messages from the backlog and attempt to dispatch them to consumers until the number of replay +# messages reaches the calculated threshold. +# Formula: threshold = min(keySharedLookAheadMsgInReplayThresholdPerConsumer * +# connected consumer count, keySharedLookAheadMsgInReplayThresholdPerSubscription). +# Setting this value to 0 will disable the limit calculated per consumer. +keySharedLookAheadMsgInReplayThresholdPerConsumer=2000 + +# For Key_Shared subscriptions, if messages cannot be dispatched to consumers due to a slow consumer +# or a blocked key hash (because of ordering constraints), the broker will continue reading more +# messages from the backlog and attempt to dispatch them to consumers until the number of replay +# messages reaches the calculated threshold. +# Formula: threshold = min(keySharedLookAheadMsgInReplayThresholdPerConsumer * +# connected consumer count, keySharedLookAheadMsgInReplayThresholdPerSubscription). +# This value should be set to a value less than 2 * managedLedgerMaxUnackedRangesToPersist. +# Setting this value to 0 will disable the limit calculated per subscription. +keySharedLookAheadMsgInReplayThresholdPerSubscription=20000 + # Broker periodically checks if subscription is stuck and unblock if flag is enabled. (Default is disabled) unblockStuckSubscriptionEnabled=false diff --git a/conf/standalone.conf b/conf/standalone.conf index d5d79e0383e1f..622949bf6c325 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -232,6 +232,25 @@ maxUnackedMessagesPerBroker=0 # limit/2 messages maxUnackedMessagesPerSubscriptionOnBrokerBlocked=0.16 +# For Key_Shared subscriptions, if messages cannot be dispatched to consumers due to a slow consumer +# or a blocked key hash (because of ordering constraints), the broker will continue reading more +# messages from the backlog and attempt to dispatch them to consumers until the number of replay +# messages reaches the calculated threshold. +# Formula: threshold = min(keySharedLookAheadMsgInReplayThresholdPerConsumer * +# connected consumer count, keySharedLookAheadMsgInReplayThresholdPerSubscription). +# Setting this value to 0 will disable the limit calculated per consumer. +keySharedLookAheadMsgInReplayThresholdPerConsumer=2000 + +# For Key_Shared subscriptions, if messages cannot be dispatched to consumers due to a slow consumer +# or a blocked key hash (because of ordering constraints), the broker will continue reading more +# messages from the backlog and attempt to dispatch them to consumers until the number of replay +# messages reaches the calculated threshold. +# Formula: threshold = min(keySharedLookAheadMsgInReplayThresholdPerConsumer * +# connected consumer count, keySharedLookAheadMsgInReplayThresholdPerSubscription). +# This value should be set to a value less than 2 * managedLedgerMaxUnackedRangesToPersist. +# Setting this value to 0 will disable the limit calculated per subscription. +keySharedLookAheadMsgInReplayThresholdPerSubscription=20000 + # Tick time to schedule task that checks topic publish rate limiting across all topics # Reducing to lower value can give more accuracy while throttling publish but # it uses more CPU to perform frequent check. (Disable publish throttling with value 0) 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 60f37f52b6b8c..42dc959426692 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 @@ -949,6 +949,36 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece + " back and unack count reaches to `limit/2`. Using a value of 0, is disabling unackedMessage-limit" + " check and broker doesn't block dispatchers") private int maxUnackedMessagesPerBroker = 0; + + @FieldContext( + category = CATEGORY_POLICIES, + doc = "For Key_Shared subscriptions, if messages cannot be dispatched to consumers due to a slow consumer" + + " or a blocked key hash (because of ordering constraints), the broker will continue reading more" + + " messages from the backlog and attempt to dispatch them to consumers until the number of replay" + + " messages reaches the calculated threshold.\n" + + "Formula: threshold = min(keySharedLookAheadMsgInReplayThresholdPerConsumer *" + + " connected consumer count, keySharedLookAheadMsgInReplayThresholdPerSubscription)" + + ".\n" + + "Setting this value to 0 will disable the limit calculated per consumer.", + dynamic = true + ) + private int keySharedLookAheadMsgInReplayThresholdPerConsumer = 2000; + + @FieldContext( + category = CATEGORY_POLICIES, + doc = "For Key_Shared subscriptions, if messages cannot be dispatched to consumers due to a slow consumer" + + " or a blocked key hash (because of ordering constraints), the broker will continue reading more" + + " messages from the backlog and attempt to dispatch them to consumers until the number of replay" + + " messages reaches the calculated threshold.\n" + + "Formula: threshold = min(keySharedLookAheadMsgInReplayThresholdPerConsumer *" + + " connected consumer count, keySharedLookAheadMsgInReplayThresholdPerSubscription)" + + ".\n" + + "This value should be set to a value less than 2 * managedLedgerMaxUnackedRangesToPersist.\n" + + "Setting this value to 0 will disable the limit calculated per subscription.\n", + dynamic = true + ) + private int keySharedLookAheadMsgInReplayThresholdPerSubscription = 20000; + @FieldContext( category = CATEGORY_POLICIES, doc = "Once broker reaches maxUnackedMessagesPerBroker limit, it blocks subscriptions which has higher " diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java index 9d29b93ca450d..fa6e1412151b6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java @@ -22,7 +22,10 @@ import java.util.ArrayList; import java.util.List; import java.util.NavigableSet; +import java.util.Optional; import java.util.Set; +import java.util.TreeSet; +import java.util.function.Predicate; import javax.annotation.concurrent.NotThreadSafe; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; @@ -146,8 +149,32 @@ public boolean containsStickyKeyHashes(Set stickyKeyHashes) { return false; } - public NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { - return messagesToRedeliver.items(maxMessagesToRead, PositionFactory::create); + public boolean containsStickyKeyHash(int stickyKeyHash) { + return !allowOutOfOrderDelivery && hashesRefCount.containsKey(stickyKeyHash); + } + + public Optional getFirstPositionInReplay() { + return messagesToRedeliver.first(PositionFactory::create); + } + + /** + * Get the messages to replay now. + * + * @param maxMessagesToRead + * the max messages to read + * @param filter + * the filter to use to select the messages to replay + * @return the messages to replay now + */ + public NavigableSet getMessagesToReplayNow(int maxMessagesToRead, Predicate filter) { + NavigableSet items = new TreeSet<>(); + messagesToRedeliver.processItems(PositionFactory::create, item -> { + if (filter.test(item)) { + items.add(item); + } + return items.size() < maxMessagesToRead; + }); + return items; } /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 264bac7cb6aab..450a446c85a78 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -135,6 +135,7 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul protected final ExecutorService dispatchMessagesThread; private final SharedConsumerAssignor assignor; protected int lastNumberOfEntriesDispatched; + protected boolean skipNextBackoff; private final Backoff retryBackoff; protected enum ReadType { Normal, Replay @@ -345,26 +346,24 @@ public synchronized void readMoreEntries() { return; } - NavigableSet messagesToReplayNow = getMessagesToReplayNow(messagesToRead); - NavigableSet messagesToReplayFiltered = filterOutEntriesWillBeDiscarded(messagesToReplayNow); - if (!messagesToReplayFiltered.isEmpty()) { + Set messagesToReplayNow = + canReplayMessages() ? getMessagesToReplayNow(messagesToRead) : Collections.emptySet(); + if (!messagesToReplayNow.isEmpty()) { if (log.isDebugEnabled()) { log.debug("[{}] Schedule replay of {} messages for {} consumers", name, - messagesToReplayFiltered.size(), consumerList.size()); + messagesToReplayNow.size(), consumerList.size()); } - havePendingReplayRead = true; - minReplayedPosition = messagesToReplayNow.first(); + updateMinReplayedPosition(); Set deletedMessages = topic.isDelayedDeliveryEnabled() - ? asyncReplayEntriesInOrder(messagesToReplayFiltered) - : asyncReplayEntries(messagesToReplayFiltered); + ? asyncReplayEntriesInOrder(messagesToReplayNow) + : asyncReplayEntries(messagesToReplayNow); // clear already acked positions from replay bucket - deletedMessages.forEach(position -> redeliveryMessages.remove(position.getLedgerId(), position.getEntryId())); // if all the entries are acked-entries and cleared up from redeliveryMessages, try to read // next entries as readCompletedEntries-callback was never called - if ((messagesToReplayFiltered.size() - deletedMessages.size()) == 0) { + if ((messagesToReplayNow.size() - deletedMessages.size()) == 0) { havePendingReplayRead = false; readMoreEntriesAsync(); } @@ -373,7 +372,11 @@ public synchronized void readMoreEntries() { log.debug("[{}] Dispatcher read is blocked due to unackMessages {} reached to max {}", name, totalUnackedMessages, topic.getMaxUnackedMessagesOnSubscription()); } - } else if (!havePendingRead && hasConsumersNeededNormalRead()) { + } else if (doesntHavePendingRead()) { + if (!isNormalReadAllowed()) { + handleNormalReadNotAllowed(); + return; + } if (shouldPauseOnAckStatePersist(ReadType.Normal)) { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Skipping read for the topic, Due to blocked on ack state persistent.", @@ -386,13 +389,9 @@ public synchronized void readMoreEntries() { consumerList.size()); } havePendingRead = true; - NavigableSet toReplay = getMessagesToReplayNow(1); - if (!toReplay.isEmpty()) { - minReplayedPosition = toReplay.first(); - redeliveryMessages.add(minReplayedPosition.getLedgerId(), minReplayedPosition.getEntryId()); - } else { - minReplayedPosition = null; - } + updateMinReplayedPosition(); + + messagesToRead = Math.min(messagesToRead, getMaxEntriesReadLimit()); // Filter out and skip read delayed messages exist in DelayedDeliveryTracker if (delayedDeliveryTracker.isPresent()) { @@ -410,14 +409,7 @@ public synchronized void readMoreEntries() { } } else { if (log.isDebugEnabled()) { - if (!messagesToReplayNow.isEmpty()) { - log.debug("[{}] [{}] Skipping read for the topic: because all entries in replay queue were" - + " filtered out due to the mechanism of Key_Shared mode, and the left consumers have" - + " no permits now", - topic.getName(), getSubscriptionName()); - } else { - log.debug("[{}] Cannot schedule next read until previous one is done", name); - } + log.debug("[{}] Cannot schedule next read until previous one is done", name); } } } else { @@ -427,6 +419,43 @@ public synchronized void readMoreEntries() { } } + /** + * Sets a hard limit on the number of entries to read from the Managed Ledger. + * Subclasses can override this method to set a different limit. + * By default, this method does not impose an additional limit. + * + * @return the maximum number of entries to read from the Managed Ledger + */ + protected int getMaxEntriesReadLimit() { + return Integer.MAX_VALUE; + } + + /** + * Checks if there's a pending read operation that hasn't completed yet. + * This allows to avoid scheduling a new read operation while the previous one is still in progress. + * @return true if there's a pending read operation + */ + protected boolean doesntHavePendingRead() { + return !havePendingRead; + } + + protected void handleNormalReadNotAllowed() { + // do nothing + } + + /** + * Controls whether replaying entries is currently enabled. + * Subclasses can override this method to temporarily disable replaying entries. + * @return true if replaying entries is currently enabled + */ + protected boolean canReplayMessages() { + return true; + } + + private void updateMinReplayedPosition() { + minReplayedPosition = getFirstPositionInReplay().orElse(null); + } + private boolean shouldPauseOnAckStatePersist(ReadType readType) { // Allows new consumers to consume redelivered messages caused by the just-closed consumer. if (readType != ReadType.Normal) { @@ -446,6 +475,10 @@ protected void reScheduleRead() { reScheduleReadInMs(MESSAGE_RATE_BACKOFF_MS); } + protected synchronized void reScheduleReadWithBackoff() { + reScheduleReadInMs(retryBackoff.next()); + } + protected void reScheduleReadInMs(long readAfterMs) { if (isRescheduleReadInProgress.compareAndSet(false, true)) { if (log.isDebugEnabled()) { @@ -697,14 +730,15 @@ private synchronized void handleSendingMessagesAndReadingMore(ReadType readType, int entriesDispatched = lastNumberOfEntriesDispatched; updatePendingBytesToDispatch(-totalBytesSize); if (triggerReadingMore) { - if (entriesDispatched > 0) { + if (entriesDispatched > 0 || skipNextBackoff) { + skipNextBackoff = false; // Reset the backoff when we successfully dispatched messages retryBackoff.reset(); // Call readMoreEntries in the same thread to trigger the next read readMoreEntries(); } else if (entriesDispatched == 0) { // If no messages were dispatched, we need to reschedule a new read with an increasing backoff delay - reScheduleReadInMs(retryBackoff.next()); + reScheduleReadWithBackoff(); } } } @@ -754,8 +788,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis int remainingMessages = 0; boolean hasChunk = false; for (int i = 0; i < metadataArray.length; i++) { - final MessageMetadata metadata = Commands.peekAndCopyMessageMetadata( - entries.get(i).getDataBuffer(), subscription.toString(), -1); + Entry entry = entries.get(i); + MessageMetadata metadata = entry instanceof EntryAndMetadata ? ((EntryAndMetadata) entry).getMetadata() + : Commands.peekAndCopyMessageMetadata(entry.getDataBuffer(), subscription.toString(), -1); if (metadata != null) { remainingMessages += metadata.getNumMessagesInBatch(); if (!hasChunk && metadata.hasUuid()) { @@ -788,22 +823,27 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis // round-robin dispatch batch size for this consumer int availablePermits = c.isWritable() ? c.getAvailablePermits() : 1; - if (c.getMaxUnackedMessages() > 0) { - // Avoid negative number - int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0); - availablePermits = Math.min(availablePermits, remainUnAckedMessages); - } if (log.isDebugEnabled() && !c.isWritable()) { log.debug("[{}-{}] consumer is not writable. dispatching only 1 message to {}; " + "availablePermits are {}", topic.getName(), name, c, c.getAvailablePermits()); } - int messagesForC = Math.min(Math.min(remainingMessages, availablePermits), - serviceConfig.getDispatcherMaxRoundRobinBatchSize()); - messagesForC = Math.max(messagesForC / avgBatchSizePerMsg, 1); - - int end = Math.min(start + messagesForC, entries.size()); + int maxMessagesInThisBatch = + Math.max(remainingMessages, serviceConfig.getDispatcherMaxRoundRobinBatchSize()); + if (c.getMaxUnackedMessages() > 0) { + // Calculate the maximum number of additional unacked messages allowed + int maxAdditionalUnackedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0); + maxMessagesInThisBatch = Math.min(maxMessagesInThisBatch, maxAdditionalUnackedMessages); + } + int maxEntriesInThisBatch = Math.min(availablePermits, + // use the average batch size per message to calculate the number of entries to + // dispatch. round up to the next integer without using floating point arithmetic. + (maxMessagesInThisBatch + avgBatchSizePerMsg - 1) / avgBatchSizePerMsg); + // pick at least one entry to dispatch + maxEntriesInThisBatch = Math.max(maxEntriesInThisBatch, 1); + + int end = Math.min(start + maxEntriesInThisBatch, entries.size()); List entriesForThisConsumer = entries.subList(start, end); // remove positions first from replay list first : sendMessages recycles entries @@ -817,6 +857,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis EntryBatchSizes batchSizes = EntryBatchSizes.get(entriesForThisConsumer.size()); EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(entriesForThisConsumer.size()); + totalEntries += filterEntriesForConsumer(metadataArray, start, entriesForThisConsumer, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, readType == ReadType.Replay, c); @@ -826,8 +867,8 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis int msgSent = sendMessageInfo.getTotalMessages(); remainingMessages -= msgSent; - start += messagesForC; - entriesToDispatch -= messagesForC; + start += maxEntriesInThisBatch; + entriesToDispatch -= maxEntriesInThisBatch; TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -(msgSent - batchIndexesAcks.getTotalAckedIndexCount())); if (log.isDebugEnabled()) { @@ -848,8 +889,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis entries.size() - start); } entries.subList(start, entries.size()).forEach(entry -> { - long stickyKeyHash = getStickyKeyHash(entry); - addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + addEntryToReplay(entry); entry.release(); }); } @@ -857,6 +897,11 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis return true; } + protected void addEntryToReplay(Entry entry) { + long stickyKeyHash = getStickyKeyHash(entry); + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + } + private boolean sendChunkedMessagesToConsumers(ReadType readType, List entries, MessageMetadata[] metadataArray) { @@ -1021,7 +1066,7 @@ protected int getFirstAvailableConsumerPermits() { return 0; } for (Consumer consumer : consumerList) { - if (consumer != null && !consumer.isBlocked()) { + if (consumer != null && !consumer.isBlocked() && consumer.cnx().isActive()) { int availablePermits = consumer.getAvailablePermits(); if (availablePermits > 0) { return availablePermits; @@ -1045,7 +1090,8 @@ private boolean isConsumerWritable() { @Override public boolean isConsumerAvailable(Consumer consumer) { - return consumer != null && !consumer.isBlocked() && consumer.getAvailablePermits() > 0; + return consumer != null && !consumer.isBlocked() && consumer.cnx().isActive() + && consumer.getAvailablePermits() > 0; } @Override @@ -1235,16 +1281,20 @@ protected synchronized NavigableSet getMessagesToReplayNow(int maxMess delayedDeliveryTracker.get().getScheduledMessages(maxMessagesToRead); messagesAvailableNow.forEach(p -> redeliveryMessages.add(p.getLedgerId(), p.getEntryId())); } - if (!redeliveryMessages.isEmpty()) { - return redeliveryMessages.getMessagesToReplayNow(maxMessagesToRead); + return redeliveryMessages.getMessagesToReplayNow(maxMessagesToRead, createFilterForReplay()); } else { return Collections.emptyNavigableSet(); } } + protected Optional getFirstPositionInReplay() { + return redeliveryMessages.getFirstPositionInReplay(); + } + /** - * This is a mode method designed for Key_Shared mode. + * Creates a stateful filter for filtering replay positions. + * This is only used for Key_Shared mode to skip replaying certain entries. * Filter out the entries that will be discarded due to the order guarantee mechanism of Key_Shared mode. * This method is in order to avoid the scenario below: * - Get positions from the Replay queue. @@ -1252,18 +1302,20 @@ protected synchronized NavigableSet getMessagesToReplayNow(int maxMess * - The order guarantee mechanism of Key_Shared mode filtered out all the entries. * - Delivery non entry to the client, but we did a BK read. */ - protected NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { - return src; + protected Predicate createFilterForReplay() { + // pick all positions from the replay + return position -> true; } /** - * This is a mode method designed for Key_Shared mode, to avoid unnecessary stuck. - * See detail {@link PersistentStickyKeyDispatcherMultipleConsumers#hasConsumersNeededNormalRead}. + * Checks if the dispatcher is allowed to read messages from the cursor. */ - protected boolean hasConsumersNeededNormalRead() { + protected boolean isNormalReadAllowed() { return true; } + + protected synchronized boolean shouldPauseDeliveryForDelayTracker() { return delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().shouldPauseAllDeliveries(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index e8e4919a9be52..d45b9394dd744 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -19,21 +19,18 @@ package org.apache.pulsar.broker.service.persistent; import com.google.common.annotations.VisibleForTesting; -import io.netty.util.concurrent.FastThreadLocal; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.NavigableSet; +import java.util.Optional; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; +import java.util.function.Predicate; import javax.annotation.Nullable; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -42,6 +39,8 @@ import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.commons.collections4.MapUtils; +import org.apache.commons.lang3.mutable.MutableBoolean; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.ConsistentHashingStickyKeyConsumerSelector; @@ -68,7 +67,7 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi private final boolean allowOutOfOrderDelivery; private final StickyKeyConsumerSelector selector; - private boolean isDispatcherStuckOnReplays = false; + private boolean skipNextReplayToTriggerLookAhead = false; private final KeySharedMode keySharedMode; /** @@ -183,22 +182,6 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE } } - private static final FastThreadLocal>> localGroupedEntries = - new FastThreadLocal>>() { - @Override - protected Map> initialValue() throws Exception { - return new HashMap<>(); - } - }; - - private static final FastThreadLocal>> localGroupedPositions = - new FastThreadLocal>>() { - @Override - protected Map> initialValue() throws Exception { - return new HashMap<>(); - } - }; - @Override protected synchronized boolean trySendMessagesToConsumers(ReadType readType, List entries) { lastNumberOfEntriesDispatched = 0; @@ -221,15 +204,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis if (!allowOutOfOrderDelivery) { // A corner case that we have to retry a readMoreEntries in order to preserver order delivery. // This may happen when consumer closed. See issue #12885 for details. - NavigableSet messagesToReplayNow = this.getMessagesToReplayNow(1); - if (messagesToReplayNow != null && !messagesToReplayNow.isEmpty()) { - Position replayPosition = messagesToReplayNow.first(); - - // We have received a message potentially from the delayed tracker and, since we're not using it - // right now, it needs to be added to the redelivery tracker or we won't attempt anymore to - // resend it (until we disconnect consumer). - redeliveryMessages.add(replayPosition.getLedgerId(), replayPosition.getEntryId()); - + Optional firstReplayPosition = getFirstPositionInReplay(); + if (firstReplayPosition.isPresent()) { + Position replayPosition = firstReplayPosition.get(); if (this.minReplayedPosition != null) { // If relayPosition is a new entry wither smaller position is inserted for redelivery during this // async read, it is possible that this relayPosition should dispatch to consumer first. So in @@ -274,96 +251,58 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } } - final Map> groupedEntries = localGroupedEntries.get(); - groupedEntries.clear(); - final Map> consumerStickyKeyHashesMap = new HashMap<>(); - - for (Entry entry : entries) { - int stickyKeyHash = getStickyKeyHash(entry); - Consumer c = selector.select(stickyKeyHash); - if (c != null) { - groupedEntries.computeIfAbsent(c, k -> new ArrayList<>()).add(entry); - consumerStickyKeyHashesMap.computeIfAbsent(c, k -> new HashSet<>()).add(stickyKeyHash); - } else { - addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); - entry.release(); - } - } - - AtomicInteger keyNumbers = new AtomicInteger(groupedEntries.size()); + // returns a boolean indicating whether look-ahead could be useful, when there's a consumer + // with available permits, and it's not able to make progress because of blocked hashes. + MutableBoolean triggerLookAhead = new MutableBoolean(); + // filter and group the entries by consumer for dispatching + final Map> entriesByConsumerForDispatching = + filterAndGroupEntriesForDispatching(entries, readType, triggerLookAhead); - int currentThreadKeyNumber = groupedEntries.size(); - if (currentThreadKeyNumber == 0) { - currentThreadKeyNumber = -1; - } - for (Map.Entry> current : groupedEntries.entrySet()) { + AtomicInteger remainingConsumersToFinishSending = new AtomicInteger(entriesByConsumerForDispatching.size()); + for (Map.Entry> current : entriesByConsumerForDispatching.entrySet()) { Consumer consumer = current.getKey(); - assert consumer != null; // checked when added to groupedEntries - List entriesWithSameKey = current.getValue(); - int entriesWithSameKeyCount = entriesWithSameKey.size(); - int availablePermits = getAvailablePermits(consumer); - int messagesForC = getRestrictedMaxEntriesForConsumer(consumer, - entriesWithSameKey.stream().map(Entry::getPosition).collect(Collectors.toList()), availablePermits, - readType, consumerStickyKeyHashesMap.get(consumer)); + List entriesForConsumer = current.getValue(); if (log.isDebugEnabled()) { log.debug("[{}] select consumer {} with messages num {}, read type is {}", - name, consumer.consumerName(), messagesForC, readType); + name, consumer.consumerName(), entriesForConsumer.size(), readType); } - - if (messagesForC < entriesWithSameKeyCount) { - // We are not able to push all the messages with given key to its consumer, - // so we discard for now and mark them for later redelivery - for (int i = messagesForC; i < entriesWithSameKeyCount; i++) { - Entry entry = entriesWithSameKey.get(i); - long stickyKeyHash = getStickyKeyHash(entry); - addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); - entry.release(); - entriesWithSameKey.set(i, null); + final ManagedLedger managedLedger = cursor.getManagedLedger(); + for (Entry entry : entriesForConsumer) { + // remove positions first from replay list first : sendMessages recycles entries + if (readType == ReadType.Replay) { + redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId()); } - } - - if (messagesForC > 0) { - final ManagedLedger managedLedger = cursor.getManagedLedger(); - for (int i = 0; i < messagesForC; i++) { - final Entry entry = entriesWithSameKey.get(i); - // remove positions first from replay list first : sendMessages recycles entries - if (readType == ReadType.Replay) { - redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId()); - } - // Add positions to individuallySentPositions if necessary - if (!allowOutOfOrderDelivery) { - final Position position = entry.getPosition(); - // Store to individuallySentPositions even if lastSentPosition is null - if ((lastSentPosition == null || position.compareTo(lastSentPosition) > 0) - && !individuallySentPositions.contains(position.getLedgerId(), position.getEntryId())) { - final Position previousPosition = managedLedger.getPreviousPosition(position); - individuallySentPositions.addOpenClosed(previousPosition.getLedgerId(), - previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); - } + // Add positions to individuallySentPositions if necessary + if (!allowOutOfOrderDelivery) { + final Position position = entry.getPosition(); + // Store to individuallySentPositions even if lastSentPosition is null + if ((lastSentPosition == null || position.compareTo(lastSentPosition) > 0) + && !individuallySentPositions.contains(position.getLedgerId(), position.getEntryId())) { + final Position previousPosition = managedLedger.getPreviousPosition(position); + individuallySentPositions.addOpenClosed(previousPosition.getLedgerId(), + previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); } } + } - SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); - EntryBatchSizes batchSizes = EntryBatchSizes.get(messagesForC); - EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(messagesForC); - totalEntries += filterEntriesForConsumer(entriesWithSameKey, batchSizes, sendMessageInfo, - batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); - consumer.sendMessages(entriesWithSameKey, batchSizes, batchIndexesAcks, - sendMessageInfo.getTotalMessages(), - sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), - getRedeliveryTracker()).addListener(future -> { - if (future.isDone() && keyNumbers.decrementAndGet() == 0) { - readMoreEntries(); - } - }); + SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + EntryBatchSizes batchSizes = EntryBatchSizes.get(entriesForConsumer.size()); + EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(entriesForConsumer.size()); + totalEntries += filterEntriesForConsumer(entriesForConsumer, batchSizes, sendMessageInfo, + batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); + consumer.sendMessages(entriesForConsumer, batchSizes, batchIndexesAcks, + sendMessageInfo.getTotalMessages(), + sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), + getRedeliveryTracker()).addListener(future -> { + if (future.isDone() && remainingConsumersToFinishSending.decrementAndGet() == 0) { + readMoreEntries(); + } + }); - TOTAL_AVAILABLE_PERMITS_UPDATER.getAndAdd(this, - -(sendMessageInfo.getTotalMessages() - batchIndexesAcks.getTotalAckedIndexCount())); - totalMessagesSent += sendMessageInfo.getTotalMessages(); - totalBytesSent += sendMessageInfo.getTotalBytes(); - } else { - currentThreadKeyNumber = keyNumbers.decrementAndGet(); - } + TOTAL_AVAILABLE_PERMITS_UPDATER.getAndAdd(this, + -(sendMessageInfo.getTotalMessages() - batchIndexesAcks.getTotalAckedIndexCount())); + totalMessagesSent += sendMessageInfo.getTotalMessages(); + totalBytesSent += sendMessageInfo.getTotalBytes(); } // Update the last sent position and remove ranges from individuallySentPositions if necessary @@ -426,41 +365,250 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis // acquire message-dispatch permits for already delivered messages acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); - if (totalMessagesSent == 0 && (recentlyJoinedConsumers == null || recentlyJoinedConsumers.isEmpty())) { - // This means, that all the messages we've just read cannot be dispatched right now. - // This condition can only happen when: - // 1. We have consumers ready to accept messages (otherwise the would not haven been triggered) - // 2. All keys in the current set of messages are routing to consumers that are currently busy - // - // The solution here is to move on and read next batch of messages which might hopefully contain - // also keys meant for other consumers. - // - // We do it unless that are "recently joined consumers". In that case, we would be looking - // ahead in the stream while the new consumers are not ready to accept the new messages, - // therefore would be most likely only increase the distance between read-position and mark-delete - // position. - isDispatcherStuckOnReplays = true; + // trigger read more messages if necessary + if (triggerLookAhead.booleanValue()) { + // When all messages get filtered and no messages are sent, we should read more entries, "look ahead" + // so that a possible next batch of messages might contain messages that can be dispatched. + // This is done only when there's a consumer with available permits, and it's not able to make progress + // because of blocked hashes. Without this rule we would be looking ahead in the stream while the + // new consumers are not ready to accept the new messages, + // therefore would be most likely only increase the distance between read-position and mark-delete position. + skipNextReplayToTriggerLookAhead = true; + // skip backoff delay before reading ahead in the "look ahead" mode to prevent any additional latency + skipNextBackoff = true; return true; - } else if (currentThreadKeyNumber == 0) { + } + + // if no messages were sent, we should retry after a backoff delay + if (entriesByConsumerForDispatching.size() == 0) { return true; } + return false; } - private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List entries, - int availablePermits, ReadType readType, Set stickyKeyHashes) { - int maxMessages = Math.min(entries.size(), availablePermits); - if (maxMessages == 0) { - return 0; + private boolean isReplayQueueSizeBelowLimit() { + return redeliveryMessages.size() < getEffectiveLookAheadLimit(); + } + + private int getEffectiveLookAheadLimit() { + return getEffectiveLookAheadLimit(serviceConfig, consumerList.size()); + } + + static int getEffectiveLookAheadLimit(ServiceConfiguration serviceConfig, int consumerCount) { + int perConsumerLimit = serviceConfig.getKeySharedLookAheadMsgInReplayThresholdPerConsumer(); + int perSubscriptionLimit = serviceConfig.getKeySharedLookAheadMsgInReplayThresholdPerSubscription(); + int effectiveLimit; + if (perConsumerLimit <= 0) { + effectiveLimit = perSubscriptionLimit; + } else { + effectiveLimit = perConsumerLimit * consumerCount; + if (perSubscriptionLimit > 0 && perSubscriptionLimit < effectiveLimit) { + effectiveLimit = perSubscriptionLimit; + } } - if (readType == ReadType.Normal && stickyKeyHashes != null - && redeliveryMessages.containsStickyKeyHashes(stickyKeyHashes)) { - // If redeliveryMessages contains messages that correspond to the same hash as the messages - // that the dispatcher is trying to send, do not send those messages for order guarantee - return 0; + if (effectiveLimit <= 0) { + // use max unacked messages limits if key shared look-ahead limits are disabled + int maxUnackedMessagesPerSubscription = serviceConfig.getMaxUnackedMessagesPerSubscription(); + if (maxUnackedMessagesPerSubscription <= 0) { + maxUnackedMessagesPerSubscription = Integer.MAX_VALUE; + } + int maxUnackedMessagesByConsumers = consumerCount * serviceConfig.getMaxUnackedMessagesPerConsumer(); + if (maxUnackedMessagesByConsumers <= 0) { + maxUnackedMessagesByConsumers = Integer.MAX_VALUE; + } + effectiveLimit = Math.min(maxUnackedMessagesPerSubscription, maxUnackedMessagesByConsumers); } + return effectiveLimit; + } + + // groups the entries by consumer and filters out the entries that should not be dispatched + // the entries are handled in the order they are received instead of first grouping them by consumer and + // then filtering them + private Map> filterAndGroupEntriesForDispatching(List entries, ReadType readType, + MutableBoolean triggerLookAhead) { + // entries grouped by consumer + Map> entriesGroupedByConsumer = new HashMap<>(); + // permits for consumer, permits are for entries/batches + Map permitsForConsumer = new HashMap<>(); + // maxLastSentPosition cache for consumers, used when recently joined consumers exist + boolean hasRecentlyJoinedConsumers = hasRecentlyJoinedConsumers(); + Map maxLastSentPositionCache = hasRecentlyJoinedConsumers ? new HashMap<>() : null; + boolean lookAheadAllowed = isReplayQueueSizeBelowLimit(); + // in normal read mode, keep track of consumers that are blocked by hash, to check if look-ahead could be useful + Set blockedByHashConsumers = lookAheadAllowed && readType == ReadType.Normal ? new HashSet<>() : null; + // in replay read mode, keep track of consumers for entries, used for look-ahead check + Set consumersForEntriesForLookaheadCheck = lookAheadAllowed ? new HashSet<>() : null; + + for (Entry entry : entries) { + int stickyKeyHash = getStickyKeyHash(entry); + Consumer consumer = selector.select(stickyKeyHash); + MutableBoolean blockedByHash = null; + boolean dispatchEntry = false; + if (consumer != null) { + if (lookAheadAllowed) { + consumersForEntriesForLookaheadCheck.add(consumer); + } + Position maxLastSentPosition = hasRecentlyJoinedConsumers ? maxLastSentPositionCache.computeIfAbsent( + consumer, __ -> resolveMaxLastSentPositionForRecentlyJoinedConsumer(consumer, readType)) : null; + blockedByHash = lookAheadAllowed && readType == ReadType.Normal ? new MutableBoolean(false) : null; + MutableInt permits = + permitsForConsumer.computeIfAbsent(consumer, + k -> new MutableInt(getAvailablePermits(consumer))); + // a consumer was found for the sticky key hash and the entry can be dispatched + if (permits.intValue() > 0 && canDispatchEntry(entry, readType, stickyKeyHash, + maxLastSentPosition, blockedByHash)) { + // decrement the permits for the consumer + permits.decrement(); + // allow the entry to be dispatched + dispatchEntry = true; + } + } + if (dispatchEntry) { + // add the entry to consumer's entry list for dispatching + List consumerEntries = + entriesGroupedByConsumer.computeIfAbsent(consumer, k -> new ArrayList<>()); + consumerEntries.add(entry); + } else { + if (blockedByHash != null && blockedByHash.isTrue()) { + // the entry is blocked by hash, add the consumer to the blocked set + blockedByHashConsumers.add(consumer); + } + // add the message to replay + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + // release the entry as it will not be dispatched + entry.release(); + } + } + // + // determine whether look-ahead could be useful for making more progress + // + if (lookAheadAllowed && entriesGroupedByConsumer.isEmpty()) { + // check if look-ahead could be useful for the consumers that are blocked by a hash that is in the replay + // queue. This check applies only to the normal read mode. + if (readType == ReadType.Normal) { + for (Consumer consumer : blockedByHashConsumers) { + // if the consumer isn't in the entriesGroupedByConsumer, it means that it won't receive any + // messages + // if it has available permits, then look-ahead could be useful for this particular consumer + // to make further progress + if (!entriesGroupedByConsumer.containsKey(consumer) + && permitsForConsumer.get(consumer).intValue() > 0) { + triggerLookAhead.setTrue(); + break; + } + } + } + // check if look-ahead could be useful for other consumers + if (!triggerLookAhead.booleanValue()) { + for (Consumer consumer : getConsumers()) { + // filter out the consumers that are already checked when the entries were processed for entries + if (!consumersForEntriesForLookaheadCheck.contains(consumer)) { + // if another consumer has available permits, then look-ahead could be useful + if (getAvailablePermits(consumer) > 0) { + triggerLookAhead.setTrue(); + break; + } + } + } + } + } + return entriesGroupedByConsumer; + } + + // checks if the entry can be dispatched to the consumer + private boolean canDispatchEntry(Entry entry, + ReadType readType, int stickyKeyHash, Position maxLastSentPosition, + MutableBoolean blockedByHash) { + // check if the entry can be replayed to a recently joined consumer + if (maxLastSentPosition != null && entry.getPosition().compareTo(maxLastSentPosition) > 0) { + return false; + } + + // If redeliveryMessages contains messages that correspond to the same hash as the entry to be dispatched + // do not send those messages for order guarantee + if (readType == ReadType.Normal && redeliveryMessages.containsStickyKeyHash(stickyKeyHash)) { + if (blockedByHash != null) { + blockedByHash.setTrue(); + } + return false; + } + + return true; + } + + /** + * Creates a filter for replaying messages. The filter is stateful and shouldn't be cached or reused. + * @see PersistentDispatcherMultipleConsumers#createFilterForReplay() + */ + @Override + protected Predicate createFilterForReplay() { + return new ReplayPositionFilter(); + } + + /** + * Filter for replaying messages. The filter is stateful for a single invocation and shouldn't be cached, shared + * or reused. This is a short-lived object, and optimizing it for the "no garbage" coding style of Pulsar is + * unnecessary since the JVM can optimize allocations for short-lived objects. + */ + private class ReplayPositionFilter implements Predicate { + // tracks the available permits for each consumer for the duration of the filter usage + // the filter is stateful and shouldn't be shared or reused later + private final Map availablePermitsMap = new HashMap<>(); + private final Map maxLastSentPositionCache = + hasRecentlyJoinedConsumers() ? new HashMap<>() : null; + + @Override + public boolean test(Position position) { + // if out of order delivery is allowed, then any position will be replayed + if (isAllowOutOfOrderDelivery()) { + return true; + } + // lookup the sticky key hash for the entry at the replay position + Long stickyKeyHash = redeliveryMessages.getHash(position.getLedgerId(), position.getEntryId()); + if (stickyKeyHash == null) { + // the sticky key hash is missing for delayed messages, the filtering will happen at the time of + // dispatch after reading the entry from the ledger + if (log.isDebugEnabled()) { + log.debug("[{}] replay of entry at position {} doesn't contain sticky key hash.", name, position); + } + return true; + } + // find the consumer for the sticky key hash + Consumer consumer = selector.select(stickyKeyHash.intValue()); + // skip replaying the message position if there's no assigned consumer + if (consumer == null) { + return false; + } + // lookup the available permits for the consumer + MutableInt availablePermits = + availablePermitsMap.computeIfAbsent(consumer, + k -> new MutableInt(getAvailablePermits(consumer))); + // skip replaying the message position if the consumer has no available permits + if (availablePermits.intValue() <= 0) { + return false; + } + // check if the entry position can be replayed to a recently joined consumer + Position maxLastSentPosition = maxLastSentPositionCache != null + ? maxLastSentPositionCache.computeIfAbsent(consumer, __ -> + resolveMaxLastSentPositionForRecentlyJoinedConsumer(consumer, ReadType.Replay)) + : null; + if (maxLastSentPosition != null && position.compareTo(maxLastSentPosition) > 0) { + return false; + } + availablePermits.decrement(); + return true; + } + } + + /** + * Contains the logic to resolve the max last sent position for a consumer + * when the consumer has recently joined. This is only applicable for key shared mode when + * allowOutOfOrderDelivery=false. + */ + private Position resolveMaxLastSentPositionForRecentlyJoinedConsumer(Consumer consumer, ReadType readType) { if (recentlyJoinedConsumers == null) { - return maxMessages; + return null; } removeConsumersFromRecentJoinedConsumers(); Position maxLastSentPosition = recentlyJoinedConsumers.get(consumer); @@ -468,7 +616,7 @@ private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List 0) { - // We have already crossed the divider line. All messages in the list are now - // newer than what we can currently dispatch to this consumer - return i; - } - } - return maxMessages; + return maxLastSentPosition; } + @Override public void markDeletePositionMoveForward() { // Execute the notification in different thread to avoid a mutex chain here // from the delete operation that was completed topic.getBrokerService().getTopicOrderedExecutor().execute(() -> { synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) { - if (recentlyJoinedConsumers != null && !recentlyJoinedConsumers.isEmpty() + if (hasRecentlyJoinedConsumers() && removeConsumersFromRecentJoinedConsumers()) { // After we process acks, we need to check whether the mark-delete position was advanced and we // can finally read more messages. It's safe to call readMoreEntries() multiple times. @@ -520,6 +660,10 @@ && removeConsumersFromRecentJoinedConsumers()) { }); } + private boolean hasRecentlyJoinedConsumers() { + return !MapUtils.isEmpty(recentlyJoinedConsumers); + } + private boolean removeConsumersFromRecentJoinedConsumers() { if (MapUtils.isEmpty(recentlyJoinedConsumers)) { return false; @@ -553,99 +697,104 @@ private synchronized Position updateIfNeededAndGetLastSentPosition() { return lastSentPosition; } + /** + * The dispatcher will skip replaying messages when all messages in the replay queue are filtered out when + * skipNextReplayToTriggerLookAhead=true. The flag gets resetted after the call. + * + * If we're stuck on replay, we want to move forward reading on the topic (until the configured look ahead + * limits kick in), instead of keep replaying the same old messages, since the consumer that these + * messages are routing to might be busy at the moment. + * + * Please see {@link ServiceConfiguration#getKeySharedLookAheadMsgInReplayThresholdPerConsumer} and + * {@link ServiceConfiguration#getKeySharedLookAheadMsgInReplayThresholdPerSubscription} for configuring the limits. + */ @Override - protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { - if (isDispatcherStuckOnReplays) { - // If we're stuck on replay, we want to move forward reading on the topic (until the overall max-unacked - // messages kicks in), instead of keep replaying the same old messages, since the consumer that these - // messages are routing to might be busy at the moment - this.isDispatcherStuckOnReplays = false; - return Collections.emptyNavigableSet(); - } else { - return super.getMessagesToReplayNow(maxMessagesToRead); + protected synchronized boolean canReplayMessages() { + if (skipNextReplayToTriggerLookAhead) { + skipNextReplayToTriggerLookAhead = false; + return false; } + return true; } private int getAvailablePermits(Consumer c) { + // skip consumers that are currently closing + if (!c.cnx().isActive()) { + return 0; + } int availablePermits = Math.max(c.getAvailablePermits(), 0); - if (c.getMaxUnackedMessages() > 0) { - // Avoid negative number - int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0); - availablePermits = Math.min(availablePermits, remainUnAckedMessages); + if (availablePermits > 0 && c.getMaxUnackedMessages() > 0) { + // Calculate the maximum number of additional unacked messages allowed + int maxAdditionalUnackedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0); + if (maxAdditionalUnackedMessages == 0) { + // if the consumer has reached the max unacked messages, then no more messages can be dispatched + return 0; + } + // Estimate the remaining permits based on the average messages per entry + // add "avgMessagesPerEntry - 1" to round up the division to the next integer without the need to use + // floating point arithmetic + int avgMessagesPerEntry = Math.max(c.getAvgMessagesPerEntry(), 1); + int estimatedRemainingPermits = + (maxAdditionalUnackedMessages + avgMessagesPerEntry - 1) / avgMessagesPerEntry; + // return the minimum of current available permits and estimated remaining permits + return Math.min(availablePermits, estimatedRemainingPermits); + } else { + return availablePermits; } - return availablePermits; } + /** + * For Key_Shared subscription, the dispatcher will not read more entries while there are pending reads + * or pending replay reads. + * @return true if there are no pending reads or pending replay reads + */ @Override - protected synchronized NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { - // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", - // So skip this filter out. - if (isAllowOutOfOrderDelivery()) { - return src; - } - if (src.isEmpty()) { - return src; - } - NavigableSet res = new TreeSet<>(); - // Group positions. - final Map> groupedPositions = localGroupedPositions.get(); - groupedPositions.clear(); - for (Position pos : src) { - Long stickyKeyHash = redeliveryMessages.getHash(pos.getLedgerId(), pos.getEntryId()); - if (stickyKeyHash == null) { - res.add(pos); - continue; - } - Consumer c = selector.select(stickyKeyHash.intValue()); - if (c == null) { - // Maybe using HashRangeExclusiveStickyKeyConsumerSelector. - continue; - } - groupedPositions.computeIfAbsent(c, k -> new ArrayList<>()).add(pos); - } - // Filter positions by the Recently Joined Position rule. - for (Map.Entry> item : groupedPositions.entrySet()) { - int availablePermits = getAvailablePermits(item.getKey()); - if (availablePermits == 0) { - continue; - } - int posCountToRead = getRestrictedMaxEntriesForConsumer(item.getKey(), item.getValue(), availablePermits, - ReadType.Replay, null); - if (posCountToRead > 0) { - res.addAll(item.getValue().subList(0, posCountToRead)); - } - } - return res; + protected boolean doesntHavePendingRead() { + return !havePendingRead && !havePendingReplayRead; } /** - * In Key_Shared mode, the consumer will not receive any entries from a normal reading if it is included in - * {@link #recentlyJoinedConsumers}, they can only receive entries from replay reads. - * If all entries in {@link #redeliveryMessages} have been filtered out due to the order guarantee mechanism, - * Broker need a normal read to make the consumers not included in @link #recentlyJoinedConsumers} will not be - * stuck. See https://github.com/apache/pulsar/pull/7105. + * For Key_Shared subscription, the dispatcher will not attempt to read more entries if the replay queue size + * has reached the limit or if there are no consumers with permits. */ @Override - protected boolean hasConsumersNeededNormalRead() { - // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", - // So the method "filterOutEntriesWillBeDiscarded" will filter out nothing, just return "true" here. - if (isAllowOutOfOrderDelivery()) { - return true; + protected boolean isNormalReadAllowed() { + // don't allow reading more if the replay queue size has reached the limit + if (!isReplayQueueSizeBelowLimit()) { + return false; } for (Consumer consumer : consumerList) { + // skip blocked consumers if (consumer == null || consumer.isBlocked()) { continue; } - if (recentlyJoinedConsumers.containsKey(consumer)) { - continue; - } - if (consumer.getAvailablePermits() > 0) { + // before reading more, check that there's at least one consumer that has permits + if (getAvailablePermits(consumer) > 0) { return true; } } return false; } + @Override + protected int getMaxEntriesReadLimit() { + // prevent the redelivery queue from growing over the limit by limiting the number of entries to read + // to the maximum number of entries that can be added to the redelivery queue + return Math.max(getEffectiveLookAheadLimit() - redeliveryMessages.size(), 1); + } + + /** + * When a normal read is not allowed, the dispatcher will reschedule a read with a backoff. + */ + @Override + protected void handleNormalReadNotAllowed() { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Skipping read for the topic since normal read isn't allowed. " + + "Rescheduling a read with a backoff.", topic.getName(), getSubscriptionName()); + } + reScheduleReadWithBackoff(); + } + @Override public SubType getType() { return SubType.Key_Shared; @@ -702,5 +851,4 @@ public Map> getConsumerKeyHashRanges() { } private static final Logger log = LoggerFactory.getLogger(PersistentStickyKeyDispatcherMultipleConsumers.class); - } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java b/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java index e42cae2580b78..cc1eae475fa2d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java @@ -22,10 +22,12 @@ import java.util.Map; import java.util.NavigableMap; import java.util.NavigableSet; +import java.util.Optional; import java.util.TreeMap; import java.util.TreeSet; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.commons.lang3.mutable.MutableObject; import org.apache.pulsar.common.util.collections.LongPairSet; import org.roaringbitmap.RoaringBitmap; @@ -93,25 +95,51 @@ public void removeUpTo(long item1, long item2) { } } + public > Optional first(LongPairSet.LongPairFunction longPairConverter) { + MutableObject> result = new MutableObject<>(Optional.empty()); + processItems(longPairConverter, item -> { + result.setValue(Optional.of(item)); + return false; + }); + return result.getValue(); + } public > NavigableSet items(int numberOfItems, LongPairSet.LongPairFunction longPairConverter) { NavigableSet items = new TreeSet<>(); + processItems(longPairConverter, item -> { + items.add(item); + return items.size() < numberOfItems; + }); + return items; + } + + public interface ItemProcessor> { + /** + * @param item + * @return false if there is no further processing required + */ + boolean process(T item); + } + + public > void processItems(LongPairSet.LongPairFunction longPairConverter, + ItemProcessor itemProcessor) { lock.readLock().lock(); try { for (Map.Entry entry : map.entrySet()) { Iterator iterator = entry.getValue().stream().iterator(); - while (iterator.hasNext() && items.size() < numberOfItems) { - items.add(longPairConverter.apply(entry.getKey(), iterator.next())); + boolean continueProcessing = true; + while (continueProcessing && iterator.hasNext()) { + T item = longPairConverter.apply(entry.getKey(), iterator.next()); + continueProcessing = itemProcessor.process(item); } - if (items.size() == numberOfItems) { + if (!continueProcessing) { break; } } } finally { lock.readLock().unlock(); } - return items; } public boolean isEmpty() { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java index bfb172d0711d4..5641816ee0b80 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java @@ -18,8 +18,31 @@ */ package org.apache.pulsar.broker; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.ObjectWriter; +import java.io.IOException; +import java.io.StringWriter; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.Arrays; import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.mockito.Mockito; +import org.slf4j.Logger; /** * Holds util methods used in test. @@ -77,4 +100,138 @@ public static T spyWithoutRecordingInvocations(T object) { .defaultAnswer(Mockito.CALLS_REAL_METHODS) .stubOnly()); } + + /** + * Uses Jackson to create a JSON string for the given object + * @param object to convert to JSON + * @return JSON string + */ + public static String toJson(Object object) { + ObjectWriter writer = ObjectMapperFactory.getMapper().writer(); + StringWriter stringWriter = new StringWriter(); + try (JsonGenerator generator = writer.createGenerator(stringWriter).useDefaultPrettyPrinter()) { + generator.writeObject(object); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return stringWriter.toString(); + } + + /** + * Logs the topic stats and internal stats for the given topic + * @param logger logger to use + * @param pulsarAdmin PulsarAdmin client to use + * @param topic topic name + */ + public static void logTopicStats(Logger logger, PulsarAdmin pulsarAdmin, String topic) { + try { + logger.info("[{}] stats: {}", topic, toJson(pulsarAdmin.topics().getStats(topic))); + logger.info("[{}] internalStats: {}", topic, + toJson(pulsarAdmin.topics().getInternalStats(topic, true))); + } catch (PulsarAdminException e) { + logger.warn("Failed to get stats for topic {}", topic, e); + } + } + + /** + * Receive messages concurrently from multiple consumers and handles them using the provided message handler. + * The message handler should return true if it wants to continue receiving more messages, false otherwise. + * + * @param messageHandler the message handler + * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages + * @param consumers the consumers to receive messages from + * @param the message value type + */ + public static void receiveMessages(BiFunction, Message, Boolean> messageHandler, + Duration quietTimeout, + Consumer... consumers) { + FutureUtil.waitForAll(Arrays.stream(consumers) + .map(consumer -> receiveMessagesAsync(consumer, quietTimeout, messageHandler)).toList()).join(); + } + + // asynchronously receive messages from a consumer and handle them using the provided message handler + // the benefit is that multiple consumers can be concurrently consumed without the need to have multiple threads + // this is useful in tests where multiple consumers are needed to test the functionality + private static CompletableFuture receiveMessagesAsync(Consumer consumer, Duration quietTimeout, + BiFunction, Message, Boolean> + messageHandler) { + CompletableFuture> receiveFuture = consumer.receiveAsync(); + return receiveFuture + .orTimeout(quietTimeout.toMillis(), TimeUnit.MILLISECONDS) + .handle((msg, t) -> { + if (t != null) { + if (t instanceof TimeoutException) { + // cancel the receive future so that Pulsar client can clean up the resources + receiveFuture.cancel(false); + return false; + } else { + throw FutureUtil.wrapToCompletionException(t); + } + } + return messageHandler.apply(consumer, msg); + }).thenComposeAsync(receiveMore -> { + if (receiveMore) { + return receiveMessagesAsync(consumer, quietTimeout, messageHandler); + } else { + return CompletableFuture.completedFuture(null); + } + }); + } + + /** + * Receive messages concurrently from multiple consumers and handles them using the provided message handler. + * The messages are received until the quiet timeout is reached or the maximum number of messages is received. + * + * @param messageHandler the message handler + * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages + * @param maxMessages the maximum number of messages to receive + * @param consumers the consumers to receive messages from + * @param the message value type + */ + public static void receiveMessagesN(BiConsumer, Message> messageHandler, + Duration quietTimeout, + int maxMessages, + Consumer... consumers) + throws ExecutionException, InterruptedException { + AtomicInteger messagesReceived = new AtomicInteger(); + receiveMessages( + (consumer, message) -> { + messageHandler.accept(consumer, message); + return messagesReceived.incrementAndGet() < maxMessages; + }, quietTimeout, consumers); + } + + /** + * Receive messages concurrently from multiple consumers and handles them using the provided message handler. + * + * @param messageHandler the message handler + * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages + * @param consumers the consumers to receive messages from + * @param the message value type + */ + public static void receiveMessagesInThreads(BiFunction, Message, Boolean> messageHandler, + final Duration quietTimeout, + Consumer... consumers) { + FutureUtil.waitForAll(Arrays.stream(consumers).sequential().map(consumer -> { + return CompletableFuture.runAsync(() -> { + try { + while (!Thread.currentThread().isInterrupted()) { + Message msg = consumer.receive((int) quietTimeout.toMillis(), TimeUnit.MILLISECONDS); + if (msg != null) { + if (!messageHandler.apply(consumer, msg)) { + break; + } + } else { + break; + } + } + } catch (PulsarClientException e) { + throw new CompletionException(e); + } + }, runnable -> { + Thread thread = new Thread(runnable, "Consumer-" + consumer.getConsumerName()); + thread.start(); + }); + }).toList()).join(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java index 1cc20b04c2137..e32af29c7e962 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java @@ -626,21 +626,23 @@ public void testGetTransactionBufferInternalStats() throws Exception { producer.newMessage(transaction).send(); transaction.abort().get(); - // Get transaction buffer internal stats and verify single snapshot stats - TransactionBufferInternalStats stats = admin.transactions() - .getTransactionBufferInternalStatsAsync(topic2, true).get(); - assertEquals(stats.snapshotType, AbortedTxnProcessor.SnapshotType.Single.toString()); - assertNotNull(stats.singleSnapshotSystemTopicInternalStats); - - // Get managed ledger internal stats for the transaction buffer snapshot topic - PersistentTopicInternalStats internalStats = admin.topics().getInternalStats( - TopicName.get(topic2).getNamespace() + "/" + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT); - verifyManagedLedgerInternalStats(stats.singleSnapshotSystemTopicInternalStats.managedLedgerInternalStats, - internalStats); - assertTrue(stats.singleSnapshotSystemTopicInternalStats.managedLedgerName - .contains(SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT)); - assertNull(stats.segmentInternalStats); - assertNull(stats.segmentIndexInternalStats); + Awaitility.await().untilAsserted(() -> { + // Get transaction buffer internal stats and verify single snapshot stats + TransactionBufferInternalStats stats = admin.transactions() + .getTransactionBufferInternalStatsAsync(topic2, true).get(); + assertEquals(stats.snapshotType, AbortedTxnProcessor.SnapshotType.Single.toString()); + assertNotNull(stats.singleSnapshotSystemTopicInternalStats); + + // Get managed ledger internal stats for the transaction buffer snapshot topic + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats( + TopicName.get(topic2).getNamespace() + "/" + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT); + verifyManagedLedgerInternalStats(stats.singleSnapshotSystemTopicInternalStats.managedLedgerInternalStats, + internalStats); + assertTrue(stats.singleSnapshotSystemTopicInternalStats.managedLedgerName + .contains(SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT)); + assertNull(stats.segmentInternalStats); + assertNull(stats.segmentIndexInternalStats); + }); // Configure segmented snapshot and set segment size pulsar.getConfig().setTransactionBufferSnapshotSegmentSize(9); @@ -652,28 +654,31 @@ public void testGetTransactionBufferInternalStats() throws Exception { producer.newMessage(transaction).send(); transaction.abort().get(); - // Get transaction buffer internal stats and verify segmented snapshot stats - stats = admin.transactions().getTransactionBufferInternalStatsAsync(topic3, true).get(); - assertEquals(stats.snapshotType, AbortedTxnProcessor.SnapshotType.Segment.toString()); - assertNull(stats.singleSnapshotSystemTopicInternalStats); - assertNotNull(stats.segmentInternalStats); - - // Get managed ledger internal stats for the transaction buffer segments topic - internalStats = admin.topics().getInternalStats( - TopicName.get(topic2).getNamespace() + "/" + - SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS); - verifyManagedLedgerInternalStats(stats.segmentInternalStats.managedLedgerInternalStats, internalStats); - assertTrue(stats.segmentInternalStats.managedLedgerName - .contains(SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS)); - - // Get managed ledger internal stats for the transaction buffer indexes topic - assertNotNull(stats.segmentIndexInternalStats); - internalStats = admin.topics().getInternalStats( - TopicName.get(topic2).getNamespace() + "/" + - SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES); - verifyManagedLedgerInternalStats(stats.segmentIndexInternalStats.managedLedgerInternalStats, internalStats); - assertTrue(stats.segmentIndexInternalStats.managedLedgerName - .contains(SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES)); + Awaitility.await().untilAsserted(() -> { + // Get transaction buffer internal stats and verify segmented snapshot stats + TransactionBufferInternalStats stats = + admin.transactions().getTransactionBufferInternalStatsAsync(topic3, true).get(); + assertEquals(stats.snapshotType, AbortedTxnProcessor.SnapshotType.Segment.toString()); + assertNull(stats.singleSnapshotSystemTopicInternalStats); + assertNotNull(stats.segmentInternalStats); + + // Get managed ledger internal stats for the transaction buffer segments topic + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats( + TopicName.get(topic2).getNamespace() + "/" + + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS); + verifyManagedLedgerInternalStats(stats.segmentInternalStats.managedLedgerInternalStats, internalStats); + assertTrue(stats.segmentInternalStats.managedLedgerName + .contains(SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS)); + + // Get managed ledger internal stats for the transaction buffer indexes topic + assertNotNull(stats.segmentIndexInternalStats); + internalStats = admin.topics().getInternalStats( + TopicName.get(topic2).getNamespace() + "/" + + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES); + verifyManagedLedgerInternalStats(stats.segmentIndexInternalStats.managedLedgerInternalStats, internalStats); + assertTrue(stats.segmentIndexInternalStats.managedLedgerName + .contains(SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES)); + }); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index c83888b8022b3..8dd2fc1c3c26d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -45,6 +45,7 @@ import javax.ws.rs.container.TimeoutHandler; import lombok.AllArgsConstructor; import lombok.Data; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationProviderTls; @@ -759,5 +760,9 @@ protected void assertOtelMetricLongSumValue(String metricName, int value) { sum -> sum.hasPointsSatisfying(point -> point.hasValue(value)))); } + protected void logTopicStats(String topic) { + BrokerTestUtil.logTopicStats(log, admin, topic); + } + private static final Logger log = LoggerFactory.getLogger(MockedPulsarServiceBaseTest.class); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/KeySharedLookAheadConfigTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/KeySharedLookAheadConfigTest.java new file mode 100644 index 0000000000000..cf028cf369d7b --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/KeySharedLookAheadConfigTest.java @@ -0,0 +1,63 @@ +/* + * 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.service.persistent; + +import static org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers.getEffectiveLookAheadLimit; +import static org.testng.Assert.assertEquals; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.testng.annotations.Test; + +public class KeySharedLookAheadConfigTest { + + @Test + public void testGetEffectiveLookAheadLimit() { + ServiceConfiguration config = new ServiceConfiguration(); + + config.setKeySharedLookAheadMsgInReplayThresholdPerConsumer(5); + config.setKeySharedLookAheadMsgInReplayThresholdPerSubscription(100); + assertEquals(getEffectiveLookAheadLimit(config, 5), 25); + assertEquals(getEffectiveLookAheadLimit(config, 100), 100); + + config.setKeySharedLookAheadMsgInReplayThresholdPerConsumer(5); + config.setKeySharedLookAheadMsgInReplayThresholdPerSubscription(0); + assertEquals(getEffectiveLookAheadLimit(config, 100), 500); + + config.setKeySharedLookAheadMsgInReplayThresholdPerConsumer(0); + config.setKeySharedLookAheadMsgInReplayThresholdPerSubscription(6000); + assertEquals(getEffectiveLookAheadLimit(config, 100), 6000); + + config.setKeySharedLookAheadMsgInReplayThresholdPerConsumer(0); + config.setKeySharedLookAheadMsgInReplayThresholdPerSubscription(0); + config.setMaxUnackedMessagesPerConsumer(0); + config.setMaxUnackedMessagesPerSubscription(0); + assertEquals(getEffectiveLookAheadLimit(config, 100), Integer.MAX_VALUE); + + config.setKeySharedLookAheadMsgInReplayThresholdPerConsumer(0); + config.setKeySharedLookAheadMsgInReplayThresholdPerSubscription(0); + config.setMaxUnackedMessagesPerConsumer(1); + config.setMaxUnackedMessagesPerSubscription(10); + assertEquals(getEffectiveLookAheadLimit(config, 100), 10); + + config.setKeySharedLookAheadMsgInReplayThresholdPerConsumer(0); + config.setKeySharedLookAheadMsgInReplayThresholdPerSubscription(0); + config.setMaxUnackedMessagesPerConsumer(22); + config.setMaxUnackedMessagesPerSubscription(0); + assertEquals(getEffectiveLookAheadLimit(config, 100), 2200); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryControllerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryControllerTest.java index 2222c8156e011..1708dc7bc2536 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryControllerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryControllerTest.java @@ -225,12 +225,12 @@ public void testGetMessagesToReplayNow(boolean allowOutOfOrderDelivery) throws E if (allowOutOfOrderDelivery) { // The entries are sorted by ledger ID but not by entry ID - Position[] actual1 = controller.getMessagesToReplayNow(3).toArray(new Position[3]); + Position[] actual1 = controller.getMessagesToReplayNow(3, item -> true).toArray(new Position[3]); Position[] expected1 = { PositionFactory.create(1, 1), PositionFactory.create(1, 2), PositionFactory.create(1, 3) }; assertEqualsNoOrder(actual1, expected1); } else { // The entries are completely sorted - Set actual2 = controller.getMessagesToReplayNow(6); + Set actual2 = controller.getMessagesToReplayNow(6, item -> true); Set expected2 = new TreeSet<>(); expected2.add(PositionFactory.create(1, 1)); expected2.add(PositionFactory.create(1, 2)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index a7ff9eb9c11f2..b78d1e554c32d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -74,6 +74,7 @@ import org.apache.pulsar.broker.service.EntryBatchSizes; import org.apache.pulsar.broker.service.RedeliveryTracker; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.TransportCnx; import org.apache.pulsar.broker.service.plugin.EntryFilterProvider; import org.apache.pulsar.common.api.proto.KeySharedMeta; import org.apache.pulsar.common.api.proto.KeySharedMode; @@ -120,7 +121,7 @@ public void setup() throws Exception { doReturn(100).when(configMock).getDispatcherMaxReadBatchSize(); doReturn(true).when(configMock).isSubscriptionKeySharedUseConsistentHashing(); doReturn(1).when(configMock).getSubscriptionKeySharedConsistentHashingReplicaPoints(); - doReturn(true).when(configMock).isDispatcherDispatchMessagesInSubscriptionThread(); + doReturn(false).when(configMock).isDispatcherDispatchMessagesInSubscriptionThread(); doReturn(false).when(configMock).isAllowOverrideEntryFilters(); doAnswer(invocation -> retryBackoffInitialTimeInMs).when(configMock).getDispatcherRetryBackoffInitialTimeInMs(); doAnswer(invocation -> retryBackoffMaxTimeInMs).when(configMock).getDispatcherRetryBackoffMaxTimeInMs(); @@ -191,7 +192,7 @@ public void setup() throws Exception { doReturn(subscriptionName).when(cursorMock).getName(); doReturn(ledgerMock).when(cursorMock).getManagedLedger(); - consumerMock = mock(Consumer.class); + consumerMock = createMockConsumer(); channelMock = mock(ChannelPromise.class); doReturn("consumer1").when(consumerMock).consumerName(); consumerMockAvailablePermits = new AtomicInteger(1000); @@ -214,6 +215,14 @@ public void setup() throws Exception { new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)); } + protected static Consumer createMockConsumer() { + Consumer consumerMock = mock(Consumer.class); + TransportCnx transportCnx = mock(TransportCnx.class); + doReturn(transportCnx).when(consumerMock).cnx(); + doReturn(true).when(transportCnx).isActive(); + return consumerMock; + } + @AfterMethod(alwaysRun = true) public void cleanup() { if (persistentDispatcher != null && !persistentDispatcher.isClosed()) { @@ -228,7 +237,7 @@ public void cleanup() { @Test(timeOut = 10000) public void testAddConsumerWhenClosed() throws Exception { persistentDispatcher.close().get(); - Consumer consumer = mock(Consumer.class); + Consumer consumer = createMockConsumer(); persistentDispatcher.addConsumer(consumer); verify(consumer, times(1)).disconnect(); assertEquals(0, persistentDispatcher.getConsumers().size()); @@ -286,7 +295,7 @@ public void testSendMessage() { .setStart(0) .setEnd(9); - Consumer consumerMock = mock(Consumer.class); + Consumer consumerMock = createMockConsumer(); doReturn(keySharedMeta).when(consumerMock).getKeySharedMeta(); persistentDispatcher.addConsumer(consumerMock); persistentDispatcher.consumerFlow(consumerMock, 1000); @@ -308,7 +317,7 @@ public void testSendMessage() { @Test public void testSkipRedeliverTemporally() { - final Consumer slowConsumerMock = mock(Consumer.class); + final Consumer slowConsumerMock = createMockConsumer(); final ChannelPromise slowChannelMock = mock(ChannelPromise.class); // add entries to redeliver and read target final List redeliverEntries = new ArrayList<>(); @@ -336,7 +345,6 @@ public void testSkipRedeliverTemporally() { // Create 2Consumers try { doReturn("consumer2").when(slowConsumerMock).consumerName(); - // Change slowConsumer availablePermits to 0 and back to normal when(slowConsumerMock.getAvailablePermits()) .thenReturn(0) .thenReturn(1); @@ -362,28 +370,24 @@ public void testSkipRedeliverTemporally() { // Change slowConsumer availablePermits to 1 // run PersistentStickyKeyDispatcherMultipleConsumers#sendMessagesToConsumers internally // and then stop to dispatch to slowConsumer - if (persistentDispatcher.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal, - redeliverEntries, true)) { - persistentDispatcher.readMoreEntriesAsync(); - } - - Awaitility.await().untilAsserted(() -> { - verify(consumerMock, times(1)).sendMessages( - argThat(arg -> { - assertEquals(arg.size(), 1); - Entry entry = arg.get(0); - assertEquals(entry.getLedgerId(), 1); - assertEquals(entry.getEntryId(), 3); - return true; - }), - any(EntryBatchSizes.class), - any(EntryBatchIndexesAcks.class), - anyInt(), - anyLong(), - anyLong(), - any(RedeliveryTracker.class) - ); - }); + persistentDispatcher.readEntriesComplete(redeliverEntries, + PersistentDispatcherMultipleConsumers.ReadType.Replay); + + verify(consumerMock, times(1)).sendMessages( + argThat(arg -> { + assertEquals(arg.size(), 1); + Entry entry = arg.get(0); + assertEquals(entry.getLedgerId(), 1); + assertEquals(entry.getEntryId(), 3); + return true; + }), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); verify(slowConsumerMock, times(0)).sendMessages( anyList(), any(EntryBatchSizes.class), @@ -421,7 +425,7 @@ public void testMessageRedelivery() throws Exception { final List readEntries = new ArrayList<>(); readEntries.add(allEntries.get(2)); // message3 - final Consumer consumer1 = mock(Consumer.class); + final Consumer consumer1 = createMockConsumer(); doReturn("consumer1").when(consumer1).consumerName(); // Change availablePermits of consumer1 to 0 and then back to normal when(consumer1.getAvailablePermits()).thenReturn(0).thenReturn(10); @@ -437,7 +441,7 @@ public void testMessageRedelivery() throws Exception { }).when(consumer1).sendMessages(anyList(), any(EntryBatchSizes.class), any(EntryBatchIndexesAcks.class), anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); - final Consumer consumer2 = mock(Consumer.class); + final Consumer consumer2 = createMockConsumer(); doReturn("consumer2").when(consumer2).consumerName(); when(consumer2.getAvailablePermits()).thenReturn(10); doReturn(true).when(consumer2).isWritable(); @@ -619,7 +623,7 @@ public void testLastSentPositionAndIndividuallySentPositions(final boolean initi PositionFactory.create(1, -1), PositionFactory.create(3, 19))), 60); // Add a consumer - final Consumer consumer1 = mock(Consumer.class); + final Consumer consumer1 = createMockConsumer(); doReturn("consumer1").when(consumer1).consumerName(); when(consumer1.getAvailablePermits()).thenReturn(1000); doReturn(true).when(consumer1).isWritable(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index e8fd537831673..ddf7b0f1d5ee2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -18,6 +18,9 @@ */ package org.apache.pulsar.client.api; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.pulsar.broker.BrokerTestUtil.receiveMessages; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; @@ -32,6 +35,7 @@ import java.lang.reflect.Field; import java.nio.file.Files; import java.nio.file.Paths; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -62,6 +66,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentStickyKeyDispatcherMultipleConsumers; @@ -91,7 +96,7 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -@Test(groups = "flaky") +@Test(groups = "broker-impl") public class KeySharedSubscriptionTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(KeySharedSubscriptionTest.class); @@ -155,6 +160,12 @@ public void resetDefaultNamespace() throws Exception { admin.topics().delete(topicName, false); } } + // reset read ahead limits to defaults + ServiceConfiguration defaultConf = new ServiceConfiguration(); + conf.setKeySharedLookAheadMsgInReplayThresholdPerSubscription( + defaultConf.getKeySharedLookAheadMsgInReplayThresholdPerSubscription()); + conf.setKeySharedLookAheadMsgInReplayThresholdPerConsumer( + defaultConf.getKeySharedLookAheadMsgInReplayThresholdPerConsumer()); } private static final Random random = new Random(System.nanoTime()); @@ -630,8 +641,11 @@ public void testOrderingWhenAddingConsumers() throws Exception { } @Test - public void testReadAheadWhenAddingConsumers() throws Exception { - String topic = "testReadAheadWhenAddingConsumers-" + UUID.randomUUID(); + public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { + String topic = "testReadAheadWithConfiguredLookAheadLimit-" + UUID.randomUUID(); + + // Set the look ahead limit to 50 for subscriptions + conf.setKeySharedLookAheadMsgInReplayThresholdPerSubscription(50); @Cleanup Producer producer = createProducer(topic, false); @@ -679,7 +693,8 @@ public void testReadAheadWhenAddingConsumers() throws Exception { // We need to ensure that dispatcher does not keep to look ahead in the topic, Position readPosition = sub.getCursor().getReadPosition(); - assertTrue(readPosition.getEntryId() < 1000); + long entryId = readPosition.getEntryId(); + assertTrue(entryId < 100); } @Test @@ -1296,7 +1311,7 @@ public void testCheckBetweenSkippingAndRecentlyJoinedConsumers(boolean preSend) redeliveryMessagesField.setAccessible(true); final MessageRedeliveryController redeliveryMessages = (MessageRedeliveryController) redeliveryMessagesField.get(dispatcher); - final Set replayMsgSet = redeliveryMessages.getMessagesToReplayNow(3); + final Set replayMsgSet = redeliveryMessages.getMessagesToReplayNow(3, item -> true); assertEquals(replayMsgSet.size(), 1); final Position replayMsg = replayMsgSet.stream().findAny().get(); assertEquals(replayMsg, PositionFactory.create(msg1Id.getLedgerId(), msg1Id.getEntryId())); @@ -2302,4 +2317,130 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO producer.close(); admin.topics().delete(topic, false); } + + @Test + public void testReadAheadLimit() throws Exception { + String topic = "testReadAheadLimit-" + UUID.randomUUID(); + int numberOfKeys = 1000; + long pauseTime = 100L; + int readAheadLimit = 20; + pulsar.getConfig().setKeySharedLookAheadMsgInReplayThresholdPerSubscription(readAheadLimit); + + @Cleanup + Producer producer = createProducer(topic, false); + + // create a consumer and close it to create a subscription + String subscriptionName = "key_shared"; + pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe() + .close(); + + Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); + PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscriptionName); + // get the dispatcher reference + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + + // create a function to use for checking the number of messages in replay + Runnable checkLimit = () -> { + assertThat(dispatcher.getNumberOfMessagesInReplay()).isLessThanOrEqualTo(readAheadLimit); + }; + + // Adding a new consumer. + @Cleanup + Consumer c1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c1") + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .startPaused(true) // start paused + .subscribe(); + + @Cleanup + Consumer c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(500) // use large receiver queue size + .subscribe(); + + @Cleanup + Consumer c3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c3") + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .startPaused(true) // start paused + .subscribe(); + + // find keys that will be assigned to c2 + List keysForC2 = new ArrayList<>(); + for (int i = 0; i < numberOfKeys; i++) { + String key = String.valueOf(i); + byte[] keyBytes = key.getBytes(UTF_8); + int hash = StickyKeyConsumerSelector.makeStickyKeyHash(keyBytes); + if (dispatcher.getSelector().select(hash).consumerName().equals("c2")) { + keysForC2.add(key); + } + } + + Set remainingMessageValues = new HashSet<>(); + // produce messages with keys that all get assigned to c2 + for (int i = 0; i < 1000; i++) { + String key = keysForC2.get(random.nextInt(keysForC2.size())); + //log.info("Producing message with key: {} value: {}", key, i); + producer.newMessage() + .key(key) + .value(i) + .send(); + remainingMessageValues.add(i); + } + + checkLimit.run(); + + Thread.sleep(pauseTime); + checkLimit.run(); + + Thread.sleep(pauseTime); + checkLimit.run(); + + // resume c1 and c3 + c1.resume(); + c3.resume(); + + Thread.sleep(pauseTime); + checkLimit.run(); + + // produce more messages + for (int i = 1000; i < 2000; i++) { + String key = String.valueOf(random.nextInt(numberOfKeys)); + producer.newMessage() + .key(key) + .value(i) + .send(); + remainingMessageValues.add(i); + checkLimit.run(); + } + + // consume the messages + receiveMessages((consumer, msg) -> { + synchronized (this) { + try { + consumer.acknowledge(msg); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } + remainingMessageValues.remove(msg.getValue()); + checkLimit.run(); + return true; + } + }, Duration.ofSeconds(2), c1, c2, c3); + assertEquals(remainingMessageValues, Collections.emptySet()); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java index ef070250ca1aa..0cf2e49d35bee 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerConsumerBase.java @@ -18,15 +18,15 @@ */ package org.apache.pulsar.client.api; +import static org.apache.pulsar.broker.BrokerTestUtil.receiveMessagesInThreads; import com.google.common.collect.Sets; - import java.lang.reflect.Method; +import java.time.Duration; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Random; import java.util.Set; - -import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -78,32 +78,16 @@ protected ReceivedMessages receiveAndAckMessages( BiFunction ackPredicate, Consumer...consumers) throws Exception { ReceivedMessages receivedMessages = new ReceivedMessages(); - while (true) { - int receivedMsgCount = 0; - for (int i = 0; i < consumers.length; i++) { - Consumer consumer = consumers[i]; - while (true) { - Message msg = consumer.receive(2, TimeUnit.SECONDS); - if (msg != null) { - receivedMsgCount++; - T v = msg.getValue(); - MessageId messageId = msg.getMessageId(); - receivedMessages.messagesReceived.add(Pair.of(msg.getMessageId(), v)); - if (ackPredicate.apply(messageId, v)) { - consumer.acknowledge(msg); - receivedMessages.messagesAcked.add(Pair.of(msg.getMessageId(), v)); - } - } else { - break; - } - } + receiveMessagesInThreads((consumer, msg) -> { + T v = msg.getValue(); + MessageId messageId = msg.getMessageId(); + receivedMessages.messagesReceived.add(Pair.of(msg.getMessageId(), v)); + if (ackPredicate.apply(messageId, v)) { + consumer.acknowledgeAsync(msg); + receivedMessages.messagesAcked.add(Pair.of(msg.getMessageId(), v)); } - // Because of the possibility of consumers getting stuck with each other, only jump out of the loop if all - // consumers could not receive messages. - if (receivedMsgCount == 0) { - break; - } - } + return true; + }, Duration.ofSeconds(2), consumers); return receivedMessages; } @@ -113,9 +97,9 @@ protected ReceivedMessages ackAllMessages(Consumer...consumers) throws protected static class ReceivedMessages { - List> messagesReceived = new ArrayList<>(); + List> messagesReceived = Collections.synchronizedList(new ArrayList<>()); - List> messagesAcked = new ArrayList<>(); + List> messagesAcked = Collections.synchronizedList(new ArrayList<>()); public boolean hasReceivedMessage(T v) { for (Pair pair : messagesReceived) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java index 1d534176e8d61..7889b19e5b29e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java @@ -151,10 +151,12 @@ public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(Subsc .until(() -> (System.currentTimeMillis() - lastActiveTime.get()) > TimeUnit.SECONDS.toMillis(5)); + logTopicStats(topic); + //Determine if all messages have been received. //If the dispatcher is stuck, we can not receive enough messages. - Assert.assertEquals(pubMessages.size(), totalMsg); - Assert.assertEquals(pubMessages.size(), recMessages.size()); + Assert.assertEquals(totalMsg, pubMessages.size()); + Assert.assertEquals(recMessages.size(), pubMessages.size()); Assert.assertTrue(recMessages.containsAll(pubMessages)); // cleanup From a875debe9144e69764bde8f04ada36a5302519e5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 19 Sep 2024 09:45:46 +0300 Subject: [PATCH 509/580] [fix][io] Upgrade mssql server docker tag in DebeziumMsSqlContainer (#23318) --- .../tests/integration/containers/DebeziumMsSqlContainer.java | 2 +- .../io/sources/debezium/DebeziumMsSqlSourceTester.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/DebeziumMsSqlContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/DebeziumMsSqlContainer.java index 357fd8724d738..61acdae37696b 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/DebeziumMsSqlContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/DebeziumMsSqlContainer.java @@ -37,7 +37,7 @@ public class DebeziumMsSqlContainer extends ChaosContainer Date: Thu, 19 Sep 2024 11:14:11 +0300 Subject: [PATCH 510/580] [improve][broker] Don't use "recently joined consumers" rules for Key_Shared in STICKY mode (#23275) --- ...tStickyKeyDispatcherMultipleConsumers.java | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index d45b9394dd744..26463ba902c58 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -66,6 +66,7 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi private final boolean allowOutOfOrderDelivery; private final StickyKeyConsumerSelector selector; + private final boolean recentlyJoinedConsumerTrackingRequired; private boolean skipNextReplayToTriggerLookAhead = false; private final KeySharedMode keySharedMode; @@ -90,10 +91,15 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi super(topic, cursor, subscription, ksm.isAllowOutOfOrderDelivery()); this.allowOutOfOrderDelivery = ksm.isAllowOutOfOrderDelivery(); - this.recentlyJoinedConsumers = allowOutOfOrderDelivery ? null : new LinkedHashMap<>(); - this.individuallySentPositions = - allowOutOfOrderDelivery ? null : new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter); this.keySharedMode = ksm.getKeySharedMode(); + // recent joined consumer tracking is required only for AUTO_SPLIT mode when out-of-order delivery is disabled + this.recentlyJoinedConsumerTrackingRequired = + keySharedMode == KeySharedMode.AUTO_SPLIT && !allowOutOfOrderDelivery; + this.recentlyJoinedConsumers = recentlyJoinedConsumerTrackingRequired ? new LinkedHashMap<>() : null; + this.individuallySentPositions = + recentlyJoinedConsumerTrackingRequired + ? new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter) + : null; switch (this.keySharedMode) { case AUTO_SPLIT: if (conf.isSubscriptionKeySharedUseConsistentHashing()) { @@ -138,7 +144,7 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { }) ).thenRun(() -> { synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) { - if (!allowOutOfOrderDelivery) { + if (recentlyJoinedConsumerTrackingRequired) { final Position lastSentPositionWhenJoining = updateIfNeededAndGetLastSentPosition(); if (lastSentPositionWhenJoining != null) { consumer.setLastSentPositionWhenJoining(lastSentPositionWhenJoining); @@ -165,7 +171,7 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE // eventually causing all consumers to get stuck. selector.removeConsumer(consumer); super.removeConsumer(consumer); - if (!allowOutOfOrderDelivery && recentlyJoinedConsumers != null) { + if (recentlyJoinedConsumerTrackingRequired) { recentlyJoinedConsumers.remove(consumer); if (consumerList.size() == 1) { recentlyJoinedConsumers.clear(); @@ -231,7 +237,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } } } + } + if (recentlyJoinedConsumerTrackingRequired) { // Update if the markDeletePosition move forward updateIfNeededAndGetLastSentPosition(); @@ -273,7 +281,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId()); } // Add positions to individuallySentPositions if necessary - if (!allowOutOfOrderDelivery) { + if (recentlyJoinedConsumerTrackingRequired) { final Position position = entry.getPosition(); // Store to individuallySentPositions even if lastSentPosition is null if ((lastSentPosition == null || position.compareTo(lastSentPosition) > 0) @@ -306,7 +314,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } // Update the last sent position and remove ranges from individuallySentPositions if necessary - if (!allowOutOfOrderDelivery && lastSentPosition != null) { + if (recentlyJoinedConsumerTrackingRequired && lastSentPosition != null) { final ManagedLedger managedLedger = cursor.getManagedLedger(); com.google.common.collect.Range range = individuallySentPositions.firstRange(); From 03330b3f7ca7dc06114d34fe34679eef73f821e7 Mon Sep 17 00:00:00 2001 From: Nikhil Erigila <60037808+nikhilerigila09@users.noreply.github.com> Date: Thu, 19 Sep 2024 14:27:36 +0530 Subject: [PATCH 511/580] [fix][broker] Fix incomplete NAR file extraction which prevents broker from starting (#23274) --- .../apache/pulsar/common/nar/NarUnpacker.java | 29 ++++++++++++++----- .../pulsar/common/nar/NarUnpackerTest.java | 11 +++++++ 2 files changed, 33 insertions(+), 7 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarUnpacker.java b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarUnpacker.java index e1806836d2833..ef802674b421a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarUnpacker.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarUnpacker.java @@ -32,7 +32,9 @@ import java.io.RandomAccessFile; import java.nio.channels.FileChannel; import java.nio.channels.FileLock; +import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.StandardCopyOption; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.Base64; @@ -86,19 +88,32 @@ static File doUnpackNar(final File nar, final File baseWorkingDirectory, Runnabl try (FileChannel channel = new RandomAccessFile(lockFile, "rw").getChannel(); FileLock lock = channel.lock()) { File narWorkingDirectory = new File(parentDirectory, md5Sum); - if (narWorkingDirectory.mkdir()) { + if (!narWorkingDirectory.exists()) { + File narExtractionTempDirectory = new File(parentDirectory, md5Sum + ".tmp"); + if (narExtractionTempDirectory.exists()) { + FileUtils.deleteFile(narExtractionTempDirectory, true); + } + if (!narExtractionTempDirectory.mkdir()) { + throw new IOException("Cannot create " + narExtractionTempDirectory); + } try { - log.info("Extracting {} to {}", nar, narWorkingDirectory); + log.info("Extracting {} to {}", nar, narExtractionTempDirectory); if (extractCallback != null) { extractCallback.run(); } - unpack(nar, narWorkingDirectory); + unpack(nar, narExtractionTempDirectory); } catch (IOException e) { log.error("There was a problem extracting the nar file. Deleting {} to clean up state.", - narWorkingDirectory, e); - FileUtils.deleteFile(narWorkingDirectory, true); + narExtractionTempDirectory, e); + try { + FileUtils.deleteFile(narExtractionTempDirectory, true); + } catch (IOException e2) { + log.error("Failed to delete temporary directory {}", narExtractionTempDirectory, e2); + } throw e; } + Files.move(narExtractionTempDirectory.toPath(), narWorkingDirectory.toPath(), + StandardCopyOption.ATOMIC_MOVE); } return narWorkingDirectory; } @@ -166,7 +181,7 @@ private static void makeFile(final InputStream inputStream, final File file) thr * @throws IOException * if cannot read file */ - private static byte[] calculateMd5sum(final File file) throws IOException { + protected static byte[] calculateMd5sum(final File file) throws IOException { try (final FileInputStream inputStream = new FileInputStream(file)) { // codeql[java/weak-cryptographic-algorithm] - md5 is sufficient for this use case final MessageDigest md5 = MessageDigest.getInstance("md5"); @@ -184,4 +199,4 @@ private static byte[] calculateMd5sum(final File file) throws IOException { throw new IllegalArgumentException(nsae); } } -} +} \ No newline at end of file diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/nar/NarUnpackerTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/nar/NarUnpackerTest.java index a1f915c8b7828..1c3a2c276537b 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/nar/NarUnpackerTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/nar/NarUnpackerTest.java @@ -118,6 +118,17 @@ public static void main(String[] args) { } } + @Test + void shouldReExtractWhenUnpackedDirectoryIsMissing() throws IOException { + AtomicInteger extractCounter = new AtomicInteger(); + + File narWorkingDirectory = NarUnpacker.doUnpackNar(sampleZipFile, extractDirectory, extractCounter::incrementAndGet); + FileUtils.deleteFile(narWorkingDirectory, true); + NarUnpacker.doUnpackNar(sampleZipFile, extractDirectory, extractCounter::incrementAndGet); + + assertEquals(extractCounter.get(), 2); + } + @Test void shouldExtractFilesOnceInDifferentProcess() throws InterruptedException { int processes = 5; From 4b3b273c1c57741f9f9da2118eb4ec5dfeee2220 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Thu, 19 Sep 2024 22:33:32 +0800 Subject: [PATCH 512/580] [improve][broker] PIP-376: Make topic policies service pluggable (#23319) --- .../pulsar/broker/ServiceConfiguration.java | 12 +- .../apache/pulsar/broker/PulsarService.java | 25 +- .../pulsar/broker/admin/AdminResource.java | 25 +- .../admin/impl/PersistentTopicsBase.java | 34 +-- .../broker/loadbalance/LoadManager.java | 4 + .../extensions/ExtensibleLoadManagerImpl.java | 4 +- .../ExtensibleLoadManagerWrapper.java | 4 + .../channel/ServiceUnitStateChannel.java | 5 + .../channel/ServiceUnitStateChannelImpl.java | 5 + .../pulsar/broker/service/AbstractTopic.java | 24 +- .../pulsar/broker/service/BrokerService.java | 123 ++-------- .../service/BrokerServiceException.java | 6 - .../SystemTopicBasedTopicPoliciesService.java | 231 ++++++------------ .../broker/service/TopicPoliciesService.java | 193 +++------------ .../broker/service/TopicPolicyListener.java | 11 +- .../nonpersistent/NonPersistentTopic.java | 2 +- .../service/persistent/PersistentTopic.java | 22 +- .../pulsar/broker/admin/NamespacesTest.java | 20 +- .../broker/admin/TopicPoliciesTest.java | 77 +++--- .../service/InmemoryTopicPoliciesService.java | 81 ++++++ ...memoryTopicPoliciesServiceServiceTest.java | 91 +++++++ .../broker/service/OneWayReplicatorTest.java | 3 +- .../service/OneWayReplicatorTestBase.java | 4 +- .../PersistentTopicInitializeDelayTest.java | 4 +- .../broker/service/PersistentTopicTest.java | 3 +- .../service/ReplicatorTopicPoliciesTest.java | 13 +- ...temTopicBasedTopicPoliciesServiceTest.java | 130 +++------- .../broker/service/TopicPolicyTestUtils.java | 74 ++++++ .../persistent/PersistentTopicTest.java | 3 +- .../systopic/PartitionedSystemTopicTest.java | 4 +- .../broker/transaction/TransactionTest.java | 8 +- .../client/api/OrphanPersistentTopicTest.java | 5 +- 32 files changed, 571 insertions(+), 679 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InmemoryTopicPoliciesService.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InmemoryTopicPoliciesServiceServiceTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java 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 42dc959426692..cdd27412e3052 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 @@ -1554,6 +1554,14 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece + "please enable the system topic first.") private boolean topicLevelPoliciesEnabled = true; + @FieldContext( + category = CATEGORY_SERVER, + doc = "The class name of the topic policies service. The default config only takes affect when the " + + "systemTopicEnable config is true" + ) + private String topicPoliciesServiceClassName = + "org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService"; + @FieldContext( category = CATEGORY_SERVER, doc = "List of interceptors for entry metadata.") @@ -3793,10 +3801,6 @@ public int getTopicOrderedExecutorThreadNum() { ? numWorkerThreadsForNonPersistentTopic : topicOrderedExecutorThreadNum; } - public boolean isSystemTopicAndTopicLevelPoliciesEnabled() { - return topicLevelPoliciesEnabled && systemTopicEnabled; - } - public Map lookupProperties() { final var map = new HashMap(); properties.forEach((key, value) -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 87196d3f3a9a6..a2f6fb9e9773b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -989,11 +989,8 @@ public void start() throws PulsarServerException { this.nsService.initialize(); // Start topic level policies service - if (config.isSystemTopicAndTopicLevelPoliciesEnabled()) { - this.topicPoliciesService = new SystemTopicBasedTopicPoliciesService(this); - } - - this.topicPoliciesService.start(); + this.topicPoliciesService = initTopicPoliciesService(); + this.topicPoliciesService.start(this); // Register heartbeat and bootstrap namespaces. this.nsService.registerBootstrapNamespaces(); @@ -2137,4 +2134,22 @@ public void initConfigMetadataSynchronizerIfNeeded() { mutex.unlock(); } } + + private TopicPoliciesService initTopicPoliciesService() throws Exception { + if (!config.isTopicLevelPoliciesEnabled()) { + return TopicPoliciesService.DISABLED; + } + final var className = Optional.ofNullable(config.getTopicPoliciesServiceClassName()) + .orElse(SystemTopicBasedTopicPoliciesService.class.getName()); + if (className.equals(SystemTopicBasedTopicPoliciesService.class.getName())) { + if (config.isSystemTopicEnabled()) { + return new SystemTopicBasedTopicPoliciesService(this); + } else { + LOG.warn("System topic is disabled while the topic policies service is {}, disable it", className); + return TopicPoliciesService.DISABLED; + } + } + return (TopicPoliciesService) Reflections.createInstance(className, + Thread.currentThread().getContextClassLoader()); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index d42dff39a8a0d..3268f07b13d88 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -45,6 +45,7 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.broker.resources.ClusterResources; +import org.apache.pulsar.broker.service.TopicPoliciesService; import org.apache.pulsar.broker.service.plugin.InvalidEntryFilterException; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; @@ -365,14 +366,8 @@ protected CompletableFuture> getTopicPoliciesAsyncWithRe protected CompletableFuture> getTopicPoliciesAsyncWithRetry(TopicName topicName, boolean isGlobal) { - try { - checkTopicLevelPolicyEnable(); - return pulsar().getTopicPoliciesService() - .getTopicPoliciesAsyncWithRetry(topicName, null, pulsar().getExecutor(), isGlobal); - } catch (Exception e) { - log.error("[{}] Failed to get topic policies {}", clientAppId(), topicName, e); - return FutureUtil.failedFuture(e); - } + final var type = isGlobal ? TopicPoliciesService.GetType.GLOBAL_ONLY : TopicPoliciesService.GetType.LOCAL_ONLY; + return pulsar().getTopicPoliciesService().getTopicPoliciesAsync(topicName, type); } protected boolean checkBacklogQuota(BacklogQuota quota, RetentionPolicies retention) { @@ -396,13 +391,6 @@ protected boolean checkBacklogQuota(BacklogQuota quota, RetentionPolicies retent return true; } - protected void checkTopicLevelPolicyEnable() { - if (!config().isSystemTopicAndTopicLevelPoliciesEnabled()) { - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Topic level policies is disabled, to enable the topic level policy and retry."); - } - } - protected DispatchRateImpl dispatchRate() { return DispatchRateImpl.builder() .dispatchThrottlingRateInMsg(config().getDispatchThrottlingRatePerTopicInMsg()) @@ -784,11 +772,8 @@ protected CompletableFuture getSchemaCompatibilityS } protected CompletableFuture getSchemaCompatibilityStrategyAsyncWithoutAuth() { - CompletableFuture future = CompletableFuture.completedFuture(null); - if (config().isSystemTopicAndTopicLevelPoliciesEnabled()) { - future = getTopicPoliciesAsyncWithRetry(topicName) - .thenApply(op -> op.map(TopicPolicies::getSchemaCompatibilityStrategy).orElse(null)); - } + CompletableFuture future = getTopicPoliciesAsyncWithRetry(topicName) + .thenApply(op -> op.map(TopicPolicies::getSchemaCompatibilityStrategy).orElse(null)); return future.thenCompose((topicSchemaCompatibilityStrategy) -> { if (!SchemaCompatibilityStrategy.isUndefined(topicSchemaCompatibilityStrategy)) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 4d04dfeda7a74..bdbd70afbaeac 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -84,6 +84,7 @@ import org.apache.pulsar.broker.service.MessageExpirer; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.TopicPoliciesService; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -447,20 +448,9 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean return CompletableFuture.completedFuture(null); } // update remote cluster - return namespaceResources().getPoliciesAsync(namespaceName) - .thenCompose(policies -> { - if (!policies.isPresent()) { - return CompletableFuture.completedFuture(null); - } - // Combine namespace level policies and topic level policies. - Set replicationClusters = policies.get().replication_clusters; - TopicPolicies topicPolicies = - pulsarService.getTopicPoliciesService().getTopicPoliciesIfExists(topicName); - if (topicPolicies != null && topicPolicies.getReplicationClusters() != null) { - replicationClusters = topicPolicies.getReplicationClustersSet(); - } - // Do check replicated clusters. - if (replicationClusters.size() == 0) { + return getReplicationClusters() + .thenCompose(replicationClusters -> { + if (replicationClusters == null || replicationClusters.isEmpty()) { return CompletableFuture.completedFuture(null); } boolean containsCurrentCluster = @@ -495,6 +485,20 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean }); } + private CompletableFuture> getReplicationClusters() { + return namespaceResources().getPoliciesAsync(namespaceName).thenCompose(optionalPolicies -> { + if (optionalPolicies.isEmpty()) { + return CompletableFuture.completedFuture(null); + } + // Query the topic-level policies only if the namespace-level policies exist + final var namespacePolicies = optionalPolicies.get(); + return pulsar().getTopicPoliciesService().getTopicPoliciesAsync(topicName, + TopicPoliciesService.GetType.DEFAULT + ).thenApply(optionalTopicPolicies -> optionalTopicPolicies.map(TopicPolicies::getReplicationClustersSet) + .orElse(namespacePolicies.replication_clusters)); + }); + } + protected void internalCreateMissedPartitions(AsyncResponse asyncResponse) { getPartitionedTopicMetadataAsync(topicName, false, false).thenAccept(metadata -> { if (metadata != null && metadata.partitions > 0) { @@ -3655,7 +3659,7 @@ protected CompletableFuture internalSetReplicatorDispatchRate(DispatchRate } protected CompletableFuture preValidation(boolean authoritative) { - if (!config().isSystemTopicAndTopicLevelPoliciesEnabled()) { + if (!config().isTopicLevelPoliciesEnabled()) { return FutureUtil.failedFuture(new RestException(Status.METHOD_NOT_ALLOWED, "Topic level policies is disabled, to enable the topic level policy and retry.")); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java index 0dd5d948480ab..db2fb2ffd0fa6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java @@ -53,6 +53,10 @@ public interface LoadManager { void start() throws PulsarServerException; + default boolean started() { + return true; + } + /** * Is centralized decision making to assign a new bundle. */ diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 8e34f2f697fb1..f22bcc836f6e9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -182,7 +182,7 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS private SplitManager splitManager; - private volatile boolean started = false; + volatile boolean started = false; private boolean configuredSystemTopics = false; @@ -320,7 +320,7 @@ private static void createSystemTopics(PulsarService pulsar) throws PulsarServer private static boolean configureSystemTopics(PulsarService pulsar) { try { if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar) - && pulsar.getConfiguration().isSystemTopicAndTopicLevelPoliciesEnabled()) { + && pulsar.getConfiguration().isTopicLevelPoliciesEnabled()) { Long threshold = pulsar.getAdminClient().topicPolicies().getCompactionThreshold(TOPIC); if (threshold == null || COMPACTION_THRESHOLD != threshold.longValue()) { pulsar.getAdminClient().topicPolicies().setCompactionThreshold(TOPIC, COMPACTION_THRESHOLD); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java index 25eb27bc58d27..6a48607977ba9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java @@ -50,6 +50,10 @@ public void start() throws PulsarServerException { loadManager.start(); } + public boolean started() { + return loadManager.started && loadManager.getServiceUnitStateChannel().started(); + } + @Override public void initialize(PulsarService pulsar) { loadManager.initialize(pulsar); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java index 9be76e1b0f44d..6319fc332a678 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java @@ -43,6 +43,11 @@ public interface ServiceUnitStateChannel extends Closeable { */ void start() throws PulsarServerException; + /** + * Whether the channel started. + */ + boolean started(); + /** * Closes the ServiceUnitStateChannel. * @throws PulsarServerException if it fails to close the channel. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index 1063f8124ece8..3ebcd1c20ca87 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -265,6 +265,11 @@ public void cleanOwnerships() { doCleanup(brokerId); } + @Override + public synchronized boolean started() { + return validateChannelState(LeaderElectionServiceStarted, false); + } + public synchronized void start() throws PulsarServerException { if (!validateChannelState(LeaderElectionServiceStarted, false)) { throw new IllegalStateException("Invalid channel state:" + channelState.name()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index 9e5d6ef7191d1..3fdfeeee6e152 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -87,7 +87,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class AbstractTopic implements Topic, TopicPolicyListener { +public abstract class AbstractTopic implements Topic, TopicPolicyListener { protected static final long POLICY_UPDATE_FAILURE_RETRY_TIME_SECONDS = 60; @@ -509,17 +509,13 @@ protected boolean isProducersExceeded(boolean isRemote) { } protected void registerTopicPolicyListener() { - if (brokerService.pulsar().getConfig().isSystemTopicAndTopicLevelPoliciesEnabled()) { - brokerService.getPulsar().getTopicPoliciesService() - .registerListener(TopicName.getPartitionedTopicName(topic), this); - } + brokerService.getPulsar().getTopicPoliciesService() + .registerListener(TopicName.getPartitionedTopicName(topic), this); } protected void unregisterTopicPolicyListener() { - if (brokerService.pulsar().getConfig().isSystemTopicAndTopicLevelPoliciesEnabled()) { - brokerService.getPulsar().getTopicPoliciesService() - .unregisterListener(TopicName.getPartitionedTopicName(topic), this); - } + brokerService.getPulsar().getTopicPoliciesService() + .unregisterListener(TopicName.getPartitionedTopicName(topic), this); } protected boolean isSameAddressProducersExceeded(Producer producer) { @@ -1253,16 +1249,8 @@ public InactiveTopicPolicies getInactiveTopicPolicies() { return topicPolicies.getInactiveTopicPolicies().get(); } - /** - * Get {@link TopicPolicies} for this topic. - * @return TopicPolicies, if they exist. Otherwise, the value will not be present. - */ - public Optional getTopicPolicies() { - return brokerService.getTopicPolicies(TopicName.get(topic)); - } - public CompletableFuture deleteTopicPolicies() { - return brokerService.deleteTopicPolicies(TopicName.get(topic)); + return brokerService.pulsar().getTopicPoliciesService().deleteTopicPoliciesAsync(TopicName.get(topic)); } protected int getWaitingProducersCount() { 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 cb5e0853d53f3..aee6532716cd8 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 @@ -63,7 +63,6 @@ import java.util.Set; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionStage; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.RejectedExecutionException; @@ -171,6 +170,7 @@ import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.policies.data.TopicType; +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.stats.Metrics; @@ -1177,15 +1177,8 @@ public CompletableFuture> getTopic(final TopicName topicName, bo } private CompletableFuture> getTopicPoliciesBypassSystemTopic(@Nonnull TopicName topicName) { - Objects.requireNonNull(topicName); - final ServiceConfiguration serviceConfiguration = pulsar.getConfiguration(); - if (serviceConfiguration.isSystemTopicAndTopicLevelPoliciesEnabled() - && !NamespaceService.isSystemServiceNamespace(topicName.getNamespace()) - && !SystemTopicNames.isTopicPoliciesSystemTopic(topicName.toString())) { - return pulsar.getTopicPoliciesService().getTopicPoliciesAsync(topicName); - } else { - return CompletableFuture.completedFuture(Optional.empty()); - } + return pulsar.getTopicPoliciesService().getTopicPoliciesAsync(topicName, + TopicPoliciesService.GetType.DEFAULT); } public CompletableFuture deleteTopic(String topic, boolean forceDelete) { @@ -1239,13 +1232,7 @@ private CompletableFuture deleteTopicInternal(String topic, boolean forceD deleteTopicAuthenticationWithRetry(topic, deleteTopicAuthenticationFuture, 5); deleteTopicAuthenticationFuture .thenCompose(__ -> deleteSchema(tn)) - .thenCompose(__ -> { - if (!SystemTopicNames.isTopicPoliciesSystemTopic(topic) - && getPulsar().getConfiguration().isSystemTopicEnabled()) { - return deleteTopicPolicies(tn); - } - return CompletableFuture.completedFuture(null); - }).whenComplete((v, ex) -> { + .thenCompose(__ -> pulsar.getTopicPoliciesService().deleteTopicPoliciesAsync(tn)).whenComplete((v, ex) -> { if (ex != null) { future.completeExceptionally(ex); return; @@ -3611,71 +3598,25 @@ private AutoTopicCreationOverride getAutoTopicCreationOverride(final TopicName t return null; } - /** - * @deprecated Avoid using the deprecated method - * #{@link org.apache.pulsar.broker.resources.NamespaceResources#getPoliciesIfCached(NamespaceName)} and blocking - * call. we can use #{@link BrokerService#isAllowAutoSubscriptionCreationAsync(TopicName)} to instead of it. - */ - @Deprecated - public boolean isAllowAutoSubscriptionCreation(final String topic) { - TopicName topicName = TopicName.get(topic); - return isAllowAutoSubscriptionCreation(topicName); - } - - /** - * @deprecated Avoid using the deprecated method - * #{@link org.apache.pulsar.broker.resources.NamespaceResources#getPoliciesIfCached(NamespaceName)} and blocking - * call. we can use #{@link BrokerService#isAllowAutoSubscriptionCreationAsync(TopicName)} to instead of it. - */ - @Deprecated - public boolean isAllowAutoSubscriptionCreation(final TopicName topicName) { - AutoSubscriptionCreationOverride autoSubscriptionCreationOverride = - getAutoSubscriptionCreationOverride(topicName); - if (autoSubscriptionCreationOverride != null) { - return autoSubscriptionCreationOverride.isAllowAutoSubscriptionCreation(); - } else { - return pulsar.getConfiguration().isAllowAutoSubscriptionCreation(); - } - } - - /** - * @deprecated Avoid using the deprecated method - * #{@link org.apache.pulsar.broker.resources.NamespaceResources#getPoliciesIfCached(NamespaceName)} and blocking - * call. we can use #{@link BrokerService#isAllowAutoSubscriptionCreationAsync(TopicName)} to instead of it. - */ - @Deprecated - private AutoSubscriptionCreationOverride getAutoSubscriptionCreationOverride(final TopicName topicName) { - Optional topicPolicies = getTopicPolicies(topicName); - if (topicPolicies.isPresent() && topicPolicies.get().getAutoSubscriptionCreationOverride() != null) { - return topicPolicies.get().getAutoSubscriptionCreationOverride(); - } - - Optional policies = - pulsar.getPulsarResources().getNamespaceResources().getPoliciesIfCached(topicName.getNamespaceObject()); - // If namespace policies have the field set, it will override the broker-level setting - if (policies.isPresent() && policies.get().autoSubscriptionCreationOverride != null) { - return policies.get().autoSubscriptionCreationOverride; - } - log.debug("No autoSubscriptionCreateOverride policy found for {}", topicName); - return null; - } - - public @Nonnull CompletionStage isAllowAutoSubscriptionCreationAsync(@Nonnull TopicName tpName) { + public @Nonnull CompletableFuture isAllowAutoSubscriptionCreationAsync(@Nonnull TopicName tpName) { requireNonNull(tpName); - // topic level policies - final var topicPolicies = getTopicPolicies(tpName); - if (topicPolicies.isPresent() && topicPolicies.get().getAutoSubscriptionCreationOverride() != null) { - return CompletableFuture.completedFuture(topicPolicies.get().getAutoSubscriptionCreationOverride() - .isAllowAutoSubscriptionCreation()); - } - // namespace level policies - return pulsar.getPulsarResources().getNamespaceResources().getPoliciesAsync(tpName.getNamespaceObject()) - .thenApply(policies -> { - if (policies.isPresent() && policies.get().autoSubscriptionCreationOverride != null) { - return policies.get().autoSubscriptionCreationOverride.isAllowAutoSubscriptionCreation(); + // Policies priority: topic level -> namespace level -> broker level + return pulsar.getTopicPoliciesService() + .getTopicPoliciesAsync(tpName, TopicPoliciesService.GetType.LOCAL_ONLY) + .thenCompose(optionalTopicPolicies -> { + Boolean allowed = optionalTopicPolicies.map(TopicPolicies::getAutoSubscriptionCreationOverride) + .map(AutoSubscriptionCreationOverrideImpl::isAllowAutoSubscriptionCreation) + .orElse(null); + if (allowed != null) { + return CompletableFuture.completedFuture(allowed); } - // broker level policies - return pulsar.getConfiguration().isAllowAutoSubscriptionCreation(); + // namespace level policies + return pulsar.getPulsarResources().getNamespaceResources().getPoliciesAsync( + tpName.getNamespaceObject() + ).thenApply(optionalPolicies -> optionalPolicies.map(__ -> __.autoSubscriptionCreationOverride) + .map(AutoSubscriptionCreationOverride::isAllowAutoSubscriptionCreation) + // broker level policies + .orElse(pulsar.getConfiguration().isAllowAutoSubscriptionCreation())); }); } @@ -3688,28 +3629,6 @@ public boolean isSystemTopic(TopicName topicName) { || SystemTopicNames.isSystemTopic(topicName); } - /** - * Get {@link TopicPolicies} for the parameterized topic. - * @param topicName - * @return TopicPolicies, if they exist. Otherwise, the value will not be present. - */ - public Optional getTopicPolicies(TopicName topicName) { - if (!pulsar().getConfig().isSystemTopicAndTopicLevelPoliciesEnabled()) { - return Optional.empty(); - } - return Optional.ofNullable(pulsar.getTopicPoliciesService() - .getTopicPoliciesIfExists(topicName)); - } - - public CompletableFuture deleteTopicPolicies(TopicName topicName) { - final PulsarService pulsarService = pulsar(); - if (!pulsarService.getConfig().isSystemTopicAndTopicLevelPoliciesEnabled()) { - return CompletableFuture.completedFuture(null); - } - return pulsar.getTopicPoliciesService() - .deleteTopicPoliciesAsync(TopicName.get(topicName.getPartitionedTopicName())); - } - public CompletableFuture deleteSchema(TopicName topicName) { // delete schema at the upper level when deleting the partitioned topic. if (topicName.isPartitioned()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java index 6abe40f811d1d..d30dfc319e098 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java @@ -218,12 +218,6 @@ public ConsumerAssignException(String msg) { } } - public static class TopicPoliciesCacheNotInitException extends BrokerServiceException { - public TopicPoliciesCacheNotInitException() { - super("Topic policies cache have not init."); - } - } - public static class TopicBacklogQuotaExceededException extends BrokerServiceException { @Getter private final BacklogQuota.RetentionPolicy retentionPolicy; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index 5156246bb5efb..18b4c610a5c9b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -37,12 +37,10 @@ import javax.annotation.Nonnull; import org.apache.commons.lang3.concurrent.ConcurrentInitializer; import org.apache.commons.lang3.concurrent.LazyInitializer; -import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.namespace.NamespaceBundleOwnershipListener; import org.apache.pulsar.broker.namespace.NamespaceService; -import org.apache.pulsar.broker.service.BrokerServiceException.TopicPoliciesCacheNotInitException; import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.client.api.Message; @@ -56,10 +54,10 @@ import org.apache.pulsar.common.events.TopicPoliciesEvent; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.util.FutureUtil; -import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -101,7 +99,7 @@ protected NamespaceEventsSystemTopicFactory initialize() { final Map> policyCacheInitMap = new ConcurrentHashMap<>(); @VisibleForTesting - final Map>> listeners = new ConcurrentHashMap<>(); + final Map> listeners = new ConcurrentHashMap<>(); private final AsyncLoadingCache> writerCaches; @@ -132,7 +130,7 @@ public SystemTopicBasedTopicPoliciesService(PulsarService pulsarService) { @Override public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName) { - if (NamespaceService.isHeartbeatNamespace(topicName.getNamespaceObject())) { + if (NamespaceService.isHeartbeatNamespace(topicName.getNamespaceObject()) || isSelf(topicName)) { return CompletableFuture.completedFuture(null); } return sendTopicPolicyEvent(topicName, ActionType.DELETE, null); @@ -216,7 +214,7 @@ private void notifyListener(Message msg) { if (msg.getValue() == null) { TopicName topicName = TopicName.get(TopicName.get(msg.getKey()).getPartitionedTopicName()); if (listeners.get(topicName) != null) { - for (TopicPolicyListener listener : listeners.get(topicName)) { + for (TopicPolicyListener listener : listeners.get(topicName)) { try { listener.onUpdate(null); } catch (Throwable error) { @@ -235,7 +233,7 @@ private void notifyListener(Message msg) { event.getNamespace(), event.getTopic()); if (listeners.get(topicName) != null) { TopicPolicies policies = event.getPolicies(); - for (TopicPolicyListener listener : listeners.get(topicName)) { + for (TopicPolicyListener listener : listeners.get(topicName)) { try { listener.onUpdate(policies); } catch (Throwable error) { @@ -246,115 +244,76 @@ private void notifyListener(Message msg) { } @Override - public TopicPolicies getTopicPolicies(TopicName topicName) throws TopicPoliciesCacheNotInitException { - return getTopicPolicies(topicName, false); - } - - @Override - public TopicPolicies getTopicPolicies(TopicName topicName, - boolean isGlobal) throws TopicPoliciesCacheNotInitException { - if (NamespaceService.isHeartbeatNamespace(topicName.getNamespaceObject())) { - return null; + public CompletableFuture> getTopicPoliciesAsync(TopicName topicName, GetType type) { + requireNonNull(topicName); + final var namespace = topicName.getNamespaceObject(); + if (NamespaceService.isHeartbeatNamespace(namespace) || isSelf(topicName)) { + return CompletableFuture.completedFuture(Optional.empty()); } - if (!policyCacheInitMap.containsKey(topicName.getNamespaceObject())) { - NamespaceName namespace = topicName.getNamespaceObject(); - prepareInitPoliciesCacheAsync(namespace); + // When the extensible load manager initializes its channel topic, it will trigger the topic policies + // initialization by calling this method. At the moment, the load manager does not start so the lookup + // for "__change_events" will fail. In this case, just return an empty policies to avoid deadlock. + final var loadManager = pulsarService.getLoadManager().get(); + if (loadManager == null || !loadManager.started()) { + return CompletableFuture.completedFuture(Optional.empty()); } - - MutablePair result = new MutablePair<>(); - policyCacheInitMap.compute(topicName.getNamespaceObject(), (k, initialized) -> { - if (initialized == null || !initialized.isDone()) { - result.setLeft(new TopicPoliciesCacheNotInitException()); + final CompletableFuture preparedFuture = prepareInitPoliciesCacheAsync(topicName.getNamespaceObject()); + final var resultFuture = new CompletableFuture>(); + preparedFuture.thenAccept(inserted -> policyCacheInitMap.compute(namespace, (___, existingFuture) -> { + if (!inserted || existingFuture != null) { + final var partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName()); + final var policies = Optional.ofNullable(switch (type) { + case DEFAULT -> Optional.ofNullable(policiesCache.get(partitionedTopicName)) + .orElseGet(() -> globalPoliciesCache.get(partitionedTopicName)); + case GLOBAL_ONLY -> globalPoliciesCache.get(partitionedTopicName); + case LOCAL_ONLY -> policiesCache.get(partitionedTopicName); + }); + resultFuture.complete(policies); } else { - TopicPolicies topicPolicies = - isGlobal ? globalPoliciesCache.get(TopicName.get(topicName.getPartitionedTopicName())) - : policiesCache.get(TopicName.get(topicName.getPartitionedTopicName())); - result.setRight(topicPolicies); - } - return initialized; - }); - - if (result.getLeft() != null) { - throw result.getLeft(); - } else { - return result.getRight(); - } - } - - @NotNull - @Override - public CompletableFuture> getTopicPoliciesAsync(@NotNull TopicName topicName, - boolean isGlobal) { - requireNonNull(topicName); - final CompletableFuture preparedFuture = prepareInitPoliciesCacheAsync(topicName.getNamespaceObject()); - return preparedFuture.thenApply(__ -> { - final TopicPolicies candidatePolicies = isGlobal - ? globalPoliciesCache.get(TopicName.get(topicName.getPartitionedTopicName())) - : policiesCache.get(TopicName.get(topicName.getPartitionedTopicName())); - return Optional.ofNullable(candidatePolicies); - }); - } - - @NotNull - @Override - public CompletableFuture> getTopicPoliciesAsync(@NotNull TopicName topicName) { - requireNonNull(topicName); - final CompletableFuture preparedFuture = prepareInitPoliciesCacheAsync(topicName.getNamespaceObject()); - return preparedFuture.thenApply(__ -> { - final TopicPolicies localPolicies = policiesCache.get(TopicName.get(topicName.getPartitionedTopicName())); - if (localPolicies != null) { - return Optional.of(localPolicies); + CompletableFuture.runAsync(() -> { + log.info("The future of {} has been removed from cache, retry getTopicPolicies again", namespace); + // Call it in another thread to avoid recursive update because getTopicPoliciesAsync() could call + // policyCacheInitMap.computeIfAbsent() + getTopicPoliciesAsync(topicName, type).whenComplete((result, e) -> { + if (e == null) { + resultFuture.complete(result); + } else { + resultFuture.completeExceptionally(e); + } + }); + }); } - return Optional.ofNullable(globalPoliciesCache.get(TopicName.get(topicName.getPartitionedTopicName()))); + return existingFuture; + })).exceptionally(e -> { + resultFuture.completeExceptionally(e); + return null; }); + return resultFuture; } - @Override - public TopicPolicies getTopicPoliciesIfExists(TopicName topicName) { - return policiesCache.get(TopicName.get(topicName.getPartitionedTopicName())); - } - - @Override - public CompletableFuture getTopicPoliciesBypassCacheAsync(TopicName topicName) { - CompletableFuture result = new CompletableFuture<>(); - try { - createSystemTopicFactoryIfNeeded(); - } catch (PulsarServerException e) { - result.complete(null); - return result; - } - SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() - .createTopicPoliciesSystemTopicClient(topicName.getNamespaceObject()); - systemTopicClient.newReaderAsync().thenAccept(r -> - fetchTopicPoliciesAsyncAndCloseReader(r, topicName, null, result)); - return result; - } - - @Override - public CompletableFuture addOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) { + public void addOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) { NamespaceName namespace = namespaceBundle.getNamespaceObject(); if (NamespaceService.isHeartbeatNamespace(namespace)) { - return CompletableFuture.completedFuture(null); + return; } synchronized (this) { if (readerCaches.get(namespace) != null) { ownedBundlesCountPerNamespace.get(namespace).incrementAndGet(); - return CompletableFuture.completedFuture(null); } else { - return prepareInitPoliciesCacheAsync(namespace); + prepareInitPoliciesCacheAsync(namespace); } } } @VisibleForTesting - @Nonnull CompletableFuture prepareInitPoliciesCacheAsync(@Nonnull NamespaceName namespace) { + @Nonnull CompletableFuture prepareInitPoliciesCacheAsync(@Nonnull NamespaceName namespace) { requireNonNull(namespace); return pulsarService.getPulsarResources().getNamespaceResources().getPoliciesAsync(namespace) .thenCompose(namespacePolicies -> { if (namespacePolicies.isEmpty() || namespacePolicies.get().deleted) { log.info("[{}] skip prepare init policies cache since the namespace is deleted", namespace); - return CompletableFuture.completedFuture(null); + return CompletableFuture.completedFuture(false); } return policyCacheInitMap.computeIfAbsent(namespace, (k) -> { @@ -384,7 +343,7 @@ public CompletableFuture addOwnedNamespaceBundleAsync(NamespaceBundle name }); // let caller know we've got an exception. return initFuture; - }); + }).thenApply(__ -> true); }); } @@ -404,22 +363,20 @@ protected CompletableFuture> createSystemT return systemTopicClient.newReaderAsync(); } - @Override - public CompletableFuture removeOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) { + private void removeOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) { NamespaceName namespace = namespaceBundle.getNamespaceObject(); if (NamespaceService.checkHeartbeatNamespace(namespace) != null || NamespaceService.checkHeartbeatNamespaceV2(namespace) != null) { - return CompletableFuture.completedFuture(null); + return; } AtomicInteger bundlesCount = ownedBundlesCountPerNamespace.get(namespace); if (bundlesCount == null || bundlesCount.decrementAndGet() <= 0) { cleanCacheAndCloseReader(namespace, true, true); } - return CompletableFuture.completedFuture(null); } @Override - public void start() { + public void start(PulsarService pulsarService) { pulsarService.getNamespaceService().addNamespaceBundleOwnershipListener( new NamespaceBundleOwnershipListener() { @@ -478,7 +435,7 @@ private void initPolicesCache(SystemTopicClient.Reader reader, Comp // replay policy message policiesCache.forEach(((topicName, topicPolicies) -> { if (listeners.get(topicName) != null) { - for (TopicPolicyListener listener : listeners.get(topicName)) { + for (TopicPolicyListener listener : listeners.get(topicName)) { try { listener.onUpdate(topicPolicies); } catch (Throwable error) { @@ -525,7 +482,7 @@ private void cleanCacheAndCloseReader(@Nonnull NamespaceName namespace, boolean * This is an async method for the background reader to continue syncing new messages. * * Note: You should not do any blocking call here. because it will affect - * #{@link SystemTopicBasedTopicPoliciesService#getTopicPoliciesAsync(TopicName)} method to block loading topic. + * #{@link SystemTopicBasedTopicPoliciesService#getTopicPoliciesAsync} method to block loading topic. */ private void readMorePoliciesAsync(SystemTopicClient.Reader reader) { if (closed.get()) { @@ -638,7 +595,8 @@ private void createSystemTopicFactoryIfNeeded() throws PulsarServerException { } } - private NamespaceEventsSystemTopicFactory getNamespaceEventsSystemTopicFactory() { + @VisibleForTesting + NamespaceEventsSystemTopicFactory getNamespaceEventsSystemTopicFactory() { try { return namespaceEventsSystemTopicFactoryLazyInitializer.get(); } catch (Exception e) { @@ -647,58 +605,6 @@ private NamespaceEventsSystemTopicFactory getNamespaceEventsSystemTopicFactory() } } - private void fetchTopicPoliciesAsyncAndCloseReader(SystemTopicClient.Reader reader, - TopicName topicName, TopicPolicies policies, - CompletableFuture future) { - if (closed.get()) { - future.completeExceptionally(new BrokerServiceException(getClass().getName() + " is closed.")); - reader.closeAsync().whenComplete((v, e) -> { - if (e != null) { - log.error("[{}] Close reader error.", topicName, e); - } - }); - return; - } - reader.hasMoreEventsAsync().whenComplete((hasMore, ex) -> { - if (ex != null) { - future.completeExceptionally(ex); - } - if (hasMore != null && hasMore) { - reader.readNextAsync().whenComplete((msg, e) -> { - if (e != null) { - future.completeExceptionally(e); - } - if (msg.getValue() != null - && EventType.TOPIC_POLICY.equals(msg.getValue().getEventType())) { - TopicPoliciesEvent topicPoliciesEvent = msg.getValue().getTopicPoliciesEvent(); - if (topicName.equals(TopicName.get( - topicPoliciesEvent.getDomain(), - topicPoliciesEvent.getTenant(), - topicPoliciesEvent.getNamespace(), - topicPoliciesEvent.getTopic())) - ) { - fetchTopicPoliciesAsyncAndCloseReader(reader, topicName, - topicPoliciesEvent.getPolicies(), future); - } else { - fetchTopicPoliciesAsyncAndCloseReader(reader, topicName, policies, future); - } - } else { - future.complete(null); - } - }); - } else { - if (!future.isDone()) { - future.complete(policies); - } - reader.closeAsync().whenComplete((v, e) -> { - if (e != null) { - log.error("[{}] Close reader error.", topicName, e); - } - }); - } - }); - } - public static String getEventKey(PulsarEvent event) { return TopicName.get(event.getTopicPoliciesEvent().getDomain(), event.getTopicPoliciesEvent().getTenant(), @@ -718,11 +624,6 @@ long getPoliciesCacheSize() { return policiesCache.size(); } - @VisibleForTesting - long getReaderCacheCount() { - return readerCaches.size(); - } - @VisibleForTesting boolean checkReaderIsCached(NamespaceName namespaceName) { return readerCaches.get(namespaceName) != null; @@ -734,7 +635,7 @@ public CompletableFuture getPoliciesCacheInit(NamespaceName namespaceName) } @Override - public void registerListener(TopicName topicName, TopicPolicyListener listener) { + public boolean registerListener(TopicName topicName, TopicPolicyListener listener) { listeners.compute(topicName, (k, topicListeners) -> { if (topicListeners == null) { topicListeners = new CopyOnWriteArrayList<>(); @@ -742,10 +643,11 @@ public void registerListener(TopicName topicName, TopicPolicyListener listener) { + public void unregisterListener(TopicName topicName, TopicPolicyListener listener) { listeners.compute(topicName, (k, topicListeners) -> { if (topicListeners != null){ topicListeners.remove(listener); @@ -763,7 +665,7 @@ protected Map getPoliciesCache() { } @VisibleForTesting - protected Map>> getListeners() { + protected Map> getListeners() { return listeners; } @@ -792,4 +694,13 @@ public void close() throws Exception { readerCaches.clear(); } } + + private static boolean isSelf(TopicName topicName) { + final var localName = topicName.getLocalName(); + if (!topicName.isPartitioned()) { + return localName.equals(SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME); + } + final var index = localName.lastIndexOf(TopicName.PARTITIONED_TOPIC_SUFFIX); + return localName.substring(0, index).equals(SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java index eca31ec230a8e..9b5d9a28ac216 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java @@ -20,38 +20,31 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import javax.annotation.Nonnull; -import org.apache.pulsar.broker.service.BrokerServiceException.TopicPoliciesCacheNotInitException; -import org.apache.pulsar.client.util.RetryUtil; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.common.classification.InterfaceAudience; import org.apache.pulsar.common.classification.InterfaceStability; -import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicPolicies; -import org.apache.pulsar.common.util.Backoff; -import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.util.FutureUtil; -import org.jetbrains.annotations.NotNull; /** * Topic policies service. */ -@InterfaceStability.Evolving +@InterfaceStability.Stable +@InterfaceAudience.LimitedPrivate public interface TopicPoliciesService extends AutoCloseable { TopicPoliciesService DISABLED = new TopicPoliciesServiceDisabled(); - long DEFAULT_GET_TOPIC_POLICY_TIMEOUT = 30_000; /** - * Delete policies for a topic async. + * Delete policies for a topic asynchronously. * * @param topicName topic name */ CompletableFuture deleteTopicPoliciesAsync(TopicName topicName); /** - * Update policies for a topic async. + * Update policies for a topic asynchronously. * * @param topicName topic name * @param policies policies for the topic name @@ -59,119 +52,56 @@ public interface TopicPoliciesService extends AutoCloseable { CompletableFuture updateTopicPoliciesAsync(TopicName topicName, TopicPolicies policies); /** - * Get policies for a topic async. - * @param topicName topic name - * @return future of the topic policies - */ - TopicPolicies getTopicPolicies(TopicName topicName) throws TopicPoliciesCacheNotInitException; - - /** - * Get policies from current cache. - * @param topicName topic name - * @return the topic policies - */ - TopicPolicies getTopicPoliciesIfExists(TopicName topicName); - - /** - * Get global policies for a topic async. - * @param topicName topic name - * @return future of the topic policies - */ - TopicPolicies getTopicPolicies(TopicName topicName, boolean isGlobal) throws TopicPoliciesCacheNotInitException; - - /** - * When getting TopicPolicies, if the initialization has not been completed, - * we will go back off and try again until time out. - * @param topicName topic name - * @param backoff back off policy - * @param isGlobal is global policies - * @return CompletableFuture<Optional<TopicPolicies>> + * It controls the behavior of {@link TopicPoliciesService#getTopicPoliciesAsync}. */ - default CompletableFuture> getTopicPoliciesAsyncWithRetry(TopicName topicName, - final Backoff backoff, ScheduledExecutorService scheduledExecutorService, boolean isGlobal) { - CompletableFuture> response = new CompletableFuture<>(); - Backoff usedBackoff = backoff == null ? new BackoffBuilder() - .setInitialTime(500, TimeUnit.MILLISECONDS) - .setMandatoryStop(DEFAULT_GET_TOPIC_POLICY_TIMEOUT, TimeUnit.MILLISECONDS) - .setMax(DEFAULT_GET_TOPIC_POLICY_TIMEOUT, TimeUnit.MILLISECONDS) - .create() : backoff; - try { - RetryUtil.retryAsynchronously(() -> { - CompletableFuture> future = new CompletableFuture<>(); - try { - future.complete(Optional.ofNullable(getTopicPolicies(topicName, isGlobal))); - } catch (BrokerServiceException.TopicPoliciesCacheNotInitException exception) { - future.completeExceptionally(exception); - } - return future; - }, usedBackoff, scheduledExecutorService, response); - } catch (Exception e) { - response.completeExceptionally(e); - } - return response; + enum GetType { + DEFAULT, // try getting the local topic policies, if not present, then get the global policies + GLOBAL_ONLY, // only get the global policies + LOCAL_ONLY, // only get the local policies } /** - * Asynchronously retrieves topic policies. - * This triggers the Pulsar broker's internal client to load policies from the - * system topic `persistent://tenant/namespace/__change_event`. - * - * @param topicName The name of the topic. - * @param isGlobal Indicates if the policies are global. - * @return A CompletableFuture containing an Optional of TopicPolicies. - * @throws NullPointerException If the topicName is null. + * Retrieve the topic policies. */ - @Nonnull - CompletableFuture> getTopicPoliciesAsync(@Nonnull TopicName topicName, boolean isGlobal); + CompletableFuture> getTopicPoliciesAsync(TopicName topicName, GetType type); /** - * Asynchronously retrieves topic policies. - * This triggers the Pulsar broker's internal client to load policies from the - * system topic `persistent://tenant/namespace/__change_event`. - * - * NOTE: If local policies are not available, it will fallback to using topic global policies. - * @param topicName The name of the topic. - * @return A CompletableFuture containing an Optional of TopicPolicies. - * @throws NullPointerException If the topicName is null. + * Start the topic policy service. */ - @Nonnull - CompletableFuture> getTopicPoliciesAsync(@Nonnull TopicName topicName); + default void start(PulsarService pulsar) { + } /** - * Get policies for a topic without cache async. - * @param topicName topic name - * @return future of the topic policies + * Close the resources if necessary. */ - CompletableFuture getTopicPoliciesBypassCacheAsync(TopicName topicName); + default void close() throws Exception { + } /** - * Add owned namespace bundle async. + * Registers a listener for topic policies updates. * - * @param namespaceBundle namespace bundle - */ - CompletableFuture addOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle); - - /** - * Remove owned namespace bundle async. + *

+ * The listener will receive the latest topic policies when they are updated. If the policies are removed, the + * listener will receive a null value. Note that not every update is guaranteed to trigger the listener. For + * instance, if the policies change from A -> B -> null -> C in quick succession, only the final state (C) is + * guaranteed to be received by the listener. + * In summary, the listener is guaranteed to receive only the latest value. + *

* - * @param namespaceBundle namespace bundle + * @return true if the listener is registered successfully */ - CompletableFuture removeOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle); + boolean registerListener(TopicName topicName, TopicPolicyListener listener); /** - * Start the topic policy service. + * Unregister the topic policies listener. */ - void start(); - - void registerListener(TopicName topicName, TopicPolicyListener listener); - - void unregisterListener(TopicName topicName, TopicPolicyListener listener); + void unregisterListener(TopicName topicName, TopicPolicyListener listener); class TopicPoliciesServiceDisabled implements TopicPoliciesService { @Override public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName) { - return FutureUtil.failedFuture(new UnsupportedOperationException("Topic policies service is disabled.")); + return CompletableFuture.completedFuture(null); } @Override @@ -180,68 +110,17 @@ public CompletableFuture updateTopicPoliciesAsync(TopicName topicName, Top } @Override - public TopicPolicies getTopicPolicies(TopicName topicName) throws TopicPoliciesCacheNotInitException { - return null; - } - - @Override - public TopicPolicies getTopicPolicies(TopicName topicName, boolean isGlobal) - throws TopicPoliciesCacheNotInitException { - return null; - } - - @NotNull - @Override - public CompletableFuture> getTopicPoliciesAsync(@NotNull TopicName topicName, - boolean isGlobal) { - return CompletableFuture.completedFuture(Optional.empty()); - } - - @NotNull - @Override - public CompletableFuture> getTopicPoliciesAsync(@NotNull TopicName topicName) { + public CompletableFuture> getTopicPoliciesAsync(TopicName topicName, GetType type) { return CompletableFuture.completedFuture(Optional.empty()); } @Override - public TopicPolicies getTopicPoliciesIfExists(TopicName topicName) { - return null; - } - - @Override - public CompletableFuture getTopicPoliciesBypassCacheAsync(TopicName topicName) { - return CompletableFuture.completedFuture(null); - } - - @Override - public CompletableFuture addOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) { - //No-op - return CompletableFuture.completedFuture(null); - } - - @Override - public CompletableFuture removeOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) { - //No-op - return CompletableFuture.completedFuture(null); - } - - @Override - public void start() { - //No-op - } - - @Override - public void registerListener(TopicName topicName, TopicPolicyListener listener) { - //No-op - } - - @Override - public void unregisterListener(TopicName topicName, TopicPolicyListener listener) { - //No-op + public boolean registerListener(TopicName topicName, TopicPolicyListener listener) { + return false; } @Override - public void close() { + public void unregisterListener(TopicName topicName, TopicPolicyListener listener) { //No-op } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPolicyListener.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPolicyListener.java index 7f7fd154ab035..a597e2ef9aedf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPolicyListener.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPolicyListener.java @@ -18,6 +18,13 @@ */ package org.apache.pulsar.broker.service; -public interface TopicPolicyListener { - void onUpdate(T data); +import org.apache.pulsar.common.classification.InterfaceAudience; +import org.apache.pulsar.common.classification.InterfaceStability; +import org.apache.pulsar.common.policies.data.TopicPolicies; + +@InterfaceStability.Stable +@InterfaceAudience.LimitedPrivate +public interface TopicPolicyListener { + + void onUpdate(TopicPolicies data); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 1b98ee2f8306d..2abd505d527cc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -102,7 +102,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class NonPersistentTopic extends AbstractTopic implements Topic, TopicPolicyListener { +public class NonPersistentTopic extends AbstractTopic implements Topic, TopicPolicyListener { // Subscriptions to this topic private final ConcurrentOpenHashMap subscriptions; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index fc47889c60aac..e951ffab1e230 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -125,6 +125,7 @@ import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.SubscriptionOption; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.TopicPoliciesService; import org.apache.pulsar.broker.service.TransportCnx; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; import org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage; @@ -1512,14 +1513,7 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, brokerService.deleteTopicAuthenticationWithRetry(topic, deleteTopicAuthenticationFuture, 5); deleteTopicAuthenticationFuture.thenCompose(ignore -> deleteSchema()) - .thenCompose(ignore -> { - if (!SystemTopicNames.isTopicPoliciesSystemTopic(topic) - && brokerService.getPulsar().getConfiguration().isSystemTopicEnabled()) { - return deleteTopicPolicies(); - } else { - return CompletableFuture.completedFuture(null); - } - }) + .thenCompose(ignore -> deleteTopicPolicies()) .thenCompose(ignore -> transactionBufferCleanupAndClose()) .whenComplete((v, ex) -> { if (ex != null) { @@ -4327,12 +4321,12 @@ private void updateSubscriptionsDispatcherRateLimiter() { } protected CompletableFuture initTopicPolicy() { - if (brokerService.pulsar().getConfig().isSystemTopicAndTopicLevelPoliciesEnabled()) { - brokerService.getPulsar().getTopicPoliciesService() - .registerListener(TopicName.getPartitionedTopicName(topic), this); - return CompletableFuture.completedFuture(null).thenRunAsync(() -> onUpdate( - brokerService.getPulsar().getTopicPoliciesService() - .getTopicPoliciesIfExists(TopicName.getPartitionedTopicName(topic))), + final var topicPoliciesService = brokerService.pulsar().getTopicPoliciesService(); + final var partitionedTopicName = TopicName.getPartitionedTopicName(topic); + if (topicPoliciesService.registerListener(partitionedTopicName, this)) { + return topicPoliciesService.getTopicPoliciesAsync(partitionedTopicName, + TopicPoliciesService.GetType.DEFAULT + ).thenAcceptAsync(optionalPolicies -> optionalPolicies.ifPresent(this::onUpdate), brokerService.getTopicOrderedExecutor()); } return CompletableFuture.completedFuture(null); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java index 1050d9f33b465..f294866095250 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java @@ -79,6 +79,7 @@ import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.namespace.OwnershipCache; import org.apache.pulsar.broker.service.AbstractTopic; +import org.apache.pulsar.broker.service.TopicPolicyTestUtils; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; @@ -113,7 +114,6 @@ import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.apache.pulsar.common.util.FutureUtil; @@ -2103,17 +2103,17 @@ public void testDeleteTopicPolicyWhenDeleteSystemTopic() throws Exception { Producer producer = pulsarClient.newProducer(Schema.STRING) .topic(systemTopic).create(); admin.topicPolicies().setMaxConsumers(systemTopic, 5); + Awaitility.await().atMost(3, TimeUnit.SECONDS).untilAsserted(() -> { + final var policies = TopicPolicyTestUtils.getTopicPoliciesBypassCache(pulsar.getTopicPoliciesService(), + TopicName.get(systemTopic)); + Assert.assertTrue(policies.isPresent()); + Assert.assertEquals(policies.get().getMaxConsumerPerTopic(), 5); + }); - Integer maxConsumerPerTopic = pulsar - .getTopicPoliciesService() - .getTopicPoliciesBypassCacheAsync(TopicName.get(systemTopic)).get() - .getMaxConsumerPerTopic(); - - assertEquals(maxConsumerPerTopic, 5); admin.topics().delete(systemTopic, true); - TopicPolicies topicPolicies = pulsar.getTopicPoliciesService() - .getTopicPoliciesBypassCacheAsync(TopicName.get(systemTopic)).get(5, TimeUnit.SECONDS); - assertNull(topicPolicies); + Awaitility.await().atMost(3, TimeUnit.SECONDS).untilAsserted(() -> assertTrue( + TopicPolicyTestUtils.getTopicPoliciesBypassCache(pulsar.getTopicPoliciesService(), TopicName.get(systemTopic)) + .isEmpty())); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java index 9f56acfb57f23..1351c41e4279e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java @@ -50,6 +50,7 @@ import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.PublishRateLimiterImpl; import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService; +import org.apache.pulsar.broker.service.TopicPolicyTestUtils; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -2092,7 +2093,7 @@ public void testTopicMaxMessageSizeApi() throws Exception{ assertNull(admin.topicPolicies().getMaxMessageSize(persistenceTopic)); admin.topicPolicies().setMaxMessageSize(persistenceTopic,10); Awaitility.await().until(() - -> pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(persistenceTopic)) != null); + -> TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(persistenceTopic)) != null); assertEquals(admin.topicPolicies().getMaxMessageSize(persistenceTopic).intValue(),10); admin.topicPolicies().removeMaxMessageSize(persistenceTopic); @@ -2138,7 +2139,7 @@ public void testTopicMaxMessageSize(TopicDomain topicDomain, boolean isPartition assertNull(admin.topicPolicies().getMaxMessageSize(topic)); // set msg size admin.topicPolicies().setMaxMessageSize(topic, 10); - Awaitility.await().until(() -> pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic)) != null); + Awaitility.await().until(() -> TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic)) != null); if (isPartitioned) { for (int i = 0; i <3; i++) { String partitionName = TopicName.get(topic).getPartition(i).toString(); @@ -2255,7 +2256,7 @@ public void testMaxSubscriptionsPerTopicApi() throws Exception { // set max subscriptions admin.topicPolicies().setMaxSubscriptionsPerTopic(topic, 10); Awaitility.await().until(() - -> pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic)) != null); + -> TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic)) != null); assertEquals(admin.topicPolicies().getMaxSubscriptionsPerTopic(topic).intValue(), 10); // remove max subscriptions admin.topicPolicies().removeMaxSubscriptionsPerTopic(topic); @@ -2278,7 +2279,7 @@ public void testMaxSubscriptionsPerTopicWithExistingSubs() throws Exception { final int topicLevelMaxSubNum = 2; admin.topicPolicies().setMaxSubscriptionsPerTopic(topic, topicLevelMaxSubNum); Awaitility.await().until(() - -> pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic)) != null); + -> TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic)) != null); List> consumerList = new ArrayList<>(); String subName = "my-sub-"; for (int i = 0; i < topicLevelMaxSubNum; i++) { @@ -2410,7 +2411,7 @@ public void testMaxSubscriptionsPerTopic() throws Exception { final int topicLevelMaxSubNum = 2; admin.topicPolicies().setMaxSubscriptionsPerTopic(topic, topicLevelMaxSubNum); Awaitility.await().until(() - -> pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic)) != null); + -> TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic)) != null); List> consumerList = new ArrayList<>(); for (int i = 0; i < topicLevelMaxSubNum; i++) { @@ -2613,7 +2614,7 @@ public void testSubscriptionTypesEnabled() throws Exception { admin.topicPolicies().setSubscriptionTypesEnabled(topic, subscriptionTypeSet); Awaitility.await().until(() - -> pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic)) != null); + -> TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic)) != null); waitTopicPoliciesApplied(topic, 0, hierarchyTopicPolicies -> { assertTrue(hierarchyTopicPolicies.getSubscriptionTypesEnabled().get() .contains(CommandSubscribe.SubType.Failover)); @@ -2836,7 +2837,7 @@ public void testPolicyIsDeleteTogetherManually() throws Exception { pulsarClient.newProducer().topic(topic).create().close(); Awaitility.await().untilAsserted(() -> - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))) + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic))) .isNull()); int maxConsumersPerSubscription = 10; @@ -2845,7 +2846,7 @@ public void testPolicyIsDeleteTogetherManually() throws Exception { Awaitility.await().untilAsserted(() -> Assertions.assertThat(pulsar.getBrokerService().getTopic(topic, false).get().isPresent()).isTrue()); Awaitility.await().untilAsserted(() -> - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))) + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic))) .isNotNull()); admin.topics().delete(topic); @@ -2853,7 +2854,7 @@ public void testPolicyIsDeleteTogetherManually() throws Exception { Awaitility.await().untilAsserted(() -> Assertions.assertThat(pulsar.getBrokerService().getTopic(topic, false).get().isPresent()).isFalse()); Awaitility.await().untilAsserted(() -> - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))) + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic))) .isNull()); } @@ -2865,8 +2866,8 @@ public void testPoliciesCanBeDeletedWithTopic() throws Exception { pulsarClient.newProducer().topic(topic2).create().close(); Awaitility.await().untilAsserted(() -> { - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))).isNull(); - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic2))).isNull(); + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic))).isNull(); + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic2))).isNull(); }); // Init Topic Policies. Send 4 messages in a row, there should be only 2 messages left after compression admin.topicPolicies().setMaxConsumersPerSubscription(topic, 1); @@ -2874,8 +2875,8 @@ public void testPoliciesCanBeDeletedWithTopic() throws Exception { admin.topicPolicies().setMaxConsumersPerSubscription(topic, 3); admin.topicPolicies().setMaxConsumersPerSubscription(topic2, 4); Awaitility.await().untilAsserted(() -> { - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))).isNotNull(); - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic2))).isNotNull(); + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic))).isNotNull(); + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic2))).isNotNull(); }); String topicPoliciesTopic = "persistent://" + myNamespace + "/" + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME; PersistentTopic persistentTopic = @@ -2908,7 +2909,7 @@ public void testPoliciesCanBeDeletedWithTopic() throws Exception { admin.topics().delete(topic, true); Awaitility.await().untilAsserted(() -> - assertNull(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic)))); + assertNull(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic)))); persistentTopic.triggerCompaction(); field = PersistentTopic.class.getDeclaredField("currentCompaction"); field.setAccessible(true); @@ -2940,7 +2941,7 @@ public void testPolicyIsDeleteTogetherAutomatically() throws Exception { pulsarClient.newProducer().topic(topic).create().close(); Awaitility.await().untilAsserted(() -> - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))) + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic))) .isNull()); int maxConsumersPerSubscription = 10; @@ -2949,7 +2950,7 @@ public void testPolicyIsDeleteTogetherAutomatically() throws Exception { Awaitility.await().untilAsserted(() -> Assertions.assertThat(pulsar.getBrokerService().getTopic(topic, false).get().isPresent()).isTrue()); Awaitility.await().untilAsserted(() -> - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))) + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic))) .isNotNull()); InactiveTopicPolicies inactiveTopicPolicies = @@ -2963,7 +2964,7 @@ public void testPolicyIsDeleteTogetherAutomatically() throws Exception { Awaitility.await().untilAsserted(() -> Assertions.assertThat(pulsar.getBrokerService().getTopic(topic, false).get().isPresent()).isFalse()); Awaitility.await().untilAsserted(() -> - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))) + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic))) .isNull()); } @@ -3009,17 +3010,17 @@ public void testLoopCreateAndDeleteTopicPolicies() throws Exception { n++; pulsarClient.newProducer().topic(topic).create().close(); Awaitility.await().untilAsserted(() -> { - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))).isNull(); + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic))).isNull(); }); admin.topicPolicies().setMaxConsumersPerSubscription(topic, 1); Awaitility.await().untilAsserted(() -> { - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))).isNotNull(); + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic))).isNotNull(); }); admin.topics().delete(topic); Awaitility.await().untilAsserted(() -> { - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))).isNull(); + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic))).isNull(); }); } } @@ -3030,42 +3031,43 @@ public void testGlobalTopicPolicies() throws Exception { pulsarClient.newProducer().topic(topic).create().close(); Awaitility.await().untilAsserted(() -> - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))) + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(topic))) .isNull()); admin.topicPolicies(true).setRetention(topic, new RetentionPolicies(1, 2)); SystemTopicBasedTopicPoliciesService topicPoliciesService = (SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService(); // check global topic policies can be added correctly. - Awaitility.await().untilAsserted(() -> assertNotNull(topicPoliciesService.getTopicPolicies(TopicName.get(topic), true))); - TopicPolicies topicPolicies = topicPoliciesService.getTopicPolicies(TopicName.get(topic), true); - assertNull(topicPoliciesService.getTopicPolicies(TopicName.get(topic))); + Awaitility.await().untilAsserted(() -> assertNotNull( + TopicPolicyTestUtils.getGlobalTopicPolicies(topicPoliciesService, TopicName.get(topic)))); + TopicPolicies topicPolicies = TopicPolicyTestUtils.getGlobalTopicPolicies(topicPoliciesService, TopicName.get(topic)); + assertNull(TopicPolicyTestUtils.getLocalTopicPolicies(topicPoliciesService, TopicName.get(topic))); assertEquals(topicPolicies.getRetentionPolicies().getRetentionTimeInMinutes(), 1); assertEquals(topicPolicies.getRetentionPolicies().getRetentionSizeInMB(), 2); // check global topic policies can be updated correctly. admin.topicPolicies(true).setRetention(topic, new RetentionPolicies(3, 4)); Awaitility.await().untilAsserted(() -> { - TopicPolicies tempPolicies = topicPoliciesService.getTopicPolicies(TopicName.get(topic), true); - assertNull(topicPoliciesService.getTopicPolicies(TopicName.get(topic))); + TopicPolicies tempPolicies = TopicPolicyTestUtils.getGlobalTopicPolicies(topicPoliciesService, TopicName.get(topic)); + assertNull(TopicPolicyTestUtils.getLocalTopicPolicies(topicPoliciesService, (TopicName.get(topic)))); assertEquals(tempPolicies.getRetentionPolicies().getRetentionTimeInMinutes(), 3); assertEquals(tempPolicies.getRetentionPolicies().getRetentionSizeInMB(), 4); }); //Local topic policies and global topic policies can exist together. admin.topicPolicies().setRetention(topic, new RetentionPolicies(10, 20)); - Awaitility.await().untilAsserted(() -> assertNotNull(topicPoliciesService.getTopicPolicies(TopicName.get(topic)))); - TopicPolicies tempPolicies = topicPoliciesService.getTopicPolicies(TopicName.get(topic), true); + Awaitility.await().untilAsserted(() -> assertNotNull(TopicPolicyTestUtils.getTopicPolicies(topicPoliciesService, (TopicName.get(topic))))); + TopicPolicies tempPolicies = TopicPolicyTestUtils.getGlobalTopicPolicies(topicPoliciesService, TopicName.get(topic)); assertEquals(tempPolicies.getRetentionPolicies().getRetentionTimeInMinutes(), 3); assertEquals(tempPolicies.getRetentionPolicies().getRetentionSizeInMB(), 4); - tempPolicies = topicPoliciesService.getTopicPolicies(TopicName.get(topic)); + tempPolicies = TopicPolicyTestUtils.getTopicPolicies(topicPoliciesService, (TopicName.get(topic))); assertEquals(tempPolicies.getRetentionPolicies().getRetentionTimeInMinutes(), 10); assertEquals(tempPolicies.getRetentionPolicies().getRetentionSizeInMB(), 20); // check remove global topic policies can be removed correctly. admin.topicPolicies(true).removeRetention(topic); - Awaitility.await().untilAsserted(() -> - assertNull(topicPoliciesService.getTopicPolicies(TopicName.get(topic), true).getRetentionPolicies())); + Awaitility.await().untilAsserted(() -> assertNull(TopicPolicyTestUtils.getGlobalTopicPolicies(topicPoliciesService, + TopicName.get(topic)).getRetentionPolicies())); } @@ -3109,7 +3111,7 @@ public void testShadowTopics() throws Exception { pulsarClient.newProducer().topic(sourceTopic).create().close(); Awaitility.await().untilAsserted(() -> - Assert.assertNull(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(sourceTopic)))); + Assert.assertNull(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), TopicName.get(sourceTopic)))); //shadow topic must exist Assert.expectThrows(PulsarAdminException.PreconditionFailedException.class, ()-> @@ -3139,16 +3141,13 @@ public void testGetTopicPoliciesWhenDeleteTopicPolicy() throws Exception { admin.topics().createNonPartitionedTopic(persistenceTopic); admin.topicPolicies().setMaxConsumers(persistenceTopic, 5); - Integer maxConsumerPerTopic = pulsar - .getTopicPoliciesService() - .getTopicPoliciesBypassCacheAsync(TopicName.get(persistenceTopic)).get() - .getMaxConsumerPerTopic(); + Integer maxConsumerPerTopic = TopicPolicyTestUtils.getTopicPoliciesBypassCache(pulsar.getTopicPoliciesService(), + TopicName.get(persistenceTopic)).orElseThrow().getMaxConsumerPerTopic(); assertEquals(maxConsumerPerTopic, 5); admin.topics().delete(persistenceTopic, true); - TopicPolicies topicPolicies =pulsar.getTopicPoliciesService() - .getTopicPoliciesBypassCacheAsync(TopicName.get(persistenceTopic)).get(5, TimeUnit.SECONDS); - assertNull(topicPolicies); + assertTrue(TopicPolicyTestUtils.getTopicPoliciesBypassCache(pulsar.getTopicPoliciesService(), + TopicName.get(persistenceTopic)).isEmpty()); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InmemoryTopicPoliciesService.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InmemoryTopicPoliciesService.java new file mode 100644 index 0000000000000..88a75fe8f0387 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InmemoryTopicPoliciesService.java @@ -0,0 +1,81 @@ +/* + * 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.service; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.TopicPolicies; + +public class InmemoryTopicPoliciesService implements TopicPoliciesService { + + private final Map cache = new HashMap<>(); + private final Map> listeners = new HashMap<>(); + + @Override + public synchronized CompletableFuture deleteTopicPoliciesAsync(TopicName topicName) { + cache.remove(topicName); + return CompletableFuture.completedFuture(null); + } + + @Override + public synchronized CompletableFuture updateTopicPoliciesAsync(TopicName topicName, TopicPolicies policies) { + final var existingPolicies = cache.get(topicName); + if (existingPolicies != policies) { + cache.put(topicName, policies); + CompletableFuture.runAsync(() -> { + final TopicPolicies latestPolicies; + final List listeners; + synchronized (InmemoryTopicPoliciesService.this) { + latestPolicies = cache.get(topicName); + listeners = this.listeners.getOrDefault(topicName, List.of()); + } + for (var listener : listeners) { + listener.onUpdate(latestPolicies); + } + }); + } + return CompletableFuture.completedFuture(null); + } + + @Override + public synchronized CompletableFuture> getTopicPoliciesAsync( + TopicName topicName, GetType type) { + return CompletableFuture.completedFuture(Optional.ofNullable(cache.get(topicName))); + } + + @Override + public synchronized boolean registerListener(TopicName topicName, TopicPolicyListener listener) { + listeners.computeIfAbsent(topicName, __ -> new ArrayList<>()).add(listener); + return true; + } + + @Override + public synchronized void unregisterListener(TopicName topicName, TopicPolicyListener listener) { + listeners.get(topicName).remove(listener); + } + + synchronized boolean containsKey(TopicName topicName) { + return cache.containsKey(topicName); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InmemoryTopicPoliciesServiceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InmemoryTopicPoliciesServiceServiceTest.java new file mode 100644 index 0000000000000..9ec16405ba853 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InmemoryTopicPoliciesServiceServiceTest.java @@ -0,0 +1,91 @@ +/* + * 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.service; + +import com.google.common.collect.Lists; +import java.util.List; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.naming.TopicName; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class InmemoryTopicPoliciesServiceServiceTest extends MockedPulsarServiceBaseTest { + + @BeforeClass + @Override + protected void setup() throws Exception { + conf.setTopicPoliciesServiceClassName(InmemoryTopicPoliciesService.class.getName()); + conf.setSystemTopicEnabled(false); // verify topic policies don't rely on system topics + super.internalSetup(); + super.setupDefaultTenantAndNamespace(); + } + + @AfterClass + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + // Shadow replicator is created by the topic policies update, this test verifies the listener can be triggered + @Test + public void testShadowReplicator() throws Exception { + final var sourceTopic = TopicName.get("test-shadow-replicator").toString(); + final var shadowTopic = sourceTopic + "-shadow"; + + admin.topics().createNonPartitionedTopic(sourceTopic); + admin.topics().createShadowTopic(shadowTopic, sourceTopic); + admin.topics().setShadowTopics(sourceTopic, Lists.newArrayList(shadowTopic)); + + @Cleanup final var producer = pulsarClient.newProducer(Schema.STRING).topic(sourceTopic).create(); + @Cleanup final var consumer = pulsarClient.newConsumer(Schema.STRING).topic(shadowTopic) + .subscriptionName("sub").subscribe(); + producer.send("msg"); + final var msg = consumer.receive(5, TimeUnit.SECONDS); + Assert.assertNotNull(msg); + Assert.assertEquals(msg.getValue(), "msg"); + + final var persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(sourceTopic).get() + .orElseThrow(); + Assert.assertEquals(TopicPolicyTestUtils.getTopicPolicies(persistentTopic).getShadowTopics(), List.of(shadowTopic)); + Assert.assertEquals(persistentTopic.getShadowReplicators().size(), 1); + } + + @Test + public void testTopicPoliciesAdmin() throws Exception { + final var topic = "test-topic-policies-admin"; + admin.topics().createNonPartitionedTopic(topic); + + Assert.assertNull(admin.topicPolicies().getCompactionThreshold(topic)); + admin.topicPolicies().setCompactionThreshold(topic, 1000); + Assert.assertEquals(admin.topicPolicies().getCompactionThreshold(topic).intValue(), 1000); + // Sleep here because "Directory not empty error" might occur if deleting the topic immediately + Thread.sleep(1000); + final var topicPoliciesService = (InmemoryTopicPoliciesService) pulsar.getTopicPoliciesService(); + Assert.assertTrue(topicPoliciesService.containsKey(TopicName.get(topic))); + admin.topics().delete(topic); + Assert.assertFalse(topicPoliciesService.containsKey(TopicName.get(topic))); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index 440e90da2b694..d684b4af7c251 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -1110,7 +1110,8 @@ public void testDifferentTopicCreationRule(ReplicationMode replicationMode) thro Awaitility.await().untilAsserted(() -> { assertEquals(admin2.namespaces().getAutoTopicCreationAsync(ns).join().getDefaultNumPartitions(), 2); // Trigger system topic __change_event's initialize. - pulsar2.getTopicPoliciesService().getTopicPoliciesAsync(TopicName.get("persistent://" + ns + "/1")); + pulsar2.getTopicPoliciesService().getTopicPoliciesAsync(TopicName.get("persistent://" + ns + "/1"), + TopicPoliciesService.GetType.DEFAULT); }); // Create non-partitioned topic. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index f3076ebdec6c9..200c8dd3b3d9f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -409,7 +409,7 @@ protected void setTopicLevelClusters(String topic, List clusters, Pulsar int partitions = ensurePartitionsAreSame(topic); admin.topics().setReplicationClusters(topic, clusters); Awaitility.await().untilAsserted(() -> { - TopicPolicies policies = pulsar.getTopicPoliciesService().getTopicPolicies(topicName); + TopicPolicies policies = TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), topicName); assertEquals(new HashSet<>(policies.getReplicationClusters()), expected); if (partitions == 0) { checkNonPartitionedTopicLevelClusters(topicName.toString(), clusters, admin, pulsar.getBrokerService()); @@ -434,7 +434,7 @@ protected void checkNonPartitionedTopicLevelClusters(String topic, List } PersistentTopic persistentTopic = (PersistentTopic) optional.get(); Set expected = new HashSet<>(clusters); - Set act = new HashSet<>(persistentTopic.getTopicPolicies().get().getReplicationClusters()); + Set act = new HashSet<>(TopicPolicyTestUtils.getTopicPolicies(persistentTopic).getReplicationClusters()); assertEquals(act, expected); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicInitializeDelayTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicInitializeDelayTest.java index ab8d4dbe5cc01..a563077e012da 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicInitializeDelayTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicInitializeDelayTest.java @@ -108,7 +108,9 @@ public MyPersistentTopic(String topic, ManagedLedger ledger, BrokerService broke SystemTopicBasedTopicPoliciesService topicPoliciesService = (SystemTopicBasedTopicPoliciesService) brokerService.getPulsar().getTopicPoliciesService(); if (topicPoliciesService.getListeners().containsKey(TopicName.get(topic)) ) { - this.onUpdate(brokerService.getPulsar().getTopicPoliciesService().getTopicPoliciesIfExists(TopicName.get(topic))); + brokerService.getPulsar().getTopicPoliciesService().getTopicPoliciesAsync(TopicName.get(topic), + TopicPoliciesService.GetType.DEFAULT + ).thenAccept(optionalPolicies -> optionalPolicies.ifPresent(this::onUpdate)); } } 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 f9171e883613b..b975041d04ee4 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 @@ -2279,7 +2279,8 @@ public void testGetReplicationClusters() throws MetadataStoreException { topicPolicies.setReplicationClusters(topicClusters); Optional optionalTopicPolicies = Optional.of(topicPolicies); topicPoliciesFuture.complete(optionalTopicPolicies); - when(topicPoliciesService.getTopicPoliciesIfExists(any())).thenReturn(topicPolicies); + when(topicPoliciesService.getTopicPoliciesAsync(any(), any())) + .thenReturn(CompletableFuture.completedFuture(Optional.of(topicPolicies))); topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); topic.initialize().join(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTopicPoliciesTest.java index c0281f073cfd4..f89ca2bdebb91 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTopicPoliciesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTopicPoliciesTest.java @@ -30,11 +30,9 @@ import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; -import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AutoSubscriptionCreationOverride; @@ -792,8 +790,7 @@ public void testReplicateAutoSubscriptionCreation() throws Exception { assertNull(admin3.topicPolicies(true).getAutoSubscriptionCreation(topic, false))); } - private void init(String namespace, String topic) - throws PulsarAdminException, PulsarClientException, PulsarServerException { + private void init(String namespace, String topic) throws Exception { final String cluster2 = pulsar2.getConfig().getClusterName(); final String cluster1 = pulsar1.getConfig().getClusterName(); final String cluster3 = pulsar3.getConfig().getClusterName(); @@ -817,11 +814,9 @@ private void init(String namespace, String topic) pulsar3.getClient().newProducer().topic(topic).create().close(); //init topic policies server - Awaitility.await().ignoreExceptions().untilAsserted(() -> { - assertNull(pulsar1.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))); - assertNull(pulsar2.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))); - assertNull(pulsar3.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))); - }); + TopicPolicyTestUtils.getTopicPolicies(pulsar1.getTopicPoliciesService(), TopicName.get(topic)); + TopicPolicyTestUtils.getTopicPolicies(pulsar2.getTopicPoliciesService(), TopicName.get(topic)); + TopicPolicyTestUtils.getTopicPolicies(pulsar3.getTopicPoliciesService(), TopicName.get(topic)); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java index 9caee00cb6134..7e3f4e14daa6d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java @@ -22,12 +22,9 @@ import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertNull; -import static org.testng.AssertJUnit.assertTrue; -import java.lang.reflect.Field; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -35,18 +32,13 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; -import org.apache.pulsar.broker.service.BrokerServiceException.TopicPoliciesCacheNotInitException; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.common.util.Backoff; -import org.apache.pulsar.common.util.BackoffBuilder; import org.apache.pulsar.common.events.PulsarEvent; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; @@ -98,7 +90,7 @@ protected void cleanup() throws Exception { @Test public void testConcurrentlyRegisterUnregisterListeners() throws ExecutionException, InterruptedException { TopicName topicName = TopicName.get("test"); - class TopicPolicyListenerImpl implements TopicPolicyListener { + class TopicPolicyListenerImpl implements TopicPolicyListener { @Override public void onUpdate(TopicPolicies data) { @@ -108,7 +100,7 @@ public void onUpdate(TopicPolicies data) { CompletableFuture f = CompletableFuture.completedFuture(null).thenRunAsync(() -> { for (int i = 0; i < 100; i++) { - TopicPolicyListener listener = new TopicPolicyListenerImpl(); + TopicPolicyListener listener = new TopicPolicyListenerImpl(); systemTopicBasedTopicPoliciesService.registerListener(topicName, listener); Assert.assertNotNull(systemTopicBasedTopicPoliciesService.listeners.get(topicName)); Assert.assertTrue(systemTopicBasedTopicPoliciesService.listeners.get(topicName).size() >= 1); @@ -117,7 +109,7 @@ public void onUpdate(TopicPolicies data) { }); for (int i = 0; i < 100; i++) { - TopicPolicyListener listener = new TopicPolicyListenerImpl(); + TopicPolicyListener listener = new TopicPolicyListenerImpl(); systemTopicBasedTopicPoliciesService.registerListener(topicName, listener); Assert.assertNotNull(systemTopicBasedTopicPoliciesService.listeners.get(topicName)); Assert.assertTrue(systemTopicBasedTopicPoliciesService.listeners.get(topicName).size() >= 1); @@ -130,7 +122,7 @@ public void onUpdate(TopicPolicies data) { } @Test - public void testGetPolicy() throws ExecutionException, InterruptedException, TopicPoliciesCacheNotInitException { + public void testGetPolicy() throws Exception { // Init topic policies TopicPolicies initPolicy = TopicPolicies.builder() @@ -145,7 +137,7 @@ public void testGetPolicy() throws ExecutionException, InterruptedException, Top // Assert broker is cache all topic policies Awaitility.await().untilAsserted(() -> - Assert.assertEquals(systemTopicBasedTopicPoliciesService.getTopicPolicies(TOPIC1) + Assert.assertEquals(TopicPolicyTestUtils.getTopicPolicies(systemTopicBasedTopicPoliciesService, TOPIC1) .getMaxConsumerPerTopic().intValue(), 10)); // Update policy for TOPIC1 @@ -185,12 +177,12 @@ public void testGetPolicy() throws ExecutionException, InterruptedException, Top systemTopicBasedTopicPoliciesService.updateTopicPoliciesAsync(TOPIC6, policies6).get(); Awaitility.await().untilAsserted(() -> { - TopicPolicies policiesGet1 = systemTopicBasedTopicPoliciesService.getTopicPolicies(TOPIC1); - TopicPolicies policiesGet2 = systemTopicBasedTopicPoliciesService.getTopicPolicies(TOPIC2); - TopicPolicies policiesGet3 = systemTopicBasedTopicPoliciesService.getTopicPolicies(TOPIC3); - TopicPolicies policiesGet4 = systemTopicBasedTopicPoliciesService.getTopicPolicies(TOPIC4); - TopicPolicies policiesGet5 = systemTopicBasedTopicPoliciesService.getTopicPolicies(TOPIC5); - TopicPolicies policiesGet6 = systemTopicBasedTopicPoliciesService.getTopicPolicies(TOPIC6); + TopicPolicies policiesGet1 = TopicPolicyTestUtils.getTopicPolicies(systemTopicBasedTopicPoliciesService, TOPIC1); + TopicPolicies policiesGet2 = TopicPolicyTestUtils.getTopicPolicies(systemTopicBasedTopicPoliciesService, TOPIC2); + TopicPolicies policiesGet3 = TopicPolicyTestUtils.getTopicPolicies(systemTopicBasedTopicPoliciesService, TOPIC3); + TopicPolicies policiesGet4 = TopicPolicyTestUtils.getTopicPolicies(systemTopicBasedTopicPoliciesService, TOPIC4); + TopicPolicies policiesGet5 = TopicPolicyTestUtils.getTopicPolicies(systemTopicBasedTopicPoliciesService, TOPIC5); + TopicPolicies policiesGet6 = TopicPolicyTestUtils.getTopicPolicies(systemTopicBasedTopicPoliciesService, TOPIC6); Assert.assertEquals(policiesGet1, policies1); Assert.assertEquals(policiesGet2, policies2); @@ -223,8 +215,8 @@ public void testGetPolicy() throws ExecutionException, InterruptedException, Top // reader for NAMESPACE1 will back fill the reader cache Awaitility.await().untilAsserted(() -> { - TopicPolicies policiesGet1 = systemTopicBasedTopicPoliciesService.getTopicPolicies(TOPIC1); - TopicPolicies policiesGet2 = systemTopicBasedTopicPoliciesService.getTopicPolicies(TOPIC2); + TopicPolicies policiesGet1 = TopicPolicyTestUtils.getTopicPolicies(systemTopicBasedTopicPoliciesService, TOPIC1); + TopicPolicies policiesGet2 = TopicPolicyTestUtils.getTopicPolicies(systemTopicBasedTopicPoliciesService, TOPIC2); Assert.assertEquals(policies1, policiesGet1); Assert.assertEquals(policies2, policiesGet2); }); @@ -235,7 +227,8 @@ public void testGetPolicy() throws ExecutionException, InterruptedException, Top Assert.assertTrue(systemTopicBasedTopicPoliciesService.checkReaderIsCached(NamespaceName.get(NAMESPACE3))); // Check get without cache - TopicPolicies policiesGet1 = systemTopicBasedTopicPoliciesService.getTopicPoliciesBypassCacheAsync(TOPIC1).get(); + TopicPolicies policiesGet1 = TopicPolicyTestUtils.getTopicPoliciesBypassCache(systemTopicBasedTopicPoliciesService, + TOPIC1).orElseThrow(); Assert.assertEquals(policies1, policiesGet1); } @@ -249,7 +242,7 @@ public void testCacheCleanup() throws Exception { Awaitility.await().untilAsserted(() -> assertNotNull(admin.topics().getMaxConsumers(topic))); Map map = systemTopicBasedTopicPoliciesService.getPoliciesCache(); - Map>> listMap = + Map> listMap = systemTopicBasedTopicPoliciesService.getListeners(); assertNotNull(map.get(topicName)); assertEquals(map.get(topicName).getMaxConsumerPerTopic().intValue(), 1000); @@ -268,7 +261,7 @@ public void testListenerCleanupByPartition() throws Exception { admin.topics().createPartitionedTopic(topic, 3); pulsarClient.newProducer().topic(topic).create().close(); - Map>> listMap = + Map> listMap = systemTopicBasedTopicPoliciesService.getListeners(); Awaitility.await().untilAsserted(() -> { // all 3 topic partition have registered the topic policy listeners. @@ -301,64 +294,6 @@ private void prepareData() throws PulsarAdminException { systemTopicBasedTopicPoliciesService = (SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService(); } - @Test - public void testGetPolicyTimeout() throws Exception { - SystemTopicBasedTopicPoliciesService service = (SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService(); - Awaitility.await().untilAsserted(() -> assertTrue(service.policyCacheInitMap.get(TOPIC1.getNamespaceObject()).isDone())); - service.policyCacheInitMap.put(TOPIC1.getNamespaceObject(), new CompletableFuture<>()); - long start = System.currentTimeMillis(); - Backoff backoff = new BackoffBuilder() - .setInitialTime(500, TimeUnit.MILLISECONDS) - .setMandatoryStop(5000, TimeUnit.MILLISECONDS) - .setMax(1000, TimeUnit.MILLISECONDS) - .create(); - try { - service.getTopicPoliciesAsyncWithRetry(TOPIC1, backoff, pulsar.getExecutor(), false).get(); - } catch (Exception e) { - assertTrue(e.getCause() instanceof TopicPoliciesCacheNotInitException); - } - long cost = System.currentTimeMillis() - start; - assertTrue("actual:" + cost, cost >= 5000 - 1000); - } - - @Test - public void testGetTopicPoliciesWithRetry() throws Exception { - Field initMapField = SystemTopicBasedTopicPoliciesService.class.getDeclaredField("policyCacheInitMap"); - initMapField.setAccessible(true); - Map initMap = (Map)initMapField.get(systemTopicBasedTopicPoliciesService); - initMap.remove(NamespaceName.get(NAMESPACE1)); - Field readerCaches = SystemTopicBasedTopicPoliciesService.class.getDeclaredField("readerCaches"); - readerCaches.setAccessible(true); - Map>> readers = (Map)readerCaches.get(systemTopicBasedTopicPoliciesService); - readers.remove(NamespaceName.get(NAMESPACE1)); - Backoff backoff = new BackoffBuilder() - .setInitialTime(500, TimeUnit.MILLISECONDS) - .setMandatoryStop(5000, TimeUnit.MILLISECONDS) - .setMax(1000, TimeUnit.MILLISECONDS) - .create(); - TopicPolicies initPolicy = TopicPolicies.builder() - .maxConsumerPerTopic(10) - .build(); - @Cleanup("shutdownNow") - ScheduledExecutorService executors = Executors.newScheduledThreadPool(1); - executors.schedule(new Runnable() { - @Override - public void run() { - try { - systemTopicBasedTopicPoliciesService.updateTopicPoliciesAsync(TOPIC1, initPolicy).get(); - } catch (Exception ignore) {} - } - }, 2000, TimeUnit.MILLISECONDS); - Awaitility.await().untilAsserted(() -> { - Optional topicPolicies = systemTopicBasedTopicPoliciesService - .getTopicPoliciesAsyncWithRetry(TOPIC1, backoff, pulsar.getExecutor(), false).get(); - Assert.assertTrue(topicPolicies.isPresent()); - if (topicPolicies.isPresent()) { - Assert.assertEquals(topicPolicies.get(), initPolicy); - } - }); - } - @Test public void testHandleNamespaceBeingDeleted() throws Exception { SystemTopicBasedTopicPoliciesService service = (SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService(); @@ -381,13 +316,13 @@ public void testGetTopicPoliciesWithCleanCache() throws Exception { ConcurrentHashMap spyPoliciesCache = spy(new ConcurrentHashMap()); FieldUtils.writeDeclaredField(topicPoliciesService, "policiesCache", spyPoliciesCache, true); - Awaitility.await().untilAsserted(() -> { - Assertions.assertThat(topicPoliciesService.getTopicPolicies(TopicName.get(topic))).isNull(); - }); + Awaitility.await().untilAsserted(() -> Assertions.assertThat( + TopicPolicyTestUtils.getTopicPolicies(topicPoliciesService, TopicName.get(topic))).isNull()); admin.topicPolicies().setMaxConsumersPerSubscription(topic, 1); Awaitility.await().untilAsserted(() -> { - Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))).isNotNull(); + Assertions.assertThat(TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), + TopicName.get(topic))).isNotNull(); }); Map>> readers = @@ -401,21 +336,18 @@ public void testGetTopicPoliciesWithCleanCache() throws Exception { CompletableFuture result = new CompletableFuture<>(); Thread thread = new Thread(() -> { - TopicPolicies topicPolicies; - for (int i = 0; i < 10; i++) { - try { - topicPolicies = topicPoliciesService.getTopicPolicies(TopicName.get(topic)); - Assert.assertNotNull(topicPolicies); - Thread.sleep(500); - } catch (BrokerServiceException.TopicPoliciesCacheNotInitException e) { - log.warn("topic policies cache not init, retry..."); - } catch (Throwable e) { - log.error("ops: ", e); - result.completeExceptionally(e); - return; + try { + for (int i = 0; i < 10; i++) { + final var policies = TopicPolicyTestUtils.getTopicPolicies(topicPoliciesService, + TopicName.get(topic)); + if (policies == null) { + throw new Exception("null policies for " + i + "th get"); + } } + result.complete(null); + } catch (Exception e) { + result.completeExceptionally(e); } - result.complete(null); }); Thread thread2 = new Thread(() -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java new file mode 100644 index 0000000000000..9cf688d62edc6 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicPolicyTestUtils.java @@ -0,0 +1,74 @@ +/* + * 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.service; + +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import lombok.Cleanup; +import org.apache.pulsar.common.events.PulsarEvent; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.TopicPolicies; + +public class TopicPolicyTestUtils { + + public static TopicPolicies getTopicPolicies(AbstractTopic topic) { + final TopicPolicies topicPolicies; + try { + topicPolicies = getTopicPolicies(topic.brokerService.getPulsar().getTopicPoliciesService(), + TopicName.get(topic.topic)); + } catch (ExecutionException | InterruptedException e) { + throw new RuntimeException(e); + } + if (topicPolicies == null) { + throw new RuntimeException("No topic policies for " + topic); + } + return topicPolicies; + } + + public static TopicPolicies getTopicPolicies(TopicPoliciesService topicPoliciesService, TopicName topicName) + throws ExecutionException, InterruptedException { + return topicPoliciesService.getTopicPoliciesAsync(topicName, TopicPoliciesService.GetType.DEFAULT).get() + .orElse(null); + } + + public static TopicPolicies getLocalTopicPolicies(TopicPoliciesService topicPoliciesService, TopicName topicName) + throws ExecutionException, InterruptedException { + return topicPoliciesService.getTopicPoliciesAsync(topicName, TopicPoliciesService.GetType.LOCAL_ONLY).get() + .orElse(null); + } + + public static TopicPolicies getGlobalTopicPolicies(TopicPoliciesService topicPoliciesService, TopicName topicName) + throws ExecutionException, InterruptedException { + return topicPoliciesService.getTopicPoliciesAsync(topicName, TopicPoliciesService.GetType.GLOBAL_ONLY).get() + .orElse(null); + } + + public static Optional getTopicPoliciesBypassCache(TopicPoliciesService topicPoliciesService, + TopicName topicName) throws Exception { + @Cleanup final var reader = ((SystemTopicBasedTopicPoliciesService) topicPoliciesService) + .getNamespaceEventsSystemTopicFactory() + .createTopicPoliciesSystemTopicClient(topicName.getNamespaceObject()) + .newReader(); + PulsarEvent event = null; + while (reader.hasMoreEvents()) { + event = reader.readNext().getValue(); + } + return Optional.ofNullable(event).map(e -> e.getTopicPoliciesEvent().getPolicies()); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java index 070f7193874c3..903443d37bb07 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentTopicTest.java @@ -715,7 +715,8 @@ public void testCheckPersistencePolicies() throws Exception { doReturn(policiesService).when(pulsar).getTopicPoliciesService(); TopicPolicies policies = new TopicPolicies(); policies.setRetentionPolicies(retentionPolicies); - doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(policiesService).getTopicPoliciesAsync(TopicName.get(topic)); + doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(policiesService) + .getTopicPoliciesAsync(TopicName.get(topic), TopicPoliciesService.GetType.DEFAULT); persistentTopic.onUpdate(policies); verify(persistentTopic, times(1)).checkPersistencePolicies(); Awaitility.await().untilAsserted(() -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/PartitionedSystemTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/PartitionedSystemTopicTest.java index a2401ebe19a06..e7bfa3278e36d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/PartitionedSystemTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/systopic/PartitionedSystemTopicTest.java @@ -37,6 +37,7 @@ import org.apache.pulsar.broker.admin.impl.BrokersBase; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.service.TopicPolicyTestUtils; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.schema.SchemaRegistry; @@ -359,7 +360,8 @@ public void testDeleteTopicSchemaAndPolicyWhenTopicIsNotLoaded() throws Exceptio PersistentTopic persistentTopic = (PersistentTopic) topic.join().get(); persistentTopic.close(); admin.topics().delete(topicName); - TopicPolicies topicPolicies = pulsar.getTopicPoliciesService().getTopicPoliciesIfExists(TopicName.get(topicName)); + TopicPolicies topicPolicies = TopicPolicyTestUtils.getTopicPolicies(pulsar.getTopicPoliciesService(), + TopicName.get(topicName)); assertNull(topicPolicies); String base = TopicName.get(topicName).getPartitionedTopicName(); String id = TopicName.get(base).getSchemaName(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index d3e0391443f0f..cc09fa212198d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -92,6 +92,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService; import org.apache.pulsar.broker.service.SystemTopicTxnBufferSnapshotService.ReferenceCountedWriter; +import org.apache.pulsar.broker.service.TopicPolicyTestUtils; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TransactionBufferSnapshotServiceFactory; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -142,7 +143,6 @@ import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; import org.apache.pulsar.common.policies.data.RetentionPolicies; -import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; @@ -550,11 +550,7 @@ public void testSubscriptionRecreateTopic() .getSubscription(subName); subscription.getPendingAckManageLedger().thenAccept(managedLedger -> { long retentionSize = managedLedger.getConfig().getRetentionSizeInMB(); - if (!originPersistentTopic.getTopicPolicies().isPresent()) { - log.error("Failed to getTopicPolicies of :" + originPersistentTopic); - Assert.fail(); - } - TopicPolicies topicPolicies = originPersistentTopic.getTopicPolicies().get(); + TopicPolicyTestUtils.getTopicPolicies(originPersistentTopic); // verify the topic policies exist Assert.assertEquals(retentionSizeInMbSetTopic, retentionSize); MLPendingAckStoreProvider mlPendingAckStoreProvider = new MLPendingAckStoreProvider(); CompletableFuture future = mlPendingAckStoreProvider.newPendingAckStore(subscription); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java index f60aeb78387ad..9396a80cf2557 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java @@ -46,7 +46,6 @@ import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferDisable; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.compaction.CompactionServiceFactory; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; @@ -110,7 +109,7 @@ public void testNoOrphanTopicAfterCreateTimeout() throws Exception { // Assert only one PersistentTopic was not closed. TopicPoliciesService topicPoliciesService = pulsar.getTopicPoliciesService(); - Map>> listeners = + Map> listeners = WhiteboxImpl.getInternalState(topicPoliciesService, "listeners"); assertEquals(listeners.get(TopicName.get(tpName)).size(), 1); @@ -217,7 +216,7 @@ public void testNoOrphanTopicIfInitFailed() throws Exception { // Assert only one PersistentTopic was not closed. TopicPoliciesService topicPoliciesService = pulsar.getTopicPoliciesService(); - Map>> listeners = + Map> listeners = WhiteboxImpl.getInternalState(topicPoliciesService, "listeners"); assertEquals(listeners.get(TopicName.get(tpName)).size(), 1); From 105192d5baff8eb48814e89817a900a116624ac3 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 20 Sep 2024 18:40:54 +0800 Subject: [PATCH 513/580] [fix][broker] Fix topic policies cannot be queried with extensible load manager (#23326) --- .../extensions/channel/ServiceUnitStateChannelImpl.java | 2 +- .../org/apache/pulsar/broker/service/BrokerService.java | 6 ++++++ .../pulsar/broker/service/persistent/PersistentTopic.java | 3 +++ 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index 3ebcd1c20ca87..5893fc4924413 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -267,7 +267,7 @@ public void cleanOwnerships() { @Override public synchronized boolean started() { - return validateChannelState(LeaderElectionServiceStarted, false); + return validateChannelState(Started, true); } public synchronized void start() throws PulsarServerException { 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 aee6532716cd8..c7a210bc543cf 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 @@ -1177,6 +1177,9 @@ public CompletableFuture> getTopic(final TopicName topicName, bo } private CompletableFuture> getTopicPoliciesBypassSystemTopic(@Nonnull TopicName topicName) { + if (ExtensibleLoadManagerImpl.isInternalTopic(topicName.toString())) { + return CompletableFuture.completedFuture(Optional.empty()); + } return pulsar.getTopicPoliciesService().getTopicPoliciesAsync(topicName, TopicPoliciesService.GetType.DEFAULT); } @@ -3601,6 +3604,9 @@ private AutoTopicCreationOverride getAutoTopicCreationOverride(final TopicName t public @Nonnull CompletableFuture isAllowAutoSubscriptionCreationAsync(@Nonnull TopicName tpName) { requireNonNull(tpName); // Policies priority: topic level -> namespace level -> broker level + if (ExtensibleLoadManagerImpl.isInternalTopic(tpName.toString())) { + return CompletableFuture.completedFuture(true); + } return pulsar.getTopicPoliciesService() .getTopicPoliciesAsync(tpName, TopicPoliciesService.GetType.LOCAL_ONLY) .thenCompose(optionalTopicPolicies -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index e951ffab1e230..9c0bdc120c474 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -4324,6 +4324,9 @@ protected CompletableFuture initTopicPolicy() { final var topicPoliciesService = brokerService.pulsar().getTopicPoliciesService(); final var partitionedTopicName = TopicName.getPartitionedTopicName(topic); if (topicPoliciesService.registerListener(partitionedTopicName, this)) { + if (ExtensibleLoadManagerImpl.isInternalTopic(topic)) { + return CompletableFuture.completedFuture(null); + } return topicPoliciesService.getTopicPoliciesAsync(partitionedTopicName, TopicPoliciesService.GetType.DEFAULT ).thenAcceptAsync(optionalPolicies -> optionalPolicies.ifPresent(this::onUpdate), From c2a0090144a48f01568c23c71f660d35674f2c75 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Fri, 20 Sep 2024 12:17:43 -0700 Subject: [PATCH 514/580] [improve][broker] Add ServiceUnitStateTableView (ExtensibleLoadManagerImpl only) (#23301) --- conf/broker.conf | 9 + pip/pip-378.md | 57 +- .../pulsar/broker/ServiceConfiguration.java | 29 + .../extensions/ExtensibleLoadManagerImpl.java | 81 +- .../extensions/channel/ServiceUnitState.java | 24 +- .../channel/ServiceUnitStateChannel.java | 126 +-- .../channel/ServiceUnitStateChannelImpl.java | 389 +++---- .../channel/ServiceUnitStateData.java | 15 + ...ServiceUnitStateDataConflictResolver.java} | 22 +- ...ceUnitStateMetadataStoreTableViewImpl.java | 155 +++ .../channel/ServiceUnitStateTableView.java | 113 ++ .../ServiceUnitStateTableViewBase.java | 92 ++ .../ServiceUnitStateTableViewImpl.java | 177 ++++ .../ServiceUnitStateTableViewSyncer.java | 281 +++++ .../extensions/store/LoadDataStore.java | 6 + .../store/TableViewLoadDataStoreImpl.java | 142 ++- .../service/persistent/PersistentTopic.java | 8 +- .../ExtensibleLoadManagerImplBaseTest.java | 50 +- .../ExtensibleLoadManagerImplTest.java | 855 +++++++++------ ...anagerImplWithAdvertisedListenersTest.java | 19 +- ...gerImplWithTransactionCoordinatorTest.java | 6 +- .../channel/ServiceUnitStateChannelTest.java | 976 ++++++++++-------- ...iceUnitStateDataConflictResolverTest.java} | 32 +- .../channel/ServiceUnitStateTest.java | 176 +++- .../extensions/store/LoadDataStoreTest.java | 59 +- .../BrokerServiceAutoTopicCreationTest.java | 6 +- .../broker/service/BrokerServiceTest.java | 4 +- .../ServiceUnitStateCompactionTest.java | 29 +- .../compaction/StrategicCompactionTest.java | 2 +- .../metadata/api/MetadataCacheConfig.java | 12 +- .../metadata/api/MetadataStoreTableView.java | 87 ++ .../cache/impl/MetadataCacheImpl.java | 13 +- .../impl/MetadataStoreTableViewImpl.java | 342 ++++++ .../metadata/tableview/impl/package-info.java | 19 + .../pulsar/metadata/MetadataCacheTest.java | 25 + .../metadata/MetadataStoreTableViewTest.java | 499 +++++++++ .../ExtensibleLoadManagerTest.java | 48 +- 37 files changed, 3721 insertions(+), 1264 deletions(-) rename pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/{ServiceUnitStateCompactionStrategy.java => ServiceUnitStateDataConflictResolver.java} (82%) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateMetadataStoreTableViewImpl.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableView.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewBase.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewSyncer.java rename pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/{ServiceUnitStateCompactionStrategyTest.java => ServiceUnitStateDataConflictResolverTest.java} (88%) create mode 100644 pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreTableView.java create mode 100644 pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/MetadataStoreTableViewImpl.java create mode 100644 pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/package-info.java create mode 100644 pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTableViewTest.java diff --git a/conf/broker.conf b/conf/broker.conf index 74130d709cdd2..125b2aa8c1b39 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1563,6 +1563,15 @@ loadBalancerNamespaceBundleSplitConditionHitCountThreshold=3 # (only used in load balancer extension logics) loadBalancerServiceUnitStateTombstoneDelayTimeInSeconds=3600 +# Name of ServiceUnitStateTableView implementation class to use +loadManagerServiceUnitStateTableViewClassName=org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl + +# Specify ServiceUnitTableViewSyncer to sync service unit(bundle) states between metadata store and +# system topic table views during migration from one to the other. One could enable this +# syncer before migration and disable it after the migration finishes. +# It accepts `MetadataStoreToSystemTopicSyncer` or `SystemTopicToMetadataStoreSyncer` to +# enable it. It accepts `None` to disable it." +loadBalancerServiceUnitTableViewSyncer=None ### --- Replication --- ### diff --git a/pip/pip-378.md b/pip/pip-378.md index 352c7fa560d1c..e44ce7339cf53 100644 --- a/pip/pip-378.md +++ b/pip/pip-378.md @@ -30,7 +30,7 @@ Add `ServiceUnitStateTableView` abstraction and make it pluggable, so users can - Introduce `MetadataStoreTableView` interface to support `ServiceUnitStateMetadataStoreTableViewImpl` implementation. - `MetadataStoreTableViewImpl` will use shadow hashmap to maintain the metadata tableview. It will initially fill the local tableview by scanning all existing items in the metadata store path. Also, new items will be updated to the tableview via metadata watch notifications. - Add `BiConsumer>> asyncReloadConsumer` in MetadataCacheConfig to listen the automatic cache async reload. This can be useful to re-sync the the shadow hashmap in MetadataStoreTableViewImpl in case it is out-dated in the worst case(e.g. network or metadata issues). -- Introduce `ServiceUnitStateTableViewSyncer` to sync system topic and metadata store table views to migrate to one from the other. This syncer can be enabled by a dynamic config, `loadBalancerServiceUnitTableViewSyncerEnabled`. +- Introduce `ServiceUnitStateTableViewSyncer` to sync system topic and metadata store table views to migrate to one from the other. This syncer can be enabled by a dynamic config, `loadBalancerServiceUnitTableViewSyncer`. ## Detailed Design @@ -243,56 +243,15 @@ public class MetadataCacheConfig { */ @Slf4j public class ServiceUnitStateTableViewSyncer implements Cloneable { - private static final int SYNC_TIMEOUT_IN_SECS = 30; - private volatile ServiceUnitStateTableView systemTopicTableView; - private volatile ServiceUnitStateTableView metadataStoreTableView; - + ... public void start(PulsarService pulsar) throws IOException { - if (!pulsar.getConfiguration().isLoadBalancerServiceUnitTableViewSyncerEnabled()) { - return; - } - try { - if (systemTopicTableView == null) { - systemTopicTableView = new ServiceUnitStateTableViewImpl(); - systemTopicTableView.start( - pulsar, - this::syncToMetadataStore, - this::syncToMetadataStore); - log.info("Successfully started ServiceUnitStateTableViewSyncer::systemTopicTableView"); - } - - if (metadataStoreTableView == null) { - metadataStoreTableView = new ServiceUnitStateMetadataStoreTableViewImpl(); - metadataStoreTableView.start( - pulsar, - this::syncToSystemTopic, - this::syncToSystemTopic); - log.info("Successfully started ServiceUnitStateTableViewSyncer::metadataStoreTableView"); - } - - } catch (Throwable e) { - log.error("Failed to start ServiceUnitStateTableViewSyncer", e); - throw e; - } + ... // sync SystemTopicTableView and MetadataStoreTableView } - private void syncToSystemTopic(String key, ServiceUnitStateData data) { - try { - systemTopicTableView.put(key, data).get(SYNC_TIMEOUT_IN_SECS, TimeUnit.SECONDS); - } catch (Throwable e) { - log.error("SystemTopicTableView failed to sync key:{}, data:{}", key, data, e); - throw new IllegalStateException(e); - } - } - private void syncToMetadataStore(String key, ServiceUnitStateData data) { - try { - metadataStoreTableView.put(key, data).get(SYNC_TIMEOUT_IN_SECS, TimeUnit.SECONDS); - } catch (Throwable e) { - log.error("metadataStoreTableView failed to sync key:{}, data:{}", key, data, e); - throw new IllegalStateException(e); - } + public void close() throws IOException { + ... // stop syncer } ... } @@ -302,14 +261,14 @@ public class ServiceUnitStateTableViewSyncer implements Cloneable { ### Configuration -- Add a `loadManagerServiceUnitStateTableViewClassName` configuration to specify `ServiceUnitStateTableView` implementation class name. -- Add a `loadBalancerServiceUnitTableViewSyncerEnabled` configuration to to enable ServiceUnitTableViewSyncer to sync metadata store and system topic ServiceUnitStateTableView during migration. +- Add a `loadManagerServiceUnitStateTableViewClassName` static configuration to specify `ServiceUnitStateTableView` implementation class name. +- Add a `loadBalancerServiceUnitTableViewSyncer` dynamic configuration to enable ServiceUnitTableViewSyncer to sync metadata store and system topic ServiceUnitStateTableView during migration. ## Backward & Forward Compatibility It will ba Backward & Forward compatible as `loadManagerServiceUnitStateTableViewClassName` will be `ServiceUnitStateTableViewImpl`(system topic implementation) by default. -We will introduce `ServiceUnitStateTableViewSyncer` to sync system topic and metadata store table views when migrating to ServiceUnitStateMetadataStoreTableViewImpl from ServiceUnitStateTableViewImpl and vice versa. This syncer can be enabled/disabled by a dynamic config, `loadBalancerServiceUnitTableViewSyncerEnabled`. The admin could enable this syncer before migration and disable it after it is finished. +We will introduce `ServiceUnitStateTableViewSyncer` dynamic config to sync system topic and metadata store table views when migrating to ServiceUnitStateMetadataStoreTableViewImpl from ServiceUnitStateTableViewImpl and vice versa. The admin could enable this syncer before migration and disable it after it is finished. ## Alternatives 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 cdd27412e3052..486587ec174a0 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 @@ -2912,6 +2912,25 @@ public double getLoadBalancerBandwidthOutResourceWeight() { ) private boolean loadBalancerMultiPhaseBundleUnload = true; + @FieldContext( + dynamic = false, + category = CATEGORY_LOAD_BALANCER, + doc = "Name of ServiceUnitStateTableView implementation class to use" + ) + private String loadManagerServiceUnitStateTableViewClassName = + "org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl"; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "Specify ServiceUnitTableViewSyncer to sync service unit(bundle) states between metadata store and " + + "system topic table views during migration from one to the other. One could enable this" + + " syncer before migration and disable it after the migration finishes. " + + "It accepts `MetadataStoreToSystemTopicSyncer` or `SystemTopicToMetadataStoreSyncer` to " + + "enable it. It accepts `None` to disable it." + ) + private ServiceUnitTableViewSyncerType loadBalancerServiceUnitTableViewSyncer = ServiceUnitTableViewSyncerType.None; + /**** --- Replication. --- ****/ @FieldContext( category = CATEGORY_REPLICATION, @@ -3810,4 +3829,14 @@ public Map lookupProperties() { }); return map; } + + public boolean isLoadBalancerServiceUnitTableViewSyncerEnabled() { + return loadBalancerServiceUnitTableViewSyncer != ServiceUnitTableViewSyncerType.None; + } + + public enum ServiceUnitTableViewSyncerType { + None, + MetadataStoreToSystemTopicSyncer, + SystemTopicToMetadataStoreSyncer; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index f22bcc836f6e9..98ef6bf36edac 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -21,10 +21,9 @@ import static java.lang.String.format; import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.Role.Follower; import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.Role.Leader; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.TOPIC; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.TOPIC; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Label.Success; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.Admin; -import static org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared.getNamespaceBundle; import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.util.ArrayList; @@ -41,20 +40,17 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; -import java.util.stream.Collectors; import lombok.Getter; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.mutable.MutableObject; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.loadbalance.LeaderElectionService; import org.apache.pulsar.broker.loadbalance.LoadManager; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewSyncer; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData; @@ -172,6 +168,9 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS private TopBundleLoadDataReporter topBundleLoadDataReporter; + @Getter + protected ServiceUnitStateTableViewSyncer serviceUnitStateTableViewSyncer; + private volatile ScheduledFuture brokerLoadDataReportTask; private volatile ScheduledFuture topBundlesLoadDataReportTask; @@ -209,46 +208,18 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS /** * Get all the bundles that are owned by this broker. */ + @Deprecated public CompletableFuture> getOwnedServiceUnitsAsync() { + return CompletableFuture.completedFuture(getOwnedServiceUnits()); + } + + public Set getOwnedServiceUnits() { if (!started) { log.warn("Failed to get owned service units, load manager is not started."); - return CompletableFuture.completedFuture(Collections.emptySet()); + return Collections.emptySet(); } - String brokerId = brokerRegistry.getBrokerId(); - Set> entrySet = serviceUnitStateChannel.getOwnershipEntrySet(); - Set ownedServiceUnits = entrySet.stream() - .filter(entry -> { - var stateData = entry.getValue(); - return stateData.state() == ServiceUnitState.Owned - && StringUtils.isNotBlank(stateData.dstBroker()) - && stateData.dstBroker().equals(brokerId); - }).map(entry -> { - var bundle = entry.getKey(); - return getNamespaceBundle(pulsar, bundle); - }).collect(Collectors.toSet()); - // Add heartbeat and SLA monitor namespace bundle. - NamespaceName heartbeatNamespace = NamespaceService.getHeartbeatNamespace(brokerId, pulsar.getConfiguration()); - NamespaceName heartbeatNamespaceV2 = NamespaceService - .getHeartbeatNamespaceV2(brokerId, pulsar.getConfiguration()); - NamespaceName slaMonitorNamespace = NamespaceService - .getSLAMonitorNamespace(brokerId, pulsar.getConfiguration()); - return pulsar.getNamespaceService().getNamespaceBundleFactory() - .getFullBundleAsync(heartbeatNamespace) - .thenAccept(fullBundle -> ownedServiceUnits.add(fullBundle)).exceptionally(e -> { - log.warn("Failed to get heartbeat namespace bundle.", e); - return null; - }).thenCompose(__ -> pulsar.getNamespaceService().getNamespaceBundleFactory() - .getFullBundleAsync(heartbeatNamespaceV2)) - .thenAccept(fullBundle -> ownedServiceUnits.add(fullBundle)).exceptionally(e -> { - log.warn("Failed to get heartbeat namespace V2 bundle.", e); - return null; - }).thenCompose(__ -> pulsar.getNamespaceService().getNamespaceBundleFactory() - .getFullBundleAsync(slaMonitorNamespace)) - .thenAccept(fullBundle -> ownedServiceUnits.add(fullBundle)).exceptionally(e -> { - log.warn("Failed to get SLA Monitor namespace bundle.", e); - return null; - }).thenApply(__ -> ownedServiceUnits); + return serviceUnitStateChannel.getOwnedServiceUnits(); } @Override @@ -317,14 +288,14 @@ private static void createSystemTopics(PulsarService pulsar) throws PulsarServer createSystemTopic(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); } - private static boolean configureSystemTopics(PulsarService pulsar) { + public static boolean configureSystemTopics(PulsarService pulsar, long target) { try { if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar) && pulsar.getConfiguration().isTopicLevelPoliciesEnabled()) { Long threshold = pulsar.getAdminClient().topicPolicies().getCompactionThreshold(TOPIC); - if (threshold == null || COMPACTION_THRESHOLD != threshold.longValue()) { - pulsar.getAdminClient().topicPolicies().setCompactionThreshold(TOPIC, COMPACTION_THRESHOLD); - log.info("Set compaction threshold: {} bytes for system topic {}.", COMPACTION_THRESHOLD, TOPIC); + if (threshold == null || target != threshold.longValue()) { + pulsar.getAdminClient().topicPolicies().setCompactionThreshold(TOPIC, target); + log.info("Set compaction threshold: {} bytes for system topic {}.", target, TOPIC); } } else { log.warn("System topic or topic level policies is disabled. " @@ -432,6 +403,7 @@ public void start() throws PulsarServerException { serviceUnitStateChannel, unloadCounter, unloadMetrics); this.splitScheduler = new SplitScheduler( pulsar, serviceUnitStateChannel, splitManager, splitCounter, splitMetrics, context); + this.serviceUnitStateTableViewSyncer = new ServiceUnitStateTableViewSyncer(); pulsar.runWhenReadyForIncomingRequests(() -> { try { @@ -799,10 +771,11 @@ public void close() throws PulsarServerException { monitorTask.cancel(true); } - this.brokerLoadDataStore.close(); - this.topBundlesLoadDataStore.close(); + this.brokerLoadDataStore.shutdown(); + this.topBundlesLoadDataStore.shutdown(); this.unloadScheduler.close(); this.splitScheduler.close(); + this.serviceUnitStateTableViewSyncer.close(); } catch (IOException ex) { throw new PulsarServerException(ex); } finally { @@ -857,6 +830,9 @@ synchronized void playLeader() { topBundlesLoadDataStore.init(); unloadScheduler.start(); serviceUnitStateChannel.scheduleOwnershipMonitor(); + if (pulsar.getConfiguration().isLoadBalancerServiceUnitTableViewSyncerEnabled()) { + serviceUnitStateTableViewSyncer.start(pulsar); + } break; } catch (Throwable e) { log.warn("The broker:{} failed to set the role. Retrying {} th ...", @@ -906,6 +882,7 @@ synchronized void playFollower() { brokerLoadDataStore.init(); topBundlesLoadDataStore.close(); topBundlesLoadDataStore.startProducer(); + serviceUnitStateTableViewSyncer.close(); break; } catch (Throwable e) { log.warn("The broker:{} failed to set the role. Retrying {} th ...", @@ -977,19 +954,27 @@ protected void monitor() { // System topic config might fail due to the race condition // with topic policy init(Topic policies cache have not init). if (!configuredSystemTopics) { - configuredSystemTopics = configureSystemTopics(pulsar); + configuredSystemTopics = configureSystemTopics(pulsar, COMPACTION_THRESHOLD); } if (role != Leader) { log.warn("Current role:{} does not match with the channel ownership:{}. " + "Playing the leader role.", role, isChannelOwner); playLeader(); } + + if (pulsar.getConfiguration().isLoadBalancerServiceUnitTableViewSyncerEnabled()) { + serviceUnitStateTableViewSyncer.start(pulsar); + } else { + serviceUnitStateTableViewSyncer.close(); + } + } else { if (role != Follower) { log.warn("Current role:{} does not match with the channel ownership:{}. " + "Playing the follower role.", role, isChannelOwner); playFollower(); } + serviceUnitStateTableViewSyncer.close(); } } catch (Throwable e) { log.error("Failed to get the channel ownership.", e); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitState.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitState.java index 42ef55593ae1a..b823a8277d376 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitState.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitState.java @@ -42,7 +42,13 @@ public enum ServiceUnitState { Deleted; // deleted in the system (semi-terminal state) - private static final Map> validTransitions = Map.of( + + public enum StorageType { + SystemTopic, + MetadataStore; + } + + private static final Map> validTransitionsOverSystemTopic = Map.of( // (Init -> all states) transitions are required // when the topic is compacted in the middle of assign, transfer or split. Init, Set.of(Free, Owned, Assigning, Releasing, Splitting, Deleted), @@ -54,12 +60,24 @@ public enum ServiceUnitState { Deleted, Set.of(Init) ); + private static final Map> validTransitionsOverMetadataStore = Map.of( + Init, Set.of(Assigning), + Free, Set.of(Assigning), + Owned, Set.of(Splitting, Releasing), + Assigning, Set.of(Owned), + Releasing, Set.of(Assigning, Free), + Splitting, Set.of(Deleted), + Deleted, Set.of(Init) + ); + private static final Set inFlightStates = Set.of( Assigning, Releasing, Splitting ); - public static boolean isValidTransition(ServiceUnitState from, ServiceUnitState to) { - Set transitions = validTransitions.get(from); + public static boolean isValidTransition(ServiceUnitState from, ServiceUnitState to, StorageType storageType) { + Set transitions = + (storageType == StorageType.SystemTopic) ? validTransitionsOverSystemTopic.get(from) + : validTransitionsOverMetadataStore.get(from); return transitions.contains(to); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java index 6319fc332a678..ac9897a20e75c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannel.java @@ -24,13 +24,14 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.loadbalance.extensions.manager.StateChangeListener; import org.apache.pulsar.broker.loadbalance.extensions.models.Split; import org.apache.pulsar.broker.loadbalance.extensions.models.Unload; +import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.metadata.api.NotificationType; -import org.apache.pulsar.metadata.api.extended.SessionEvent; /** * Defines the ServiceUnitStateChannel interface. @@ -56,92 +57,39 @@ public interface ServiceUnitStateChannel extends Closeable { void close() throws PulsarServerException; /** - * Asynchronously gets the current owner broker of the system topic in this channel. - * @return the service url without the protocol prefix, 'http://'. e.g. broker-xyz:abcd - * - * ServiceUnitStateChannel elects the separate leader as the owner broker of the system topic in this channel. + * Asynchronously gets the current owner broker of this channel. + * @return a future of owner brokerId to track the completion of the operation */ CompletableFuture> getChannelOwnerAsync(); /** - * Asynchronously checks if the current broker is the owner broker of the system topic in this channel. - * @return True if the current broker is the owner. Otherwise, false. + * Asynchronously checks if the current broker is the owner broker of this channel. + * @return a future of check result to track the completion of the operation */ CompletableFuture isChannelOwnerAsync(); /** - * Checks if the current broker is the owner broker of the system topic in this channel. + * Checks if the current broker is the owner broker of this channel. * @return True if the current broker is the owner. Otherwise, false. + * @throws ExecutionException + * @throws InterruptedException + * @throws TimeoutException */ - boolean isChannelOwner(); - - /** - * Handles the metadata session events to track - * if the connection between the broker and metadata store is stable or not. - * This will be registered as a metadata SessionEvent listener. - * - * The stability of the metadata connection is important - * to determine how to handle the broker deletion(unavailable) event notified from the metadata store. - * - * Please refer to handleBrokerRegistrationEvent(String broker, NotificationType type) for more details. - * - * @param event metadata session events - */ - void handleMetadataSessionEvent(SessionEvent event); - - /** - * Handles the broker registration event from the broker registry. - * This will be registered as a broker registry listener. - * - * Case 1: If NotificationType is Deleted, - * it will schedule a clean-up operation to release the ownerships of the deleted broker. - * - * Sub-case1: If the metadata connection has been stable for long time, - * it will immediately execute the cleanup operation to guarantee high-availability. - * - * Sub-case2: If the metadata connection has been stable only for short time, - * it will defer the clean-up operation for some time and execute it. - * This is to gracefully handle the case when metadata connection is flaky -- - * If the deleted broker comes back very soon, - * we better cancel the clean-up operation for high-availability. - * - * Sub-case3: If the metadata connection is unstable, - * it will not schedule the clean-up operation, as the broker-metadata connection is lost. - * The brokers will continue to serve existing topics connections, - * and we better not to interrupt the existing topic connections for high-availability. - * - * - * Case 2: If NotificationType is Created, - * it will cancel any scheduled clean-up operation if still not executed. - * - * @param broker notified broker - * @param type notification type - */ - void handleBrokerRegistrationEvent(String broker, NotificationType type); + boolean isChannelOwner() throws ExecutionException, InterruptedException, TimeoutException; /** * Asynchronously gets the current owner broker of the service unit. * - * * @param serviceUnit (e.g. bundle) - * @return the future object of the owner broker - * - * Case 1: If the service unit is owned, it returns the completed future object with the current owner. - * Case 2: If the service unit's assignment is ongoing, it returns the non-completed future object. - * Sub-case1: If the assigned broker is available and finally takes the ownership, - * the future object will complete and return the owner broker. - * Sub-case2: If the assigned broker does not take the ownership in time, - * the future object will time out. - * Case 3: If none of them, it returns Optional.empty(). + * @return a future of owner brokerId to track the completion of the operation */ CompletableFuture> getOwnerAsync(String serviceUnit); /** - * Gets the assigned broker of the service unit. - * + * Asynchronously gets the assigned broker of the service unit. * * @param serviceUnit (e.g. bundle)) - * @return the future object of the assigned broker + * @return assigned brokerId */ Optional getAssigned(String serviceUnit); @@ -149,47 +97,39 @@ public interface ServiceUnitStateChannel extends Closeable { /** * Checks if the target broker is the owner of the service unit. * - * * @param serviceUnit (e.g. bundle) - * @param targetBroker - * @return true if the target broker is the owner. false if unknown. + * @param targetBrokerId + * @return true if the target brokerId is the owner brokerId. false if unknown. */ - boolean isOwner(String serviceUnit, String targetBroker); + boolean isOwner(String serviceUnit, String targetBrokerId); /** * Checks if the current broker is the owner of the service unit. * - * * @param serviceUnit (e.g. bundle)) * @return true if the current broker is the owner. false if unknown. */ boolean isOwner(String serviceUnit); /** - * Asynchronously publishes the service unit assignment event to the system topic in this channel. - * It de-duplicates assignment events if there is any ongoing assignment event for the same service unit. + * Asynchronously publishes the service unit assignment event to this channel. * @param serviceUnit (e.g bundle) - * @param broker the assigned broker - * @return the completable future object with the owner broker - * case 1: If the assigned broker is available and takes the ownership, - * the future object will complete and return the owner broker. - * The returned owner broker could be different from the input broker (due to assignment race-condition). - * case 2: If the assigned broker does not take the ownership in time, - * the future object will time out. + * @param brokerId the assigned brokerId + * @return a future of owner brokerId to track the completion of the operation */ - CompletableFuture publishAssignEventAsync(String serviceUnit, String broker); + CompletableFuture publishAssignEventAsync(String serviceUnit, String brokerId); /** - * Asynchronously publishes the service unit unload event to the system topic in this channel. + * Asynchronously publishes the service unit unload event to this channel. * @param unload (unload specification object) - * @return the completable future object staged from the event message sendAsync. + * @return a future to track the completion of the operation */ CompletableFuture publishUnloadEventAsync(Unload unload); /** - * Asynchronously publishes the bundle split event to the system topic in this channel. + * Asynchronously publishes the bundle split event to this channel. * @param split (split specification object) - * @return the completable future object staged from the event message sendAsync. + * @return a future to track the completion of the operation */ CompletableFuture publishSplitEventAsync(Split split); @@ -200,18 +140,24 @@ public interface ServiceUnitStateChannel extends Closeable { List getMetrics(); /** - * Add a state change listener. + * Adds a state change listener. * * @param listener State change listener. */ void listen(StateChangeListener listener); /** - * Returns service unit ownership entry set. - * @return a set of service unit ownership entries + * Asynchronously returns service unit ownership entry set. + * @return a set of service unit ownership entries to track the completion of the operation */ Set> getOwnershipEntrySet(); + /** + * Asynchronously returns service units owned by this broker. + * @return a set of owned service units to track the completion of the operation + */ + Set getOwnedServiceUnits(); + /** * Schedules ownership monitor to periodically check and correct invalid ownership states. */ @@ -223,7 +169,7 @@ public interface ServiceUnitStateChannel extends Closeable { void cancelOwnershipMonitor(); /** - * Cleans the service unit ownerships from the current broker's channel. + * Cleans(gives up) any service unit ownerships from this broker. */ void cleanOwnerships(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index 5893fc4924413..ddbc9eacac921 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -32,6 +32,7 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.isInFlightState; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.Closed; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.Constructed; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.Disabled; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.LeaderElectionServiceStarted; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.Started; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Assign; @@ -42,7 +43,6 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MetadataState.Unstable; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; -import static org.apache.pulsar.common.topics.TopicCompactionStrategy.TABLE_VIEW_TAG; import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionLost; import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionReestablished; import com.google.common.annotations.VisibleForTesting; @@ -86,39 +86,27 @@ import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; -import org.apache.pulsar.client.api.CompressionType; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceBundleFactory; import org.apache.pulsar.common.naming.NamespaceBundleSplitAlgorithm; import org.apache.pulsar.common.naming.NamespaceBundles; -import org.apache.pulsar.common.naming.TopicDomain; -import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.common.topics.TopicCompactionStrategy; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.Reflections; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.extended.SessionEvent; @Slf4j public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { - public static final String TOPIC = TopicName.get( - TopicDomain.persistent.value(), - SYSTEM_NAMESPACE, - "loadbalancer-service-unit-state").toString(); - public static final CompressionType MSG_COMPRESSION_TYPE = CompressionType.ZSTD; private static final int OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS = 5000; private static final int OWNERSHIP_CLEAN_UP_WAIT_RETRY_DELAY_IN_MILLIS = 100; public static final long VERSION_ID_INIT = 1; // initial versionId public static final long MAX_CLEAN_UP_DELAY_TIME_IN_SECS = 3 * 60; // 3 mins private static final long MIN_CLEAN_UP_DELAY_TIME_IN_SECS = 0; // 0 secs to clean immediately private static final long MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS = 10; - private static final int MAX_OUTSTANDING_PUB_MESSAGES = 500; private static final long MAX_OWNED_BUNDLE_COUNT_DELAY_TIME_IN_MILLIS = 10 * 60 * 1000; private final PulsarService pulsar; private final ServiceConfiguration config; @@ -129,8 +117,8 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private final StateChangeListeners stateChangeListeners; private BrokerRegistry brokerRegistry; private LeaderElectionService leaderElectionService; - private TableView tableview; - private Producer producer; + + private ServiceUnitStateTableView tableview; private ScheduledFuture monitorTask; private SessionEvent lastMetadataSessionEvent = SessionReestablished; private long lastMetadataSessionEventTimestamp = 0; @@ -166,7 +154,8 @@ public enum EventType { public static class Counters { private final AtomicLong total; private final AtomicLong failure; - public Counters(){ + + public Counters() { total = new AtomicLong(); failure = new AtomicLong(); } @@ -181,11 +170,13 @@ enum ChannelState { Closed(0), Constructed(1), LeaderElectionServiceStarted(2), - Started(3); + Started(3), + Disabled(4); ChannelState(int id) { this.id = id; } + int id; } @@ -234,6 +225,7 @@ public ServiceUnitStateChannelImpl(PulsarService pulsar) { this.channelState = Constructed; } + @Override public void scheduleOwnershipMonitor() { if (monitorTask == null) { this.monitorTask = this.pulsar.getLoadManagerExecutor() @@ -251,6 +243,7 @@ public void scheduleOwnershipMonitor() { } } + @Override public void cancelOwnershipMonitor() { if (monitorTask != null) { monitorTask.cancel(false); @@ -262,7 +255,7 @@ public void cancelOwnershipMonitor() { @Override public void cleanOwnerships() { - doCleanup(brokerId); + doCleanup(brokerId, true); } @Override @@ -270,6 +263,22 @@ public synchronized boolean started() { return validateChannelState(Started, true); } + private ServiceUnitStateTableView createServiceUnitStateTableView() { + ServiceConfiguration conf = pulsar.getConfiguration(); + try { + ServiceUnitStateTableView tableview = + Reflections.createInstance(conf.getLoadManagerServiceUnitStateTableViewClassName(), + ServiceUnitStateTableView.class, Thread.currentThread().getContextClassLoader()); + log.info("Created service unit state tableview: {}", tableview.getClass().getCanonicalName()); + return tableview; + } catch (Throwable e) { + log.error("Error when trying to create service unit state tableview: {}.", + conf.getLoadManagerServiceUnitStateTableViewClassName(), e); + throw e; + } + } + + @Override public synchronized void start() throws PulsarServerException { if (!validateChannelState(LeaderElectionServiceStarted, false)) { throw new IllegalStateException("Invalid channel state:" + channelState.name()); @@ -289,55 +298,17 @@ public synchronized void start() throws PulsarServerException { } this.channelState = LeaderElectionServiceStarted; - if (producer != null) { - producer.close(); - if (debug) { - log.info("Closed the channel producer."); - } - } - PulsarClusterMetadataSetup.createTenantIfAbsent - (pulsar.getPulsarResources(), SYSTEM_NAMESPACE.getTenant(), config.getClusterName()); + (pulsar.getPulsarResources(), SYSTEM_NAMESPACE.getTenant(), + pulsar.getConfiguration().getClusterName()); PulsarClusterMetadataSetup.createNamespaceIfAbsent - (pulsar.getPulsarResources(), SYSTEM_NAMESPACE, config.getClusterName(), - config.getDefaultNumberOfNamespaceBundles()); - - ExtensibleLoadManagerImpl.createSystemTopic(pulsar, TOPIC); + (pulsar.getPulsarResources(), SYSTEM_NAMESPACE, pulsar.getConfiguration().getClusterName(), + pulsar.getConfiguration().getDefaultNumberOfNamespaceBundles()); - producer = pulsar.getClient().newProducer(schema) - .enableBatching(true) - .compressionType(MSG_COMPRESSION_TYPE) - .maxPendingMessages(MAX_OUTSTANDING_PUB_MESSAGES) - .blockIfQueueFull(true) - .topic(TOPIC) - .create(); - - if (debug) { - log.info("Successfully started the channel producer."); - } + tableview = createServiceUnitStateTableView(); + tableview.start(pulsar, this::handleEvent, this::handleExisting); - if (tableview != null) { - tableview.close(); - if (debug) { - log.info("Closed the channel tableview."); - } - } - tableview = pulsar.getClient().newTableViewBuilder(schema) - .topic(TOPIC) - .loadConf(Map.of( - "topicCompactionStrategyClassName", - ServiceUnitStateCompactionStrategy.class.getName())) - .create(); - tableview.listen(this::handleEvent); - tableview.forEach(this::handleExisting); - var strategy = (ServiceUnitStateCompactionStrategy) TopicCompactionStrategy.getInstance(TABLE_VIEW_TAG); - if (strategy == null) { - String err = TABLE_VIEW_TAG + "tag TopicCompactionStrategy is null."; - log.error(err); - throw new IllegalStateException(err); - } - strategy.setSkippedMsgHandler((key, value) -> handleSkippedEvent(key)); if (debug) { log.info("Successfully started the channel tableview."); } @@ -378,23 +349,15 @@ protected LeaderElectionService getLeaderElectionService() { .get().getLeaderElectionService(); } + @Override public synchronized void close() throws PulsarServerException { channelState = Closed; - boolean debug = debug(); try { leaderElectionService = null; + if (tableview != null) { tableview.close(); tableview = null; - if (debug) { - log.info("Successfully closed the channel tableview."); - } - } - - if (producer != null) { - producer.close(); - producer = null; - log.info("Successfully closed the channel producer."); } if (brokerRegistry != null) { @@ -432,6 +395,7 @@ private boolean debug() { return ExtensibleLoadManagerImpl.debug(config, log); } + @Override public CompletableFuture> getChannelOwnerAsync() { if (!validateChannelState(LeaderElectionServiceStarted, true)) { return CompletableFuture.failedFuture( @@ -442,6 +406,7 @@ public CompletableFuture> getChannelOwnerAsync() { .thenApply(leader -> leader.map(LeaderBroker::getBrokerId)); } + @Override public CompletableFuture isChannelOwnerAsync() { return getChannelOwnerAsync().thenApply(owner -> { if (owner.isPresent()) { @@ -453,19 +418,14 @@ public CompletableFuture isChannelOwnerAsync() { } }); } - - public boolean isChannelOwner() { - try { - return isChannelOwnerAsync().get( - MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS, TimeUnit.SECONDS); - } catch (InterruptedException | ExecutionException | TimeoutException e) { - String msg = "Failed to get the channel owner."; - log.error(msg, e); - throw new RuntimeException(msg, e); - } + @Override + public boolean isChannelOwner() throws ExecutionException, InterruptedException, TimeoutException { + return isChannelOwnerAsync().get( + MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS, TimeUnit.SECONDS); } - public boolean isOwner(String serviceUnit, String targetBroker) { + @Override + public boolean isOwner(String serviceUnit, String targetBrokerId) { if (!validateChannelState(Started, true)) { throw new IllegalStateException("Invalid channel state:" + channelState.name()); } @@ -474,12 +434,13 @@ public boolean isOwner(String serviceUnit, String targetBroker) { return false; } var owner = ownerFuture.join(); - if (owner.isPresent() && StringUtils.equals(targetBroker, owner.get())) { + if (owner.isPresent() && StringUtils.equals(targetBrokerId, owner.get())) { return true; } return false; } + @Override public boolean isOwner(String serviceUnit) { return isOwner(serviceUnit, brokerId); } @@ -512,13 +473,22 @@ private CompletableFuture> getActiveOwnerAsync( }).thenApply(Optional::ofNullable); } + /** + * Case 1: If the service unit is owned, it returns the completed future object with the current owner. + * Case 2: If the service unit's assignment is ongoing, it returns the non-completed future object. + * Sub-case1: If the assigned broker is available and finally takes the ownership, + * the future object will complete and return the owner broker. + * Sub-case2: If the assigned broker does not take the ownership in time, + * the future object will time out. + * Case 3: If none of them, it returns Optional.empty(). + */ + @Override public CompletableFuture> getOwnerAsync(String serviceUnit) { if (!validateChannelState(Started, true)) { return CompletableFuture.failedFuture( new IllegalStateException("Invalid channel state:" + channelState.name())); } - - ServiceUnitStateData data = tableview.get(serviceUnit); + var data = tableview.get(serviceUnit); ServiceUnitState state = state(data); ownerLookUpCounters.get(state).getTotal().incrementAndGet(); switch (state) { @@ -549,18 +519,19 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) { } default -> { ownerLookUpCounters.get(state).getFailure().incrementAndGet(); - String errorMsg = String.format("Failed to process service unit state data: %s when get owner.", data); + String errorMsg = + String.format("Failed to process service unit state data: %s when get owner.", data); log.error(errorMsg); return CompletableFuture.failedFuture(new IllegalStateException(errorMsg)); } } } - private Optional getOwner(String serviceUnit) { + private Optional getOwnerNow(String serviceUnit) { if (!validateChannelState(Started, true)) { throw new IllegalStateException("Invalid channel state:" + channelState.name()); } - ServiceUnitStateData data = tableview.get(serviceUnit); + var data = tableview.get(serviceUnit); ServiceUnitState state = state(data); switch (state) { case Owned -> { @@ -578,13 +549,14 @@ private Optional getOwner(String serviceUnit) { } } + @Override public Optional getAssigned(String serviceUnit) { if (!validateChannelState(Started, true)) { return Optional.empty(); } - ServiceUnitStateData data = tableview.get(serviceUnit); + var data = tableview.get(serviceUnit); if (data == null) { return Optional.empty(); } @@ -607,22 +579,23 @@ public Optional getAssigned(String serviceUnit) { return Optional.empty(); } default -> { - log.warn("Trying to get the assigned broker from unknown state:{} serviceUnit:{}", state, serviceUnit); + log.warn("Trying to get the assigned broker from unknown state:{} serviceUnit:{}", state, + serviceUnit); return Optional.empty(); } } } - private long getNextVersionId(String serviceUnit) { - var data = tableview.get(serviceUnit); - return getNextVersionId(data); + private Long getNextVersionId(String serviceUnit) { + return getNextVersionId(tableview.get(serviceUnit)); } private long getNextVersionId(ServiceUnitStateData data) { return data == null ? VERSION_ID_INIT : data.versionId() + 1; } - public CompletableFuture publishAssignEventAsync(String serviceUnit, String broker) { + @Override + public CompletableFuture publishAssignEventAsync(String serviceUnit, String brokerId) { if (!validateChannelState(Started, true)) { return CompletableFuture.failedFuture( new IllegalStateException("Invalid channel state:" + channelState.name())); @@ -631,7 +604,8 @@ public CompletableFuture publishAssignEventAsync(String serviceUnit, Str eventCounters.get(eventType).getTotal().incrementAndGet(); CompletableFuture getOwnerRequest = dedupeGetOwnerRequest(serviceUnit); - pubAsync(serviceUnit, new ServiceUnitStateData(Assigning, broker, getNextVersionId(serviceUnit))) + pubAsync(serviceUnit, + new ServiceUnitStateData(Assigning, brokerId, getNextVersionId(serviceUnit))) .whenComplete((__, ex) -> { if (ex != null) { getOwnerRequests.remove(serviceUnit, getOwnerRequest); @@ -641,11 +615,12 @@ public CompletableFuture publishAssignEventAsync(String serviceUnit, Str eventCounters.get(eventType).getFailure().incrementAndGet(); } }); + return getOwnerRequest; } private CompletableFuture publishOverrideEventAsync(String serviceUnit, - ServiceUnitStateData override) { + ServiceUnitStateData override) { if (!validateChannelState(Started, true)) { throw new IllegalStateException("Invalid channel state:" + channelState.name()); } @@ -697,12 +672,17 @@ public CompletableFuture publishSplitEventAsync(Split split) { } private void handleEvent(String serviceUnit, ServiceUnitStateData data) { + long totalHandledRequests = getHandlerTotalCounter(data).incrementAndGet(); if (debug()) { log.info("{} received a handle request for serviceUnit:{}, data:{}. totalHandledRequests:{}", brokerId, serviceUnit, data, totalHandledRequests); } + if (channelState == Disabled) { + return; + } + ServiceUnitState state = state(data); try { switch (state) { @@ -715,7 +695,7 @@ private void handleEvent(String serviceUnit, ServiceUnitStateData data) { case Init -> handleInitEvent(serviceUnit); default -> throw new IllegalStateException("Failed to handle channel data:" + data); } - } catch (Throwable e){ + } catch (Throwable e) { log.error("Failed to handle the event. serviceUnit:{}, data:{}, handlerFailureCount:{}", serviceUnit, data, getHandlerFailureCounter(data).incrementAndGet(), e); throw e; @@ -914,26 +894,24 @@ private void handleInitEvent(String serviceUnit) { log(null, serviceUnit, null, null); } - private CompletableFuture pubAsync(String serviceUnit, ServiceUnitStateData data) { - CompletableFuture future = new CompletableFuture<>(); - producer.newMessage() - .key(serviceUnit) - .value(data) - .sendAsync() - .whenComplete((messageId, e) -> { + private CompletableFuture pubAsync(String serviceUnit, ServiceUnitStateData data) { + return tableview.put(serviceUnit, data) + .whenComplete((__, e) -> { if (e != null) { log.error("Failed to publish the message: serviceUnit:{}, data:{}", serviceUnit, data, e); - future.completeExceptionally(e); - } else { - future.complete(messageId); } }); - return future; } - private CompletableFuture tombstoneAsync(String serviceUnit) { - return pubAsync(serviceUnit, null); + private CompletableFuture tombstoneAsync(String serviceUnit) { + return tableview.delete(serviceUnit) + .whenComplete((__, e) -> { + if (e != null) { + log.error("Failed to tombstone the serviceUnit:{}}", + serviceUnit, e); + } + }); } private boolean isTargetBroker(String broker) { @@ -943,11 +921,12 @@ private boolean isTargetBroker(String broker) { return broker.equals(brokerId); } + private CompletableFuture deferGetOwner(String serviceUnit) { var future = new CompletableFuture().orTimeout(inFlightStateWaitingTimeInMillis, TimeUnit.MILLISECONDS) .exceptionally(e -> { - var ownerAfter = getOwner(serviceUnit); + var ownerAfter = getOwnerNow(serviceUnit); log.warn("{} failed to wait for owner for serviceUnit:{}; Trying to " + "return the current owner:{}", brokerId, serviceUnit, ownerAfter, e); @@ -967,7 +946,7 @@ private CompletableFuture dedupeGetOwnerRequest(String serviceUnit) { var requested = new MutableObject>(); try { return getOwnerRequests.computeIfAbsent(serviceUnit, k -> { - var ownerBefore = getOwner(serviceUnit); + var ownerBefore = getOwnerNow(serviceUnit); if (ownerBefore != null && ownerBefore.isPresent()) { // Here, we do the broker active check first with the computeIfAbsent lock requested.setValue(brokerRegistry.lookupAsync(ownerBefore.get()) @@ -1073,7 +1052,6 @@ private CompletableFuture splitServiceUnit(String serviceUnit, ServiceUnit } - @VisibleForTesting protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, NamespaceBundleFactory bundleFactory, @@ -1093,7 +1071,7 @@ protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, .thenAccept(__ -> // Update bundled_topic cache for load-report-generation pulsar.getBrokerService().refreshTopicToStatsMaps(parentBundle)) .thenAccept(__ -> pubAsync(parentBundle.toString(), new ServiceUnitStateData( - Deleted, null, parentData.sourceBroker(), getNextVersionId(parentData)))) + Deleted, null, parentData.sourceBroker(), getNextVersionId(parentData)))) .thenAccept(__ -> { double splitBundleTime = TimeUnit.NANOSECONDS.toMillis((System.nanoTime() - startTime)); log.info("Successfully split {} parent namespace-bundle to {} in {} ms", @@ -1109,7 +1087,7 @@ protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService, log.warn("Failed to update bundle range in metadata store. Retrying {} th / {} limit", counter.get(), NamespaceService.BUNDLE_SPLIT_RETRY_LIMIT, ex); pulsar.getExecutor().schedule(() -> splitServiceUnitOnceAndRetry( - namespaceService, bundleFactory, algorithm, parentBundle, childBundles, + namespaceService, bundleFactory, algorithm, parentBundle, childBundles, boundaries, parentData, counter, startTime, completionFuture), 100, MILLISECONDS); } else { @@ -1156,45 +1134,43 @@ private CompletableFuture getSplitNamespaceBundles(NamespaceSe NamespaceBundle parentBundle, List childBundles, List boundaries) { - CompletableFuture future = new CompletableFuture(); final var debug = debug(); - var targetNsBundle = bundleFactory.getBundles(parentBundle.getNamespaceObject()); - boolean found = false; - try { - targetNsBundle.validateBundle(parentBundle); - } catch (IllegalArgumentException e) { - if (debug) { - log.info("Namespace bundles do not contain the parent bundle:{}", - parentBundle); - } - for (var childBundle : childBundles) { - try { - targetNsBundle.validateBundle(childBundle); - if (debug) { - log.info("Namespace bundles contain the child bundle:{}", - childBundle); + return bundleFactory.getBundlesAsync(parentBundle.getNamespaceObject()) + .thenCompose(targetNsBundle -> { + boolean found = false; + try { + targetNsBundle.validateBundle(parentBundle); + } catch (IllegalArgumentException e) { + if (debug) { + log.info("Namespace bundles do not contain the parent bundle:{}", + parentBundle); + } + for (var childBundle : childBundles) { + try { + targetNsBundle.validateBundle(childBundle); + if (debug) { + log.info("Namespace bundles contain the child bundle:{}", + childBundle); + } + } catch (Exception ex) { + throw FutureUtil.wrapToCompletionException( + new BrokerServiceException.ServiceUnitNotReadyException( + "Namespace bundles do not contain the child bundle:" + childBundle, e)); + } + } + found = true; + } catch (Exception e) { + throw FutureUtil.wrapToCompletionException( + new BrokerServiceException.ServiceUnitNotReadyException( + "Failed to validate the parent bundle in the namespace bundles.", e)); } - } catch (Exception ex) { - future.completeExceptionally( - new BrokerServiceException.ServiceUnitNotReadyException( - "Namespace bundles do not contain the child bundle:" + childBundle, e)); - return future; - } - } - found = true; - } catch (Exception e) { - future.completeExceptionally( - new BrokerServiceException.ServiceUnitNotReadyException( - "Failed to validate the parent bundle in the namespace bundles.", e)); - return future; - } - if (found) { - future.complete(targetNsBundle); - return future; - } else { - return namespaceService.getSplitBoundary(parentBundle, algorithm, boundaries) - .thenApply(splitBundlesPair -> splitBundlesPair.getLeft()); - } + if (found) { + return CompletableFuture.completedFuture(targetNsBundle); + } else { + return namespaceService.getSplitBoundary(parentBundle, algorithm, boundaries) + .thenApply(splitBundlesPair -> splitBundlesPair.getLeft()); + } + }); } private CompletableFuture updateSplitNamespaceBundlesAsync( @@ -1215,7 +1191,12 @@ private CompletableFuture updateSplitNamespaceBundlesAsync( }); } - public void handleMetadataSessionEvent(SessionEvent e) { + /** + * The stability of the metadata connection is important + * to determine how to handle the broker deletion(unavailable) event notified from the metadata store. + */ + @VisibleForTesting + protected void handleMetadataSessionEvent(SessionEvent e) { if (e == SessionReestablished || e == SessionLost) { lastMetadataSessionEvent = e; lastMetadataSessionEventTimestamp = System.currentTimeMillis(); @@ -1224,7 +1205,30 @@ public void handleMetadataSessionEvent(SessionEvent e) { } } - public void handleBrokerRegistrationEvent(String broker, NotificationType type) { + /** + * Case 1: If NotificationType is Deleted, + * it will schedule a clean-up operation to release the ownerships of the deleted broker. + * + * Sub-case1: If the metadata connection has been stable for long time, + * it will immediately execute the cleanup operation to guarantee high-availability. + * + * Sub-case2: If the metadata connection has been stable only for short time, + * it will defer the clean-up operation for some time and execute it. + * This is to gracefully handle the case when metadata connection is flaky -- + * If the deleted broker comes back very soon, + * we better cancel the clean-up operation for high-availability. + * + * Sub-case3: If the metadata connection is unstable, + * it will not schedule the clean-up operation, as the broker-metadata connection is lost. + * The brokers will continue to serve existing topics connections, + * and we better not to interrupt the existing topic connections for high-availability. + * + * + * Case 2: If NotificationType is Created, + * it will cancel any scheduled clean-up operation if still not executed. + */ + @VisibleForTesting + protected void handleBrokerRegistrationEvent(String broker, NotificationType type) { if (type == NotificationType.Created) { log.info("BrokerRegistry detected the broker:{} registry has been created.", broker); handleBrokerCreationEvent(broker); @@ -1263,8 +1267,13 @@ private void handleBrokerCreationEvent(String broker) { } private void handleBrokerDeletionEvent(String broker) { - if (!isChannelOwner()) { - log.warn("This broker is not the leader now. Ignoring BrokerDeletionEvent for broker {}.", broker); + try { + if (!isChannelOwner()) { + log.warn("This broker is not the leader now. Ignoring BrokerDeletionEvent for broker {}.", broker); + return; + } + } catch (Exception e) { + log.error("Failed to handle broker deletion event.", e); return; } MetadataState state = getMetadataState(); @@ -1291,7 +1300,7 @@ private void scheduleCleanup(String broker, long delayInSecs) { var future = CompletableFuture .runAsync(() -> { try { - doCleanup(broker); + doCleanup(broker, false); } catch (Throwable e) { log.error("Failed to run the cleanup job for the broker {}, " + "totalCleanupErrorCnt:{}.", @@ -1316,7 +1325,9 @@ private void scheduleCleanup(String broker, long delayInSecs) { } - private void overrideOwnership(String serviceUnit, ServiceUnitStateData orphanData, String inactiveBroker) { + private void overrideOwnership(String serviceUnit, ServiceUnitStateData orphanData, String inactiveBroker, + boolean gracefully) { + final var version = getNextVersionId(orphanData); try { selectBroker(serviceUnit, inactiveBroker) @@ -1330,9 +1341,14 @@ private void overrideOwnership(String serviceUnit, ServiceUnitStateData orphanDa } else if (orphanData.state() == Owned) { // if Owned, set orphan.dstBroker() as source to clean it up in case it is still // alive. - return new ServiceUnitStateData(Owned, selectedBroker, - selectedBroker.equals(orphanData.dstBroker()) ? null : - orphanData.dstBroker(), + var sourceBroker = selectedBroker.equals(orphanData.dstBroker()) ? null : + orphanData.dstBroker(); + // if gracefully, try to release ownership first + var overrideState = gracefully && sourceBroker != null ? Releasing : Owned; + return new ServiceUnitStateData( + overrideState, + selectedBroker, + sourceBroker, true, version); } else { // if Assigning or Releasing, set orphan.sourceBroker() as source @@ -1395,7 +1411,7 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max System.currentTimeMillis() - started); } - private synchronized void doCleanup(String broker) { + private synchronized void doCleanup(String broker, boolean gracefully) { try { if (getChannelOwnerAsync().get(MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS, TimeUnit.SECONDS) .isEmpty()) { @@ -1422,14 +1438,14 @@ private synchronized void doCleanup(String broker) { if (serviceUnit.startsWith(SYSTEM_NAMESPACE.toString())) { orphanSystemServiceUnits.put(serviceUnit, stateData); } else { - overrideOwnership(serviceUnit, stateData, broker); + overrideOwnership(serviceUnit, stateData, broker, gracefully); } orphanServiceUnitCleanupCnt++; } } try { - producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + tableview.flush(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS); } catch (Exception e) { log.error("Failed to flush the in-flight non-system bundle override messages.", e); } @@ -1449,11 +1465,11 @@ private synchronized void doCleanup(String broker) { // clean system bundles in the end for (var orphanSystemServiceUnit : orphanSystemServiceUnits.entrySet()) { log.info("Overriding orphan system service unit:{}", orphanSystemServiceUnit.getKey()); - overrideOwnership(orphanSystemServiceUnit.getKey(), orphanSystemServiceUnit.getValue(), broker); + overrideOwnership(orphanSystemServiceUnit.getKey(), orphanSystemServiceUnit.getValue(), broker, gracefully); } try { - producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + tableview.flush(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS); } catch (Exception e) { log.error("Failed to flush the in-flight system bundle override messages.", e); } @@ -1486,8 +1502,13 @@ private CompletableFuture> selectBroker(String serviceUnit, Str @VisibleForTesting protected void monitorOwnerships(List brokers) { - if (!isChannelOwner()) { - log.warn("This broker is not the leader now. Skipping ownership monitor."); + try { + if (!isChannelOwner()) { + log.warn("This broker is not the leader now. Skipping ownership monitor."); + return; + } + } catch (Exception e) { + log.error("Failed to monitor ownerships", e); return; } @@ -1571,13 +1592,13 @@ protected void monitorOwnerships(List brokers) { for (var etr : timedOutInFlightStateServiceUnits.entrySet()) { var orphanServiceUnit = etr.getKey(); var orphanData = etr.getValue(); - overrideOwnership(orphanServiceUnit, orphanData, null); + overrideOwnership(orphanServiceUnit, orphanData, null, false); orphanServiceUnitCleanupCnt++; } } try { - producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + tableview.flush(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS); } catch (Exception e) { log.error("Failed to flush the in-flight messages.", e); } @@ -1638,10 +1659,8 @@ private int getTotalOwnedServiceUnitCnt() { if (lastOwnEventHandledAt > lastOwnedServiceUnitCountAt || now - lastOwnedServiceUnitCountAt > MAX_OWNED_BUNDLE_COUNT_DELAY_TIME_IN_MILLIS) { int cnt = 0; - for (var data : tableview.values()) { - if (data.state() == Owned && isTargetBroker(data.dstBroker())) { - cnt++; - } + for (var e : tableview.ownedServiceUnits()) { + cnt++; } lastOwnedServiceUnitCountAt = now; totalOwnedServiceUnitCnt = cnt; @@ -1787,7 +1806,25 @@ public Set> getOwnershipEntrySet() { return tableview.entrySet(); } + @Override + public Set getOwnedServiceUnits() { + if (!validateChannelState(Started, true)) { + throw new IllegalStateException("Invalid channel state:" + channelState.name()); + } + return tableview.ownedServiceUnits(); + } + public static ServiceUnitStateChannel get(PulsarService pulsar) { return ExtensibleLoadManagerImpl.get(pulsar.getLoadManager().get()).getServiceUnitStateChannel(); } + + @VisibleForTesting + protected void disable() { + channelState = Disabled; + } + + @VisibleForTesting + protected void enable() { + channelState = Started; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java index 307d3a4acb175..e85134e611632 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java @@ -75,4 +75,19 @@ public ServiceUnitStateData(ServiceUnitState state, String dstBroker, boolean fo public static ServiceUnitState state(ServiceUnitStateData data) { return data == null ? ServiceUnitState.Init : data.state(); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + ServiceUnitStateData that = (ServiceUnitStateData) o; + + return versionId == that.versionId; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategy.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java similarity index 82% rename from pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategy.java rename to pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java index 6a98b79be81d0..b1dbb6fac8709 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategy.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java @@ -20,21 +20,27 @@ import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.MetadataStore; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.SystemTopic; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; import com.google.common.annotations.VisibleForTesting; import java.util.function.BiConsumer; +import lombok.Setter; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.topics.TopicCompactionStrategy; -public class ServiceUnitStateCompactionStrategy implements TopicCompactionStrategy { +public class ServiceUnitStateDataConflictResolver implements TopicCompactionStrategy { private final Schema schema; private BiConsumer skippedMsgHandler; private boolean checkBrokers = true; - public ServiceUnitStateCompactionStrategy() { + @Setter + private ServiceUnitState.StorageType storageType = SystemTopic; + + public ServiceUnitStateDataConflictResolver() { schema = Schema.JSON(ServiceUnitStateData.class); } @@ -70,8 +76,16 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to } else if (from.versionId() >= to.versionId()) { return true; } else if (from.versionId() < to.versionId() - 1) { // Compacted - return false; + // If the system topic is compacted, to.versionId can be bigger than from.versionId by 2 or more. + // e.g. (Owned, v1) -> (Owned, v3) + return storageType != SystemTopic; } // else from.versionId() == to.versionId() - 1 // continue to check further + } else { + // If `from` is null, to.versionId should start at 1 over metadata store. + // In this case, to.versionId can be bigger than 1 over the system topic, if compacted. + if (storageType == MetadataStore) { + return to.versionId() != 1; + } } if (to.force()) { @@ -80,7 +94,7 @@ public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to ServiceUnitState prevState = state(from); ServiceUnitState state = state(to); - if (!ServiceUnitState.isValidTransition(prevState, state)) { + if (!ServiceUnitState.isValidTransition(prevState, state, storageType)) { return true; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateMetadataStoreTableViewImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateMetadataStoreTableViewImpl.java new file mode 100644 index 0000000000000..f488b31c77415 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateMetadataStoreTableViewImpl.java @@ -0,0 +1,155 @@ +/* + * 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.loadbalance.extensions.channel; + +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.MetadataStore; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.MetadataStoreTableView; +import org.apache.pulsar.metadata.tableview.impl.MetadataStoreTableViewImpl; + +@Slf4j +public class ServiceUnitStateMetadataStoreTableViewImpl extends ServiceUnitStateTableViewBase { + public static final String PATH_PREFIX = "/service_unit_state"; + private static final String VALID_PATH_REG_EX = "^\\/service_unit_state\\/.*\\/0x[0-9a-fA-F]{8}_0x[0-9a-fA-F]{8}$"; + private static final Pattern VALID_PATH_PATTERN; + + static { + try { + VALID_PATH_PATTERN = Pattern.compile(VALID_PATH_REG_EX); + } catch (PatternSyntaxException error) { + log.error("Invalid regular expression {}", VALID_PATH_REG_EX, error); + throw new IllegalArgumentException(error); + } + } + private ServiceUnitStateDataConflictResolver conflictResolver; + private volatile MetadataStoreTableView tableview; + + public void start(PulsarService pulsar, + BiConsumer tailItemListener, + BiConsumer existingItemListener) + throws MetadataStoreException { + init(pulsar); + conflictResolver = new ServiceUnitStateDataConflictResolver(); + conflictResolver.setStorageType(MetadataStore); + tableview = new MetadataStoreTableViewImpl<>(ServiceUnitStateData.class, + pulsar.getBrokerId(), + pulsar.getLocalMetadataStore(), + PATH_PREFIX, + this::resolveConflict, + this::validateServiceUnitPath, + List.of(this::updateOwnedServiceUnits, tailItemListener), + List.of(this::updateOwnedServiceUnits, existingItemListener), + TimeUnit.SECONDS.toMillis(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds()) + ); + tableview.start(); + + } + + protected boolean resolveConflict(ServiceUnitStateData prev, ServiceUnitStateData cur) { + return !conflictResolver.shouldKeepLeft(prev, cur); + } + + + protected boolean validateServiceUnitPath(String path) { + try { + var matcher = VALID_PATH_PATTERN.matcher(path); + return matcher.matches(); + } catch (Exception e) { + return false; + } + } + + + @Override + public void close() throws IOException { + if (tableview != null) { + tableview = null; + log.info("Successfully closed the channel tableview."); + } + } + + private boolean isValidState() { + if (tableview == null) { + return false; + } + return true; + } + + @Override + public ServiceUnitStateData get(String key) { + if (!isValidState()) { + throw new IllegalStateException(INVALID_STATE_ERROR_MSG); + } + return tableview.get(key); + } + + @Override + public CompletableFuture put(String key, @NonNull ServiceUnitStateData value) { + if (!isValidState()) { + return CompletableFuture.failedFuture(new IllegalStateException(INVALID_STATE_ERROR_MSG)); + } + return tableview.put(key, value).exceptionally(e -> { + if (e.getCause() instanceof MetadataStoreTableView.ConflictException) { + return null; + } + throw FutureUtil.wrapToCompletionException(e); + }); + } + + @Override + public void flush(long waitDurationInMillis) { + // no-op + } + + @Override + public CompletableFuture delete(String key) { + if (!isValidState()) { + return CompletableFuture.failedFuture(new IllegalStateException(INVALID_STATE_ERROR_MSG)); + } + return tableview.delete(key).exceptionally(e -> { + if (e.getCause() instanceof MetadataStoreException.NotFoundException) { + return null; + } + throw FutureUtil.wrapToCompletionException(e); + }); + } + + + @Override + public Set> entrySet() { + if (!isValidState()) { + throw new IllegalStateException(INVALID_STATE_ERROR_MSG); + } + return tableview.entrySet(); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableView.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableView.java new file mode 100644 index 0000000000000..5ac57fe5c19c6 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableView.java @@ -0,0 +1,113 @@ +/* + * 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.loadbalance.extensions.channel; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.function.BiConsumer; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.common.naming.NamespaceBundle; + +/** + * Given that the ServiceUnitStateChannel event-sources service unit (bundle) ownership states via a persistent store + * and reacts to ownership changes, the ServiceUnitStateTableView provides an interface to the + * ServiceUnitStateChannel's persistent store and its locally replicated ownership view (tableview) with listener + * registration. It initially populates its local table view by scanning existing items in the remote store. The + * ServiceUnitStateTableView receives notifications whenever ownership states are updated in the remote store, and + * upon notification, it applies the updates to its local tableview with the listener logic. + */ +public interface ServiceUnitStateTableView extends Closeable { + + /** + * Starts the tableview. + * It initially populates its local table view by scanning existing items in the remote store, and it starts + * listening to service unit ownership changes from the remote store. + * @param pulsar pulsar service reference + * @param tailItemListener listener to listen tail(newly updated) items + * @param existingItemListener listener to listen existing items + * @throws IOException if it fails to init the tableview. + */ + void start(PulsarService pulsar, + BiConsumer tailItemListener, + BiConsumer existingItemListener) throws IOException; + + + /** + * Closes the tableview. + * @throws IOException if it fails to close the tableview. + */ + void close() throws IOException; + + /** + * Gets one item from the local tableview. + * @param key the key to get + * @return value if exists. Otherwise, null. + */ + ServiceUnitStateData get(String key); + + /** + * Tries to put the item in the persistent store. + * If it completes, all peer tableviews (including the local one) will be notified and be eventually consistent + * with this put value. + * + * It ignores put operation if the input value conflicts with the existing one in the persistent store. + * + * @param key the key to put + * @param value the value to put + * @return a future to track the completion of the operation + */ + CompletableFuture put(String key, ServiceUnitStateData value); + + /** + * Tries to delete the item from the persistent store. + * All peer tableviews (including the local one) will be notified and be eventually consistent with this deletion. + * + * It ignores delete operation if the key is not present in the persistent store. + * + * @param key the key to delete + * @return a future to track the completion of the operation + */ + CompletableFuture delete(String key); + + /** + * Returns the entry set of the items in the local tableview. + * @return entry set + */ + Set> entrySet(); + + /** + * Returns service units (namespace bundles) owned by this broker. + * @return a set of owned service units (namespace bundles) + */ + Set ownedServiceUnits(); + + /** + * Tries to flush any batched or buffered updates. + * @param waitDurationInMillis time to wait until complete. + * @throws ExecutionException + * @throws InterruptedException + * @throws TimeoutException + */ + void flush(long waitDurationInMillis) throws ExecutionException, InterruptedException, TimeoutException; +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewBase.java new file mode 100644 index 0000000000000..b690ef101e168 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewBase.java @@ -0,0 +1,92 @@ +/* + * 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.loadbalance.extensions.channel; + +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; +import java.util.Collections; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; +import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.common.naming.NamespaceBundle; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.metadata.api.MetadataStoreException; + +/** + * ServiceUnitStateTableView base class. + */ +@Slf4j +abstract class ServiceUnitStateTableViewBase implements ServiceUnitStateTableView { + protected static final String INVALID_STATE_ERROR_MSG = "The tableview has not been started."; + private final Map ownedServiceUnitsMap = new ConcurrentHashMap<>(); + private final Set ownedServiceUnits = Collections.unmodifiableSet(ownedServiceUnitsMap.keySet()); + private String brokerId; + private PulsarService pulsar; + protected void init(PulsarService pulsar) throws MetadataStoreException { + this.pulsar = pulsar; + this.brokerId = pulsar.getBrokerId(); + // Add heartbeat and SLA monitor namespace bundle. + NamespaceName heartbeatNamespace = + NamespaceService.getHeartbeatNamespace(brokerId, pulsar.getConfiguration()); + NamespaceName heartbeatNamespaceV2 = NamespaceService + .getHeartbeatNamespaceV2(brokerId, pulsar.getConfiguration()); + NamespaceName slaMonitorNamespace = NamespaceService + .getSLAMonitorNamespace(brokerId, pulsar.getConfiguration()); + try { + pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundleAsync(heartbeatNamespace) + .thenAccept(fullBundle -> ownedServiceUnitsMap.put(fullBundle, true)) + .thenCompose(__ -> pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundleAsync(heartbeatNamespaceV2)) + .thenAccept(fullBundle -> ownedServiceUnitsMap.put(fullBundle, true)) + .thenCompose(__ -> pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundleAsync(slaMonitorNamespace)) + .thenAccept(fullBundle -> ownedServiceUnitsMap.put(fullBundle, true)) + .thenApply(__ -> null).get(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), + TimeUnit.SECONDS); + } catch (Exception e) { + throw new MetadataStoreException(e); + } + } + + @Override + public Set ownedServiceUnits() { + return ownedServiceUnits; + } + + protected void updateOwnedServiceUnits(String key, ServiceUnitStateData val) { + NamespaceBundle namespaceBundle = LoadManagerShared.getNamespaceBundle(pulsar, key); + var state = ServiceUnitStateData.state(val); + ownedServiceUnitsMap.compute(namespaceBundle, (k, v) -> { + if (state == Owned && brokerId.equals(val.dstBroker())) { + return true; + } else if (state == Splitting && brokerId.equals(val.sourceBroker())) { + return true; + } else { + return null; + } + }); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java new file mode 100644 index 0000000000000..8dfaddcdabca1 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java @@ -0,0 +1,177 @@ +/* + * 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.loadbalance.extensions.channel; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; +import java.io.IOException; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.function.BiConsumer; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.TableView; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; + +@Slf4j +public class ServiceUnitStateTableViewImpl extends ServiceUnitStateTableViewBase { + + public static final String TOPIC = TopicName.get( + TopicDomain.persistent.value(), + SYSTEM_NAMESPACE, + "loadbalancer-service-unit-state").toString(); + private static final int MAX_OUTSTANDING_PUB_MESSAGES = 500; + public static final CompressionType MSG_COMPRESSION_TYPE = CompressionType.ZSTD; + private volatile Producer producer; + private volatile TableView tableview; + + public void start(PulsarService pulsar, + BiConsumer tailItemListener, + BiConsumer existingItemListener) throws IOException { + boolean debug = ExtensibleLoadManagerImpl.debug(pulsar.getConfiguration(), log); + + init(pulsar); + + var schema = Schema.JSON(ServiceUnitStateData.class); + + ExtensibleLoadManagerImpl.createSystemTopic(pulsar, TOPIC); + + if (producer != null) { + producer.close(); + if (debug) { + log.info("Closed the channel producer."); + } + } + + producer = pulsar.getClient().newProducer(schema) + .enableBatching(true) + .compressionType(MSG_COMPRESSION_TYPE) + .maxPendingMessages(MAX_OUTSTANDING_PUB_MESSAGES) + .blockIfQueueFull(true) + .topic(TOPIC) + .create(); + + if (debug) { + log.info("Successfully started the channel producer."); + } + + if (tableview != null) { + tableview.close(); + if (debug) { + log.info("Closed the channel tableview."); + } + } + + tableview = pulsar.getClient().newTableViewBuilder(schema) + .topic(TOPIC) + .loadConf(Map.of( + "topicCompactionStrategyClassName", + ServiceUnitStateDataConflictResolver.class.getName())) + .create(); + tableview.listen(this::updateOwnedServiceUnits); + tableview.listen(tailItemListener); + tableview.forEach(this::updateOwnedServiceUnits); + tableview.forEach(existingItemListener); + + } + + private boolean isValidState() { + if (tableview == null || producer == null) { + return false; + } + return true; + } + + + @Override + public void close() throws IOException { + + if (tableview != null) { + tableview.close(); + tableview = null; + log.info("Successfully closed the channel tableview."); + } + + if (producer != null) { + producer.close(); + producer = null; + log.info("Successfully closed the channel producer."); + } + } + + @Override + public ServiceUnitStateData get(String key) { + if (!isValidState()) { + throw new IllegalStateException(INVALID_STATE_ERROR_MSG); + } + return tableview.get(key); + } + + @Override + public CompletableFuture put(String key, ServiceUnitStateData value) { + if (!isValidState()) { + return CompletableFuture.failedFuture(new IllegalStateException(INVALID_STATE_ERROR_MSG)); + } + CompletableFuture future = new CompletableFuture<>(); + producer.newMessage() + .key(key) + .value(value) + .sendAsync() + .whenComplete((messageId, e) -> { + if (e != null) { + log.error("Failed to publish the message: serviceUnit:{}, data:{}", + key, value, e); + future.completeExceptionally(e); + } else { + future.complete(null); + } + }); + return future; + } + + @Override + public void flush(long waitDurationInMillis) throws InterruptedException, TimeoutException, ExecutionException { + if (!isValidState()) { + throw new IllegalStateException(INVALID_STATE_ERROR_MSG); + } + producer.flushAsync().get(waitDurationInMillis, MILLISECONDS); + } + + @Override + public CompletableFuture delete(String key) { + return put(key, null); + } + + @Override + public Set> entrySet() { + if (!isValidState()) { + throw new IllegalStateException(INVALID_STATE_ERROR_MSG); + } + return tableview.entrySet(); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewSyncer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewSyncer.java new file mode 100644 index 0000000000000..10ab39a66d279 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewSyncer.java @@ -0,0 +1,281 @@ +/* + * 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.loadbalance.extensions.channel; + +import static org.apache.pulsar.broker.ServiceConfiguration.ServiceUnitTableViewSyncerType.SystemTopicToMetadataStoreSyncer; +import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.COMPACTION_THRESHOLD; +import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.configureSystemTopics; +import com.fasterxml.jackson.core.JsonProcessingException; +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.ObjectMapperFactory; + +/** + * Defines ServiceUnitTableViewSyncer. + * It syncs service unit(bundle) states between metadata store and system topic table views. + * One could enable this syncer before migration from one to the other and disable it after the migration finishes. + */ +@Slf4j +public class ServiceUnitStateTableViewSyncer implements Closeable { + private static final int MAX_CONCURRENT_SYNC_COUNT = 100; + private static final int SYNC_WAIT_TIME_IN_SECS = 300; + private PulsarService pulsar; + private volatile ServiceUnitStateTableView systemTopicTableView; + private volatile ServiceUnitStateTableView metadataStoreTableView; + private volatile boolean isActive = false; + + + public void start(PulsarService pulsar) + throws IOException, TimeoutException, InterruptedException, ExecutionException { + if (!pulsar.getConfiguration().isLoadBalancerServiceUnitTableViewSyncerEnabled()) { + return; + } + + if (isActive) { + return; + } + this.pulsar = pulsar; + + try { + + syncExistingItems(); + // disable compaction + if (!configureSystemTopics(pulsar, 0)) { + throw new IllegalStateException("Failed to disable compaction"); + } + syncTailItems(); + + isActive = true; + + } catch (Throwable e) { + log.error("Failed to start ServiceUnitStateTableViewSyncer", e); + throw e; + } + } + + private CompletableFuture syncToSystemTopic(String key, ServiceUnitStateData data) { + return systemTopicTableView.put(key, data); + } + + private CompletableFuture syncToMetadataStore(String key, ServiceUnitStateData data) { + return metadataStoreTableView.put(key, data); + } + + private void dummy(String key, ServiceUnitStateData data) { + } + + private void syncExistingItems() + throws IOException, ExecutionException, InterruptedException, TimeoutException { + long started = System.currentTimeMillis(); + @Cleanup + ServiceUnitStateTableView metadataStoreTableView = new ServiceUnitStateMetadataStoreTableViewImpl(); + metadataStoreTableView.start( + pulsar, + this::dummy, + this::dummy + ); + + @Cleanup + ServiceUnitStateTableView systemTopicTableView = new ServiceUnitStateTableViewImpl(); + systemTopicTableView.start( + pulsar, + this::dummy, + this::dummy + ); + + + var syncer = pulsar.getConfiguration().getLoadBalancerServiceUnitTableViewSyncer(); + if (syncer == SystemTopicToMetadataStoreSyncer) { + clean(metadataStoreTableView); + syncExistingItemsToMetadataStore(systemTopicTableView); + } else { + clean(systemTopicTableView); + syncExistingItemsToSystemTopic(metadataStoreTableView, systemTopicTableView); + } + + if (!waitUntilSynced(metadataStoreTableView, systemTopicTableView, started)) { + throw new TimeoutException( + syncer + " failed to sync existing items in tableviews. MetadataStoreTableView.size: " + + metadataStoreTableView.entrySet().size() + + ", SystemTopicTableView.size: " + systemTopicTableView.entrySet().size() + " in " + + SYNC_WAIT_TIME_IN_SECS + " secs"); + } + + log.info("Synced existing items MetadataStoreTableView.size:{} , " + + "SystemTopicTableView.size: {} in {} secs", + metadataStoreTableView.entrySet().size(), systemTopicTableView.entrySet().size(), + TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - started)); + } + + private void syncTailItems() throws InterruptedException, IOException, TimeoutException { + long started = System.currentTimeMillis(); + + if (metadataStoreTableView != null) { + metadataStoreTableView.close(); + metadataStoreTableView = null; + } + + if (systemTopicTableView != null) { + systemTopicTableView.close(); + systemTopicTableView = null; + } + + this.metadataStoreTableView = new ServiceUnitStateMetadataStoreTableViewImpl(); + this.metadataStoreTableView.start( + pulsar, + this::syncToSystemTopic, + this::dummy + ); + log.info("Started MetadataStoreTableView"); + + this.systemTopicTableView = new ServiceUnitStateTableViewImpl(); + this.systemTopicTableView.start( + pulsar, + this::syncToMetadataStore, + this::dummy + ); + log.info("Started SystemTopicTableView"); + + var syncer = pulsar.getConfiguration().getLoadBalancerServiceUnitTableViewSyncer(); + if (!waitUntilSynced(metadataStoreTableView, systemTopicTableView, started)) { + throw new TimeoutException( + syncer + " failed to sync tableviews. MetadataStoreTableView.size: " + + metadataStoreTableView.entrySet().size() + + ", SystemTopicTableView.size: " + systemTopicTableView.entrySet().size() + " in " + + SYNC_WAIT_TIME_IN_SECS + " secs"); + } + + + log.info("Successfully started ServiceUnitStateTableViewSyncer MetadataStoreTableView.size:{} , " + + "SystemTopicTableView.size: {} in {} secs", + metadataStoreTableView.entrySet().size(), systemTopicTableView.entrySet().size(), + TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - started)); + } + + private void syncExistingItemsToMetadataStore(ServiceUnitStateTableView src) + throws JsonProcessingException, ExecutionException, InterruptedException, TimeoutException { + // Directly use store to sync existing items to metadataStoreTableView(otherwise, they are conflicted out) + var store = pulsar.getLocalMetadataStore(); + var writer = ObjectMapperFactory.getMapper().writer(); + var opTimeout = pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(); + List> futures = new ArrayList<>(); + var srcIter = src.entrySet().iterator(); + while (srcIter.hasNext()) { + var e = srcIter.next(); + futures.add(store.put(ServiceUnitStateMetadataStoreTableViewImpl.PATH_PREFIX + "/" + e.getKey(), + writer.writeValueAsBytes(e.getValue()), Optional.empty()).thenApply(__ -> null)); + if (futures.size() == MAX_CONCURRENT_SYNC_COUNT || !srcIter.hasNext()) { + FutureUtil.waitForAll(futures).get(opTimeout, TimeUnit.SECONDS); + } + } + } + + private void syncExistingItemsToSystemTopic(ServiceUnitStateTableView src, + ServiceUnitStateTableView dst) + throws ExecutionException, InterruptedException, TimeoutException { + var opTimeout = pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(); + List> futures = new ArrayList<>(); + var srcIter = src.entrySet().iterator(); + while (srcIter.hasNext()) { + var e = srcIter.next(); + futures.add(dst.put(e.getKey(), e.getValue())); + if (futures.size() == MAX_CONCURRENT_SYNC_COUNT || !srcIter.hasNext()) { + FutureUtil.waitForAll(futures).get(opTimeout, TimeUnit.SECONDS); + } + } + } + + private void clean(ServiceUnitStateTableView dst) + throws ExecutionException, InterruptedException, TimeoutException { + var opTimeout = pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(); + var dstIter = dst.entrySet().iterator(); + List> futures = new ArrayList<>(); + while (dstIter.hasNext()) { + var e = dstIter.next(); + futures.add(dst.delete(e.getKey())); + if (futures.size() == MAX_CONCURRENT_SYNC_COUNT || !dstIter.hasNext()) { + FutureUtil.waitForAll(futures).get(opTimeout, TimeUnit.SECONDS); + } + } + } + + private boolean waitUntilSynced(ServiceUnitStateTableView srt, ServiceUnitStateTableView dst, long started) + throws InterruptedException { + while (srt.entrySet().size() != dst.entrySet().size()) { + if (TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - started) + > SYNC_WAIT_TIME_IN_SECS) { + return false; + } + Thread.sleep(100); + } + return true; + } + + @Override + public void close() throws IOException { + if (!isActive) { + return; + } + + if (!configureSystemTopics(pulsar, COMPACTION_THRESHOLD)) { + throw new IllegalStateException("Failed to enable compaction"); + } + + try { + if (systemTopicTableView != null) { + systemTopicTableView.close(); + systemTopicTableView = null; + log.info("Closed SystemTopicTableView"); + } + } catch (Exception e) { + log.error("Failed to close SystemTopicTableView", e); + throw e; + } + + try { + if (metadataStoreTableView != null) { + metadataStoreTableView.close(); + metadataStoreTableView = null; + log.info("Closed MetadataStoreTableView"); + } + } catch (Exception e) { + log.error("Failed to close MetadataStoreTableView", e); + throw e; + } + + log.info("Successfully closed ServiceUnitStateTableViewSyncer."); + isActive = false; + } + + public boolean isActive() { + return isActive; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java index a7deeeaad8a5c..8096d1908b928 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java @@ -103,4 +103,10 @@ public interface LoadDataStore extends Closeable { */ void startProducer() throws LoadDataStoreException; + /** + * Shutdowns the data store. + */ + default void shutdown() throws IOException { + close(); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java index e9289d3ccdac2..c9d18676cfa99 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java @@ -43,20 +43,17 @@ public class TableViewLoadDataStoreImpl implements LoadDataStore { private static final long LOAD_DATA_REPORT_UPDATE_MAX_INTERVAL_MULTIPLIER_BEFORE_RESTART = 2; + private static final String SHUTDOWN_ERR_MSG = "This load store tableview has been shutdown"; private static final long INIT_TIMEOUT_IN_SECS = 5; - private volatile TableView tableView; private volatile long tableViewLastUpdateTimestamp; - + private volatile long producerLastPublishTimestamp; private volatile Producer producer; - private final ServiceConfiguration conf; - private final PulsarClient client; - private final String topic; - private final Class clazz; + private volatile boolean isShutdown; public TableViewLoadDataStoreImpl(PulsarService pulsar, String topic, Class clazz) throws LoadDataStoreException { @@ -65,6 +62,7 @@ public TableViewLoadDataStoreImpl(PulsarService pulsar, String topic, Class c this.client = pulsar.getClient(); this.topic = topic; this.clazz = clazz; + this.isShutdown = false; } catch (Exception e) { throw new LoadDataStoreException(e); } @@ -72,41 +70,76 @@ public TableViewLoadDataStoreImpl(PulsarService pulsar, String topic, Class c @Override public synchronized CompletableFuture pushAsync(String key, T loadData) { - validateProducer(); - return producer.newMessage().key(key).value(loadData).sendAsync().thenAccept(__ -> {}); + String msg = validateProducer(); + if (StringUtils.isNotBlank(msg)) { + return CompletableFuture.failedFuture(new IllegalStateException(msg)); + } + return producer.newMessage().key(key).value(loadData).sendAsync() + .thenAccept(__ -> producerLastPublishTimestamp = System.currentTimeMillis()); } @Override public synchronized CompletableFuture removeAsync(String key) { - validateProducer(); - return producer.newMessage().key(key).value(null).sendAsync().thenAccept(__ -> {}); + String msg = validateProducer(); + if (StringUtils.isNotBlank(msg)) { + return CompletableFuture.failedFuture(new IllegalStateException(msg)); + } + return producer.newMessage().key(key).value(null).sendAsync() + .thenAccept(__ -> producerLastPublishTimestamp = System.currentTimeMillis()); } @Override public synchronized Optional get(String key) { - validateTableView(); + String msg = validateTableView(); + if (StringUtils.isNotBlank(msg)) { + throw new IllegalStateException(msg); + } return Optional.ofNullable(tableView.get(key)); } @Override public synchronized void forEach(BiConsumer action) { - validateTableView(); + String msg = validateTableView(); + if (StringUtils.isNotBlank(msg)) { + throw new IllegalStateException(msg); + } tableView.forEach(action); } public synchronized Set> entrySet() { - validateTableView(); + String msg = validateTableView(); + if (StringUtils.isNotBlank(msg)) { + throw new IllegalStateException(msg); + } return tableView.entrySet(); } @Override public synchronized int size() { - validateTableView(); + String msg = validateTableView(); + if (StringUtils.isNotBlank(msg)) { + throw new IllegalStateException(msg); + } return tableView.size(); } + private void validateState() { + if (isShutdown) { + throw new IllegalStateException(SHUTDOWN_ERR_MSG); + } + } + + + @Override + public synchronized void init() throws IOException { + validateState(); + close(); + start(); + } + @Override public synchronized void closeTableView() throws IOException { + validateState(); if (tableView != null) { tableView.close(); tableView = null; @@ -115,16 +148,26 @@ public synchronized void closeTableView() throws IOException { @Override public synchronized void start() throws LoadDataStoreException { + validateState(); startProducer(); startTableView(); } + private synchronized void closeProducer() throws IOException { + validateState(); + if (producer != null) { + producer.close(); + producer = null; + } + } @Override public synchronized void startTableView() throws LoadDataStoreException { + validateState(); if (tableView == null) { try { tableView = client.newTableViewBuilder(Schema.JSON(clazz)).topic(topic).createAsync() .get(INIT_TIMEOUT_IN_SECS, TimeUnit.SECONDS); + tableViewLastUpdateTimestamp = System.currentTimeMillis(); tableView.forEachAndListen((k, v) -> tableViewLastUpdateTimestamp = System.currentTimeMillis()); } catch (Exception e) { @@ -133,13 +176,14 @@ public synchronized void startTableView() throws LoadDataStoreException { } } } - @Override public synchronized void startProducer() throws LoadDataStoreException { + validateState(); if (producer == null) { try { producer = client.newProducer(Schema.JSON(clazz)).topic(topic).createAsync() .get(INIT_TIMEOUT_IN_SECS, TimeUnit.SECONDS); + producerLastPublishTimestamp = System.currentTimeMillis(); } catch (Exception e) { producer = null; throw new LoadDataStoreException(e); @@ -149,38 +193,63 @@ public synchronized void startProducer() throws LoadDataStoreException { @Override public synchronized void close() throws IOException { - if (producer != null) { - producer.close(); - producer = null; - } + validateState(); + closeProducer(); closeTableView(); } @Override - public synchronized void init() throws IOException { + public synchronized void shutdown() throws IOException { close(); - start(); + isShutdown = true; } - private void validateProducer() { - if (producer == null) { + private String validateProducer() { + if (isShutdown) { + return SHUTDOWN_ERR_MSG; + } + String restartReason = getRestartReason(producer, producerLastPublishTimestamp); + if (StringUtils.isNotBlank(restartReason)) { try { + closeProducer(); startProducer(); - log.info("Restarted producer on {}", topic); + log.info("Restarted producer on {}, {}", topic, restartReason); } catch (Exception e) { - log.error("Failed to restart producer on {}", topic, e); - throw new RuntimeException(e); + String msg = "Failed to restart producer on " + topic + ", restart reason: " + restartReason; + log.error(msg, e); + return msg; } } + return null; } - private void validateTableView() { + private String validateTableView() { + if (isShutdown) { + return SHUTDOWN_ERR_MSG; + } + String restartReason = getRestartReason(tableView, tableViewLastUpdateTimestamp); + if (StringUtils.isNotBlank(restartReason)) { + try { + closeTableView(); + startTableView(); + log.info("Restarted tableview on {}, {}", topic, restartReason); + } catch (Exception e) { + String msg = "Failed to tableview on " + topic + ", restart reason: " + restartReason; + log.error(msg, e); + return msg; + } + } + return null; + } + + private String getRestartReason(Object obj, long lastUpdateTimestamp) { + String restartReason = null; - if (tableView == null) { - restartReason = "table view is null"; + if (obj == null) { + restartReason = "object is null"; } else { - long inactiveDuration = System.currentTimeMillis() - tableViewLastUpdateTimestamp; + long inactiveDuration = System.currentTimeMillis() - lastUpdateTimestamp; long threshold = TimeUnit.MINUTES.toMillis(conf.getLoadBalancerReportUpdateMaxIntervalMinutes()) * LOAD_DATA_REPORT_UPDATE_MAX_INTERVAL_MULTIPLIER_BEFORE_RESTART; if (inactiveDuration > threshold) { @@ -189,17 +258,6 @@ private void validateTableView() { TimeUnit.MILLISECONDS.toSeconds(threshold)); } } - - if (StringUtils.isNotBlank(restartReason)) { - tableViewLastUpdateTimestamp = 0; - try { - closeTableView(); - startTableView(); - log.info("Restarted tableview on {}, {}", topic, restartReason); - } catch (Exception e) { - log.error("Failed to restart tableview on {}", topic, e); - throw new RuntimeException(e); - } - } + return restartReason; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 9c0bdc120c474..d664d6812adaa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -21,6 +21,7 @@ import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.commons.lang3.StringUtils.isBlank; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.TOPIC; import static org.apache.pulsar.broker.service.persistent.SubscribeRateLimiter.isSubscribeRateEnabled; import static org.apache.pulsar.common.naming.SystemTopicNames.isEventSystemTopic; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; @@ -91,8 +92,7 @@ import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateCompactionStrategy; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateDataConflictResolver; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.resources.NamespaceResources.PartitionedTopicResources; import org.apache.pulsar.broker.service.AbstractReplicator; @@ -247,8 +247,8 @@ public static boolean isDedupCursorName(String name) { // TODO: Create compaction strategy from topic policy when exposing strategic compaction to users. private static Map strategicCompactionMap = Map.of( - ServiceUnitStateChannelImpl.TOPIC, - new ServiceUnitStateCompactionStrategy()); + TOPIC, + new ServiceUnitStateDataConflictResolver()); private CompletableFuture currentOffload = CompletableFuture.completedFuture( (MessageIdImpl) MessageId.earliest); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java index e9fafa9c30317..bb224cdf7c40e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java @@ -22,26 +22,35 @@ import static org.mockito.Mockito.spy; import com.google.common.collect.Sets; import com.google.common.io.Resources; +import java.util.ArrayList; +import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateMetadataStoreTableViewImpl; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl; import org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.LookupService; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.util.FutureUtil; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; public abstract class ExtensibleLoadManagerImplBaseTest extends MockedPulsarServiceBaseTest { @@ -67,8 +76,21 @@ public abstract class ExtensibleLoadManagerImplBaseTest extends MockedPulsarServ protected LookupService lookupService; - protected ExtensibleLoadManagerImplBaseTest(String defaultTestNamespace) { + protected String serviceUnitStateTableViewClassName; + + protected ArrayList clients = new ArrayList<>(); + + @DataProvider(name = "serviceUnitStateTableViewClassName") + public static Object[][] serviceUnitStateTableViewClassName() { + return new Object[][]{ + {ServiceUnitStateTableViewImpl.class.getName()}, + {ServiceUnitStateMetadataStoreTableViewImpl.class.getName()} + }; + } + + protected ExtensibleLoadManagerImplBaseTest(String defaultTestNamespace, String serviceUnitStateTableViewClassName) { this.defaultTestNamespace = defaultTestNamespace; + this.serviceUnitStateTableViewClassName = serviceUnitStateTableViewClassName; } @Override @@ -82,6 +104,8 @@ protected ServiceConfiguration updateConfig(ServiceConfiguration conf) { conf.setForceDeleteNamespaceAllowed(true); conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + conf.setLoadManagerServiceUnitStateTableViewClassName(serviceUnitStateTableViewClassName); + conf.setLoadBalancerReportUpdateMaxIntervalMinutes(1); conf.setLoadBalancerSheddingEnabled(false); conf.setLoadBalancerDebugModeEnabled(true); conf.setWebServicePortTls(Optional.of(0)); @@ -117,20 +141,44 @@ protected void setup() throws Exception { admin.namespaces().setNamespaceReplicationClusters(defaultTestNamespace, Sets.newHashSet(this.conf.getClusterName())); lookupService = (LookupService) FieldUtils.readDeclaredField(pulsarClient, "lookup", true); + + for (int i = 0; i < 4; i++) { + clients.add(pulsarClient(lookupUrl.toString(), 100)); + } } + private static PulsarClient pulsarClient(String url, int intervalInMillis) throws PulsarClientException { + return + PulsarClient.builder() + .serviceUrl(url) + .statsInterval(intervalInMillis, TimeUnit.MILLISECONDS).build(); + } + + @Override @AfterClass(alwaysRun = true) protected void cleanup() throws Exception { + List> futures = new ArrayList<>(); + for (PulsarClient client : clients) { + futures.add(client.closeAsync()); + } + futures.add(pulsar2.closeAsync()); + if (additionalPulsarTestContext != null) { additionalPulsarTestContext.close(); additionalPulsarTestContext = null; } super.internalCleanup(); + try { + FutureUtil.waitForAll(futures).join(); + } catch (Throwable e) { + // skip error + } pulsar1 = pulsar2 = null; primaryLoadManager = secondaryLoadManager = null; channel1 = channel2 = null; lookupService = null; + } @BeforeMethod(alwaysRun = true) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index 51966f420bf25..4f6a006918318 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -18,8 +18,7 @@ */ package org.apache.pulsar.broker.loadbalance.extensions; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Releasing; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelTest.overrideTableView; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.TOPIC; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.Admin; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.Bandwidth; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.MsgRate; @@ -44,7 +43,6 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; -import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -91,7 +89,8 @@ import org.apache.pulsar.broker.loadbalance.LeaderElectionService; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateMetadataStoreTableViewImpl; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData; @@ -102,6 +101,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.models.UnloadCounter; import org.apache.pulsar.broker.loadbalance.extensions.reporter.BrokerLoadDataReporter; import org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder; +import org.apache.pulsar.broker.loadbalance.extensions.store.TableViewLoadDataStoreImpl; import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.broker.lookup.LookupResult; import org.apache.pulsar.broker.namespace.LookupOptions; @@ -138,6 +138,7 @@ import org.awaitility.Awaitility; import org.testng.AssertJUnit; import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; /** @@ -148,19 +149,20 @@ @SuppressWarnings("unchecked") public class ExtensibleLoadManagerImplTest extends ExtensibleLoadManagerImplBaseTest { - public ExtensibleLoadManagerImplTest() { - super("public/test"); + @Factory(dataProvider = "serviceUnitStateTableViewClassName") + public ExtensibleLoadManagerImplTest(String serviceUnitStateTableViewClassName) { + super("public/test", serviceUnitStateTableViewClassName); } @Test public void testAssignInternalTopic() throws Exception { Optional brokerLookupData1 = primaryLoadManager.assign( - Optional.of(TopicName.get(ServiceUnitStateChannelImpl.TOPIC)), - getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get(), + Optional.of(TopicName.get(TOPIC)), + getBundleAsync(pulsar1, TopicName.get(TOPIC)).get(), LookupOptions.builder().build()).get(); Optional brokerLookupData2 = secondaryLoadManager.assign( - Optional.of(TopicName.get(ServiceUnitStateChannelImpl.TOPIC)), - getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get(), + Optional.of(TopicName.get(TOPIC)), + getBundleAsync(pulsar1, TopicName.get(TOPIC)).get(), LookupOptions.builder().build()).get(); assertEquals(brokerLookupData1, brokerLookupData2); assertTrue(brokerLookupData1.isPresent()); @@ -320,7 +322,7 @@ public void testUnloadUponTopicLookupFailure() throws Exception { try { pulsar1.getBrokerService().getTopics().put(topicName.toString(), future1); pulsar2.getBrokerService().getTopics().put(topicName.toString(), future2); - CompletableFuture.delayedExecutor(2, TimeUnit.SECONDS).execute(() -> { + CompletableFuture.delayedExecutor(1, TimeUnit.SECONDS).execute(() -> { future1.completeExceptionally(new CompletionException( new BrokerServiceException.ServiceUnitNotReadyException("Please redo the lookup"))); future2.completeExceptionally(new CompletionException( @@ -417,7 +419,7 @@ public boolean test(NamespaceBundle namespaceBundle) { } } - @Test(timeOut = 30 * 1000) + @Test(timeOut = 30 * 1000, priority = 1000) public void testNamespaceOwnershipListener() throws Exception { Pair topicAndBundle = getBundleIsNotOwnByChangeEventTopic("test-namespace-ownership-listener"); @@ -457,14 +459,17 @@ public boolean test(NamespaceBundle namespaceBundle) { assertEquals(unloadCount.get(), 0); }); - ServiceUnitStateChannelImpl channel = new ServiceUnitStateChannelImpl(pulsar1); - channel.start(); + @Cleanup + ServiceUnitStateChannelImpl channel3 = new ServiceUnitStateChannelImpl(pulsar1); + channel3.start(); + @Cleanup + ServiceUnitStateChannelImpl channel4 = new ServiceUnitStateChannelImpl(pulsar2); + channel4.start(); Awaitility.await().untilAsserted(() -> { assertEquals(onloadCount.get(), 2); assertEquals(unloadCount.get(), 0); }); - channel.close(); } @DataProvider(name = "isPersistentTopicSubscriptionTypeTest") @@ -484,30 +489,31 @@ public Object[][] isPersistentTopicSubscriptionTypeTest() { @Test(timeOut = 30_000, dataProvider = "isPersistentTopicSubscriptionTypeTest") public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, SubscriptionType subscriptionType) throws Exception { - testTransferClientReconnectionWithoutLookup(topicDomain, subscriptionType, defaultTestNamespace, admin, - lookupUrl.toString(), pulsar1, pulsar2, primaryLoadManager, secondaryLoadManager); + testTransferClientReconnectionWithoutLookup(clients, topicDomain, subscriptionType, defaultTestNamespace, + admin, lookupUrl.toString(), pulsar1, pulsar2, primaryLoadManager, secondaryLoadManager); } @Test(enabled = false) - public static void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, - SubscriptionType subscriptionType, - String defaultTestNamespace, - PulsarAdmin admin, String brokerServiceUrl, - PulsarService pulsar1, PulsarService pulsar2, - ExtensibleLoadManager primaryLoadManager, - ExtensibleLoadManager secondaryLoadManager) + public static void testTransferClientReconnectionWithoutLookup( + List clients, + TopicDomain topicDomain, + SubscriptionType subscriptionType, + String defaultTestNamespace, + PulsarAdmin admin, String brokerServiceUrl, + PulsarService pulsar1, PulsarService pulsar2, + ExtensibleLoadManager primaryLoadManager, + ExtensibleLoadManager secondaryLoadManager) throws Exception { var id = String.format("test-tx-client-reconnect-%s-%s", subscriptionType, UUID.randomUUID()); var topic = String.format("%s://%s/%s", topicDomain.toString(), defaultTestNamespace, id); var topicName = TopicName.get(topic); var timeoutMs = 30_000; - var clients = new ArrayList(); var consumers = new ArrayList>(); + var lookups = new ArrayList>(); + int clientId = 0; try { - var lookups = new ArrayList(); - var pulsarClient = pulsarClient(brokerServiceUrl, 0); - clients.add(pulsarClient); + var pulsarClient = clients.get(clientId++); @Cleanup var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); lookups.add(spyLookupService(pulsarClient)); @@ -515,8 +521,7 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic var consumerCount = subscriptionType == SubscriptionType.Exclusive ? 1 : 3; for (int i = 0; i < consumerCount; i++) { - var client = pulsarClient(brokerServiceUrl, 0); - clients.add(client); + var client = clients.get(clientId++); var consumer = client.newConsumer(Schema.STRING). subscriptionName(id). subscriptionType(subscriptionType). @@ -544,8 +549,8 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic } checkOwnershipState(broker, bundle, primaryLoadManager, secondaryLoadManager, pulsar1); - var messageCountBeforeUnloading = 100; - var messageCountAfterUnloading = 100; + var messageCountBeforeUnloading = 10; + var messageCountAfterUnloading = 10; var messageCount = messageCountBeforeUnloading + messageCountAfterUnloading; var semMessagesReadyToSend = new Semaphore(0); @@ -561,6 +566,7 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic semMessagesReadyToSend.release(messageCountBeforeUnloading); admin.namespaces() .unloadNamespaceBundle(defaultTestNamespace, bundle.getBundleRange(), dstBrokerUrl); + //log.info("### unloaded."); semMessagesReadyToSend.release(messageCountAfterUnloading); } catch (InterruptedException | PulsarAdminException e) { fail(); @@ -579,6 +585,7 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic pendingMessages.add(message); } producer.send(message); + //log.info("### producer sent: {}", message); } } catch (PulsarClientException | InterruptedException e) { fail(); @@ -594,10 +601,11 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic } while (!producerFuture.isDone() || !pendingMessages.isEmpty()) { try { - var message = consumer.receive(1500, TimeUnit.MILLISECONDS); + var message = consumer.receive(200, TimeUnit.MILLISECONDS); if (message != null) { consumer.acknowledge(message); pendingMessages.remove(message.getValue()); + //log.info("### consumer received: {}", message.getValue()); } } catch (PulsarClientException e) { // Retry read @@ -620,15 +628,17 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic assertTrue(producer.isConnected()); assertTrue(consumers.stream().allMatch(Consumer::isConnected)); - for (LookupService lookupService : lookups) { - verify(lookupService, never()).getBroker(topicName); + for (var lookupService : lookups) { + verify(lookupService.getRight(), never()).getBroker(topicName); } } finally { for (var consumer: consumers) { consumer.close(); } - for (var client: clients) { - client.close(); + + clientId = 0; + for (var lookup : lookups) { + resetLookupService(clients.get(clientId++), lookup.getLeft()); } } } @@ -636,12 +646,13 @@ public static void testTransferClientReconnectionWithoutLookup(TopicDomain topic @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicSubscriptionTypeTest") public void testUnloadClientReconnectionWithLookup(TopicDomain topicDomain, SubscriptionType subscriptionType) throws Exception { - testUnloadClientReconnectionWithLookup(topicDomain, subscriptionType, defaultTestNamespace, admin, - lookupUrl.toString(), pulsar1); + testUnloadClientReconnectionWithLookup(clients, topicDomain, subscriptionType, defaultTestNamespace, + admin, lookupUrl.toString(), pulsar1); } @Test(enabled = false) - public static void testUnloadClientReconnectionWithLookup(TopicDomain topicDomain, + public static void testUnloadClientReconnectionWithLookup(List clients, + TopicDomain topicDomain, SubscriptionType subscriptionType, String defaultTestNamespace, PulsarAdmin admin, @@ -653,9 +664,10 @@ public static void testUnloadClientReconnectionWithLookup(TopicDomain topicDomai var topicName = TopicName.get(topic); var consumers = new ArrayList>(); + Pair lookup = null; + PulsarClient pulsarClient = null; try { - @Cleanup - var pulsarClient = pulsarClient(brokerServiceUrl, 0); + pulsarClient = clients.get(0); var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); var consumerCount = subscriptionType == SubscriptionType.Exclusive ? 1 : 3; @@ -666,7 +678,7 @@ public static void testUnloadClientReconnectionWithLookup(TopicDomain topicDomai Awaitility.await() .until(() -> producer.isConnected() && consumers.stream().allMatch(Consumer::isConnected)); - var lookup = spyLookupService(pulsarClient); + lookup = spyLookupService(pulsarClient); final CountDownLatch cdl = new CountDownLatch(3); @@ -687,7 +699,7 @@ public static void testUnloadClientReconnectionWithLookup(TopicDomain topicDomai boolean messageSent = false; while (true) { var recvFutures = consumers.stream(). - map(consumer -> consumer.receiveAsync().orTimeout(1000, TimeUnit.MILLISECONDS)). + map(consumer -> consumer.receiveAsync().orTimeout(200, TimeUnit.MILLISECONDS)). collect(Collectors.toList()); if (!messageSent) { @@ -712,11 +724,12 @@ public static void testUnloadClientReconnectionWithLookup(TopicDomain topicDomai assertTrue(producer.isConnected()); assertTrue(consumers.stream().allMatch(Consumer::isConnected)); assertTrue(unloadNamespaceBundle.isDone()); - verify(lookup, times(1 + consumerCount)).getBroker(topicName); + verify(lookup.getRight(), times(1 + consumerCount)).getBroker(topicName); } finally { for (var consumer : consumers) { consumer.close(); } + resetLookupService(pulsarClient, lookup.getLeft()); } } @@ -727,11 +740,13 @@ public Object[][] isPersistentTopicTest() { @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicTest") public void testOptimizeUnloadDisable(TopicDomain topicDomain) throws Exception { - testOptimizeUnloadDisable(topicDomain, defaultTestNamespace, admin, lookupUrl.toString(), pulsar1, pulsar2); + testOptimizeUnloadDisable(clients, topicDomain, defaultTestNamespace, admin, lookupUrl.toString(), pulsar1, + pulsar2); } @Test(enabled = false) - public static void testOptimizeUnloadDisable(TopicDomain topicDomain, + public static void testOptimizeUnloadDisable(List clients, + TopicDomain topicDomain, String defaultTestNamespace, PulsarAdmin admin, String brokerServiceUrl, @@ -744,9 +759,8 @@ public static void testOptimizeUnloadDisable(TopicDomain topicDomain, pulsar1.getConfig().setLoadBalancerMultiPhaseBundleUnload(false); pulsar2.getConfig().setLoadBalancerMultiPhaseBundleUnload(false); - @Cleanup - var pulsarClient = pulsarClient(brokerServiceUrl, 0); - + var pulsarClient = clients.get(0); + Pair lookup = null; @Cleanup var producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); @@ -755,64 +769,77 @@ public static void testOptimizeUnloadDisable(TopicDomain topicDomain, Awaitility.await().until(() -> producer.isConnected() && consumer.isConnected()); - var lookup = spyLookupService(pulsarClient); - - final CountDownLatch cdl = new CountDownLatch(3); + try { + lookup = spyLookupService(pulsarClient); - NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).get(); - var srcBrokerServiceUrl = admin.lookups().lookupTopic(topic); - var dstBroker = srcBrokerServiceUrl.equals(pulsar1.getBrokerServiceUrl()) ? pulsar2 : pulsar1; + final CountDownLatch cdl = new CountDownLatch(3); - CompletableFuture unloadNamespaceBundle = CompletableFuture.runAsync(() -> { - try { - cdl.await(); - admin.namespaces().unloadNamespaceBundle(defaultTestNamespace, bundle.getBundleRange(), - dstBroker.getBrokerId()); - } catch (InterruptedException | PulsarAdminException e) { - fail(); - } - }); + NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).get(); + var srcBrokerServiceUrl = admin.lookups().lookupTopic(topic); + var dstBroker = srcBrokerServiceUrl.equals(pulsar1.getBrokerServiceUrl()) ? pulsar2 : pulsar1; - MutableInt sendCount = new MutableInt(); - Awaitility.await().atMost(20, TimeUnit.SECONDS).ignoreExceptions().until(() -> { - var message = String.format("message-%d", sendCount.getValue()); - - AtomicBoolean messageSent = new AtomicBoolean(false); - while (true) { - var recvFuture = consumer.receiveAsync().orTimeout(1000, TimeUnit.MILLISECONDS); - if (!messageSent.get()) { - producer.sendAsync(message).thenAccept(messageId -> { - if (messageId != null) { - messageSent.set(true); - } - }).get(1000, TimeUnit.MILLISECONDS); + CompletableFuture unloadNamespaceBundle = CompletableFuture.runAsync(() -> { + try { + cdl.await(); + admin.namespaces().unloadNamespaceBundle(defaultTestNamespace, bundle.getBundleRange(), + dstBroker.getBrokerId()); + } catch (InterruptedException | PulsarAdminException e) { + fail(); } + }); - if (topicDomain == TopicDomain.non_persistent) { - // No need to wait for message receipt, we're only trying to stress the consumer lookup pathway. - break; - } - var msg = recvFuture.get(); - if (Objects.equals(msg.getValue(), message)) { - break; + MutableInt sendCount = new MutableInt(); + Awaitility.await().atMost(20, TimeUnit.SECONDS).ignoreExceptions().until(() -> { + var message = String.format("message-%d", sendCount.getValue()); + + AtomicBoolean messageSent = new AtomicBoolean(false); + while (true) { + var recvFuture = consumer.receiveAsync().orTimeout(200, TimeUnit.MILLISECONDS); + if (!messageSent.get()) { + producer.sendAsync(message).thenAccept(messageId -> { + if (messageId != null) { + messageSent.set(true); + } + }).get(200, TimeUnit.MILLISECONDS); + } + + if (topicDomain == TopicDomain.non_persistent) { + // No need to wait for message receipt, we're only trying to stress the consumer lookup pathway. + break; + } + var msg = recvFuture.get(); + if (Objects.equals(msg.getValue(), message)) { + break; + } } - } - cdl.countDown(); - return sendCount.incrementAndGet() == 10; - }); + cdl.countDown(); + return sendCount.incrementAndGet() == 10; + }); - assertTrue(producer.isConnected()); - assertTrue(consumer.isConnected()); - assertTrue(unloadNamespaceBundle.isDone()); - verify(lookup, times(2)).getBroker(topicName); + Pair finalLookup = lookup; + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> { + assertTrue(producer.isConnected()); + assertTrue(consumer.isConnected()); + assertTrue(unloadNamespaceBundle.isDone()); + verify(finalLookup.getRight(), times(2)).getBroker(topicName); + }); + } finally { + resetLookupService(pulsarClient, lookup.getLeft()); + } } - protected static LookupService spyLookupService(PulsarClient client) throws IllegalAccessException { + protected static Pair spyLookupService(PulsarClient client) throws IllegalAccessException { LookupService svc = (LookupService) FieldUtils.readDeclaredField(client, "lookup", true); var lookup = spy(svc); FieldUtils.writeDeclaredField(client, "lookup", lookup, true); - return lookup; + return Pair.of(svc, lookup); + } + + protected static void resetLookupService(PulsarClient client, LookupService lookup) throws IllegalAccessException { + FieldUtils.writeDeclaredField(client, "lookup", lookup, true); } protected static void checkOwnershipState(String broker, NamespaceBundle bundle, @@ -1043,52 +1070,117 @@ public CompletableFuture> filterAsync(Map topicAndBundle = - getBundleIsNotOwnByChangeEventTopic("testDeployAndRollbackLoadManager"); - TopicName topicName = topicAndBundle.getLeft(); - NamespaceBundle bundle = topicAndBundle.getRight(); - String topic = topicName.toString(); - - String lookupResult1 = pulsar3.getAdminClient().lookups().lookupTopic(topic); - assertEquals(lookupResult1, pulsar3.getBrokerServiceUrl()); - - String lookupResult2 = pulsar1.getAdminClient().lookups().lookupTopic(topic); - String lookupResult3 = pulsar2.getAdminClient().lookups().lookupTopic(topic); - assertEquals(lookupResult1, lookupResult2); - assertEquals(lookupResult1, lookupResult3); - - LookupOptions options = LookupOptions.builder() - .authoritative(false) - .requestHttps(false) - .readOnly(false) - .loadTopicsInBundle(false).build(); - Optional webServiceUrl1 = + // Test rollback to modular load manager. + ServiceConfiguration defaultConf = getDefaultConf(); + defaultConf.setAllowAutoTopicCreation(true); + defaultConf.setForceDeleteNamespaceAllowed(true); + defaultConf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); + defaultConf.setLoadBalancerSheddingEnabled(false); + try (var additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf)) { + // start pulsar3 with old load manager + @Cleanup + var pulsar3 = additionalPulsarTestContext.getPulsarService(); + Pair topicAndBundle = + getBundleIsNotOwnByChangeEventTopic("testDeployAndRollbackLoadManager"); + TopicName topicName = topicAndBundle.getLeft(); + NamespaceBundle bundle = topicAndBundle.getRight(); + String topic = topicName.toString(); + + String lookupResult1 = pulsar3.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult1, pulsar3.getBrokerServiceUrl()); + + String lookupResult2 = pulsar1.getAdminClient().lookups().lookupTopic(topic); + String lookupResult3 = pulsar2.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult1, lookupResult2); + assertEquals(lookupResult1, lookupResult3); + + LookupOptions options = LookupOptions.builder() + .authoritative(false) + .requestHttps(false) + .readOnly(false) + .loadTopicsInBundle(false).build(); + Optional webServiceUrl1 = + pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl1.isPresent()); + assertEquals(webServiceUrl1.get().toString(), pulsar3.getWebServiceAddress()); + + Optional webServiceUrl2 = + pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl2.isPresent()); + assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); + + Optional webServiceUrl3 = + pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl3.isPresent()); + assertEquals(webServiceUrl3.get().toString(), webServiceUrl1.get().toString()); + + List pulsarServices = List.of(pulsar1, pulsar2, pulsar3); + for (PulsarService pulsarService : pulsarServices) { + // Test lookup heartbeat namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupHeartbeatOwner(pulsarService, + pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); + } + // Test lookup SLA namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupSLANamespaceOwner(pulsarService, + pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); + } + } + + // Test deploy new broker with new load manager + ServiceConfiguration conf = getDefaultConf(); + conf.setAllowAutoTopicCreation(true); + conf.setForceDeleteNamespaceAllowed(true); + conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + conf.setLoadManagerServiceUnitStateTableViewClassName(serviceUnitStateTableViewClassName); + try (var additionPulsarTestContext = createAdditionalPulsarTestContext(conf)) { + @Cleanup + var pulsar4 = additionPulsarTestContext.getPulsarService(); + + Set availableCandidates = Sets.newHashSet(pulsar1.getBrokerServiceUrl(), + pulsar2.getBrokerServiceUrl(), + pulsar4.getBrokerServiceUrl()); + String lookupResult4 = pulsar4.getAdminClient().lookups().lookupTopic(topic); + assertTrue(availableCandidates.contains(lookupResult4)); + + String lookupResult5 = pulsar1.getAdminClient().lookups().lookupTopic(topic); + String lookupResult6 = pulsar2.getAdminClient().lookups().lookupTopic(topic); + String lookupResult7 = pulsar3.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult4, lookupResult5); + assertEquals(lookupResult4, lookupResult6); + assertEquals(lookupResult4, lookupResult7); + + Set availableWebUrlCandidates = Sets.newHashSet(pulsar1.getWebServiceAddress(), + pulsar2.getWebServiceAddress(), + pulsar4.getWebServiceAddress()); + + webServiceUrl1 = pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); assertTrue(webServiceUrl1.isPresent()); - assertEquals(webServiceUrl1.get().toString(), pulsar3.getWebServiceAddress()); + assertTrue(availableWebUrlCandidates.contains(webServiceUrl1.get().toString())); - Optional webServiceUrl2 = + webServiceUrl2 = pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); assertTrue(webServiceUrl2.isPresent()); assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); - Optional webServiceUrl3 = + // The pulsar3 will redirect to pulsar4 + webServiceUrl3 = pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); assertTrue(webServiceUrl3.isPresent()); - assertEquals(webServiceUrl3.get().toString(), webServiceUrl1.get().toString()); + // It will redirect to pulsar4 + assertTrue(availableWebUrlCandidates.contains(webServiceUrl3.get().toString())); + + var webServiceUrl4 = + pulsar4.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl4.isPresent()); + assertEquals(webServiceUrl4.get().toString(), webServiceUrl1.get().toString()); - List pulsarServices = List.of(pulsar1, pulsar2, pulsar3); + pulsarServices = List.of(pulsar1, pulsar2, pulsar3, pulsar4); for (PulsarService pulsarService : pulsarServices) { // Test lookup heartbeat namespace's topic for (PulsarService pulsar : pulsarServices) { @@ -1101,111 +1193,269 @@ public void testDeployAndRollbackLoadManager() throws Exception { pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); } } + // Check if the broker is available + var wrapper = (ExtensibleLoadManagerWrapper) pulsar4.getLoadManager().get(); + var loadManager4 = spy((ExtensibleLoadManagerImpl) + FieldUtils.readField(wrapper, "loadManager", true)); + loadManager4.getBrokerRegistry().unregister(); + + NamespaceName slaMonitorNamespace = + getSLAMonitorNamespace(pulsar4.getBrokerId(), pulsar.getConfiguration()); + String slaMonitorTopic = slaMonitorNamespace.getPersistentTopicName("test"); + String result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertNotEquals(result, pulsar4.getBrokerServiceUrl()); + + Producer producer = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer.send("t1"); + + // Test re-register broker and check the lookup result + loadManager4.getBrokerRegistry().register(); + + result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertEquals(result, pulsar4.getBrokerServiceUrl()); + + producer.send("t2"); + Producer producer1 = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer1.send("t3"); + + producer.close(); + producer1.close(); + @Cleanup + Consumer consumer = pulsar.getClient().newConsumer(Schema.STRING) + .topic(slaMonitorTopic) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName("test") + .subscribe(); + // receive message t1 t2 t3 + assertEquals(consumer.receive().getValue(), "t1"); + assertEquals(consumer.receive().getValue(), "t2"); + assertEquals(consumer.receive().getValue(), "t3"); + } + } + } - // Test deploy new broker with new load manager - ServiceConfiguration conf = getDefaultConf(); - conf.setAllowAutoTopicCreation(true); - conf.setForceDeleteNamespaceAllowed(true); - conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); - conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); - try (var additionPulsarTestContext = createAdditionalPulsarTestContext(conf)) { - var pulsar4 = additionPulsarTestContext.getPulsarService(); - - Set availableCandidates = Sets.newHashSet(pulsar1.getBrokerServiceUrl(), - pulsar2.getBrokerServiceUrl(), - pulsar4.getBrokerServiceUrl()); - String lookupResult4 = pulsar4.getAdminClient().lookups().lookupTopic(topic); - assertTrue(availableCandidates.contains(lookupResult4)); - - String lookupResult5 = pulsar1.getAdminClient().lookups().lookupTopic(topic); - String lookupResult6 = pulsar2.getAdminClient().lookups().lookupTopic(topic); - String lookupResult7 = pulsar3.getAdminClient().lookups().lookupTopic(topic); - assertEquals(lookupResult4, lookupResult5); - assertEquals(lookupResult4, lookupResult6); - assertEquals(lookupResult4, lookupResult7); - - Set availableWebUrlCandidates = Sets.newHashSet(pulsar1.getWebServiceAddress(), - pulsar2.getWebServiceAddress(), - pulsar4.getWebServiceAddress()); - - webServiceUrl1 = - pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); - assertTrue(webServiceUrl1.isPresent()); - assertTrue(availableWebUrlCandidates.contains(webServiceUrl1.get().toString())); - - webServiceUrl2 = - pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); - assertTrue(webServiceUrl2.isPresent()); - assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); - - // The pulsar3 will redirect to pulsar4 - webServiceUrl3 = - pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); - assertTrue(webServiceUrl3.isPresent()); - // It will redirect to pulsar4 - assertTrue(availableWebUrlCandidates.contains(webServiceUrl3.get().toString())); - - var webServiceUrl4 = - pulsar4.getNamespaceService().getWebServiceUrl(bundle, options); - assertTrue(webServiceUrl4.isPresent()); - assertEquals(webServiceUrl4.get().toString(), webServiceUrl1.get().toString()); - - pulsarServices = List.of(pulsar1, pulsar2, pulsar3, pulsar4); - for (PulsarService pulsarService : pulsarServices) { - // Test lookup heartbeat namespace's topic - for (PulsarService pulsar : pulsarServices) { - assertLookupHeartbeatOwner(pulsarService, - pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); - } - // Test lookup SLA namespace's topic - for (PulsarService pulsar : pulsarServices) { - assertLookupSLANamespaceOwner(pulsarService, - pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); - } + @Test(priority = 200) + public void testLoadBalancerServiceUnitTableViewSyncer() throws Exception { + + Pair topicAndBundle = + getBundleIsNotOwnByChangeEventTopic("testLoadBalancerServiceUnitTableViewSyncer"); + TopicName topicName = topicAndBundle.getLeft(); + NamespaceBundle bundle = topicAndBundle.getRight(); + String topic = topicName.toString(); + + String lookupResultBefore1 = pulsar1.getAdminClient().lookups().lookupTopic(topic); + String lookupResultBefore2 = pulsar2.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResultBefore1, lookupResultBefore2); + + LookupOptions options = LookupOptions.builder() + .authoritative(false) + .requestHttps(false) + .readOnly(false) + .loadTopicsInBundle(false).build(); + Optional webServiceUrlBefore1 = + pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrlBefore1.isPresent()); + + Optional webServiceUrlBefore2 = + pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrlBefore2.isPresent()); + assertEquals(webServiceUrlBefore2.get().toString(), webServiceUrlBefore1.get().toString()); + + + String syncerTyp = serviceUnitStateTableViewClassName.equals(ServiceUnitStateTableViewImpl.class.getName()) ? + "SystemTopicToMetadataStoreSyncer" : "MetadataStoreToSystemTopicSyncer"; + pulsar.getAdminClient().brokers() + .updateDynamicConfiguration("loadBalancerServiceUnitTableViewSyncer", syncerTyp); + makeSecondaryAsLeader(); + makePrimaryAsLeader(); + Awaitility.waitAtMost(10, TimeUnit.SECONDS) + .untilAsserted(() -> assertTrue(primaryLoadManager.getServiceUnitStateTableViewSyncer().isActive())); + Awaitility.waitAtMost(10, TimeUnit.SECONDS) + .untilAsserted(() -> assertFalse(secondaryLoadManager.getServiceUnitStateTableViewSyncer().isActive())); + ServiceConfiguration defaultConf = getDefaultConf(); + defaultConf.setAllowAutoTopicCreation(true); + defaultConf.setForceDeleteNamespaceAllowed(true); + defaultConf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getCanonicalName()); + defaultConf.setLoadBalancerSheddingEnabled(false); + defaultConf.setLoadManagerServiceUnitStateTableViewClassName(ServiceUnitStateTableViewImpl.class.getName()); + try (var additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf)) { + // start pulsar3 with ServiceUnitStateTableViewImpl + @Cleanup + var pulsar3 = additionalPulsarTestContext.getPulsarService(); + + String lookupResult1 = pulsar3.getAdminClient().lookups().lookupTopic(topic); + String lookupResult2 = pulsar1.getAdminClient().lookups().lookupTopic(topic); + String lookupResult3 = pulsar2.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult1, lookupResult2); + assertEquals(lookupResult1, lookupResult3); + assertEquals(lookupResult1, lookupResultBefore1); + + Optional webServiceUrl1 = + pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl1.isPresent()); + + Optional webServiceUrl2 = + pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl2.isPresent()); + assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); + + Optional webServiceUrl3 = + pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl3.isPresent()); + assertEquals(webServiceUrl3.get().toString(), webServiceUrl1.get().toString()); + + assertEquals(webServiceUrl3.get().toString(), webServiceUrlBefore1.get().toString()); + + List pulsarServices = List.of(pulsar1, pulsar2, pulsar3); + for (PulsarService pulsarService : pulsarServices) { + // Test lookup heartbeat namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupHeartbeatOwner(pulsarService, + pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); + } + // Test lookup SLA namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupSLANamespaceOwner(pulsarService, + pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); + } + } + + // Start broker4 with ServiceUnitStateMetadataStoreTableViewImpl + ServiceConfiguration conf = getDefaultConf(); + conf.setAllowAutoTopicCreation(true); + conf.setForceDeleteNamespaceAllowed(true); + conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getCanonicalName()); + conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + conf.setLoadManagerServiceUnitStateTableViewClassName( + ServiceUnitStateMetadataStoreTableViewImpl.class.getName()); + try (var additionPulsarTestContext = createAdditionalPulsarTestContext(conf)) { + @Cleanup + var pulsar4 = additionPulsarTestContext.getPulsarService(); + + Set availableCandidates = Sets.newHashSet( + pulsar1.getBrokerServiceUrl(), + pulsar2.getBrokerServiceUrl(), + pulsar3.getBrokerServiceUrl(), + pulsar4.getBrokerServiceUrl()); + String lookupResult4 = pulsar4.getAdminClient().lookups().lookupTopic(topic); + assertTrue(availableCandidates.contains(lookupResult4)); + + String lookupResult5 = pulsar1.getAdminClient().lookups().lookupTopic(topic); + String lookupResult6 = pulsar2.getAdminClient().lookups().lookupTopic(topic); + String lookupResult7 = pulsar3.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult4, lookupResult5); + assertEquals(lookupResult4, lookupResult6); + assertEquals(lookupResult4, lookupResult7); + assertEquals(lookupResult4, lookupResultBefore1); + + + Pair topicAndBundle2 = + getBundleIsNotOwnByChangeEventTopic("testLoadBalancerServiceUnitTableViewSyncer2"); + String topic2 = topicAndBundle2.getLeft().toString(); + + String lookupResult8 = pulsar1.getAdminClient().lookups().lookupTopic(topic2); + String lookupResult9 = pulsar2.getAdminClient().lookups().lookupTopic(topic2); + String lookupResult10 = pulsar3.getAdminClient().lookups().lookupTopic(topic2); + String lookupResult11 = pulsar4.getAdminClient().lookups().lookupTopic(topic2); + assertEquals(lookupResult9, lookupResult8); + assertEquals(lookupResult10, lookupResult8); + assertEquals(lookupResult11, lookupResult8); + + Set availableWebUrlCandidates = Sets.newHashSet( + pulsar1.getWebServiceAddress(), + pulsar2.getWebServiceAddress(), + pulsar3.getWebServiceAddress(), + pulsar4.getWebServiceAddress()); + + webServiceUrl1 = + pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl1.isPresent()); + assertTrue(availableWebUrlCandidates.contains(webServiceUrl1.get().toString())); + + webServiceUrl2 = + pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl2.isPresent()); + assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); + + webServiceUrl3 = + pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl3.isPresent()); + assertTrue(availableWebUrlCandidates.contains(webServiceUrl3.get().toString())); + + var webServiceUrl4 = + pulsar4.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl4.isPresent()); + assertEquals(webServiceUrl4.get().toString(), webServiceUrl1.get().toString()); + assertEquals(webServiceUrl4.get().toString(), webServiceUrlBefore1.get().toString()); + + pulsarServices = List.of(pulsar1, pulsar2, pulsar3, pulsar4); + for (PulsarService pulsarService : pulsarServices) { + // Test lookup heartbeat namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupHeartbeatOwner(pulsarService, + pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); + } + // Test lookup SLA namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupSLANamespaceOwner(pulsarService, + pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); } - // Check if the broker is available - var wrapper = (ExtensibleLoadManagerWrapper) pulsar4.getLoadManager().get(); - var loadManager4 = spy((ExtensibleLoadManagerImpl) - FieldUtils.readField(wrapper, "loadManager", true)); - loadManager4.getBrokerRegistry().unregister(); - - NamespaceName slaMonitorNamespace = - getSLAMonitorNamespace(pulsar4.getBrokerId(), pulsar.getConfiguration()); - String slaMonitorTopic = slaMonitorNamespace.getPersistentTopicName("test"); - String result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); - assertNotNull(result); - log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); - assertNotEquals(result, pulsar4.getBrokerServiceUrl()); - - Producer producer = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); - producer.send("t1"); - - // Test re-register broker and check the lookup result - loadManager4.getBrokerRegistry().register(); - - result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); - assertNotNull(result); - log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); - assertEquals(result, pulsar4.getBrokerServiceUrl()); - - producer.send("t2"); - Producer producer1 = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); - producer1.send("t3"); - - producer.close(); - producer1.close(); - @Cleanup - Consumer consumer = pulsar.getClient().newConsumer(Schema.STRING) - .topic(slaMonitorTopic) - .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) - .subscriptionName("test") - .subscribe(); - // receive message t1 t2 t3 - assertEquals(consumer.receive().getValue(), "t1"); - assertEquals(consumer.receive().getValue(), "t2"); - assertEquals(consumer.receive().getValue(), "t3"); } + // Check if the broker is available + var wrapper = (ExtensibleLoadManagerWrapper) pulsar4.getLoadManager().get(); + var loadManager4 = spy((ExtensibleLoadManagerImpl) + FieldUtils.readField(wrapper, "loadManager", true)); + loadManager4.getBrokerRegistry().unregister(); + + NamespaceName slaMonitorNamespace = + getSLAMonitorNamespace(pulsar4.getBrokerId(), pulsar.getConfiguration()); + String slaMonitorTopic = slaMonitorNamespace.getPersistentTopicName("test"); + String result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertNotEquals(result, pulsar4.getBrokerServiceUrl()); + + Producer producer = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer.send("t1"); + + // Test re-register broker and check the lookup result + loadManager4.getBrokerRegistry().register(); + + result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertEquals(result, pulsar4.getBrokerServiceUrl()); + + producer.send("t2"); + Producer producer1 = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer1.send("t3"); + + producer.close(); + producer1.close(); + @Cleanup + Consumer consumer = pulsar.getClient().newConsumer(Schema.STRING) + .topic(slaMonitorTopic) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName("test") + .subscribe(); + // receive message t1 t2 t3 + assertEquals(consumer.receive().getValue(), "t1"); + assertEquals(consumer.receive().getValue(), "t2"); + assertEquals(consumer.receive().getValue(), "t3"); } + } + + pulsar.getAdminClient().brokers() + .deleteDynamicConfiguration("loadBalancerServiceUnitTableViewSyncer"); + makeSecondaryAsLeader(); + Awaitility.waitAtMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> assertFalse(primaryLoadManager.getServiceUnitStateTableViewSyncer().isActive())); + Awaitility.waitAtMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> assertFalse(secondaryLoadManager.getServiceUnitStateTableViewSyncer().isActive())); } private void assertLookupHeartbeatOwner(PulsarService pulsar, @@ -1272,12 +1522,13 @@ private void makeSecondaryAsLeader() throws Exception { }); } - @Test(timeOut = 30 * 1000) + @Test(timeOut = 30 * 1000, priority = 2100) public void testRoleChangeIdempotency() throws Exception { makePrimaryAsLeader(); - var topBundlesLoadDataStorePrimary = primaryLoadManager.getTopBundlesLoadDataStore(); + var topBundlesLoadDataStorePrimary = + (TableViewLoadDataStoreImpl) primaryLoadManager.getTopBundlesLoadDataStore(); var topBundlesLoadDataStorePrimarySpy = spy(topBundlesLoadDataStorePrimary); AtomicInteger countPri = new AtomicInteger(3); AtomicInteger countPri2 = new AtomicInteger(3); @@ -1286,19 +1537,18 @@ public void testRoleChangeIdempotency() throws Exception { throw new RuntimeException(); } // Call the real method - reset(); - return null; + return invocationOnMock.callRealMethod(); }).when(topBundlesLoadDataStorePrimarySpy).startTableView(); doAnswer(invocationOnMock -> { if (countPri2.decrementAndGet() > 0) { throw new RuntimeException(); } // Call the real method - reset(); - return null; + return invocationOnMock.callRealMethod(); }).when(topBundlesLoadDataStorePrimarySpy).closeTableView(); - var topBundlesLoadDataStoreSecondary = secondaryLoadManager.getTopBundlesLoadDataStore(); + var topBundlesLoadDataStoreSecondary = + (TableViewLoadDataStoreImpl) secondaryLoadManager.getTopBundlesLoadDataStore(); var topBundlesLoadDataStoreSecondarySpy = spy(topBundlesLoadDataStoreSecondary); AtomicInteger countSec = new AtomicInteger(3); AtomicInteger countSec2 = new AtomicInteger(3); @@ -1306,17 +1556,14 @@ public void testRoleChangeIdempotency() throws Exception { if (countSec.decrementAndGet() > 0) { throw new RuntimeException(); } - // Call the real method - reset(); - return null; + return invocationOnMock.callRealMethod(); }).when(topBundlesLoadDataStoreSecondarySpy).startTableView(); doAnswer(invocationOnMock -> { if (countSec2.decrementAndGet() > 0) { throw new RuntimeException(); } // Call the real method - reset(); - return null; + return invocationOnMock.callRealMethod(); }).when(topBundlesLoadDataStoreSecondarySpy).closeTableView(); try { @@ -1325,8 +1572,6 @@ public void testRoleChangeIdempotency() throws Exception { FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStoreSecondarySpy, true); - - primaryLoadManager.playLeader(); secondaryLoadManager.playFollower(); verify(topBundlesLoadDataStorePrimarySpy, times(3)).startTableView(); @@ -1334,6 +1579,13 @@ public void testRoleChangeIdempotency() throws Exception { verify(topBundlesLoadDataStoreSecondarySpy, times(0)).startTableView(); verify(topBundlesLoadDataStoreSecondarySpy, times(3)).closeTableView(); + } finally { + FieldUtils.writeDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", + topBundlesLoadDataStorePrimary, true); + FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", + topBundlesLoadDataStoreSecondary, true); + } + primaryLoadManager.playFollower(); secondaryLoadManager.playFollower(); @@ -1350,14 +1602,9 @@ public void testRoleChangeIdempotency() throws Exception { assertEquals(ExtensibleLoadManagerImpl.Role.Follower, secondaryLoadManager.getRole()); - } finally { - FieldUtils.writeDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", - topBundlesLoadDataStorePrimary, true); - FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", - topBundlesLoadDataStoreSecondary, true); - } + } - @Test(timeOut = 30 * 1000) + @Test(timeOut = 30 * 1000, priority = 2000) public void testRoleChange() throws Exception { makePrimaryAsLeader(); @@ -1375,15 +1622,18 @@ public void testRoleChange() throws Exception { topBundlesExpected.getTopBundlesLoadData().clear(); topBundlesExpected.getTopBundlesLoadData().add(new TopBundlesLoadData.BundleLoadData(bundle, new NamespaceBundleStats())); - follower.getBrokerLoadDataStore().pushAsync(key, brokerLoadExpected); - follower.getTopBundlesLoadDataStore().pushAsync(bundle, topBundlesExpected); - Awaitility.await().atMost(30, TimeUnit.SECONDS).untilAsserted(() -> { assertNotNull(FieldUtils.readDeclaredField(leader.getTopBundlesLoadDataStore(), "tableView", true)); assertNull(FieldUtils.readDeclaredField(follower.getTopBundlesLoadDataStore(), "tableView", true)); + for (String internalTopic : ExtensibleLoadManagerImpl.INTERNAL_TOPICS) { + if (serviceUnitStateTableViewClassName + .equals(ServiceUnitStateMetadataStoreTableViewImpl.class.getCanonicalName()) + && internalTopic.equals(TOPIC)) { + continue; + } assertTrue(leader.pulsar.getBrokerService().getTopicReference(internalTopic) .isPresent()); assertTrue(follower.pulsar.getBrokerService().getTopicReference(internalTopic) @@ -1394,22 +1644,9 @@ public void testRoleChange() throws Exception { assertFalse(follower.pulsar.getNamespaceService() .isServiceUnitOwnedAsync(TopicName.get(internalTopic)).get()); } - - var actualBrokerLoadLeader = leader.getBrokerLoadDataStore().get(key); - if (actualBrokerLoadLeader.isPresent()) { - assertEquals(actualBrokerLoadLeader.get(), brokerLoadExpected); - } - - var actualTopBundlesLeader = leader.getTopBundlesLoadDataStore().get(bundle); - if (actualTopBundlesLeader.isPresent()) { - assertEquals(actualTopBundlesLeader.get(), topBundlesExpected); - } - - var actualBrokerLoadFollower = follower.getBrokerLoadDataStore().get(key); - if (actualBrokerLoadFollower.isPresent()) { - assertEquals(actualBrokerLoadFollower.get(), brokerLoadExpected); - } }); + follower.getBrokerLoadDataStore().pushAsync(key, brokerLoadExpected).get(3, TimeUnit.SECONDS); + follower.getTopBundlesLoadDataStore().pushAsync(bundle, topBundlesExpected).get(3, TimeUnit.SECONDS); makeSecondaryAsLeader(); @@ -1419,14 +1656,16 @@ public void testRoleChange() throws Exception { brokerLoadExpected.update(usage, 1, 0, 0, 0, 0, 0, conf); topBundlesExpected.getTopBundlesLoadData().get(0).stats().msgRateIn = 1; - follower.getBrokerLoadDataStore().pushAsync(key, brokerLoadExpected); - follower.getTopBundlesLoadDataStore().pushAsync(bundle, topBundlesExpected); - Awaitility.await().atMost(30, TimeUnit.SECONDS).ignoreExceptions().untilAsserted(() -> { assertNotNull(FieldUtils.readDeclaredField(leader2.getTopBundlesLoadDataStore(), "tableView", true)); assertNull(FieldUtils.readDeclaredField(follower2.getTopBundlesLoadDataStore(), "tableView", true)); for (String internalTopic : ExtensibleLoadManagerImpl.INTERNAL_TOPICS) { + if (serviceUnitStateTableViewClassName + .equals(ServiceUnitStateMetadataStoreTableViewImpl.class.getCanonicalName()) + && internalTopic.equals(TOPIC)) { + continue; + } assertTrue(leader2.pulsar.getBrokerService().getTopicReference(internalTopic) .isPresent()); assertTrue(follower2.pulsar.getBrokerService().getTopicReference(internalTopic) @@ -1437,17 +1676,10 @@ public void testRoleChange() throws Exception { assertFalse(follower2.pulsar.getNamespaceService() .isServiceUnitOwnedAsync(TopicName.get(internalTopic)).get()); } - - - var actualBrokerLoadLeader = leader2.getBrokerLoadDataStore().get(key); - assertEquals(actualBrokerLoadLeader.get(), brokerLoadExpected); - - var actualTopBundlesLeader = leader2.getTopBundlesLoadDataStore().get(bundle); - assertEquals(actualTopBundlesLeader.get(), topBundlesExpected); - - var actualBrokerLoadFollower = follower2.getBrokerLoadDataStore().get(key); - assertEquals(actualBrokerLoadFollower.get(), brokerLoadExpected); }); + + follower2.getBrokerLoadDataStore().pushAsync(key, brokerLoadExpected).get(3, TimeUnit.SECONDS); + follower2.getTopBundlesLoadDataStore().pushAsync(bundle, topBundlesExpected).get(3, TimeUnit.SECONDS); } @Test @@ -1647,7 +1879,7 @@ SplitDecision.Reason.Unknown, new AtomicLong(6)) assertEquals(actual, expected); } - @Test + @Test(priority = 100) public void testDisableBroker() throws Exception { // Test rollback to modular load manager. ServiceConfiguration defaultConf = getDefaultConf(); @@ -1658,7 +1890,9 @@ public void testDisableBroker() throws Exception { defaultConf.setLoadBalancerSheddingEnabled(false); defaultConf.setLoadBalancerDebugModeEnabled(true); defaultConf.setTopicLevelPoliciesEnabled(false); + defaultConf.setLoadManagerServiceUnitStateTableViewClassName(serviceUnitStateTableViewClassName); try (var additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf)) { + @Cleanup var pulsar3 = additionalPulsarTestContext.getPulsarService(); ExtensibleLoadManagerImpl ternaryLoadManager = spy((ExtensibleLoadManagerImpl) FieldUtils.readField(pulsar3.getLoadManager().get(), "loadManager", true)); @@ -1767,15 +2001,13 @@ public void testGetOwnedServiceUnitsAndGetOwnedNamespaceStatus() throws Exceptio .getFullBundle(slaMonitorNamespacePulsar2); - Set ownedServiceUnitsByPulsar1 = primaryLoadManager.getOwnedServiceUnitsAsync() - .get(5, TimeUnit.SECONDS); + Set ownedServiceUnitsByPulsar1 = primaryLoadManager.getOwnedServiceUnits(); log.info("Owned service units: {}", ownedServiceUnitsByPulsar1); // heartbeat namespace bundle will own by pulsar1 assertTrue(ownedServiceUnitsByPulsar1.contains(bundle1)); assertTrue(ownedServiceUnitsByPulsar1.contains(bundle2)); assertTrue(ownedServiceUnitsByPulsar1.contains(slaBundle1)); - Set ownedServiceUnitsByPulsar2 = secondaryLoadManager.getOwnedServiceUnitsAsync() - .get(5, TimeUnit.SECONDS); + Set ownedServiceUnitsByPulsar2 = secondaryLoadManager.getOwnedServiceUnits(); log.info("Owned service units: {}", ownedServiceUnitsByPulsar2); assertTrue(ownedServiceUnitsByPulsar2.contains(bundle3)); assertTrue(ownedServiceUnitsByPulsar2.contains(bundle4)); @@ -1811,8 +2043,7 @@ private void assertOwnedServiceUnits( ExtensibleLoadManagerImpl extensibleLoadManager, NamespaceBundle bundle) throws PulsarAdminException { Awaitility.await().untilAsserted(() -> { - Set ownedBundles = extensibleLoadManager.getOwnedServiceUnitsAsync() - .get(5, TimeUnit.SECONDS); + Set ownedBundles = extensibleLoadManager.getOwnedServiceUnits(); assertTrue(ownedBundles.contains(bundle)); }); Map ownedNamespaces = @@ -1828,8 +2059,7 @@ private void assertOwnedServiceUnits( public void testGetOwnedServiceUnitsWhenLoadManagerNotStart() throws Exception { ExtensibleLoadManagerImpl loadManager = new ExtensibleLoadManagerImpl(); - Set ownedServiceUnits = loadManager.getOwnedServiceUnitsAsync() - .get(5, TimeUnit.SECONDS); + Set ownedServiceUnits = loadManager.getOwnedServiceUnits(); assertNotNull(ownedServiceUnits); assertTrue(ownedServiceUnits.isEmpty()); } @@ -1858,6 +2088,11 @@ public void testHealthcheck() throws PulsarAdminException { @Test(timeOut = 30 * 1000) public void compactionScheduleTest() { + if (serviceUnitStateTableViewClassName.equals( + ServiceUnitStateMetadataStoreTableViewImpl.class.getCanonicalName())) { + // no topic compaction happens + return; + } Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) .atMost(30, TimeUnit.SECONDS) @@ -1866,37 +2101,11 @@ public void compactionScheduleTest() { primaryLoadManager.monitor(); secondaryLoadManager.monitor(); var threshold = admin.topicPolicies() - .getCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC, false); + .getCompactionThreshold(TOPIC, false); AssertJUnit.assertEquals(5 * 1024 * 1024, threshold == null ? 0 : threshold.longValue()); }); } - @Test(timeOut = 10 * 1000) - public void unloadTimeoutCheckTest() - throws Exception { - Pair topicAndBundle = getBundleIsNotOwnByChangeEventTopic("unload-timeout"); - String topic = topicAndBundle.getLeft().toString(); - var bundle = topicAndBundle.getRight().toString(); - var releasing = new ServiceUnitStateData(Releasing, pulsar2.getBrokerId(), pulsar1.getBrokerId(), 1); - overrideTableView(channel1, bundle, releasing); - var topicFuture = pulsar1.getBrokerService().getOrCreateTopic(topic); - - - try { - topicFuture.get(1, TimeUnit.SECONDS); - } catch (Exception e) { - log.info("getOrCreateTopic failed", e); - if (!(e.getCause() instanceof BrokerServiceException.ServiceUnitNotReadyException && e.getMessage() - .contains("Please redo the lookup"))) { - fail(); - } - } - - pulsar1.getBrokerService() - .unloadServiceUnit(topicAndBundle.getRight(), true, true, 5, - TimeUnit.SECONDS).get(2, TimeUnit.SECONDS); - } - private static abstract class MockBrokerFilter implements BrokerFilter { @Override @@ -1905,12 +2114,4 @@ public String name() { } } - - protected static PulsarClient pulsarClient(String url, int intervalInSecs) throws PulsarClientException { - return - PulsarClient.builder() - .serviceUrl(url) - .statsInterval(intervalInSecs, TimeUnit.SECONDS).build(); - } - } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithAdvertisedListenersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithAdvertisedListenersTest.java index bec7d4d78fe7e..b9c945fe81571 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithAdvertisedListenersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithAdvertisedListenersTest.java @@ -25,6 +25,7 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.TopicDomain; import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; /** @@ -36,8 +37,10 @@ public class ExtensibleLoadManagerImplWithAdvertisedListenersTest extends ExtensibleLoadManagerImplBaseTest { public String brokerServiceUrl; - public ExtensibleLoadManagerImplWithAdvertisedListenersTest() { - super("public/test"); + + @Factory(dataProvider = "serviceUnitStateTableViewClassName") + public ExtensibleLoadManagerImplWithAdvertisedListenersTest(String serviceUnitStateTableViewClassName) { + super("public/test", serviceUnitStateTableViewClassName); } @Override @@ -69,7 +72,9 @@ public Object[][] isPersistentTopicSubscriptionTypeTest() { @Test(timeOut = 30_000, dataProvider = "isPersistentTopicSubscriptionTypeTest") public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, SubscriptionType subscriptionType) throws Exception { - ExtensibleLoadManagerImplTest.testTransferClientReconnectionWithoutLookup(topicDomain, subscriptionType, + ExtensibleLoadManagerImplTest.testTransferClientReconnectionWithoutLookup( + clients, + topicDomain, subscriptionType, defaultTestNamespace, admin, brokerServiceUrl, pulsar1, pulsar2, primaryLoadManager, secondaryLoadManager); @@ -78,7 +83,9 @@ public void testTransferClientReconnectionWithoutLookup(TopicDomain topicDomain, @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicSubscriptionTypeTest") public void testUnloadClientReconnectionWithLookup(TopicDomain topicDomain, SubscriptionType subscriptionType) throws Exception { - ExtensibleLoadManagerImplTest.testUnloadClientReconnectionWithLookup(topicDomain, subscriptionType, + ExtensibleLoadManagerImplTest.testUnloadClientReconnectionWithLookup( + clients, + topicDomain, subscriptionType, defaultTestNamespace, admin, brokerServiceUrl, pulsar1); @@ -91,7 +98,9 @@ public Object[][] isPersistentTopicTest() { @Test(timeOut = 30 * 1000, dataProvider = "isPersistentTopicTest") public void testOptimizeUnloadDisable(TopicDomain topicDomain) throws Exception { - ExtensibleLoadManagerImplTest.testOptimizeUnloadDisable(topicDomain, defaultTestNamespace, admin, + ExtensibleLoadManagerImplTest.testOptimizeUnloadDisable( + clients, + topicDomain, defaultTestNamespace, admin, brokerServiceUrl, pulsar1, pulsar2); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithTransactionCoordinatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithTransactionCoordinatorTest.java index ed99b502b7e29..1d3f02f4e717d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithTransactionCoordinatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplWithTransactionCoordinatorTest.java @@ -21,13 +21,15 @@ import static org.testng.Assert.assertEquals; import org.apache.pulsar.broker.ServiceConfiguration; import org.awaitility.Awaitility; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "broker") public class ExtensibleLoadManagerImplWithTransactionCoordinatorTest extends ExtensibleLoadManagerImplBaseTest { - public ExtensibleLoadManagerImplWithTransactionCoordinatorTest() { - super("public/test-elb-with-tx"); + @Factory(dataProvider = "serviceUnitStateTableViewClassName") + public ExtensibleLoadManagerImplWithTransactionCoordinatorTest(String serviceUnitStateTableViewClassName) { + super("public/test-elb-with-tx", serviceUnitStateTableViewClassName); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java index e569f0d32d573..92cdf61f44269 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java @@ -30,6 +30,7 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Unload; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MAX_CLEAN_UP_DELAY_TIME_IN_SECS; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.TOPIC; import static org.apache.pulsar.metadata.api.extended.SessionEvent.ConnectionLost; import static org.apache.pulsar.metadata.api.extended.SessionEvent.Reconnected; import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionLost; @@ -37,23 +38,22 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; -import static org.testng.Assert.assertNotEquals; -import static org.testng.Assert.assertThrows; -import static org.testng.Assert.expectThrows; -import static org.testng.Assert.fail; -import static org.testng.AssertJUnit.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.expectThrows; +import static org.testng.Assert.fail; +import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertNull; import java.lang.reflect.Field; @@ -70,13 +70,14 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import lombok.Cleanup; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.loadbalance.LeaderElectionService; import org.apache.pulsar.broker.loadbalance.extensions.BrokerRegistryImpl; @@ -86,12 +87,14 @@ import org.apache.pulsar.broker.loadbalance.extensions.models.Unload; import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStore; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.testcontext.PulsarTestContext; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.TypedMessageBuilder; -import org.apache.pulsar.client.impl.TableViewImpl; +import org.apache.pulsar.client.api.TableView; +import org.apache.pulsar.common.naming.NamespaceBundle; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.MetadataStoreTableView; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.coordination.LeaderElectionState; import org.apache.pulsar.metadata.api.extended.SessionEvent; @@ -99,6 +102,8 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "broker") @@ -109,6 +114,8 @@ public class ServiceUnitStateChannelTest extends MockedPulsarServiceBaseTest { private PulsarService pulsar2; private ServiceUnitStateChannel channel1; private ServiceUnitStateChannel channel2; + private String namespaceName; + private String namespaceName2; private String brokerId1; private String brokerId2; private String brokerId3; @@ -131,19 +138,40 @@ public class ServiceUnitStateChannelTest extends MockedPulsarServiceBaseTest { private ExtensibleLoadManagerImpl loadManager; - @BeforeClass - @Override - protected void setup() throws Exception { + private final String serviceUnitStateTableViewClassName; + + @DataProvider(name = "serviceUnitStateTableViewClassName") + public static Object[][] serviceUnitStateTableViewClassName() { + return new Object[][]{ + {ServiceUnitStateTableViewImpl.class.getName()}, + {ServiceUnitStateMetadataStoreTableViewImpl.class.getName()} + }; + } + + @Factory(dataProvider = "serviceUnitStateTableViewClassName") + public ServiceUnitStateChannelTest(String serviceUnitStateTableViewClassName) { + this.serviceUnitStateTableViewClassName = serviceUnitStateTableViewClassName; + } + + private void updateConfig(ServiceConfiguration conf) { conf.setAllowAutoTopicCreation(true); conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); conf.setLoadBalancerDebugModeEnabled(true); conf.setBrokerServiceCompactionMonitorIntervalInSeconds(10); + conf.setLoadManagerServiceUnitStateTableViewClassName(serviceUnitStateTableViewClassName); + } + + @BeforeClass + @Override + protected void setup() throws Exception { + updateConfig(conf); super.internalSetup(conf); - admin.tenants().createTenant("pulsar", createDefaultTenantInfo()); - admin.namespaces().createNamespace("pulsar/system"); - admin.tenants().createTenant("public", createDefaultTenantInfo()); - admin.namespaces().createNamespace("public/default"); + namespaceName = "my-tenant/my-ns"; + namespaceName2 = "my-tenant/my-ns2"; + admin.tenants().createTenant("my-tenant", createDefaultTenantInfo()); + admin.namespaces().createNamespace(namespaceName); + admin.namespaces().createNamespace(namespaceName2); pulsar1 = pulsar; registry = new BrokerRegistryImpl(pulsar); @@ -151,7 +179,9 @@ protected void setup() throws Exception { doReturn(mock(LoadDataStore.class)).when(loadManagerContext).brokerLoadDataStore(); doReturn(mock(LoadDataStore.class)).when(loadManagerContext).topBundleLoadDataStore(); loadManager = mock(ExtensibleLoadManagerImpl.class); - additionalPulsarTestContext = createAdditionalPulsarTestContext(getDefaultConf()); + var conf2 = getDefaultConf(); + updateConfig(conf2); + additionalPulsarTestContext = createAdditionalPulsarTestContext(conf2); pulsar2 = additionalPulsarTestContext.getPulsarService(); channel1 = createChannel(pulsar1); @@ -165,22 +195,23 @@ protected void setup() throws Exception { FieldUtils.readDeclaredField(channel2, "brokerId", true); brokerId3 = "broker-3"; - bundle = "public/default/0x00000000_0xffffffff"; - bundle1 = "public/default/0x00000000_0xfffffff0"; - bundle2 = "public/default/0xfffffff0_0xffffffff"; - bundle3 = "public/default3/0x00000000_0xffffffff"; + bundle = namespaceName + "/0x00000000_0xffffffff"; + bundle1 = namespaceName + "/0x00000000_0xfffffff0"; + bundle2 = namespaceName + "/0xfffffff0_0xffffffff"; + bundle3 = namespaceName2 + "/0x00000000_0xffffffff"; childBundle1Range = "0x7fffffff_0xffffffff"; childBundle2Range = "0x00000000_0x7fffffff"; - childBundle11 = "public/default/" + childBundle1Range; - childBundle12 = "public/default/" + childBundle2Range; + childBundle11 = namespaceName + "/" + childBundle1Range; + childBundle12 = namespaceName + "/" + childBundle2Range; - childBundle31 = "public/default3/" + childBundle1Range; - childBundle32 = "public/default3/" + childBundle2Range; + childBundle31 = namespaceName2 + "/" + childBundle1Range; + childBundle32 = namespaceName2 + "/" + childBundle2Range; } @BeforeMethod protected void initChannels() throws Exception { + disableChannels(); cleanTableViews(); cleanOwnershipMonitorCounters(channel1); cleanOwnershipMonitorCounters(channel2); @@ -188,6 +219,7 @@ protected void initChannels() throws Exception { cleanOpsCounters(channel2); cleanMetadataState(channel1); cleanMetadataState(channel2); + enableChannels(); } @@ -205,7 +237,7 @@ protected void cleanup() throws Exception { super.internalCleanup(); } - @Test(priority = -1) + @Test(priority = 0) public void channelOwnerTest() throws Exception { var channelOwner1 = channel1.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); var channelOwner2 = channel2.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); @@ -232,7 +264,7 @@ public void channelOwnerTest() throws Exception { } } - @Test(priority = 0) + @Test(priority = 100) public void channelValidationTest() throws ExecutionException, InterruptedException, IllegalAccessException, PulsarServerException, TimeoutException { @@ -256,7 +288,7 @@ public void channelValidationTest() ServiceUnitStateChannelImpl.ChannelState.LeaderElectionServiceStarted, true); assertNotNull(channel.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get()); - Future closeFuture = executor.submit(()->{ + Future closeFuture = executor.submit(() -> { try { channel.close(); } catch (PulsarServerException e) { @@ -289,7 +321,7 @@ private int validateChannelStart(ServiceUnitStateChannelImpl channel) try { channel.isChannelOwnerAsync().get(2, TimeUnit.SECONDS); } catch (ExecutionException e) { - if(e.getCause() instanceof IllegalStateException){ + if (e.getCause() instanceof IllegalStateException) { errorCnt++; } } @@ -316,7 +348,7 @@ private int validateChannelStart(ServiceUnitStateChannelImpl channel) } try { channel.publishUnloadEventAsync( - new Unload(brokerId1, bundle, Optional.of(brokerId2))) + new Unload(brokerId1, bundle, Optional.of(brokerId2))) .get(2, TimeUnit.SECONDS); } catch (ExecutionException e) { if (e.getCause() instanceof IllegalStateException) { @@ -382,35 +414,33 @@ public void assignmentTestWhenOneAssignmentFails() assertEquals(0, getOwnerRequests1.size()); assertEquals(0, getOwnerRequests2.size()); - var producer = (Producer) FieldUtils.readDeclaredField(channel1, - "producer", true); - var spyProducer = spy(producer); - var msg = mock(TypedMessageBuilder.class); - var future = spy(CompletableFuture.failedFuture(new RuntimeException())); - doReturn(msg).when(spyProducer).newMessage(); - doReturn(msg).when(msg).key(any()); - doReturn(msg).when(msg).value(any()); - doReturn(future).when(msg).sendAsync(); - - FieldUtils.writeDeclaredField(channel1, "producer", spyProducer, true); - - var owner1 = channel1.getOwnerAsync(bundle); - var owner2 = channel2.getOwnerAsync(bundle); - - assertTrue(owner1.get().isEmpty()); - assertTrue(owner2.get().isEmpty()); + var tableView = getTableView(channel1); + var spyTableView = spy(tableView); + var future = CompletableFuture.failedFuture(new RuntimeException()); + doReturn(future).when(spyTableView).put(any(), any()); - var owner3 = channel1.publishAssignEventAsync(bundle, brokerId1); - var owner4 = channel2.publishAssignEventAsync(bundle, brokerId2); - assertTrue(owner3.isCompletedExceptionally()); - assertNotNull(owner4); - String ownerAddrOpt2 = owner4.get(5, TimeUnit.SECONDS); - assertEquals(ownerAddrOpt2, brokerId2); - waitUntilNewOwner(channel1, bundle, brokerId2); - assertEquals(0, getOwnerRequests1.size()); - assertEquals(0, getOwnerRequests2.size()); + try { + setTableView(channel1, spyTableView); + + var owner1 = channel1.getOwnerAsync(bundle); + var owner2 = channel2.getOwnerAsync(bundle); + + assertTrue(owner1.get().isEmpty()); + assertTrue(owner2.get().isEmpty()); + var owner3 = channel1.publishAssignEventAsync(bundle, brokerId1); + var owner4 = channel2.publishAssignEventAsync(bundle, brokerId2); + + assertTrue(owner3.isCompletedExceptionally()); + assertNotNull(owner4); + String ownerAddrOpt2 = owner4.get(5, TimeUnit.SECONDS); + assertEquals(ownerAddrOpt2, brokerId2); + waitUntilNewOwner(channel1, bundle, brokerId2); + assertEquals(0, getOwnerRequests1.size()); + assertEquals(0, getOwnerRequests2.size()); + } finally { + setTableView(channel1, tableView); + } - FieldUtils.writeDeclaredField(channel1, "producer", producer, true); } @Test(priority = 4) @@ -423,7 +453,6 @@ public void transferTest() assertTrue(owner1.get().isEmpty()); assertTrue(owner2.get().isEmpty()); - channel1.publishAssignEventAsync(bundle, brokerId1); waitUntilNewOwner(channel1, bundle, brokerId1); waitUntilNewOwner(channel2, bundle, brokerId1); @@ -468,78 +497,78 @@ public void transferTestWhenDestBrokerFails() assertEquals(ownerAddr1, ownerAddr2); assertEquals(ownerAddr1, Optional.of(brokerId1)); - var producer = (Producer) FieldUtils.readDeclaredField(channel1, - "producer", true); - var spyProducer = spy(producer); - var msg = mock(TypedMessageBuilder.class); + var tableView = getTableView(channel2); + var spyTableView = spy(tableView); var future = CompletableFuture.failedFuture(new RuntimeException()); - doReturn(msg).when(spyProducer).newMessage(); - doReturn(msg).when(msg).key(any()); - doReturn(msg).when(msg).value(any()); - doReturn(future).when(msg).sendAsync(); - FieldUtils.writeDeclaredField(channel2, "producer", spyProducer, true); - FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 3 * 1000, true); - FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 3 * 1000, true); - Unload unload = new Unload(brokerId1, bundle, Optional.of(brokerId2)); - channel1.publishUnloadEventAsync(unload); - // channel1 is broken. the ownership transfer won't be complete. - waitUntilState(channel1, bundle); - waitUntilState(channel2, bundle); - var owner1 = channel1.getOwnerAsync(bundle); - var owner2 = channel2.getOwnerAsync(bundle); - - assertTrue(owner1.isDone()); - assertEquals(brokerId2, owner1.get().get()); - assertFalse(owner2.isDone()); - - assertEquals(0, getOwnerRequests1.size()); - assertEquals(1, getOwnerRequests2.size()); - - // In 10 secs, the getOwnerAsync requests(lookup requests) should time out. - Awaitility.await().atMost(10, TimeUnit.SECONDS) - .untilAsserted(() -> assertTrue(owner2.isCompletedExceptionally())); - - assertEquals(0, getOwnerRequests2.size()); - - // recovered, check the monitor update state : Assigned -> Owned - doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) - .when(loadManager).selectAsync(any(), any(), any()); - FieldUtils.writeDeclaredField(channel2, "producer", producer, true); - FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); - FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); - - ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( - List.of(brokerId1, brokerId2)); - ((ServiceUnitStateChannelImpl) channel2).monitorOwnerships( - List.of(brokerId1, brokerId2)); - - - waitUntilNewOwner(channel1, bundle, brokerId1); - waitUntilNewOwner(channel2, bundle, brokerId1); - ownerAddr1 = channel1.getOwnerAsync(bundle).get(); - ownerAddr2 = channel2.getOwnerAsync(bundle).get(); - - assertEquals(ownerAddr1, ownerAddr2); - assertEquals(ownerAddr1, Optional.of(brokerId1)); - - var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; - validateMonitorCounters(leader, - 0, - 0, - 1, - 0, - 0, - 0, - 0); + doReturn(future).when(spyTableView).put(any(), any()); + try { + setTableView(channel2, spyTableView); + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 3 * 1000, true); + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 3 * 1000, true); + Unload unload = new Unload(brokerId1, bundle, Optional.of(brokerId2)); + channel1.publishUnloadEventAsync(unload); + // channel2 is broken. the ownership transfer won't be complete. + waitUntilState(channel1, bundle); + waitUntilState(channel2, bundle); + var owner1 = channel1.getOwnerAsync(bundle); + var owner2 = channel2.getOwnerAsync(bundle); + + assertTrue(owner1.isDone()); + assertEquals(brokerId2, owner1.get().get()); + assertFalse(owner2.isDone()); + + assertEquals(0, getOwnerRequests1.size()); + assertEquals(1, getOwnerRequests2.size()); + + // In 10 secs, the getOwnerAsync requests(lookup requests) should time out. + Awaitility.await().atMost(10, TimeUnit.SECONDS) + .untilAsserted(() -> assertTrue(owner2.isCompletedExceptionally())); + + assertEquals(0, getOwnerRequests2.size()); + + // recovered, check the monitor update state : Assigned -> Owned + doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) + .when(loadManager).selectAsync(any(), any(), any()); + } finally { + setTableView(channel2, tableView); + } - FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 30 * 1000, true); - FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + try { + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 1, true); + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 1, true); + ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( + List.of(brokerId1, brokerId2)); + ((ServiceUnitStateChannelImpl) channel2).monitorOwnerships( + List.of(brokerId1, brokerId2)); + + + waitUntilNewOwner(channel1, bundle, brokerId1); + waitUntilNewOwner(channel2, bundle, brokerId1); + ownerAddr1 = channel1.getOwnerAsync(bundle).get(); + ownerAddr2 = channel2.getOwnerAsync(bundle).get(); + + assertEquals(ownerAddr1, ownerAddr2); + assertEquals(ownerAddr1, Optional.of(brokerId1)); + + var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; + validateMonitorCounters(leader, + 0, + 0, + 1, + 0, + 0, + 0, + 0); + } finally { + FieldUtils.writeDeclaredField(channel1, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + } } @@ -563,11 +592,7 @@ public void splitAndRetryTest() throws Exception { if (count.decrementAndGet() > 0) { return future; } - // Call the real method - reset(namespaceService); - doReturn(CompletableFuture.completedFuture(List.of("test-topic-1", "test-topic-2"))) - .when(namespaceService).getOwnedTopicListForNamespaceBundle(any()); - return future; + return invocationOnMock.callRealMethod(); }).when(namespaceService).updateNamespaceBundles(any(), any()); doReturn(namespaceService).when(pulsar1).getNamespaceService(); doReturn(CompletableFuture.completedFuture(List.of("test-topic-1", "test-topic-2"))) @@ -587,11 +612,10 @@ public void splitAndRetryTest() throws Exception { validateEventCounters(channel1, 1, 0, 1, 0, 0, 0); validateEventCounters(channel2, 0, 0, 0, 0, 0, 0); // Verify the retry count - verify(((ServiceUnitStateChannelImpl) channel1), times(badVersionExceptionCount + 1)) + verify(((ServiceUnitStateChannelImpl) channel1), times(badVersionExceptionCount)) .splitServiceUnitOnceAndRetry(any(), any(), any(), any(), any(), any(), any(), any(), anyLong(), any()); - waitUntilNewOwner(channel1, childBundle11, brokerId1); waitUntilNewOwner(channel1, childBundle12, brokerId1); waitUntilNewOwner(channel2, childBundle11, brokerId1); @@ -604,12 +628,12 @@ public void splitAndRetryTest() throws Exception { // try the monitor and check the monitor moves `Deleted` -> `Init` FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel1, "stateTombstoneDelayTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, "stateTombstoneDelayTimeInMillis", 1, true); @@ -630,10 +654,15 @@ public void splitAndRetryTest() throws Exception { 0, 0); - cleanTableView(channel1, childBundle11); - cleanTableView(channel2, childBundle11); - cleanTableView(channel1, childBundle12); - cleanTableView(channel2, childBundle12); + try { + disableChannels(); + overrideTableView(channel1, childBundle11, null); + overrideTableView(channel2, childBundle11, null); + overrideTableView(channel1, childBundle12, null); + overrideTableView(channel2, childBundle12, null); + } finally { + enableChannels(); + } FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); @@ -649,6 +678,7 @@ public void splitAndRetryTest() throws Exception { @Test(priority = 7) public void handleMetadataSessionEventTest() throws IllegalAccessException { var ts = System.currentTimeMillis(); + ServiceUnitStateChannelImpl channel1 = (ServiceUnitStateChannelImpl) this.channel1; channel1.handleMetadataSessionEvent(SessionReestablished); var lastMetadataSessionEvent = getLastMetadataSessionEvent(channel1); var lastMetadataSessionEventTimestamp = getLastMetadataSessionEventTimestamp(channel1); @@ -692,7 +722,7 @@ public void handleBrokerCreationEventTest() throws IllegalAccessException { String broker = "broker-1"; var future = new CompletableFuture(); cleanupJobs.put(broker, future); - channel1.handleBrokerRegistrationEvent(broker, NotificationType.Created); + ((ServiceUnitStateChannelImpl) channel1).handleBrokerRegistrationEvent(broker, NotificationType.Created); assertEquals(0, cleanupJobs.size()); assertTrue(future.isCancelled()); } @@ -705,14 +735,14 @@ public void handleBrokerDeletionEventTest() var cleanupJobs2 = getCleanupJobs(channel2); var leaderCleanupJobsTmp = spy(cleanupJobs1); var followerCleanupJobsTmp = spy(cleanupJobs2); - var leaderChannel = channel1; - var followerChannel = channel2; + ServiceUnitStateChannelImpl leaderChannel = (ServiceUnitStateChannelImpl) channel1; + ServiceUnitStateChannelImpl followerChannel = (ServiceUnitStateChannelImpl) channel2; String leader = channel1.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); String leader2 = channel2.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); assertEquals(leader, leader2); if (leader.equals(brokerId2)) { - leaderChannel = channel2; - followerChannel = channel1; + leaderChannel = (ServiceUnitStateChannelImpl) channel2; + followerChannel = (ServiceUnitStateChannelImpl) channel1; var tmp = followerCleanupJobsTmp; followerCleanupJobsTmp = leaderCleanupJobsTmp; leaderCleanupJobsTmp = tmp; @@ -754,8 +784,10 @@ public void handleBrokerDeletionEventTest() System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); followerChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); - leaderChannel.handleBrokerRegistrationEvent(brokerId2, NotificationType.Deleted); - followerChannel.handleBrokerRegistrationEvent(brokerId2, NotificationType.Deleted); + leaderChannel.handleBrokerRegistrationEvent(brokerId2, + NotificationType.Deleted); + followerChannel.handleBrokerRegistrationEvent(brokerId2, + NotificationType.Deleted); waitUntilNewOwner(channel1, bundle1, brokerId2); waitUntilNewOwner(channel2, bundle1, brokerId2); @@ -912,7 +944,7 @@ public void handleBrokerDeletionEventTest() true); } - @Test(priority = 10) + @Test(priority = 2000) public void conflictAndCompactionTest() throws Exception { String bundle = String.format("%s/%s", "public/default", "0x0000000a_0xffffffff"); var owner1 = channel1.getOwnerAsync(bundle); @@ -941,16 +973,21 @@ public void conflictAndCompactionTest() throws Exception { assertNull(ex); assertEquals(Optional.of(brokerId1), channel2.getOwnerAsync(bundle).get()); assertEquals(Optional.of(brokerId1), channel1.getOwnerAsync(bundle).get()); + if (serviceUnitStateTableViewClassName.equals( + ServiceUnitStateMetadataStoreTableViewImpl.class.getCanonicalName())) { + // no compaction + return; + } - var compactor = spy (pulsar1.getStrategicCompactor()); + var compactor = spy(pulsar1.getStrategicCompactor()); Field strategicCompactorField = FieldUtils.getDeclaredField(PulsarService.class, "strategicCompactor", true); FieldUtils.writeField(strategicCompactorField, pulsar1, compactor, true); FieldUtils.writeField(strategicCompactorField, pulsar2, compactor, true); var threshold = admin.topicPolicies() - .getCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC); + .getCompactionThreshold(TOPIC); admin.topicPolicies() - .setCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC, 0); + .setCompactionThreshold(TOPIC, 0); try { Awaitility.await() @@ -959,7 +996,7 @@ public void conflictAndCompactionTest() throws Exception { .untilAsserted(() -> { channel1.publishAssignEventAsync(bundle, brokerId1); verify(compactor, times(1)) - .compact(eq(ServiceUnitStateChannelImpl.TOPIC), any()); + .compact(eq(TOPIC), any()); }); @@ -976,7 +1013,7 @@ public void conflictAndCompactionTest() throws Exception { "inFlightStateWaitingTimeInMillis", 30 * 1000, true); if (threshold != null) { admin.topicPolicies() - .setCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC, threshold); + .setCompactionThreshold(TOPIC, threshold); } } @@ -985,36 +1022,40 @@ public void conflictAndCompactionTest() throws Exception { @Test(priority = 11) public void ownerLookupCountTests() throws IllegalAccessException { + try { + disableChannels(); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, "b1", 1)); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, "b1", 1)); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, "b1", 1)); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, "b1", 1)); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, "b1", 1)); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, "b1", 1)); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, "b1", 1)); - channel1.getOwnerAsync(bundle); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, "b1", 1)); + channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, "b1", 1)); - channel1.getOwnerAsync(bundle); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, "b1", 1)); + channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, "b1", 1)); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, "b1", 1)); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); - overrideTableView(channel1, bundle, null); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); - channel1.getOwnerAsync(bundle); + overrideTableView(channel1, bundle, null); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); + channel1.getOwnerAsync(bundle); - validateOwnerLookUpCounters(channel1, 2, 3, 2, 1, 1, 2, 3); + validateOwnerLookUpCounters(channel1, 2, 3, 2, 1, 1, 2, 3); + } finally { + enableChannels(); + } } @@ -1062,12 +1103,12 @@ public void unloadTest() // test monitor if Free -> Init FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel1, "stateTombstoneDelayTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, "stateTombstoneDelayTimeInMillis", 1, true); @@ -1114,16 +1155,11 @@ public void assignTestWhenDestBrokerProducerFails() assertEquals(Optional.empty(), channel1.getOwnerAsync(bundle).get()); assertEquals(Optional.empty(), channel2.getOwnerAsync(bundle).get()); - var producer = (Producer) FieldUtils.readDeclaredField(channel1, - "producer", true); - var spyProducer = spy(producer); - var msg = mock(TypedMessageBuilder.class); + var tableview = getTableView(channel1); + var tableviewSpy = spy(tableview); var future = CompletableFuture.failedFuture(new RuntimeException()); - doReturn(msg).when(spyProducer).newMessage(); - doReturn(msg).when(msg).key(any()); - doReturn(msg).when(msg).value(any()); - doReturn(future).when(msg).sendAsync(); - FieldUtils.writeDeclaredField(channel2, "producer", spyProducer, true); + doReturn(future).when(tableviewSpy).put(any(), any()); + setTableView(channel2, tableviewSpy); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 3 * 1000, true); FieldUtils.writeDeclaredField(channel2, @@ -1145,11 +1181,11 @@ public void assignTestWhenDestBrokerProducerFails() .untilAsserted(() -> assertTrue(owner2.isCompletedExceptionally())); // recovered, check the monitor update state : Assigned -> Owned - FieldUtils.writeDeclaredField(channel2, "producer", producer, true); + setTableView(channel2, tableview); FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); ((ServiceUnitStateChannelImpl) channel1).monitorOwnerships( List.of(brokerId1, brokerId2)); @@ -1183,7 +1219,7 @@ public void assignTestWhenDestBrokerProducerFails() } @Test(priority = 14) - public void splitTestWhenProducerFails() + public void splitTestWhenTableViewPutFails() throws ExecutionException, InterruptedException, IllegalAccessException { @@ -1202,16 +1238,11 @@ public void splitTestWhenProducerFails() assertEquals(brokerId1, channel1.getOwnerAsync(bundle).get().get()); assertEquals(brokerId1, channel2.getOwnerAsync(bundle).get().get()); - var producer = (Producer) FieldUtils.readDeclaredField(channel1, - "producer", true); - var spyProducer = spy(producer); - var msg = mock(TypedMessageBuilder.class); + var tableview = getTableView(channel1); + var tableviewSpy = spy(tableview); var future = CompletableFuture.failedFuture(new RuntimeException()); - doReturn(msg).when(spyProducer).newMessage(); - doReturn(msg).when(msg).key(any()); - doReturn(msg).when(msg).value(any()); - doReturn(future).when(msg).sendAsync(); - FieldUtils.writeDeclaredField(channel1, "producer", spyProducer, true); + doReturn(future).when(tableviewSpy).put(any(), any()); + setTableView(channel1, tableviewSpy); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 3 * 1000, true); FieldUtils.writeDeclaredField(channel2, @@ -1230,11 +1261,11 @@ public void splitTestWhenProducerFails() // recovered, check the monitor update state : Splitting -> Owned - FieldUtils.writeDeclaredField(channel1, "producer", producer, true); + setTableView(channel1, tableview); FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; @@ -1261,6 +1292,7 @@ public void splitTestWhenProducerFails() @Test(priority = 15) public void testIsOwner() throws IllegalAccessException { + var owner1 = channel1.isOwner(bundle); var owner2 = channel2.isOwner(bundle); @@ -1298,91 +1330,102 @@ public void testIsOwner() throws IllegalAccessException { assertTrue(owner1); assertFalse(owner2); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId1, 1)); - assertFalse(channel1.isOwner(bundle)); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId1, 1)); - assertTrue(channel1.isOwner(bundle)); + try { + disableChannels(); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId1, 1)); + assertFalse(channel1.isOwner(bundle)); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId1, 1)); + assertTrue(channel1.isOwner(bundle)); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, null, brokerId1, 1)); - assertFalse(channel1.isOwner(bundle)); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, null, brokerId1, 1)); + assertFalse(channel1.isOwner(bundle)); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId1, 1)); - assertTrue(channel1.isOwner(bundle)); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId1, 1)); + assertTrue(channel1.isOwner(bundle)); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, null, brokerId1, 1)); - assertFalse(channel1.isOwner(bundle)); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, null, brokerId1, 1)); + assertFalse(channel1.isOwner(bundle)); - overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, null, brokerId1, 1)); - assertFalse(channel1.isOwner(bundle)); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, null, brokerId1, 1)); + assertFalse(channel1.isOwner(bundle)); - overrideTableView(channel1, bundle, null); - assertFalse(channel1.isOwner(bundle)); + overrideTableView(channel1, bundle, null); + assertFalse(channel1.isOwner(bundle)); + } finally { + enableChannels(); + } } - @Test(priority = 15) + @Test(priority = 16) public void testGetOwnerAsync() throws Exception { + try { + disableChannels(); + overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId1, 1)); + var owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(brokerId1, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId2, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(brokerId2, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + assertFalse(owner.isDone()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId2, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(brokerId2, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + assertFalse(owner.isDone()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, brokerId2, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(brokerId2, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, null, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(Optional.empty(), owner.get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(brokerId1, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId2, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(brokerId2, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, null, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(Optional.empty(), owner.get()); + + overrideTableView(channel1, bundle, null); + owner = channel1.getOwnerAsync(bundle); + //assertTrue(owner.isDone()); + assertEquals(Optional.empty(), owner.get()); + + overrideTableView(channel1, bundle1, new ServiceUnitStateData(Deleted, null, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle1); + //assertTrue(owner.isDone()); + assertTrue(owner.isCompletedExceptionally()); + } finally { + enableChannels(); + } - overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId1, 1)); - var owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(brokerId1, owner.get().get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId2, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(brokerId2, owner.get().get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId1, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(!owner.isDone()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId2, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(brokerId2, owner.get().get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, brokerId1, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(!owner.isDone()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, brokerId2, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(brokerId2, owner.get().get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, null, brokerId1, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(Optional.empty(), owner.get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId1, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(brokerId1, owner.get().get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId2, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(brokerId2, owner.get().get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, null, brokerId1, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(Optional.empty(), owner.get()); - - overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, null, brokerId1, 1)); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertTrue(owner.isCompletedExceptionally()); - - overrideTableView(channel1, bundle, null); - owner = channel1.getOwnerAsync(bundle); - assertTrue(owner.isDone()); - assertEquals(Optional.empty(), owner.get()); } - @Test(priority = 16) + @Test(priority = 17) public void splitAndRetryFailureTest() throws Exception { channel1.publishAssignEventAsync(bundle3, brokerId1); waitUntilNewOwner(channel1, bundle3, brokerId1); @@ -1395,6 +1438,7 @@ public void splitAndRetryFailureTest() throws Exception { NamespaceService namespaceService = pulsar1.getNamespaceService(); CompletableFuture future = new CompletableFuture<>(); + int badVersionExceptionCount = 10; AtomicInteger count = new AtomicInteger(badVersionExceptionCount); future.completeExceptionally(new MetadataStoreException.BadVersionException("BadVersion")); @@ -1402,12 +1446,8 @@ public void splitAndRetryFailureTest() throws Exception { if (count.decrementAndGet() > 0) { return future; } - // Call the real method - reset(namespaceService); - doReturn(CompletableFuture.completedFuture(List.of("test-topic-1", "test-topic-2"))) - .when(namespaceService).getOwnedTopicListForNamespaceBundle(any()); - return future; - }).when(namespaceService).updateNamespaceBundlesForPolicies(any(), any()); + return invocationOnMock.callRealMethod(); + }).when(namespaceService).updateNamespaceBundles(any(), any()); doReturn(namespaceService).when(pulsar1).getNamespaceService(); doReturn(CompletableFuture.completedFuture(List.of("test-topic-1", "test-topic-2"))) .when(namespaceService).getOwnedTopicListForNamespaceBundle(any()); @@ -1419,9 +1459,9 @@ public void splitAndRetryFailureTest() throws Exception { channel1.publishSplitEventAsync(split); FieldUtils.writeDeclaredField(channel1, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 1 , true); + "inFlightStateWaitingTimeInMillis", 1, true); Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) @@ -1429,33 +1469,33 @@ public void splitAndRetryFailureTest() throws Exception { .untilAsserted(() -> { assertEquals(3, count.get()); }); - var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; + ServiceUnitStateChannelImpl leader = + (ServiceUnitStateChannelImpl) (channel1.isChannelOwnerAsync().get() ? channel1 : channel2); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) .when(loadManager).selectAsync(any(), any(), any()); - ((ServiceUnitStateChannelImpl) leader) - .monitorOwnerships(List.of(brokerId1, brokerId2)); + leader.monitorOwnerships(List.of(brokerId1, brokerId2)); waitUntilState(leader, bundle3, Init); waitUntilState(channel1, bundle3, Init); waitUntilState(channel2, bundle3, Init); - - - validateHandlerCounters(channel1, 1, 0, 3, 0, 0, 0, 2, 1, 0, 0, 1, 0, 1, 0); - validateHandlerCounters(channel2, 1, 0, 3, 0, 0, 0, 2, 0, 0, 0, 1, 0, 1, 0); - validateEventCounters(channel1, 1, 0, 1, 0, 0, 0); - validateEventCounters(channel2, 0, 0, 0, 0, 0, 0); - waitUntilNewOwner(channel1, childBundle31, brokerId1); waitUntilNewOwner(channel1, childBundle32, brokerId1); waitUntilNewOwner(channel2, childBundle31, brokerId1); waitUntilNewOwner(channel2, childBundle32, brokerId1); + assertEquals(Optional.of(brokerId1), channel1.getOwnerAsync(childBundle31).get()); assertEquals(Optional.of(brokerId1), channel1.getOwnerAsync(childBundle32).get()); assertEquals(Optional.of(brokerId1), channel2.getOwnerAsync(childBundle31).get()); assertEquals(Optional.of(brokerId1), channel2.getOwnerAsync(childBundle32).get()); + validateHandlerCounters(channel1, 1, 0, 3, 0, 0, 0, 2, 1, 0, 0, 1, 0, 1, 0); + validateHandlerCounters(channel2, 1, 0, 3, 0, 0, 0, 2, 0, 0, 0, 1, 0, 1, 0); + validateEventCounters(channel1, 1, 0, 1, 0, 0, 0); + validateEventCounters(channel2, 0, 0, 0, 0, 0, 0); + + // try the monitor and check the monitor moves `Deleted` -> `Init` FieldUtils.writeDeclaredField(channel1, @@ -1493,18 +1533,18 @@ public void splitAndRetryFailureTest() throws Exception { "stateTombstoneDelayTimeInMillis", 300 * 1000, true); } - @Test(priority = 17) + @Test(priority = 18) public void testOverrideInactiveBrokerStateData() throws IllegalAccessException, ExecutionException, InterruptedException, TimeoutException { - var leaderChannel = channel1; - var followerChannel = channel2; + ServiceUnitStateChannelImpl leaderChannel = (ServiceUnitStateChannelImpl) channel1; + ServiceUnitStateChannelImpl followerChannel = (ServiceUnitStateChannelImpl) channel2; String leader = channel1.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); String leader2 = channel2.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); assertEquals(leader, leader2); if (leader.equals(brokerId2)) { - leaderChannel = channel2; - followerChannel = channel1; + leaderChannel = (ServiceUnitStateChannelImpl) channel2; + followerChannel = (ServiceUnitStateChannelImpl) channel1; } String broker = brokerId1; @@ -1516,20 +1556,25 @@ public void testOverrideInactiveBrokerStateData() String freeBundle = "public/free/0xfffffff0_0xffffffff"; String deletedBundle = "public/deleted/0xfffffff0_0xffffffff"; String ownedBundle = "public/owned/0xfffffff0_0xffffffff"; - overrideTableViews(releasingBundle, - new ServiceUnitStateData(Releasing, null, broker, 1)); - overrideTableViews(splittingBundle, - new ServiceUnitStateData(Splitting, null, broker, - Map.of(childBundle1Range, Optional.empty(), - childBundle2Range, Optional.empty()), 1)); - overrideTableViews(assigningBundle, - new ServiceUnitStateData(Assigning, broker, null, 1)); - overrideTableViews(freeBundle, - new ServiceUnitStateData(Free, null, broker, 1)); - overrideTableViews(deletedBundle, - new ServiceUnitStateData(Deleted, null, broker, 1)); - overrideTableViews(ownedBundle, - new ServiceUnitStateData(Owned, broker, null, 1)); + try { + disableChannels(); + overrideTableViews(releasingBundle, + new ServiceUnitStateData(Releasing, null, broker, 1)); + overrideTableViews(splittingBundle, + new ServiceUnitStateData(Splitting, null, broker, + Map.of(childBundle1Range, Optional.empty(), + childBundle2Range, Optional.empty()), 1)); + overrideTableViews(assigningBundle, + new ServiceUnitStateData(Assigning, broker, null, 1)); + overrideTableViews(freeBundle, + new ServiceUnitStateData(Free, null, broker, 1)); + overrideTableViews(deletedBundle, + new ServiceUnitStateData(Deleted, null, broker, 1)); + overrideTableViews(ownedBundle, + new ServiceUnitStateData(Owned, broker, null, 1)); + } finally { + enableChannels(); + } // test stable metadata state doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) @@ -1558,7 +1603,7 @@ public void testOverrideInactiveBrokerStateData() } - @Test(priority = 18) + @Test(priority = 19) public void testOverrideOrphanStateData() throws IllegalAccessException, ExecutionException, InterruptedException, TimeoutException { @@ -1586,28 +1631,33 @@ public void testOverrideOrphanStateData() String ownedBundle2 = "public/owned2SourceBundle/0xfffffff0_0xffffffff"; String ownedBundle3 = "public/owned3/0xfffffff0_0xffffffff"; String inactiveBroker = "broker-inactive-1"; - overrideTableViews(releasingBundle1, - new ServiceUnitStateData(Releasing, broker, brokerId2, 1)); - overrideTableViews(releasingBundle2, - new ServiceUnitStateData(Releasing, brokerId2, brokerId3, 1)); - overrideTableViews(splittingBundle, - new ServiceUnitStateData(Splitting, null, broker, - Map.of(childBundle1Range, Optional.empty(), - childBundle2Range, Optional.empty()), 1)); - overrideTableViews(assigningBundle1, - new ServiceUnitStateData(Assigning, broker, null, 1)); - overrideTableViews(assigningBundle2, - new ServiceUnitStateData(Assigning, broker, brokerId2, 1)); - overrideTableViews(freeBundle, - new ServiceUnitStateData(Free, null, broker, 1)); - overrideTableViews(deletedBundle, - new ServiceUnitStateData(Deleted, null, broker, 1)); - overrideTableViews(ownedBundle1, - new ServiceUnitStateData(Owned, broker, null, 1)); - overrideTableViews(ownedBundle2, - new ServiceUnitStateData(Owned, broker, inactiveBroker, 1)); - overrideTableViews(ownedBundle3, - new ServiceUnitStateData(Owned, inactiveBroker, broker, 1)); + try { + disableChannels(); + overrideTableViews(releasingBundle1, + new ServiceUnitStateData(Releasing, broker, brokerId2, 1)); + overrideTableViews(releasingBundle2, + new ServiceUnitStateData(Releasing, brokerId2, brokerId3, 1)); + overrideTableViews(splittingBundle, + new ServiceUnitStateData(Splitting, null, broker, + Map.of(childBundle1Range, Optional.empty(), + childBundle2Range, Optional.empty()), 1)); + overrideTableViews(assigningBundle1, + new ServiceUnitStateData(Assigning, broker, null, 1)); + overrideTableViews(assigningBundle2, + new ServiceUnitStateData(Assigning, broker, brokerId2, 1)); + overrideTableViews(freeBundle, + new ServiceUnitStateData(Free, null, broker, 1)); + overrideTableViews(deletedBundle, + new ServiceUnitStateData(Deleted, null, broker, 1)); + overrideTableViews(ownedBundle1, + new ServiceUnitStateData(Owned, broker, null, 1)); + overrideTableViews(ownedBundle2, + new ServiceUnitStateData(Owned, broker, inactiveBroker, 1)); + overrideTableViews(ownedBundle3, + new ServiceUnitStateData(Owned, inactiveBroker, broker, 1)); + } finally { + enableChannels(); + } // test stable metadata state @@ -1654,36 +1704,41 @@ public void testOverrideOrphanStateData() cleanTableViews(); } - @Test(priority = 19) + @Test(priority = 20) public void testActiveGetOwner() throws Exception { // case 1: the bundle owner is empty String broker = brokerId2; String bundle = "public/owned/0xfffffff0_0xffffffff"; - overrideTableViews(bundle, null); - assertEquals(Optional.empty(), channel1.getOwnerAsync(bundle).get()); - - // case 2: the bundle ownership is transferring, and the dst broker is not the channel owner - overrideTableViews(bundle, - new ServiceUnitStateData(Releasing, broker, brokerId1, 1)); - assertEquals(Optional.of(broker), channel1.getOwnerAsync(bundle).get()); - - - // case 3: the bundle ownership is transferring, and the dst broker is the channel owner - overrideTableViews(bundle, - new ServiceUnitStateData(Assigning, brokerId1, brokerId2, 1)); - assertTrue(!channel1.getOwnerAsync(bundle).isDone()); - - // case 4: the bundle ownership is found - overrideTableViews(bundle, - new ServiceUnitStateData(Owned, broker, null, 1)); - var owner = channel1.getOwnerAsync(bundle).get(5, TimeUnit.SECONDS).get(); - assertEquals(owner, broker); + try { + disableChannels(); + overrideTableViews(bundle, null); + assertEquals(Optional.empty(), channel1.getOwnerAsync(bundle).get()); + + // case 2: the bundle ownership is transferring, and the dst broker is not the channel owner + overrideTableViews(bundle, + new ServiceUnitStateData(Releasing, broker, brokerId1, 1)); + assertEquals(Optional.of(broker), channel1.getOwnerAsync(bundle).get()); + + + // case 3: the bundle ownership is transferring, and the dst broker is the channel owner + overrideTableViews(bundle, + new ServiceUnitStateData(Assigning, brokerId1, brokerId2, 1)); + assertFalse(channel1.getOwnerAsync(bundle).isDone()); + + // case 4: the bundle ownership is found + overrideTableViews(bundle, + new ServiceUnitStateData(Owned, broker, null, 1)); + var owner = channel1.getOwnerAsync(bundle).get(5, TimeUnit.SECONDS).get(); + assertEquals(owner, broker); + } finally { + enableChannels(); + } // case 5: the owner lookup gets delayed var spyRegistry = spy(new BrokerRegistryImpl(pulsar)); FieldUtils.writeDeclaredField(channel1, - "brokerRegistry", spyRegistry , true); + "brokerRegistry", spyRegistry, true); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1000, true); var delayedFuture = new CompletableFuture(); @@ -1692,7 +1747,7 @@ public void testActiveGetOwner() throws Exception { try { Thread.sleep(500); } catch (InterruptedException e) { - Thread.currentThread().interrupt();; + Thread.currentThread().interrupt(); } delayedFuture.complete(Optional.of(broker)); }); @@ -1716,12 +1771,12 @@ public void testActiveGetOwner() throws Exception { // case 7: the ownership cleanup(no new owner) by the leader channel doReturn(CompletableFuture.completedFuture(Optional.empty())) .when(loadManager).selectAsync(any(), any(), any()); - var leaderChannel = channel1; + ServiceUnitStateChannelImpl leaderChannel = (ServiceUnitStateChannelImpl) channel1; String leader1 = channel1.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); String leader2 = channel2.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); assertEquals(leader1, leader2); if (leader1.equals(brokerId2)) { - leaderChannel = channel2; + leaderChannel = (ServiceUnitStateChannelImpl) channel2; } leaderChannel.handleMetadataSessionEvent(SessionReestablished); FieldUtils.writeDeclaredField(leaderChannel, "lastMetadataSessionEventTimestamp", @@ -1739,8 +1794,13 @@ public void testActiveGetOwner() throws Exception { assertTrue(System.currentTimeMillis() - start < 20_000); // case 8: simulate ownership cleanup(brokerId1 as the new owner) by the leader channel - overrideTableViews(bundle, - new ServiceUnitStateData(Owned, broker, null, 1)); + try { + disableChannels(); + overrideTableViews(bundle, + new ServiceUnitStateData(Owned, broker, null, 1)); + } finally { + enableChannels(); + } doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) .when(loadManager).selectAsync(any(), any(), any()); leaderChannel.handleMetadataSessionEvent(SessionReestablished); @@ -1758,12 +1818,12 @@ public void testActiveGetOwner() throws Exception { FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); FieldUtils.writeDeclaredField(channel1, - "brokerRegistry", registry , true); + "brokerRegistry", registry, true); cleanTableViews(); } - @Test(priority = 20) + @Test(priority = 21) public void testGetOwnershipEntrySetBeforeChannelStart() { var tmpChannel = new ServiceUnitStateChannelImpl(pulsar1); try { @@ -1775,6 +1835,33 @@ public void testGetOwnershipEntrySetBeforeChannelStart() { } } + @Test(priority = 22) + public void unloadTimeoutCheckTest() + throws Exception { + + String topic = "persistent://" + namespaceName + "/test-topic"; + NamespaceBundle bundleName = pulsar.getNamespaceService().getBundle(TopicName.get(topic)); + var releasing = new ServiceUnitStateData(Releasing, pulsar2.getBrokerId(), pulsar1.getBrokerId(), 1); + + try { + disableChannels(); + overrideTableView(channel1, bundleName.toString(), releasing); + var topicFuture = pulsar1.getBrokerService().getOrCreateTopic(topic); + topicFuture.get(1, TimeUnit.SECONDS); + } catch (Exception e) { + if (!(e.getCause() instanceof BrokerServiceException.ServiceUnitNotReadyException && e.getMessage() + .contains("Please redo the lookup"))) { + fail(); + } + } finally { + enableChannels(); + } + + pulsar1.getBrokerService() + .unloadServiceUnit(bundleName, true, true, 5, + TimeUnit.SECONDS).get(2, TimeUnit.SECONDS); + } + private static ConcurrentHashMap>> getOwnerRequests( ServiceUnitStateChannel channel) throws IllegalAccessException { @@ -1846,10 +1933,21 @@ private static void waitUntilNewOwner(ServiceUnitStateChannel channel, String se }); } - private static void waitUntilState(ServiceUnitStateChannel channel, String key) + private static ServiceUnitStateTableView getTableView(ServiceUnitStateChannel channel) throws IllegalAccessException { - TableViewImpl tv = (TableViewImpl) + return (ServiceUnitStateTableView) FieldUtils.readField(channel, "tableview", true); + } + + private static void setTableView(ServiceUnitStateChannel channel, + ServiceUnitStateTableView tableView) + throws IllegalAccessException { + FieldUtils.writeField(channel, "tableview", tableView, true); + } + + private static void waitUntilState(ServiceUnitStateChannel channel, String key) + throws IllegalAccessException { + var tv = getTableView(channel); Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) .atMost(10, TimeUnit.SECONDS) @@ -1865,8 +1963,7 @@ private static void waitUntilState(ServiceUnitStateChannel channel, String key) private static void waitUntilState(ServiceUnitStateChannel channel, String key, ServiceUnitState expected) throws IllegalAccessException { - TableViewImpl tv = (TableViewImpl) - FieldUtils.readField(channel, "tableview", true); + var tv = getTableView(channel); Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) .atMost(10, TimeUnit.SECONDS) @@ -1879,8 +1976,7 @@ private static void waitUntilState(ServiceUnitStateChannel channel, String key, private void waitUntilStateWithMonitor(ServiceUnitStateChannel channel, String key, ServiceUnitState expected) throws IllegalAccessException { - TableViewImpl tv = (TableViewImpl) - FieldUtils.readField(channel, "tableview", true); + var tv = getTableView(channel); Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) .atMost(10, TimeUnit.SECONDS) @@ -1893,28 +1989,50 @@ private void waitUntilStateWithMonitor(ServiceUnitStateChannel channel, String k }); } - private static void cleanTableView(ServiceUnitStateChannel channel, String serviceUnit) + private void cleanTableViews() throws IllegalAccessException { - var tv = (TableViewImpl) - FieldUtils.readField(channel, "tableview", true); - var cache = (ConcurrentMap) - FieldUtils.readField(tv, "data", true); - cache.remove(serviceUnit); + cleanTableView(channel1); + cleanTableView(channel2); } - private void cleanTableViews() - throws IllegalAccessException { - var tv1 = (TableViewImpl) - FieldUtils.readField(channel1, "tableview", true); - var cache1 = (ConcurrentMap) - FieldUtils.readField(tv1, "data", true); - cache1.clear(); - - var tv2 = (TableViewImpl) - FieldUtils.readField(channel2, "tableview", true); - var cache2 = (ConcurrentMap) - FieldUtils.readField(tv2, "data", true); - cache2.clear(); + private void cleanTableView(ServiceUnitStateChannel channel) throws IllegalAccessException { + var getOwnerRequests = (Map>) + FieldUtils.readField(channel, "getOwnerRequests", true); + getOwnerRequests.clear(); + var tv = getTableView(channel); + if (serviceUnitStateTableViewClassName.equals(ServiceUnitStateTableViewImpl.class.getCanonicalName())) { + var tableview = (TableView) + FieldUtils.readField(tv, "tableview", true); + var cache = (ConcurrentMap) + FieldUtils.readField(tableview, "data", true); + cache.clear(); + } else { + var tableview = (MetadataStoreTableView) + FieldUtils.readField(tv, "tableview", true); + var handlerCounters = + (Map) + FieldUtils.readDeclaredField(channel, "handlerCounters", true); + var initCounter = handlerCounters.get(Init).getTotal(); + var deletedCounter = new AtomicLong(initCounter.get()); + try { + var set = tableview.entrySet(); + for (var e : set) { + try { + tableview.delete(e.getKey()).join(); + deletedCounter.incrementAndGet(); + } catch (CompletionException ex) { + if (!(ex.getCause() instanceof MetadataStoreException.NotFoundException)) { + throw ex; + } + } + } + Awaitility.await().ignoreNoExceptions().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> { + assertEquals(initCounter.get(), deletedCounter.get()); + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } } private void overrideTableViews(String serviceUnit, ServiceUnitStateData val) throws IllegalAccessException { @@ -1923,20 +2041,54 @@ private void overrideTableViews(String serviceUnit, ServiceUnitStateData val) th } @Test(enabled = false) - public static void overrideTableView(ServiceUnitStateChannel channel, String serviceUnit, ServiceUnitStateData val) - throws IllegalAccessException { - var tv = (TableViewImpl) - FieldUtils.readField(channel, "tableview", true); + public static void overrideTableView(ServiceUnitStateChannel channel, + String serviceUnit, ServiceUnitStateData val) throws IllegalAccessException { var getOwnerRequests = (Map>) FieldUtils.readField(channel, "getOwnerRequests", true); - var cache = (ConcurrentMap) - FieldUtils.readField(tv, "data", true); - if(val == null){ - cache.remove(serviceUnit); - } else { - cache.put(serviceUnit, val); - } getOwnerRequests.clear(); + var tv = getTableView(channel); + + var handlerCounters = + (Map) + FieldUtils.readDeclaredField(channel, "handlerCounters", true); + + var cur = tv.get(serviceUnit); + if (cur != null) { + long intCountStart = handlerCounters.get(Init).getTotal().get(); + var deletedCount = new AtomicLong(0); + tv.delete(serviceUnit).join(); + deletedCount.incrementAndGet(); + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(3, TimeUnit.SECONDS) + .untilAsserted(() -> { + assertEquals( + handlerCounters.get(Init).getTotal().get() + - intCountStart, deletedCount.get()); + assertNull(tv.get(serviceUnit)); + }); + } + + + + if (val != null) { + long stateCountStart = handlerCounters.get(state(val)).getTotal().get(); + var stateCount = new AtomicLong(0); + tv.put(serviceUnit, val).join(); + stateCount.incrementAndGet(); + + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(3, TimeUnit.SECONDS) + .untilAsserted(() -> { + assertEquals( + handlerCounters.get(state(val)).getTotal().get() + - stateCountStart, stateCount.get()); + assertEquals(val, tv.get(serviceUnit)); + }); + } + + } private static void cleanOpsCounters(ServiceUnitStateChannel channel) @@ -1945,7 +2097,7 @@ private static void cleanOpsCounters(ServiceUnitStateChannel channel) (Map) FieldUtils.readDeclaredField(channel, "handlerCounters", true); - for(var val : handlerCounters.values()){ + for (var val : handlerCounters.values()) { val.getFailure().set(0); val.getTotal().set(0); } @@ -1954,7 +2106,7 @@ private static void cleanOpsCounters(ServiceUnitStateChannel channel) (Map) FieldUtils.readDeclaredField(channel, "eventCounters", true); - for(var val : eventCounters.values()){ + for (var val : eventCounters.values()) { val.getFailure().set(0); val.getTotal().set(0); } @@ -1963,7 +2115,7 @@ private static void cleanOpsCounters(ServiceUnitStateChannel channel) (Map) FieldUtils.readDeclaredField(channel, "ownerLookUpCounters", true); - for(var val : ownerLookUpCounters.values()){ + for (var val : ownerLookUpCounters.values()) { val.getFailure().set(0); val.getTotal().set(0); } @@ -1980,7 +2132,7 @@ private void cleanOwnershipMonitorCounters(ServiceUnitStateChannel channel) thro } private void cleanMetadataState(ServiceUnitStateChannel channel) throws IllegalAccessException { - channel.handleMetadataSessionEvent(SessionReestablished); + ((ServiceUnitStateChannelImpl) channel).handleMetadataSessionEvent(SessionReestablished); FieldUtils.writeDeclaredField(channel, "lastMetadataSessionEventTimestamp", 0L, true); } @@ -2058,7 +2210,7 @@ private static void validateOwnerLookUpCounters(ServiceUnitStateChannel channel, long free, long deleted, long init - ) + ) throws IllegalAccessException { var ownerLookUpCounters = (Map) @@ -2126,4 +2278,14 @@ ServiceUnitStateChannelImpl createChannel(PulsarService pulsar) return channel; } + + private void disableChannels() { + ((ServiceUnitStateChannelImpl) channel1).disable(); + ((ServiceUnitStateChannelImpl) channel2).disable(); + } + + private void enableChannels() { + ((ServiceUnitStateChannelImpl) channel1).enable(); + ((ServiceUnitStateChannelImpl) channel2).enable(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategyTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolverTest.java similarity index 88% rename from pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategyTest.java rename to pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolverTest.java index 049da191a80ab..d336e8918ec5e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategyTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolverTest.java @@ -25,13 +25,15 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Releasing; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.MetadataStore; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.SystemTopic; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import org.testng.annotations.Test; @Test(groups = "broker") -public class ServiceUnitStateCompactionStrategyTest { - ServiceUnitStateCompactionStrategy strategy = new ServiceUnitStateCompactionStrategy(); +public class ServiceUnitStateDataConflictResolverTest { + ServiceUnitStateDataConflictResolver strategy = new ServiceUnitStateDataConflictResolver(); String dst = "dst"; String src = "src"; @@ -91,6 +93,32 @@ public void testVersionId(){ } + @Test + public void testStoreType(){ + ServiceUnitStateDataConflictResolver strategy = new ServiceUnitStateDataConflictResolver(); + strategy.setStorageType(SystemTopic); + assertFalse(strategy.shouldKeepLeft( + null, + new ServiceUnitStateData(Owned, dst, 1))); + assertFalse(strategy.shouldKeepLeft( + null, + new ServiceUnitStateData(Owned, dst, 2))); + assertFalse(strategy.shouldKeepLeft( + new ServiceUnitStateData(Owned, dst, 1), + new ServiceUnitStateData(Owned, dst, 3))); + + strategy.setStorageType(MetadataStore); + assertFalse(strategy.shouldKeepLeft( + null, + new ServiceUnitStateData(Owned, dst, 1))); + assertTrue(strategy.shouldKeepLeft( + null, + new ServiceUnitStateData(Owned, dst, 2))); + assertTrue(strategy.shouldKeepLeft( + new ServiceUnitStateData(Owned, dst, 1), + new ServiceUnitStateData(Owned, dst, 3))); + } + @Test public void testForce(){ assertFalse(strategy.shouldKeepLeft( diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTest.java index 620266aee46a1..0a5f012ad40a7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTest.java @@ -25,6 +25,8 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Releasing; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.MetadataStore; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.SystemTopic; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import org.testng.annotations.Test; @@ -54,63 +56,123 @@ public void testActive() { } @Test - public void testTransitions() { - - assertFalse(ServiceUnitState.isValidTransition(Init, Init)); - assertTrue(ServiceUnitState.isValidTransition(Init, Free)); - assertTrue(ServiceUnitState.isValidTransition(Init, Owned)); - assertTrue(ServiceUnitState.isValidTransition(Init, Assigning)); - assertTrue(ServiceUnitState.isValidTransition(Init, Releasing)); - assertTrue(ServiceUnitState.isValidTransition(Init, Splitting)); - assertTrue(ServiceUnitState.isValidTransition(Init, Deleted)); - - assertTrue(ServiceUnitState.isValidTransition(Free, Init)); - assertFalse(ServiceUnitState.isValidTransition(Free, Free)); - assertFalse(ServiceUnitState.isValidTransition(Free, Owned)); - assertTrue(ServiceUnitState.isValidTransition(Free, Assigning)); - assertFalse(ServiceUnitState.isValidTransition(Free, Releasing)); - assertFalse(ServiceUnitState.isValidTransition(Free, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Free, Deleted)); - - assertFalse(ServiceUnitState.isValidTransition(Assigning, Init)); - assertFalse(ServiceUnitState.isValidTransition(Assigning, Free)); - assertFalse(ServiceUnitState.isValidTransition(Assigning, Assigning)); - assertTrue(ServiceUnitState.isValidTransition(Assigning, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Assigning, Releasing)); - assertFalse(ServiceUnitState.isValidTransition(Assigning, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Assigning, Deleted)); - - assertFalse(ServiceUnitState.isValidTransition(Owned, Init)); - assertFalse(ServiceUnitState.isValidTransition(Owned, Free)); - assertFalse(ServiceUnitState.isValidTransition(Owned, Assigning)); - assertFalse(ServiceUnitState.isValidTransition(Owned, Owned)); - assertTrue(ServiceUnitState.isValidTransition(Owned, Releasing)); - assertTrue(ServiceUnitState.isValidTransition(Owned, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Owned, Deleted)); - - assertFalse(ServiceUnitState.isValidTransition(Releasing, Init)); - assertTrue(ServiceUnitState.isValidTransition(Releasing, Free)); - assertTrue(ServiceUnitState.isValidTransition(Releasing, Assigning)); - assertFalse(ServiceUnitState.isValidTransition(Releasing, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Releasing, Releasing)); - assertFalse(ServiceUnitState.isValidTransition(Releasing, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Releasing, Deleted)); - - assertFalse(ServiceUnitState.isValidTransition(Splitting, Init)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Free)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Assigning)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Releasing)); - assertFalse(ServiceUnitState.isValidTransition(Splitting, Splitting)); - assertTrue(ServiceUnitState.isValidTransition(Splitting, Deleted)); - - assertTrue(ServiceUnitState.isValidTransition(Deleted, Init)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Free)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Assigning)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Owned)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Releasing)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Splitting)); - assertFalse(ServiceUnitState.isValidTransition(Deleted, Deleted)); + public void testTransitionsOverSystemTopic() { + + assertFalse(ServiceUnitState.isValidTransition(Init, Init, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Init, Free, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Init, Owned, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Init, Assigning, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Init, Releasing, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Init, Splitting, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Init, Deleted, SystemTopic)); + + assertTrue(ServiceUnitState.isValidTransition(Free, Init, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Free, Free, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Free, Owned, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Free, Assigning, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Free, Releasing, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Free, Splitting, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Free, Deleted, SystemTopic)); + + assertFalse(ServiceUnitState.isValidTransition(Assigning, Init, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Free, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Assigning, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Assigning, Owned, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Releasing, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Splitting, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Deleted, SystemTopic)); + + assertFalse(ServiceUnitState.isValidTransition(Owned, Init, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Free, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Assigning, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Owned, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Releasing, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Splitting, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Deleted, SystemTopic)); + + assertFalse(ServiceUnitState.isValidTransition(Releasing, Init, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Releasing, Free, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Releasing, Assigning, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Owned, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Releasing, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Splitting, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Deleted, SystemTopic)); + + assertFalse(ServiceUnitState.isValidTransition(Splitting, Init, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Free, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Assigning, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Owned, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Releasing, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Splitting, SystemTopic)); + assertTrue(ServiceUnitState.isValidTransition(Splitting, Deleted, SystemTopic)); + + assertTrue(ServiceUnitState.isValidTransition(Deleted, Init, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Free, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Assigning, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Owned, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Releasing, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Splitting, SystemTopic)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Deleted, SystemTopic)); + } + + @Test + public void testTransitionsOverMetadataStore() { + + assertFalse(ServiceUnitState.isValidTransition(Init, Init, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Init, Free, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Init, Owned, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Init, Assigning, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Init, Releasing, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Init, Splitting, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Init, Deleted, MetadataStore)); + + assertFalse(ServiceUnitState.isValidTransition(Free, Init, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Free, Free, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Free, Owned, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Free, Assigning, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Free, Releasing, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Free, Splitting, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Free, Deleted, MetadataStore)); + + assertFalse(ServiceUnitState.isValidTransition(Assigning, Init, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Free, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Assigning, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Assigning, Owned, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Releasing, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Splitting, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Assigning, Deleted, MetadataStore)); + + assertFalse(ServiceUnitState.isValidTransition(Owned, Init, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Free, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Assigning, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Owned, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Releasing, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Owned, Splitting, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Owned, Deleted, MetadataStore)); + + assertFalse(ServiceUnitState.isValidTransition(Releasing, Init, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Releasing, Free, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Releasing, Assigning, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Owned, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Releasing, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Splitting, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Releasing, Deleted, MetadataStore)); + + assertFalse(ServiceUnitState.isValidTransition(Splitting, Init, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Free, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Assigning, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Owned, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Releasing, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Splitting, Splitting, MetadataStore)); + assertTrue(ServiceUnitState.isValidTransition(Splitting, Deleted, MetadataStore)); + + assertTrue(ServiceUnitState.isValidTransition(Deleted, Init, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Free, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Assigning, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Owned, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Releasing, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Splitting, MetadataStore)); + assertFalse(ServiceUnitState.isValidTransition(Deleted, Deleted, MetadataStore)); } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java index d25cba2bd1bdd..3267e67ad2c3e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java @@ -18,8 +18,12 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.store; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertThrows; import static org.testng.AssertJUnit.assertTrue; import com.google.common.collect.Sets; @@ -33,6 +37,7 @@ import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; +import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -75,8 +80,6 @@ public void testPushGetAndRemove() throws Exception { @Cleanup LoadDataStore loadDataStore = LoadDataStoreFactory.create(pulsar, topic, MyClass.class); - loadDataStore.startProducer(); - loadDataStore.startTableView(); MyClass myClass1 = new MyClass("1", 1); loadDataStore.pushAsync("key1", myClass1).get(); @@ -109,8 +112,6 @@ public void testForEach() throws Exception { @Cleanup LoadDataStore loadDataStore = LoadDataStoreFactory.create(pulsar, topic, Integer.class); - loadDataStore.startProducer(); - loadDataStore.startTableView(); Map map = new HashMap<>(); for (int i = 0; i < 10; i++) { @@ -134,9 +135,6 @@ public void testTableViewRestart() throws Exception { String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); LoadDataStore loadDataStore = LoadDataStoreFactory.create(pulsar, topic, Integer.class); - loadDataStore.startProducer(); - - loadDataStore.startTableView(); loadDataStore.pushAsync("1", 1).get(); Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.size(), 1)); assertEquals(loadDataStore.get("1").get(), 1); @@ -150,6 +148,31 @@ public void testTableViewRestart() throws Exception { Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.get("1").get(), 3)); } + @Test + public void testProducerRestart() throws Exception { + String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); + var loadDataStore = + (TableViewLoadDataStoreImpl) spy(LoadDataStoreFactory.create(pulsar, topic, Integer.class)); + + // happy case + loadDataStore.pushAsync("1", 1).get(); + Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.size(), 1)); + assertEquals(loadDataStore.get("1").get(), 1); + verify(loadDataStore, times(1)).startProducer(); + + // loadDataStore will restart producer if null. + FieldUtils.writeField(loadDataStore, "producer", null, true); + loadDataStore.pushAsync("1", 2).get(); + Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.get("1").get(), 2)); + verify(loadDataStore, times(2)).startProducer(); + + // loadDataStore will restart producer if too slow. + FieldUtils.writeField(loadDataStore, "producerLastPublishTimestamp", 0 , true); + loadDataStore.pushAsync("1", 3).get(); + Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.get("1").get(), 3)); + verify(loadDataStore, times(3)).startProducer(); + } + @Test public void testProducerStop() throws Exception { String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); @@ -165,4 +188,26 @@ public void testProducerStop() throws Exception { loadDataStore.removeAsync("2").get(); } + @Test + public void testShutdown() throws Exception { + String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); + LoadDataStore loadDataStore = + LoadDataStoreFactory.create(pulsar, topic, Integer.class); + loadDataStore.start(); + loadDataStore.shutdown(); + + Assert.assertTrue(loadDataStore.pushAsync("2", 2).isCompletedExceptionally()); + Assert.assertTrue(loadDataStore.removeAsync("2").isCompletedExceptionally()); + assertThrows(IllegalStateException.class, () -> loadDataStore.get("2")); + assertThrows(IllegalStateException.class, loadDataStore::size); + assertThrows(IllegalStateException.class, loadDataStore::entrySet); + assertThrows(IllegalStateException.class, () -> loadDataStore.forEach((k, v) -> {})); + assertThrows(IllegalStateException.class, loadDataStore::init); + assertThrows(IllegalStateException.class, loadDataStore::start); + assertThrows(IllegalStateException.class, loadDataStore::startProducer); + assertThrows(IllegalStateException.class, loadDataStore::startTableView); + assertThrows(IllegalStateException.class, loadDataStore::close); + assertThrows(IllegalStateException.class, loadDataStore::closeTableView); + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java index ea5365bcf4b2c..a4460187d2377 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.TOPIC; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; @@ -32,7 +33,6 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; import org.apache.pulsar.client.admin.ListNamespaceTopicsOptions; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.MessageId; @@ -547,7 +547,7 @@ public void testExtensibleLoadManagerImplInternalTopicAutoCreations() tenantInfo.setAllowedClusters(Set.of(configClusterName)); admin.tenants().createTenant("pulsar", tenantInfo); admin.namespaces().createNamespace(namespaceName); - admin.topics().createNonPartitionedTopic(ServiceUnitStateChannelImpl.TOPIC); + admin.topics().createNonPartitionedTopic(TOPIC); admin.topics().createNonPartitionedTopic(ExtensibleLoadManagerImpl.BROKER_LOAD_DATA_STORE_TOPIC); admin.topics().createNonPartitionedTopic(ExtensibleLoadManagerImpl.TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); @@ -560,7 +560,7 @@ public void testExtensibleLoadManagerImplInternalTopicAutoCreations() // The created persistent topic correctly can be found by // pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); - Producer producer = pulsarClient.newProducer().topic(ServiceUnitStateChannelImpl.TOPIC).create(); + Producer producer = pulsarClient.newProducer().topic(TOPIC).create(); // The created non-persistent topics cannot be found, as we did topics.clear() try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 2f27d5917f025..5398b5aa57b8b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.TOPIC; import static org.apache.pulsar.common.naming.SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN; import static org.apache.pulsar.common.naming.SystemTopicNames.TRANSACTION_COORDINATOR_LOG; import static org.mockito.ArgumentMatchers.anyString; @@ -82,7 +83,6 @@ import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceException; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -1759,7 +1759,7 @@ public void testMetricsNonPersistentTopicLoadFails() throws Exception { public void testIsSystemTopicAllowAutoTopicCreationAsync() throws Exception { BrokerService brokerService = pulsar.getBrokerService(); assertFalse(brokerService.isAllowAutoTopicCreationAsync( - ServiceUnitStateChannelImpl.TOPIC).get()); + TOPIC).get()); assertTrue(brokerService.isAllowAutoTopicCreationAsync( "persistent://pulsar/system/my-system-topic").get()); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java index 9140216810826..a834fa1fde1e3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java @@ -24,9 +24,10 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigning; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Releasing; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.StorageType.SystemTopic; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.isValidTransition; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MSG_COMPRESSION_TYPE; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.MSG_COMPRESSION_TYPE; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; @@ -61,7 +62,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateCompactionStrategy; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateDataConflictResolver; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -92,7 +93,7 @@ public class ServiceUnitStateCompactionTest extends MockedPulsarServiceBaseTest private ScheduledExecutorService compactionScheduler; private BookKeeper bk; private Schema schema; - private ServiceUnitStateCompactionStrategy strategy; + private ServiceUnitStateDataConflictResolver strategy; private ServiceUnitState testState = Init; @@ -118,7 +119,7 @@ private ServiceUnitStateData testValue(String broker) { private ServiceUnitState nextValidState(ServiceUnitState from) { List candidates = Arrays.stream(ServiceUnitState.values()) - .filter(to -> isValidTransition(from, to)) + .filter(to -> isValidTransition(from, to, SystemTopic)) .collect(Collectors.toList()); var state= candidates.get(RANDOM.nextInt(candidates.size())); return state; @@ -127,7 +128,7 @@ private ServiceUnitState nextValidState(ServiceUnitState from) { private ServiceUnitState nextValidStateNonSplit(ServiceUnitState from) { List candidates = Arrays.stream(ServiceUnitState.values()) .filter(to -> to != Init && to != Splitting && to != Deleted - && isValidTransition(from, to)) + && isValidTransition(from, to, SystemTopic)) .collect(Collectors.toList()); var state= candidates.get(RANDOM.nextInt(candidates.size())); return state; @@ -135,7 +136,7 @@ && isValidTransition(from, to)) private ServiceUnitState nextInvalidState(ServiceUnitState from) { List candidates = Arrays.stream(ServiceUnitState.values()) - .filter(to -> !isValidTransition(from, to)) + .filter(to -> !isValidTransition(from, to, SystemTopic)) .collect(Collectors.toList()); if (candidates.size() == 0) { return Init; @@ -157,7 +158,7 @@ public void setup() throws Exception { new ThreadFactoryBuilder().setNameFormat("compaction-%d").setDaemon(true).build()); bk = pulsar.getBookKeeperClientFactory().create(this.conf, null, null, Optional.empty(), null).get(); schema = Schema.JSON(ServiceUnitStateData.class); - strategy = new ServiceUnitStateCompactionStrategy(); + strategy = new ServiceUnitStateDataConflictResolver(); strategy.checkBrokers(false); testState = Init; @@ -329,10 +330,10 @@ public void testCompactionWithTableview() throws Exception { .topic("persistent://my-property/use/my-ns/my-topic1") .loadConf(Map.of( "topicCompactionStrategyClassName", - ServiceUnitStateCompactionStrategy.class.getName())) + ServiceUnitStateDataConflictResolver.class.getName())) .create(); - ((ServiceUnitStateCompactionStrategy) + ((ServiceUnitStateDataConflictResolver) FieldUtils.readDeclaredField(tv, "compactionStrategy", true)) .checkBrokers(false); TestData testData = generateTestData(); @@ -364,7 +365,7 @@ public void testCompactionWithTableview() throws Exception { .topic(topic) .loadConf(Map.of( "topicCompactionStrategyClassName", - ServiceUnitStateCompactionStrategy.class.getName())) + ServiceUnitStateDataConflictResolver.class.getName())) .create(); for(var etr : tableview.entrySet()){ @@ -531,7 +532,7 @@ public void testSlowTableviewAfterCompaction() throws Exception { .subscriptionName("fastTV") .loadConf(Map.of( strategyClassName, - ServiceUnitStateCompactionStrategy.class.getName())) + ServiceUnitStateDataConflictResolver.class.getName())) .create(); var defaultConf = getDefaultConf(); @@ -544,7 +545,7 @@ public void testSlowTableviewAfterCompaction() throws Exception { .subscriptionName("slowTV") .loadConf(Map.of( strategyClassName, - ServiceUnitStateCompactionStrategy.class.getName())) + ServiceUnitStateDataConflictResolver.class.getName())) .create(); var semaphore = new Semaphore(0); @@ -616,7 +617,7 @@ public void testSlowTableviewAfterCompaction() throws Exception { .topic(topic) .loadConf(Map.of( strategyClassName, - ServiceUnitStateCompactionStrategy.class.getName())) + ServiceUnitStateDataConflictResolver.class.getName())) .create(); Awaitility.await() .pollInterval(200, TimeUnit.MILLISECONDS) @@ -651,7 +652,7 @@ public void testSlowReceiveTableviewAfterCompaction() throws Exception { .subscriptionName("slowTV") .loadConf(Map.of( strategyClassName, - ServiceUnitStateCompactionStrategy.class.getName())) + ServiceUnitStateDataConflictResolver.class.getName())) .create(); // Configure retention to ensue data is retained for reader diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java index d1ff46cbc02d5..8f67e412267a0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.compaction; -import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MSG_COMPRESSION_TYPE; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl.MSG_COMPRESSION_TYPE; import static org.testng.Assert.assertEquals; import java.util.ArrayList; diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCacheConfig.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCacheConfig.java index 55b159071fda4..2bc042aebb308 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCacheConfig.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCacheConfig.java @@ -18,7 +18,9 @@ */ package org.apache.pulsar.metadata.api; +import java.util.Optional; import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; import lombok.Builder; import lombok.Getter; import lombok.ToString; @@ -29,7 +31,7 @@ @Builder @Getter @ToString -public class MetadataCacheConfig { +public class MetadataCacheConfig { private static final long DEFAULT_CACHE_REFRESH_TIME_MILLIS = TimeUnit.MINUTES.toMillis(5); /** @@ -47,4 +49,12 @@ public class MetadataCacheConfig { */ @Builder.Default private final long expireAfterWriteMillis = 2 * DEFAULT_CACHE_REFRESH_TIME_MILLIS; + + /** + * Specifies cache reload consumer behavior when the cache is refreshed automatically at refreshAfterWriteMillis + * frequency. + */ + @Builder.Default + private final BiConsumer>> asyncReloadConsumer = null; + } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreTableView.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreTableView.java new file mode 100644 index 0000000000000..64de22890a0f1 --- /dev/null +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataStoreTableView.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.metadata.api; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +/** + * Defines metadata store tableview. + * MetadataStoreTableView initially fills existing items to its local tableview and eventually + * synchronize remote updates to its local tableview from the remote metadata store. + * This abstraction can help replicate metadata in memory from metadata store. + */ +public interface MetadataStoreTableView { + + class ConflictException extends RuntimeException { + public ConflictException(String msg) { + super(msg); + } + } + + /** + * Starts the tableview by filling existing items to its local tableview from the remote metadata store. + */ + void start() throws MetadataStoreException; + + /** + * Gets one item from the local tableview. + *

+ * If the key is not found, return null. + * + * @param key the key to check + * @return value if exists. Otherwise, null. + */ + T get(String key); + + /** + * Tries to put the item in the persistent store. + * All peer tableviews (including the local one) will be notified and be eventually consistent with this put value. + *

+ * This operation can fail if the input value conflicts with the existing one. + * + * @param key the key to check on the tableview + * @return a future to track the completion of the operation + * @throws MetadataStoreTableView.ConflictException + * if the input value conflicts with the existing one. + */ + CompletableFuture put(String key, T value); + + /** + * Tries to delete the item from the persistent store. + * All peer tableviews (including the local one) will be notified and be eventually consistent with this deletion. + *

+ * This can fail if the item is not present in the metadata store. + * + * @param key the key to check on the tableview + * @return a future to track the completion of the operation + * @throws MetadataStoreException.NotFoundException + * if the key is not present in the metadata store. + */ + CompletableFuture delete(String key); + + /** + * Returns the entry set of the items in the local tableview. + * @return entry set + */ + Set> entrySet(); +} + diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java index ee394b0267c88..4c7f34aa5c16e 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java @@ -61,15 +61,15 @@ public class MetadataCacheImpl implements MetadataCache, Consumer>> objCache; - public MetadataCacheImpl(MetadataStore store, TypeReference typeRef, MetadataCacheConfig cacheConfig) { + public MetadataCacheImpl(MetadataStore store, TypeReference typeRef, MetadataCacheConfig cacheConfig) { this(store, new JSONMetadataSerdeTypeRef<>(typeRef), cacheConfig); } - public MetadataCacheImpl(MetadataStore store, JavaType type, MetadataCacheConfig cacheConfig) { + public MetadataCacheImpl(MetadataStore store, JavaType type, MetadataCacheConfig cacheConfig) { this(store, new JSONMetadataSerdeSimpleType<>(type), cacheConfig); } - public MetadataCacheImpl(MetadataStore store, MetadataSerde serde, MetadataCacheConfig cacheConfig) { + public MetadataCacheImpl(MetadataStore store, MetadataSerde serde, MetadataCacheConfig cacheConfig) { this.store = store; if (store instanceof MetadataStoreExtended) { this.storeExtended = (MetadataStoreExtended) store; @@ -98,7 +98,12 @@ public CompletableFuture>> asyncReload( Optional> oldValue, Executor executor) { if (store instanceof AbstractMetadataStore && ((AbstractMetadataStore) store).isConnected()) { - return readValueFromStore(key); + return readValueFromStore(key).thenApply(val -> { + if (cacheConfig.getAsyncReloadConsumer() != null) { + cacheConfig.getAsyncReloadConsumer().accept(key, val); + } + return val; + }); } else { // Do not try to refresh the cache item if we know that we're not connected to the // metadata store diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/MetadataStoreTableViewImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/MetadataStoreTableViewImpl.java new file mode 100644 index 0000000000000..4f9aad0ba658b --- /dev/null +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/MetadataStoreTableViewImpl.java @@ -0,0 +1,342 @@ +/* + * 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.metadata.tableview.impl; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.LongAdder; +import java.util.function.BiConsumer; +import java.util.function.BiPredicate; +import java.util.function.Predicate; +import lombok.Builder; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang.mutable.MutableBoolean; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.CacheGetResult; +import org.apache.pulsar.metadata.api.MetadataCache; +import org.apache.pulsar.metadata.api.MetadataCacheConfig; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.MetadataStoreTableView; +import org.apache.pulsar.metadata.api.NotificationType; + +@Slf4j +public class MetadataStoreTableViewImpl implements MetadataStoreTableView { + + private static final int FILL_TIMEOUT_IN_MILLIS = 300_000; + private static final int MAX_CONCURRENT_METADATA_OPS_DURING_FILL = 50; + private static final long CACHE_REFRESH_FREQUENCY_IN_MILLIS = 600_000; + private final ConcurrentMap data; + private final Map immutableData; + private final String name; + private final MetadataStore store; + private final MetadataCache cache; + private final Predicate listenPathValidator; + private final BiPredicate conflictResolver; + private final List> tailItemListeners; + private final List> existingItemListeners; + private final long timeoutInMillis; + private final String pathPrefix; + + /** + * Construct MetadataStoreTableViewImpl. + * + * @param clazz clazz of the value type + * @param name metadata store tableview name + * @param store metadata store + * @param pathPrefix metadata store path prefix + * @param listenPathValidator path validator to listen + * @param conflictResolver resolve conflicts for concurrent puts + * @param tailItemListeners listener for tail item(recently updated) notifications + * @param existingItemListeners listener for existing items in metadata store + * @param timeoutInMillis timeout duration for each sync operation. + * @throws MetadataStoreException if init fails. + */ + @Builder + public MetadataStoreTableViewImpl(@NonNull Class clazz, + @NonNull String name, + @NonNull MetadataStore store, + @NonNull String pathPrefix, + @NonNull BiPredicate conflictResolver, + Predicate listenPathValidator, + List> tailItemListeners, + List> existingItemListeners, + long timeoutInMillis) { + this.name = name; + this.data = new ConcurrentHashMap<>(); + this.immutableData = Collections.unmodifiableMap(data); + this.pathPrefix = pathPrefix; + this.conflictResolver = conflictResolver; + this.listenPathValidator = listenPathValidator; + this.tailItemListeners = new ArrayList<>(); + if (tailItemListeners != null) { + this.tailItemListeners.addAll(tailItemListeners); + } + this.existingItemListeners = new ArrayList<>(); + if (existingItemListeners != null) { + this.existingItemListeners.addAll(existingItemListeners); + } + this.timeoutInMillis = timeoutInMillis; + this.store = store; + this.cache = store.getMetadataCache(clazz, + MetadataCacheConfig.builder() + .expireAfterWriteMillis(-1) + .refreshAfterWriteMillis(CACHE_REFRESH_FREQUENCY_IN_MILLIS) + .asyncReloadConsumer(this::consumeAsyncReload) + .build()); + store.registerListener(this::handleNotification); + } + + public void start() throws MetadataStoreException { + fill(); + } + + private void consumeAsyncReload(String path, Optional> cached) { + if (!isValidPath(path)) { + return; + } + String key = getKey(path); + var val = getValue(cached); + handleTailItem(key, val); + } + + private boolean isValidPath(String path) { + if (listenPathValidator != null && !listenPathValidator.test(path)) { + return false; + } + return true; + } + + private T getValue(Optional> cached) { + return cached.map(CacheGetResult::getValue).orElse(null); + } + + boolean updateData(String key, T cur) { + MutableBoolean updated = new MutableBoolean(); + data.compute(key, (k, prev) -> { + if (Objects.equals(prev, cur)) { + if (log.isDebugEnabled()) { + log.debug("{} skipped item key={} value={} prev={}", + name, key, cur, prev); + } + updated.setValue(false); + return prev; + } else { + updated.setValue(true); + return cur; + } + }); + return updated.booleanValue(); + } + + private void handleTailItem(String key, T val) { + if (updateData(key, val)) { + if (log.isDebugEnabled()) { + log.debug("{} applying item key={} value={}", + name, + key, + val); + } + for (var listener : tailItemListeners) { + try { + listener.accept(key, val); + } catch (Throwable e) { + log.error("{} failed to listen tail item key:{}, val:{}", + name, + key, val, e); + } + } + } + + } + + private CompletableFuture doHandleNotification(String path) { + if (!isValidPath(path)) { + return CompletableFuture.completedFuture(null); + } + return cache.get(path).thenAccept(valOpt -> { + String key = getKey(path); + var val = valOpt.orElse(null); + handleTailItem(key, val); + }).exceptionally(e -> { + log.error("{} failed to handle notification for path:{}", name, path, e); + return null; + }); + } + + private void handleNotification(org.apache.pulsar.metadata.api.Notification notification) { + + if (notification.getType() == NotificationType.ChildrenChanged) { + return; + } + + String path = notification.getPath(); + + doHandleNotification(path); + } + + + private CompletableFuture handleExisting(String path) { + if (!isValidPath(path)) { + return CompletableFuture.completedFuture(null); + } + return cache.get(path) + .thenAccept(valOpt -> { + valOpt.ifPresent(val -> { + String key = getKey(path); + updateData(key, val); + if (log.isDebugEnabled()) { + log.debug("{} applying existing item key={} value={}", + name, + key, + val); + } + for (var listener : existingItemListeners) { + try { + listener.accept(key, val); + } catch (Throwable e) { + log.error("{} failed to listen existing item key:{}, val:{}", name, key, val, + e); + throw e; + } + } + }); + }); + } + + private void fill() throws MetadataStoreException { + final var deadline = System.currentTimeMillis() + FILL_TIMEOUT_IN_MILLIS; + log.info("{} start filling existing items under the pathPrefix:{}", name, pathPrefix); + ConcurrentLinkedDeque q = new ConcurrentLinkedDeque<>(); + List> futures = new ArrayList<>(); + q.add(pathPrefix); + LongAdder count = new LongAdder(); + while (!q.isEmpty()) { + var now = System.currentTimeMillis(); + if (now >= deadline) { + String err = name + " failed to fill existing items in " + + TimeUnit.MILLISECONDS.toSeconds(FILL_TIMEOUT_IN_MILLIS) + " secs. Filled count:" + + count.sum(); + log.error(err); + throw new MetadataStoreException(err); + } + int size = Math.min(MAX_CONCURRENT_METADATA_OPS_DURING_FILL, q.size()); + for (int i = 0; i < size; i++) { + String path = q.poll(); + futures.add(store.getChildren(path) + .thenCompose(children -> { + // The path is leaf + if (children.isEmpty()) { + count.increment(); + return handleExisting(path); + } else { + for (var child : children) { + q.add(path + "/" + child); + } + return CompletableFuture.completedFuture(null); + } + })); + } + try { + FutureUtil.waitForAll(futures).get( + Math.min(timeoutInMillis, deadline - now), + TimeUnit.MILLISECONDS); + } catch (Throwable e) { + Throwable c = FutureUtil.unwrapCompletionException(e); + log.error("{} failed to fill existing items", name, c); + throw new MetadataStoreException(c); + } + futures.clear(); + } + log.info("{} completed filling existing items with size:{}", name, count.sum()); + } + + + private String getPath(String key) { + return pathPrefix + "/" + key; + } + + private String getKey(String path) { + return path.replaceFirst(pathPrefix + "/", ""); + } + + public boolean exists(String key) { + return immutableData.containsKey(key); + } + + public T get(String key) { + return data.get(key); + } + + public CompletableFuture put(String key, T value) { + String path = getPath(key); + return cache.readModifyUpdateOrCreate(path, (old) -> { + if (conflictResolver.test(old.orElse(null), value)) { + return value; + } else { + throw new ConflictException( + String.format("Failed to update from old:%s to value:%s", old, value)); + } + }).thenCompose(__ -> doHandleNotification(path)); // immediately notify local tableview + } + + public CompletableFuture delete(String key) { + String path = getPath(key); + return cache.delete(path) + .thenCompose(__ -> doHandleNotification(path)); // immediately notify local tableview + } + + public int size() { + return immutableData.size(); + } + + public boolean isEmpty() { + return immutableData.isEmpty(); + } + + public Set> entrySet() { + return immutableData.entrySet(); + } + + public Set keySet() { + return immutableData.keySet(); + } + + public Collection values() { + return immutableData.values(); + } + + public void forEach(BiConsumer action) { + immutableData.forEach(action); + } + +} diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/package-info.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/package-info.java new file mode 100644 index 0000000000000..2c47770610b05 --- /dev/null +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/tableview/impl/package-info.java @@ -0,0 +1,19 @@ +/* + * 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.metadata.tableview.impl; diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java index bac5807360453..6992c69b7252e 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataCacheTest.java @@ -47,6 +47,7 @@ import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.metadata.api.CacheGetResult; import org.apache.pulsar.metadata.api.MetadataCache; +import org.apache.pulsar.metadata.api.MetadataCacheConfig; import org.apache.pulsar.metadata.api.MetadataSerde; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreConfig; @@ -622,4 +623,28 @@ public void testPut(String provider, Supplier urlSupplier) throws Except assertEquals(cache2.get(key).get().orElse(-1), 2); }); } + + @Test(dataProvider = "impl") + public void testAsyncReloadConsumer(String provider, Supplier urlSupplier) throws Exception { + @Cleanup + MetadataStore store = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + + List refreshed = new ArrayList<>(); + MetadataCache objCache = store.getMetadataCache(MyClass.class, + MetadataCacheConfig.builder().refreshAfterWriteMillis(100) + .asyncReloadConsumer((k, v) -> v.map(vv -> refreshed.add(vv.getValue()))).build()); + + String key1 = newKey(); + + MyClass value1 = new MyClass("a", 1); + objCache.create(key1, value1); + + MyClass value2 = new MyClass("a", 2); + store.put(key1, ObjectMapperFactory.getMapper().writer().writeValueAsBytes(value2), Optional.empty()) + .join(); + + Awaitility.await().untilAsserted(() -> { + refreshed.contains(value2); + }); + } } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTableViewTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTableViewTest.java new file mode 100644 index 0000000000000..5a2ea32890dbd --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/MetadataStoreTableViewTest.java @@ -0,0 +1,499 @@ +/* + * 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.metadata; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import com.fasterxml.jackson.databind.type.TypeFactory; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import lombok.Cleanup; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.api.MetadataStoreConfig; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; +import org.apache.pulsar.metadata.cache.impl.JSONMetadataSerdeSimpleType; +import org.apache.pulsar.metadata.impl.MetadataStoreFactoryImpl; +import org.apache.pulsar.metadata.tableview.impl.MetadataStoreTableViewImpl; +import org.awaitility.Awaitility; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Slf4j +public class MetadataStoreTableViewTest extends BaseMetadataStoreTest { + + LinkedBlockingDeque> tails; + LinkedBlockingDeque> existings; + + @BeforeMethod + void init(){ + tails = new LinkedBlockingDeque<>(); + existings = new LinkedBlockingDeque<>(); + } + + private void tailListener(String k, Integer v){ + tails.add(Pair.of(k, v)); + } + + private void existingListener(String k, Integer v){ + existings.add(Pair.of(k, v)); + } + + MetadataStoreTableViewImpl createTestTableView(MetadataStore store, String prefix, + Supplier urlSupplier) + throws Exception { + var tv = MetadataStoreTableViewImpl.builder() + .name("test") + .clazz(Integer.class) + .store(store) + .pathPrefix(prefix) + .conflictResolver((old, cur) -> { + if (old == null || cur == null) { + return true; + } + return old < cur; + }) + .listenPathValidator((path) -> path.startsWith(prefix) && path.contains("my")) + .tailItemListeners(List.of(this::tailListener)) + .existingItemListeners(List.of(this::existingListener)) + .timeoutInMillis(5_000) + .build(); + tv.start(); + return tv; + } + + private void assertGet(MetadataStoreTableViewImpl tv, String path, Integer expected) { + assertEquals(tv.get(path), expected); + //Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> assertEquals(tv.get(path), expected)); + } + + + @Test(dataProvider = "impl") + public void emptyTableViewTest(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + assertFalse(tv.exists("non-existing-key")); + assertFalse(tv.exists("non-existing-key/child")); + assertNull(tv.get("non-existing-key")); + assertNull(tv.get("non-existing-key/child")); + + try { + tv.delete("non-existing-key").join(); + fail("Should have failed"); + } catch (CompletionException e) { + assertException(e, NotFoundException.class); + } + + } + + @Test(dataProvider = "impl") + public void concurrentPutTest(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + int data = 1; + String path = "my"; + int concurrent = 50; + List> futureList = new ArrayList<>(); + for (int i = 0; i < concurrent; i++) { + futureList.add(tv.put(path, data).exceptionally(ex -> { + if (!(ex.getCause() instanceof MetadataStoreTableViewImpl.ConflictException)) { + fail("fail to execute concurrent put", ex); + } + return null; + })); + } + FutureUtil.waitForAll(futureList).join(); + + assertGet(tv, path, data); + } + + @Test(dataProvider = "impl") + public void conflictResolverTest(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + String key1 = "my"; + + tv.put(key1, 0).join(); + tv.put(key1, 0).exceptionally(ex -> { + if (!(ex.getCause() instanceof MetadataStoreTableViewImpl.ConflictException)) { + fail("fail to execute concurrent put", ex); + } + return null; + }).join(); + assertGet(tv, key1, 0); + tv.put(key1, 1).join(); + assertGet(tv, key1, 1); + tv.put(key1, 0).exceptionally(ex -> { + if (!(ex.getCause() instanceof MetadataStoreTableViewImpl.ConflictException)) { + fail("fail to execute concurrent put", ex); + } + return null; + }).join(); + assertGet(tv, key1, 1); + } + + @Test(dataProvider = "impl") + public void deleteTest(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + String key1 = "key"; + tv.put(key1, 0).join(); + tv.delete(key1).join(); + assertNull(tv.get(key1)); + } + + @Test(dataProvider = "impl") + public void mapApiTest(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + assertTrue(tv.isEmpty()); + assertEquals(tv.size(), 0); + + String key1 = "my1"; + String key2 = "my2"; + + int val1 = 1; + int val2 = 2; + + tv.put(key1, val1).join(); + tv.put(key2, val2).join(); + assertGet(tv, key1, 1); + assertGet(tv, key2, 2); + + assertFalse(tv.isEmpty()); + assertEquals(tv.size(), 2); + + List actual = new ArrayList<>(); + tv.forEach((k, v) -> { + actual.add(k + "," + v); + }); + assertEquals(actual, List.of(key1 + "," + val1, key2 + "," + val2)); + + var values = tv.values(); + assertEquals(values.size(), 2); + assertTrue(values.containsAll(List.of(val1, val2))); + + var keys = tv.keySet(); + assertEquals(keys.size(), 2); + assertTrue(keys.containsAll(List.of(key1, key2))); + + var entries = tv.entrySet(); + assertEquals(entries.size(), 2); + assertTrue(entries.containsAll(Map.of(key1, val1, key2, val2).entrySet())); + } + + @Test(dataProvider = "impl") + public void notificationListeners(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + String keyPrefix = "tenant/ns"; + String key1 = keyPrefix + "/my-1"; + int val1 = 1; + + assertGet(tv, key1, null); + + // listen on put + tv.put(key1, val1).join(); + var kv = tails.poll(3, TimeUnit.SECONDS); + assertEquals(kv, Pair.of(key1, val1)); + assertEquals(tv.get(key1), val1); + + // listen on modified + int val2 = 2; + tv.put(key1, val2).join(); + kv = tails.poll(3, TimeUnit.SECONDS); + assertEquals(kv, Pair.of(key1, val2)); + assertEquals(tv.get(key1), val2); + + // no listen on the parent + int val0 = 0; + String childKey = key1 + "/my-child-1"; + tv.put(childKey, val0).join(); + kv = tails.poll(3, TimeUnit.SECONDS); + assertEquals(kv, Pair.of(childKey, val0)); + kv = tails.poll(3, TimeUnit.SECONDS); + assertNull(kv); + assertEquals(tv.get(key1), val2); + assertEquals(tv.get(childKey), val0); + + tv.put(childKey, val1).join(); + kv = tails.poll(3, TimeUnit.SECONDS); + assertEquals(kv, Pair.of(childKey, val1)); + kv = tails.poll(3, TimeUnit.SECONDS); + assertNull(kv); + assertEquals(tv.get(key1), val2); + assertEquals(tv.get(childKey), val1); + + tv.delete(childKey).join(); + kv = tails.poll(3, TimeUnit.SECONDS); + assertEquals(kv, Pair.of(childKey, null)); + kv = tails.poll(3, TimeUnit.SECONDS); + assertNull(kv); + assertEquals(tv.get(key1), val2); + assertNull(tv.get(childKey)); + + // No listen on the filtered key + String noListenKey = keyPrefix + "/to-be-filtered"; + tv.put(noListenKey, val0).join(); + kv = tails.poll(3, TimeUnit.SECONDS); + assertNull(kv); + assertEquals(tv.get(key1), val2); + assertNull(tv.get(noListenKey)); + + // Trigger deleted notification + tv.delete(key1).join(); + kv = tails.poll(3, TimeUnit.SECONDS); + assertEquals(kv, Pair.of(key1, null)); + assertNull(tv.get(key1)); + } + + @Test(dataProvider = "impl") + public void testConcurrentPutGetOneKey(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + String key = "my"; + int val = 0; + int maxValue = 50; + tv.put(key, val).join(); + + AtomicInteger successWrites = new AtomicInteger(0); + Runnable task = new Runnable() { + @SneakyThrows + @Override + public void run() { + for (int k = 0; k < 1000; k++) { + var kv = tails.poll(3, TimeUnit.SECONDS); + if (kv == null) { + break; + } + Integer val = kv.getRight() + 1; + if (val <= maxValue) { + CompletableFuture putResult = + tv.put(key, val).thenRun(successWrites::incrementAndGet); + try { + putResult.get(); + } catch (Exception ignore) { + } + log.info("Put value {} success:{}. ", val, !putResult.isCompletedExceptionally()); + } else { + break; + } + } + } + }; + CompletableFuture t1 = CompletableFuture.completedFuture(null).thenRunAsync(task); + CompletableFuture t2 = CompletableFuture.completedFuture(null).thenRunAsync(task); + task.run(); + t1.join(); + t2.join(); + assertFalse(t1.isCompletedExceptionally()); + assertFalse(t2.isCompletedExceptionally()); + + assertEquals(successWrites.get(), maxValue); + assertEquals(tv.get(key), maxValue); + } + + @Test(dataProvider = "impl") + public void testConcurrentPut(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + String k = "my"; + int v = 0; + CompletableFuture f1 = + CompletableFuture.runAsync(() -> tv.put(k, v).join()); + CompletableFuture f2 = + CompletableFuture.runAsync(() -> tv.put(k, v).join()); + Awaitility.await().until(() -> f1.isDone() && f2.isDone()); + assertTrue(f1.isCompletedExceptionally() && !f2.isCompletedExceptionally() || + ! f1.isCompletedExceptionally() && f2.isCompletedExceptionally()); + } + + @Test(dataProvider = "impl") + public void testConcurrentDelete(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + + String k = "my"; + tv.put(k, 0).join(); + CompletableFuture f1 = + CompletableFuture.runAsync(() -> tv.delete(k).join()); + CompletableFuture f2 = + CompletableFuture.runAsync(() -> tv.delete(k).join()); + Awaitility.await().until(() -> f1.isDone() && f2.isDone()); + assertTrue(f1.isCompletedExceptionally() && !f2.isCompletedExceptionally() || + ! f1.isCompletedExceptionally() && f2.isCompletedExceptionally()); + } + + @Test(dataProvider = "impl") + public void testClosedMetadataStore(String provider, Supplier urlSupplier) throws Exception { + String prefix = newKey(); + String k = "my"; + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store, prefix, urlSupplier); + store.close(); + try { + tv.put(k, 0).get(); + fail(); + } catch (Exception e) { + assertTrue(e.getCause() instanceof MetadataStoreException.AlreadyClosedException); + } + try { + tv.delete(k).get(); + fail(); + } catch (Exception e) { + assertTrue(e.getCause() instanceof MetadataStoreException.AlreadyClosedException); + } + + } + + + @Test(dataProvider = "distributedImpl") + public void testGetIfCachedDistributed(String provider, Supplier urlSupplier) throws Exception { + + String prefix = newKey(); + String k = "my"; + @Cleanup + MetadataStore store1 = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv1 = createTestTableView(store1, prefix, urlSupplier); + @Cleanup + MetadataStore store2 = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv2 = createTestTableView(store2, prefix, urlSupplier); + + + assertNull(tv1.get(k)); + assertNull(tv2.get(k)); + + tv1.put(k, 0).join(); + assertGet(tv1, k, 0); + Awaitility.await() + .untilAsserted(() -> assertEquals(tv2.get(k), 0)); + + tv2.put(k, 1).join(); + assertGet(tv2, k, 1); + Awaitility.await() + .untilAsserted(() -> assertEquals(tv1.get(k), 1)); + + tv1.delete(k).join(); + assertGet(tv1, k, null); + Awaitility.await() + .untilAsserted(() -> assertNull(tv2.get(k))); + } + + @Test(dataProvider = "distributedImpl") + public void testInitialFill(String provider, Supplier urlSupplier) throws Exception { + + String prefix = newKey(); + String k1 = "tenant-1/ns-1/my-1"; + String k2 = "tenant-1/ns-1/my-2"; + String k3 = "tenant-1/ns-2/my-3"; + String k4 = "tenant-2/ns-3/my-4"; + String k5 = "tenant-2/ns-3/your-1"; + @Cleanup + MetadataStore store = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl btv = createTestTableView(store, prefix, urlSupplier); + + assertFalse(btv.exists(k1)); + + var serde = new JSONMetadataSerdeSimpleType<>( + TypeFactory.defaultInstance().constructSimpleType(Integer.class, null)); + store.put(prefix + "/" + k1, serde.serialize(prefix + "/" + k1, 0), Optional.empty()).join(); + store.put(prefix + "/" + k2, serde.serialize(prefix + "/" + k2, 1), Optional.empty()).join(); + store.put(prefix + "/" + k3, serde.serialize(prefix + "/" + k3, 2), Optional.empty()).join(); + store.put(prefix + "/" + k4, serde.serialize(prefix + "/" + k4, 3), Optional.empty()).join(); + store.put(prefix + "/" + k5, serde.serialize(prefix + "/" + k5, 4), Optional.empty()).join(); + + var expected = new HashSet<>(Set.of(Pair.of(k1, 0), Pair.of(k2, 1), Pair.of(k3, 2), Pair.of(k4, 3))); + var tailExpected = new HashSet<>(expected); + + for (int i = 0; i < 4; i++) { + var kv = tails.poll(3, TimeUnit.SECONDS); + assertTrue(tailExpected.remove(kv)); + } + assertNull(tails.poll(3, TimeUnit.SECONDS)); + assertTrue(tailExpected.isEmpty()); + + @Cleanup + MetadataStore store2 = MetadataStoreFactoryImpl.create(urlSupplier.get(), MetadataStoreConfig.builder().build()); + MetadataStoreTableViewImpl tv = createTestTableView(store2, prefix, urlSupplier); + + var existingExpected = new HashSet<>(Set.of(Pair.of(k1, 0), Pair.of(k2, 1), Pair.of(k3, 2), Pair.of(k4, 3))); + var entrySetExpected = expected.stream().collect(Collectors.toMap(Pair::getLeft, Pair::getRight)).entrySet(); + + + for (int i = 0; i < 4; i++) { + var kv = existings.poll(3, TimeUnit.SECONDS); + assertTrue(existingExpected.remove(kv)); + } + assertNull(existings.poll(3, TimeUnit.SECONDS)); + assertTrue(existingExpected.isEmpty()); + + assertEquals(tv.get(k1), 0); + assertEquals(tv.get(k2), 1); + assertEquals(tv.get(k3), 2); + assertEquals(tv.get(k4), 3); + assertNull(tv.get(k5)); + + assertEquals(tv.entrySet(), entrySetExpected); + } +} diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java index ee7497010adfc..759e689b24d0f 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java @@ -57,6 +57,8 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @@ -78,6 +80,20 @@ public class ExtensibleLoadManagerTest extends TestRetrySupport { private PulsarCluster pulsarCluster = null; private String hosts; private PulsarAdmin admin; + protected String serviceUnitStateTableViewClassName; + + @Factory(dataProvider = "serviceUnitStateTableViewClassName") + public ExtensibleLoadManagerTest(String serviceUnitStateTableViewClassName) { + this.serviceUnitStateTableViewClassName = serviceUnitStateTableViewClassName; + } + + @DataProvider(name = "serviceUnitStateTableViewClassName") + public static Object[][] serviceUnitStateTableViewClassName() { + return new Object[][]{ + {"org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateMetadataStoreTableViewImpl"}, + {"org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl"} + }; + } @BeforeClass(alwaysRun = true) public void setup() throws Exception { @@ -87,6 +103,8 @@ public void setup() throws Exception { "org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl"); brokerEnvs.put("loadBalancerLoadSheddingStrategy", "org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder"); + brokerEnvs.put("loadManagerServiceUnitStateTableViewClassName", + serviceUnitStateTableViewClassName); brokerEnvs.put("forceDeleteNamespaceAllowed", "true"); brokerEnvs.put("loadBalancerDebugModeEnabled", "true"); brokerEnvs.put("PULSAR_MEM", "-Xmx512M"); @@ -226,17 +244,17 @@ public void testSplitBundleAdminApi() throws Exception { long mid = bundleRanges.get(0) + (bundleRanges.get(1) - bundleRanges.get(0)) / 2; Awaitility.waitAtMost(10, TimeUnit.SECONDS).pollDelay(100, TimeUnit.MILLISECONDS) .untilAsserted( - () -> { - BundlesData bundlesData = admin.namespaces().getBundles(DEFAULT_NAMESPACE); - assertEquals(bundlesData.getNumBundles(), numBundles + 1); - String lowBundle = String.format("0x%08x", bundleRanges.get(0)); - String midBundle = String.format("0x%08x", mid); - String highBundle = String.format("0x%08x", bundleRanges.get(1)); - assertTrue(bundlesData.getBoundaries().contains(lowBundle)); - assertTrue(bundlesData.getBoundaries().contains(midBundle)); - assertTrue(bundlesData.getBoundaries().contains(highBundle)); - } - ); + () -> { + BundlesData bundlesData = admin.namespaces().getBundles(DEFAULT_NAMESPACE); + assertEquals(bundlesData.getNumBundles(), numBundles + 1); + String lowBundle = String.format("0x%08x", bundleRanges.get(0)); + String midBundle = String.format("0x%08x", mid); + String highBundle = String.format("0x%08x", bundleRanges.get(1)); + assertTrue(bundlesData.getBoundaries().contains(lowBundle)); + assertTrue(bundlesData.getBoundaries().contains(midBundle)); + assertTrue(bundlesData.getBoundaries().contains(highBundle)); + } + ); // Test split bundle with invalid bundle range. @@ -393,10 +411,10 @@ public void testIsolationPolicy() throws Exception { } Awaitility.await().atMost(60, TimeUnit.SECONDS).ignoreExceptions().untilAsserted( - () -> { - List activeBrokers = admin.brokers().getActiveBrokersAsync().get(5, TimeUnit.SECONDS); - assertEquals(activeBrokers.size(), 1); - } + () -> { + List activeBrokers = admin.brokers().getActiveBrokersAsync().get(5, TimeUnit.SECONDS); + assertEquals(activeBrokers.size(), 1); + } ); Awaitility.await().atMost(60, TimeUnit.SECONDS).ignoreExceptions().untilAsserted( From f5c1ad24d7753eb203da3c7f3e4912b005b76044 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Sat, 21 Sep 2024 15:02:24 +0800 Subject: [PATCH 515/580] [improve][broker] Replace ConcurrentOpenHashMap with ConcurrentHashMap in BrokerService (#23320) --- .../broker/admin/v2/NonPersistentTopics.java | 10 +- .../broker/namespace/NamespaceService.java | 6 +- .../apache/pulsar/broker/rest/TopicsBase.java | 12 +- .../pulsar/broker/service/BrokerService.java | 151 ++++-------------- .../pulsar/broker/service/PulsarStats.java | 5 +- .../prometheus/NamespaceStatsAggregator.java | 2 +- .../prometheus/TransactionAggregator.java | 2 +- .../pulsar/broker/admin/AdminApi2Test.java | 5 +- .../DelayedDeliveryTrackerFactoryTest.java | 8 +- .../namespace/NamespaceServiceTest.java | 6 +- .../service/AbstractReplicatorTest.java | 4 +- .../BrokerServiceAutoTopicCreationTest.java | 12 +- .../broker/service/BrokerServiceTest.java | 2 +- .../broker/service/OneWayReplicatorTest.java | 8 +- .../service/ReplicatorAdminTlsTest.java | 4 +- .../ReplicatorAdminTlsWithKeyStoreTest.java | 4 +- .../service/ReplicatorGlobalNSTest.java | 7 +- .../pulsar/broker/service/ReplicatorTest.java | 6 +- .../persistent/TopicDuplicationTest.java | 9 +- .../broker/stats/PrometheusMetricsTest.java | 74 ++++----- .../NamespaceStatsAggregatorTest.java | 15 +- .../TopicTransactionBufferRecoverTest.java | 19 +-- .../transaction/TransactionProduceTest.java | 2 +- .../broker/transaction/TransactionTest.java | 4 +- .../buffer/TransactionLowWaterMarkTest.java | 10 +- .../PendingAckInMemoryDeleteTest.java | 19 +-- .../pulsar/client/api/TopicReaderTest.java | 2 +- .../impl/BrokerClientIntegrationTest.java | 5 +- .../client/impl/TransactionEndToEndTest.java | 15 +- 29 files changed, 130 insertions(+), 298 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index 9f58aa4ca9d44..edf4303e1adef 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -62,7 +62,6 @@ import org.apache.pulsar.common.policies.data.stats.NonPersistentPartitionedTopicStatsImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentTopicStatsImpl; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -478,18 +477,17 @@ public void getListFromBundle( } else { validateNamespaceBundleOwnershipAsync(namespaceName, policies.bundles, bundleRange, true, true) .thenAccept(nsBundle -> { - ConcurrentOpenHashMap> bundleTopics = - pulsar().getBrokerService() - .getMultiLayerTopicsMap().get(namespaceName.toString()); + final var bundleTopics = pulsar().getBrokerService().getMultiLayerTopicsMap() + .get(namespaceName.toString()); if (bundleTopics == null || bundleTopics.isEmpty()) { asyncResponse.resume(Collections.emptyList()); return; } final List topicList = new ArrayList<>(); String bundleKey = namespaceName.toString() + "/" + nsBundle.getBundleRange(); - ConcurrentOpenHashMap topicMap = bundleTopics.get(bundleKey); + final var topicMap = bundleTopics.get(bundleKey); if (topicMap != null) { - topicList.addAll(topicMap.keys().stream() + topicList.addAll(topicMap.keySet().stream() .filter(name -> !TopicName.get(name).isPersistent()) .collect(Collectors.toList())); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 92188f5e6eeee..0b1661fb9540a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -1618,10 +1618,10 @@ public CompletableFuture> getListOfNonPersistentTopics(NamespaceNam // Non-persistent topics don't have managed ledgers. So we have to retrieve them from local // cache. List topics = new ArrayList<>(); - synchronized (pulsar.getBrokerService().getMultiLayerTopicMap()) { - if (pulsar.getBrokerService().getMultiLayerTopicMap() + synchronized (pulsar.getBrokerService().getMultiLayerTopicsMap()) { + if (pulsar.getBrokerService().getMultiLayerTopicsMap() .containsKey(namespaceName.toString())) { - pulsar.getBrokerService().getMultiLayerTopicMap().get(namespaceName.toString()) + pulsar.getBrokerService().getMultiLayerTopicsMap().get(namespaceName.toString()) .forEach((__, bundle) -> bundle.forEach((topicName, topic) -> { if (topic instanceof NonPersistentTopic && ((NonPersistentTopic) topic).isActive()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java index 8f55df1107d0f..bf6e7350186c2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java @@ -38,7 +38,9 @@ import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import javax.ws.rs.container.AsyncResponse; @@ -95,7 +97,6 @@ import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.ObjectMapperFactory; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; import org.apache.pulsar.websocket.data.ProducerAck; import org.apache.pulsar.websocket.data.ProducerAcks; import org.apache.pulsar.websocket.data.ProducerMessage; @@ -122,8 +123,9 @@ protected void publishMessages(AsyncResponse asyncResponse, ProducerMessages req .thenAccept(schemaMeta -> { // Both schema version and schema data are necessary. if (schemaMeta.getLeft() != null && schemaMeta.getRight() != null) { - internalPublishMessages(topicName, request, pulsar().getBrokerService() - .getOwningTopics().get(topic).values(), asyncResponse, + final var partitionIndexes = pulsar().getBrokerService().getOwningTopics() + .getOrDefault(topic, Set.of()).stream().toList(); + internalPublishMessages(topicName, request, partitionIndexes, asyncResponse, AutoConsumeSchema.getSchema(schemaMeta.getLeft().toSchemaInfo()), schemaMeta.getRight()); } else { @@ -446,7 +448,7 @@ private CompletableFuture lookUpBrokerForTopic(TopicName partitionedTopicN } pulsar().getBrokerService().getOwningTopics().computeIfAbsent(partitionedTopicName .getPartitionedTopicName(), - (key) -> ConcurrentOpenHashSet.newBuilder().build()) + __ -> ConcurrentHashMap.newKeySet()) .add(partitionedTopicName.getPartitionIndex()); completeLookup(Pair.of(Collections.emptyList(), false), redirectAddresses, future); } else { @@ -517,7 +519,7 @@ private CompletableFuture addSchema(SchemaData schemaData) { // Only need to add to first partition the broker owns since the schema id in schema registry are // same for all partitions which is the partitionedTopicName List partitions = pulsar().getBrokerService().getOwningTopics() - .get(topicName.getPartitionedTopicName()).values(); + .get(topicName.getPartitionedTopicName()).stream().toList(); CompletableFuture result = new CompletableFuture<>(); for (int index = 0; index < partitions.size(); index++) { CompletableFuture future = new CompletableFuture<>(); 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 c7a210bc543cf..6b0be07c8f7a8 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 @@ -63,6 +63,7 @@ import java.util.Set; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.RejectedExecutionException; @@ -177,8 +178,6 @@ import org.apache.pulsar.common.util.FieldParser; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.GracefulExecutorServicesShutdown; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; import org.apache.pulsar.common.util.netty.ChannelFutures; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.apache.pulsar.common.util.netty.NettyFutureUtil; @@ -217,27 +216,26 @@ public class BrokerService implements Closeable { private final PulsarService pulsar; private final ManagedLedgerFactory managedLedgerFactory; - private final ConcurrentOpenHashMap>> topics; + private final Map>> topics = new ConcurrentHashMap<>(); - private final ConcurrentOpenHashMap replicationClients; - private final ConcurrentOpenHashMap clusterAdmins; + private final Map replicationClients = new ConcurrentHashMap<>(); + private final Map clusterAdmins = new ConcurrentHashMap<>(); // Multi-layer topics map: // Namespace --> Bundle --> topicName --> topic - private final ConcurrentOpenHashMap>> - multiLayerTopicsMap; + private final Map>> multiLayerTopicsMap = new ConcurrentHashMap<>(); // Keep track of topics and partitions served by this broker for fast lookup. @Getter - private final ConcurrentOpenHashMap> owningTopics; + private final Map> owningTopics = new ConcurrentHashMap<>(); private long numberOfNamespaceBundles = 0; private final EventLoopGroup acceptorGroup; private final EventLoopGroup workerGroup; private final OrderedExecutor topicOrderedExecutor; // offline topic backlog cache - private final ConcurrentOpenHashMap offlineTopicStatCache; - private final ConcurrentOpenHashMap dynamicConfigurationMap; - private final ConcurrentOpenHashMap> configRegisteredListeners; + private final Map offlineTopicStatCache = new ConcurrentHashMap<>(); + private final Map dynamicConfigurationMap; + private final Map> configRegisteredListeners = new ConcurrentHashMap<>(); private final ConcurrentLinkedQueue pendingTopicLoadingQueue; @@ -297,7 +295,7 @@ public class BrokerService implements Closeable { private final int maxUnackedMessages; public final int maxUnackedMsgsPerDispatcher; private final AtomicBoolean blockedDispatcherOnHighUnackedMsgs = new AtomicBoolean(false); - private final ConcurrentOpenHashSet blockedDispatchers; + private final Set blockedDispatchers = ConcurrentHashMap.newKeySet(); private final ReadWriteLock lock = new ReentrantReadWriteLock(); @VisibleForTesting private final DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory; @@ -335,28 +333,9 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws this.preciseTopicPublishRateLimitingEnable = pulsar.getConfiguration().isPreciseTopicPublishRateLimiterEnable(); this.managedLedgerFactory = pulsar.getManagedLedgerFactory(); - this.topics = - ConcurrentOpenHashMap.>>newBuilder() - .build(); - this.replicationClients = - ConcurrentOpenHashMap.newBuilder().build(); - this.clusterAdmins = - ConcurrentOpenHashMap.newBuilder().build(); this.keepAliveIntervalSeconds = pulsar.getConfiguration().getKeepAliveIntervalSeconds(); - this.configRegisteredListeners = - ConcurrentOpenHashMap.>newBuilder().build(); this.pendingTopicLoadingQueue = Queues.newConcurrentLinkedQueue(); - - this.multiLayerTopicsMap = ConcurrentOpenHashMap.>>newBuilder() - .build(); - this.owningTopics = ConcurrentOpenHashMap.>newBuilder() - .build(); this.pulsarStats = new PulsarStats(pulsar); - this.offlineTopicStatCache = - ConcurrentOpenHashMap.newBuilder().build(); this.topicOrderedExecutor = OrderedExecutor.newBuilder() .numThreads(pulsar.getConfiguration().getTopicOrderedExecutorThreadNum()) @@ -403,8 +382,6 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws .build(); this.authenticationService = new AuthenticationService(pulsar.getConfiguration(), pulsar.getOpenTelemetry().getOpenTelemetry()); - this.blockedDispatchers = - ConcurrentOpenHashSet.newBuilder().build(); this.topicFactory = createPersistentTopicFactory(); // update dynamic configuration and register-listener updateConfigurationAndRegisterListeners(); @@ -500,7 +477,7 @@ private int getPendingTopicLoadRequests() { public void addTopicEventListener(TopicEventsListener... listeners) { topicEventsDispatcher.addTopicEventListener(listeners); - getTopics().keys().forEach(topic -> + topics.keySet().forEach(topic -> TopicEventsDispatcher.notify(listeners, topic, TopicEvent.LOAD, EventStage.SUCCESS, null)); } @@ -572,13 +549,8 @@ private ServerBootstrap defaultServerBootstrap() { } public Map getTopicStats(NamespaceBundle bundle) { - ConcurrentOpenHashMap topicMap = getMultiLayerTopicMap() - .computeIfAbsent(bundle.getNamespaceObject().toString(), k -> { - return ConcurrentOpenHashMap - .>newBuilder().build(); - }).computeIfAbsent(bundle.toString(), k -> { - return ConcurrentOpenHashMap.newBuilder().build(); - }); + final var topicMap = multiLayerTopicsMap.computeIfAbsent(bundle.getNamespaceObject().toString(), + __ -> new ConcurrentHashMap<>()).computeIfAbsent(bundle.toString(), __ -> new ConcurrentHashMap<>()); Map topicStatsMap = new HashMap<>(); topicMap.forEach((name, topic) -> { @@ -2047,14 +2019,10 @@ private void addTopicToStatsMaps(TopicName topicName, Topic topic) { if (namespaceBundle != null) { synchronized (multiLayerTopicsMap) { String serviceUnit = namespaceBundle.toString(); - multiLayerTopicsMap // - .computeIfAbsent(topicName.getNamespace(), - k -> ConcurrentOpenHashMap.>newBuilder() - .build()) // - .computeIfAbsent(serviceUnit, - k -> ConcurrentOpenHashMap.newBuilder().build()) // - .put(topicName.toString(), topic); + multiLayerTopicsMap.computeIfAbsent(topicName.getNamespace(), + __ -> new ConcurrentHashMap<>() + ).computeIfAbsent(serviceUnit, __ -> new ConcurrentHashMap<>() + ).put(topicName.toString(), topic); } } invalidateOfflineTopicStatCache(topicName); @@ -2353,7 +2321,7 @@ private CompletableFuture unloadServiceUnit(NamespaceBundle serviceUnit } public void cleanUnloadedTopicFromCache(NamespaceBundle serviceUnit) { - for (String topic : topics.keys()) { + for (String topic : topics.keySet()) { TopicName topicName = TopicName.get(topic); if (serviceUnit.includes(topicName) && getTopicReference(topic).isPresent()) { log.info("[{}][{}] Clean unloaded topic from cache.", serviceUnit.toString(), topic); @@ -2418,10 +2386,9 @@ private void removeTopicFromCache(String topic, NamespaceBundle namespaceBundle, topicEventsDispatcher.notify(topic, TopicEvent.UNLOAD, EventStage.BEFORE); synchronized (multiLayerTopicsMap) { - ConcurrentOpenHashMap> namespaceMap = multiLayerTopicsMap - .get(namespaceName); + final var namespaceMap = multiLayerTopicsMap.get(namespaceName); if (namespaceMap != null) { - ConcurrentOpenHashMap bundleMap = namespaceMap.get(bundleName); + final var bundleMap = namespaceMap.get(bundleName); if (bundleMap != null) { bundleMap.remove(topic); if (bundleMap.isEmpty()) { @@ -2462,10 +2429,6 @@ public long getNumberOfNamespaceBundles() { return this.numberOfNamespaceBundles; } - public ConcurrentOpenHashMap>> getTopics() { - return topics; - } - private void handleMetadataChanges(Notification n) { if (n.getType() == NotificationType.Modified && NamespaceResources.pathIsFromNamespace(n.getPath())) { @@ -2660,10 +2623,6 @@ public EventLoopGroup executor() { return workerGroup; } - public ConcurrentOpenHashMap getReplicationClients() { - return replicationClients; - } - public boolean isAuthenticationEnabled() { return pulsar.getConfiguration().isAuthenticationEnabled(); } @@ -2693,17 +2652,17 @@ public AuthenticationService getAuthenticationService() { } public List getAllTopicsFromNamespaceBundle(String namespace, String bundle) { - ConcurrentOpenHashMap> map1 = multiLayerTopicsMap.get(namespace); + final var map1 = multiLayerTopicsMap.get(namespace); if (map1 == null) { return Collections.emptyList(); } - ConcurrentOpenHashMap map2 = map1.get(bundle); + final var map2 = map1.get(bundle); if (map2 == null) { return Collections.emptyList(); } - return map2.values(); + return map2.values().stream().toList(); } /** @@ -3080,60 +3039,17 @@ private void createDynamicConfigPathIfNotExist() { } } - /** - * Updates pulsar.ServiceConfiguration's dynamic field with value persistent into zk-dynamic path. It also validates - * dynamic-value before updating it and throws {@code IllegalArgumentException} if validation fails - */ - private void updateDynamicServiceConfiguration() { - Optional> configCache = Optional.empty(); - - try { - configCache = - pulsar().getPulsarResources().getDynamicConfigResources().getDynamicConfiguration(); - - // create dynamic-config if not exist. - if (!configCache.isPresent()) { - pulsar().getPulsarResources().getDynamicConfigResources() - .setDynamicConfigurationWithCreate(n -> new HashMap<>()); - } - } catch (Exception e) { - log.warn("Failed to read dynamic broker configuration", e); - } - - configCache.ifPresent(stringStringMap -> stringStringMap.forEach((key, value) -> { - // validate field - if (dynamicConfigurationMap.containsKey(key) && dynamicConfigurationMap.get(key).validator != null) { - if (!dynamicConfigurationMap.get(key).validator.test(value)) { - log.error("Failed to validate dynamic config {} with value {}", key, value); - throw new IllegalArgumentException( - String.format("Failed to validate dynamic-config %s/%s", key, value)); - } - } - // update field value - try { - Field field = ServiceConfiguration.class.getDeclaredField(key); - if (field != null && field.isAnnotationPresent(FieldContext.class)) { - field.setAccessible(true); - field.set(pulsar().getConfiguration(), FieldParser.value(value, field)); - log.info("Successfully updated {}/{}", key, value); - } - } catch (Exception e) { - log.warn("Failed to update service configuration {}/{}, {}", key, value, e.getMessage()); - } - })); - } - public DelayedDeliveryTrackerFactory getDelayedDeliveryTrackerFactory() { return delayedDeliveryTrackerFactory; } public List getDynamicConfiguration() { - return dynamicConfigurationMap.keys(); + return dynamicConfigurationMap.keySet().stream().toList(); } public Map getRuntimeConfiguration() { Map configMap = new HashMap<>(); - ConcurrentOpenHashMap runtimeConfigurationMap = getRuntimeConfigurationMap(); + ConcurrentHashMap runtimeConfigurationMap = getRuntimeConfigurationMap(); runtimeConfigurationMap.forEach((key, value) -> { configMap.put(key, String.valueOf(value)); }); @@ -3151,9 +3067,8 @@ public boolean validateDynamicConfiguration(String key, String value) { return true; } - private ConcurrentOpenHashMap prepareDynamicConfigurationMap() { - ConcurrentOpenHashMap dynamicConfigurationMap = - ConcurrentOpenHashMap.newBuilder().build(); + private Map prepareDynamicConfigurationMap() { + final var dynamicConfigurationMap = new ConcurrentHashMap(); try { for (Field field : ServiceConfiguration.class.getDeclaredFields()) { if (field != null && field.isAnnotationPresent(FieldContext.class)) { @@ -3172,9 +3087,8 @@ private ConcurrentOpenHashMap prepareDynamicConfigurationMa return dynamicConfigurationMap; } - private ConcurrentOpenHashMap getRuntimeConfigurationMap() { - ConcurrentOpenHashMap runtimeConfigurationMap = - ConcurrentOpenHashMap.newBuilder().build(); + private ConcurrentHashMap getRuntimeConfigurationMap() { + final var runtimeConfigurationMap = new ConcurrentHashMap(); for (Field field : ServiceConfiguration.class.getDeclaredFields()) { if (field != null && field.isAnnotationPresent(FieldContext.class)) { field.setAccessible(true); @@ -3337,11 +3251,6 @@ public OrderedExecutor getTopicOrderedExecutor() { return topicOrderedExecutor; } - public ConcurrentOpenHashMap>> - getMultiLayerTopicMap() { - return multiLayerTopicsMap; - } - /** * If per-broker unacked message reached to limit then it blocks dispatcher if its unacked message limit has been * reached to {@link #maxUnackedMsgsPerDispatcher}. @@ -3393,7 +3302,7 @@ public void checkUnAckMessageDispatching() { } else if (blockedDispatcherOnHighUnackedMsgs.get() && unAckedMessages < maxUnackedMessages / 2) { // unblock broker-dispatching if received enough acked messages back if (blockedDispatcherOnHighUnackedMsgs.compareAndSet(true, false)) { - unblockDispatchersOnUnAckMessages(blockedDispatchers.values()); + unblockDispatchersOnUnAckMessages(blockedDispatchers.stream().toList()); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java index 7ffc7818d4c2d..b96e00a8909d6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarStats.java @@ -38,7 +38,6 @@ import org.apache.pulsar.broker.stats.NamespaceStats; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; import org.apache.pulsar.utils.StatsOutputStream; import org.slf4j.Logger; @@ -101,9 +100,7 @@ public ClusterReplicationMetrics getClusterReplicationMetrics() { return clusterReplicationMetrics; } - public synchronized void updateStats( - ConcurrentOpenHashMap>> - topicsMap) { + public synchronized void updateStats(Map>> topicsMap) { StatsOutputStream topicStatsStream = new StatsOutputStream(tempTopicStatsBuf); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java index 25c875778c05c..110a8aa82f112 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java @@ -83,7 +83,7 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b Optional compactorMXBean = getCompactorMXBean(pulsar); LongAdder topicsCount = new LongAdder(); Map localNamespaceTopicCount = new HashMap<>(); - pulsar.getBrokerService().getMultiLayerTopicMap().forEach((namespace, bundlesMap) -> { + pulsar.getBrokerService().getMultiLayerTopicsMap().forEach((namespace, bundlesMap) -> { namespaceStats.reset(); topicsCount.reset(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TransactionAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TransactionAggregator.java index 3da061f6ffef2..df2638b3bb810 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TransactionAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TransactionAggregator.java @@ -56,7 +56,7 @@ public static void generate(PulsarService pulsar, PrometheusMetricStreams stream if (includeTopicMetrics) { - pulsar.getBrokerService().getMultiLayerTopicMap().forEach((namespace, bundlesMap) -> + pulsar.getBrokerService().getMultiLayerTopicsMap().forEach((namespace, bundlesMap) -> bundlesMap.forEach((bundle, topicsMap) -> topicsMap.forEach((name, topic) -> { if (topic instanceof PersistentTopic) { topic.getSubscriptions().values().forEach(subscription -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java index df9862691d6d5..900babbecf4ad 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java @@ -114,9 +114,7 @@ import org.apache.pulsar.common.policies.data.*; import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; import org.apache.pulsar.common.protocol.schema.SchemaData; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; -import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -2913,8 +2911,7 @@ public void testMaxProducersPerTopicUnlimited() throws Exception { } private AtomicInteger injectSchemaCheckCounterForTopic(String topicName) { - ConcurrentOpenHashMap>> topics = - WhiteboxImpl.getInternalState(pulsar.getBrokerService(), "topics"); + final var topics = pulsar.getBrokerService().getTopics(); AbstractTopic topic = (AbstractTopic) topics.get(topicName).join().get(); AbstractTopic spyTopic = Mockito.spy(topic); AtomicInteger counter = new AtomicInteger(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java index bb6ef9d363652..9861ab5723732 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java @@ -24,12 +24,10 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.Subscription; -import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.*; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.mockito.Mockito; import org.testng.Assert; @@ -166,11 +164,7 @@ public void testPublishDelayMessagesAndCreateBucketDelayDeliveryTrackerFailed() Mockito.doReturn(brokerService).when(topic).getBrokerService(); // Set Mocked topic to BrokerService - Field topics = BrokerService.class.getDeclaredField("topics"); - topics.setAccessible(true); - @SuppressWarnings("unchecked") - ConcurrentOpenHashMap>> topicMap = - (ConcurrentOpenHashMap>>) topics.get(brokerService); + final var topicMap = brokerService.getTopics(); topicMap.put(topicName, CompletableFuture.completedFuture(Optional.of(topic))); // Create consumer diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java index 6b2669275dfdb..951247bd68861 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java @@ -87,7 +87,6 @@ import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.Notification; @@ -300,8 +299,7 @@ public void testUnloadNamespaceBundleFailure() throws Exception { final String topicName = "persistent://my-property/use/my-ns/my-topic1"; pulsarClient.newConsumer().topic(topicName).subscriptionName("my-subscriber-name").subscribe(); - ConcurrentOpenHashMap>> topics = pulsar.getBrokerService() - .getTopics(); + final var topics = pulsar.getBrokerService().getTopics(); Topic spyTopic = spy(topics.get(topicName).get().get()); topics.clear(); CompletableFuture> topicFuture = CompletableFuture.completedFuture(Optional.of(spyTopic)); @@ -331,7 +329,7 @@ public void testUnloadNamespaceBundleWithStuckTopic() throws Exception { final String topicName = "persistent://my-property/use/my-ns/my-topic1"; Consumer consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("my-subscriber-name") .subscribe(); - ConcurrentOpenHashMap>> topics = pulsar.getBrokerService().getTopics(); + final var topics = pulsar.getBrokerService().getTopics(); Topic spyTopic = spy(topics.get(topicName).get().get()); topics.clear(); CompletableFuture> topicFuture = CompletableFuture.completedFuture(Optional.of(spyTopic)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java index 7415a40ad5553..374296e68671d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java @@ -28,6 +28,7 @@ import io.netty.util.internal.DefaultPriorityQueue; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; @@ -45,7 +46,6 @@ import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; import org.testng.Assert; @@ -74,7 +74,7 @@ public void testRetryStartProducerStoppedByTopicRemove() throws Exception { when(remoteClient.getCnxPool()).thenReturn(connectionPool); final ProducerConfigurationData producerConf = new ProducerConfigurationData(); final ProducerBuilderImpl producerBuilder = mock(ProducerBuilderImpl.class); - final ConcurrentOpenHashMap>> topics = new ConcurrentOpenHashMap<>(); + final var topics = new ConcurrentHashMap>>(); when(broker.executor()).thenReturn(eventLoopGroup); when(broker.getTopics()).thenReturn(topics); when(remoteClient.newProducer(any(Schema.class))).thenReturn(producerBuilder); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java index a4460187d2377..3e735ee4c85b8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java @@ -26,11 +26,11 @@ import java.util.List; -import java.util.Optional; +import java.util.Map; import java.util.Set; import java.util.UUID; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import lombok.Cleanup; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.client.admin.ListNamespaceTopicsOptions; @@ -45,7 +45,6 @@ import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicType; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -552,10 +551,9 @@ public void testExtensibleLoadManagerImplInternalTopicAutoCreations() admin.topics().createNonPartitionedTopic(ExtensibleLoadManagerImpl.TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); // clear the topics to test the auto creation of non-persistent topics. - ConcurrentOpenHashMap>> topics = - pulsar.getBrokerService().getTopics(); - ConcurrentOpenHashMap>> oldTopics = new ConcurrentOpenHashMap<>(); - topics.forEach((key, val) -> oldTopics.put(key, val)); + final var topics = pulsar.getBrokerService().getTopics(); + final var oldTopics = topics.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, + Map.Entry::getValue)); topics.clear(); // The created persistent topic correctly can be found by diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 5398b5aa57b8b..17209c83c13ea 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -1630,7 +1630,7 @@ public void testGetTopic() throws Exception { producer1.close(); PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(topicName.toString(), false).get().get(); persistentTopic.close().join(); - List topics = new ArrayList<>(pulsar.getBrokerService().getTopics().keys()); + List topics = new ArrayList<>(pulsar.getBrokerService().getTopics().keySet()); topics.removeIf(item -> item.contains(SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME)); Assert.assertEquals(topics.size(), 0); @Cleanup diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java index d684b4af7c251..a8f8d7ecbbd47 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTest.java @@ -92,7 +92,6 @@ import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; import org.glassfish.jersey.client.JerseyClient; @@ -293,8 +292,7 @@ private Runnable injectMockReplicatorProducerBuilder( }); // Inject spy client. - ConcurrentOpenHashMap - replicationClients = WhiteboxImpl.getInternalState(brokerService, "replicationClients"); + final var replicationClients = brokerService.getReplicationClients(); PulsarClientImpl internalClient = (PulsarClientImpl) replicationClients.get(cluster2); PulsarClient spyClient = spy(internalClient); assertTrue(replicationClients.remove(cluster2, internalClient)); @@ -1141,9 +1139,9 @@ public void testDifferentTopicCreationRule(ReplicationMode replicationMode) thro return t.startsWith(tp); }; Awaitility.await().untilAsserted(() -> { - List topics1 = pulsar1.getBrokerService().getTopics().keys() + List topics1 = pulsar1.getBrokerService().getTopics().keySet() .stream().filter(topicNameFilter).collect(Collectors.toList()); - List topics2 = pulsar2.getBrokerService().getTopics().keys() + List topics2 = pulsar2.getBrokerService().getTopics().keySet() .stream().filter(topicNameFilter).collect(Collectors.toList()); Collections.sort(topics1); Collections.sort(topics2); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorAdminTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorAdminTlsTest.java index a5d14ca0487dc..c214378fd94a3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorAdminTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorAdminTlsTest.java @@ -23,10 +23,8 @@ import static org.testng.Assert.assertTrue; import java.util.List; import java.util.Optional; -import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -55,7 +53,7 @@ public void testReplicationAdmin() throws Exception { ns.getClusterPulsarAdmin(cluster3, Optional.of(admin1.clusters().getCluster(cluster3))); // verify the admin - ConcurrentOpenHashMap clusterAdmins = ns.getClusterAdmins(); + final var clusterAdmins = ns.getClusterAdmins(); assertFalse(clusterAdmins.isEmpty()); clusterAdmins.forEach((cluster, admin) -> { ClientConfigurationData clientConfigData = ((PulsarAdminImpl) admin).getClientConfigData(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorAdminTlsWithKeyStoreTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorAdminTlsWithKeyStoreTest.java index 3d3eb3faa727f..451bdd10eb106 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorAdminTlsWithKeyStoreTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorAdminTlsWithKeyStoreTest.java @@ -23,10 +23,8 @@ import static org.testng.Assert.assertTrue; import java.util.List; import java.util.Optional; -import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -56,7 +54,7 @@ public void testReplicationAdmin() throws Exception { ns.getClusterPulsarAdmin(cluster3, Optional.of(admin1.clusters().getCluster(cluster3))); // verify the admin - ConcurrentOpenHashMap clusterAdmins = ns.getClusterAdmins(); + final var clusterAdmins = ns.getClusterAdmins(); assertFalse(clusterAdmins.isEmpty()); clusterAdmins.forEach((cluster, admin) -> { ClientConfigurationData clientConfigData = ((PulsarAdminImpl) admin).getClientConfigData(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java index 514e0207fbfb1..a1f147cbb6273 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java @@ -34,7 +34,6 @@ import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -169,9 +168,9 @@ public Void call() throws Exception { Thread.sleep(1000L); // Make sure that the internal replicators map contains remote cluster info - ConcurrentOpenHashMap replicationClients1 = ns1.getReplicationClients(); - ConcurrentOpenHashMap replicationClients2 = ns2.getReplicationClients(); - ConcurrentOpenHashMap replicationClients3 = ns3.getReplicationClients(); + final var replicationClients1 = ns1.getReplicationClients(); + final var replicationClients2 = ns2.getReplicationClients(); + final var replicationClients3 = ns3.getReplicationClients(); Assert.assertNotNull(replicationClients1.get("r2")); Assert.assertNotNull(replicationClients1.get("r3")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index 1c47abab775b3..8e115e14b3770 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -234,11 +234,7 @@ public void testConcurrentReplicator() throws Exception { final Method startRepl = PersistentTopic.class.getDeclaredMethod("startReplicator", String.class); startRepl.setAccessible(true); - Field replClientField = BrokerService.class.getDeclaredField("replicationClients"); - replClientField.setAccessible(true); - ConcurrentOpenHashMap replicationClients = - (ConcurrentOpenHashMap) replClientField - .get(pulsar1.getBrokerService()); + final var replicationClients = pulsar1.getBrokerService().getReplicationClients(); replicationClients.put("r3", pulsarClient); admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r2", "r3")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java index ddc5eeab1d20e..f1940a2899978 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java @@ -27,7 +27,6 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; -import java.lang.reflect.Field; import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -45,7 +44,6 @@ import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; import org.testng.Assert; @@ -591,12 +589,7 @@ public void testFinishTakeSnapshotWhenTopicLoading() throws Exception { return false; }); - Field field2 = BrokerService.class.getDeclaredField("topics"); - field2.setAccessible(true); - ConcurrentOpenHashMap>> topics = - (ConcurrentOpenHashMap>>) - field2.get(pulsar.getBrokerService()); - + final var topics = pulsar.getBrokerService().getTopics(); try { pulsar.getBrokerService().getTopic(topic, false).join().get(); Assert.fail(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index 4df2d36a95303..a92f5a4acc208 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -424,16 +424,16 @@ public void testPerTopicStats() throws Exception { // There should be 2 metrics with different tags for each topic List cm = (List) metrics.get("pulsar_storage_write_latency_le_1"); assertEquals(cm.size(), 2); - assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); cm = (List) metrics.get("pulsar_producers_count"); assertEquals(cm.size(), 2); - assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); - assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); + assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); cm = (List) metrics.get("pulsar_topic_load_times_count"); @@ -446,33 +446,33 @@ public void testPerTopicStats() throws Exception { cm = (List) metrics.get("pulsar_in_bytes_total"); assertEquals(cm.size(), 2); - assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); cm = (List) metrics.get("pulsar_in_messages_total"); assertEquals(cm.size(), 2); - assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); cm = (List) metrics.get("pulsar_out_bytes_total"); assertEquals(cm.size(), 2); - assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(0).tags.get("subscription"), "test"); - assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(1).tags.get("subscription"), "test"); cm = (List) metrics.get("pulsar_out_messages_total"); assertEquals(cm.size(), 2); - assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(0).tags.get("subscription"), "test"); - assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); assertEquals(cm.get(1).tags.get("subscription"), "test"); @@ -1086,26 +1086,26 @@ public void testPerProducerStats() throws Exception { List cm = (List) metrics.get("pulsar_producer_msg_rate_in"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); - assertEquals(cm.get(0).tags.get("producer_name"), "producer2"); - assertEquals(cm.get(0).tags.get("producer_id"), "1"); + assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(0).tags.get("producer_name"), "producer1"); + assertEquals(cm.get(0).tags.get("producer_id"), "0"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); - assertEquals(cm.get(1).tags.get("producer_name"), "producer1"); - assertEquals(cm.get(1).tags.get("producer_id"), "0"); + assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(1).tags.get("producer_name"), "producer2"); + assertEquals(cm.get(1).tags.get("producer_id"), "1"); cm = (List) metrics.get("pulsar_producer_msg_throughput_in"); assertEquals(cm.size(), 2); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); - assertEquals(cm.get(0).tags.get("producer_name"), "producer2"); - assertEquals(cm.get(0).tags.get("producer_id"), "1"); + assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(0).tags.get("producer_name"), "producer1"); + assertEquals(cm.get(0).tags.get("producer_id"), "0"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); - assertEquals(cm.get(1).tags.get("producer_name"), "producer1"); - assertEquals(cm.get(1).tags.get("producer_id"), "0"); + assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(1).tags.get("producer_name"), "producer2"); + assertEquals(cm.get(1).tags.get("producer_id"), "1"); p1.close(); p2.close(); @@ -1155,42 +1155,42 @@ public void testPerConsumerStats() throws Exception { List cm = (List) metrics.get("pulsar_out_bytes_total"); assertEquals(cm.size(), 4); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("subscription"), "test"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("subscription"), "test"); - assertEquals(cm.get(1).tags.get("consumer_id"), "1"); + assertEquals(cm.get(1).tags.get("consumer_id"), "0"); assertEquals(cm.get(2).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(2).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(2).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(2).tags.get("subscription"), "test"); assertEquals(cm.get(3).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(3).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(3).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(3).tags.get("subscription"), "test"); - assertEquals(cm.get(3).tags.get("consumer_id"), "0"); + assertEquals(cm.get(3).tags.get("consumer_id"), "1"); cm = (List) metrics.get("pulsar_out_messages_total"); assertEquals(cm.size(), 4); assertEquals(cm.get(0).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(0).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(0).tags.get("subscription"), "test"); assertEquals(cm.get(1).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); + assertEquals(cm.get(1).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); assertEquals(cm.get(1).tags.get("subscription"), "test"); - assertEquals(cm.get(1).tags.get("consumer_id"), "1"); + assertEquals(cm.get(1).tags.get("consumer_id"), "0"); assertEquals(cm.get(2).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(2).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(2).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(2).tags.get("subscription"), "test"); assertEquals(cm.get(3).tags.get("namespace"), "my-property/use/my-ns"); - assertEquals(cm.get(3).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic1"); + assertEquals(cm.get(3).tags.get("topic"), "persistent://my-property/use/my-ns/my-topic2"); assertEquals(cm.get(3).tags.get("subscription"), "test"); - assertEquals(cm.get(3).tags.get("consumer_id"), "0"); + assertEquals(cm.get(3).tags.get("consumer_id"), "1"); p1.close(); p2.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java index cf923df0411dd..45e3fb253bf11 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java @@ -21,6 +21,8 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.when; import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.impl.ManagedLedgerMBeanImpl; import org.apache.bookkeeper.mledger.util.StatsBuckets; @@ -47,17 +49,14 @@ public class NamespaceStatsAggregatorTest { protected PulsarService pulsar; private BrokerService broker; - private ConcurrentOpenHashMap>> - multiLayerTopicsMap; + private Map>> multiLayerTopicsMap; @BeforeMethod(alwaysRun = true) public void setup() throws Exception { - multiLayerTopicsMap = ConcurrentOpenHashMap.>>newBuilder() - .build(); + multiLayerTopicsMap = new ConcurrentHashMap<>(); pulsar = Mockito.mock(PulsarService.class); broker = Mockito.mock(BrokerService.class); - doReturn(multiLayerTopicsMap).when(broker).getMultiLayerTopicMap(); + doReturn(multiLayerTopicsMap).when(broker).getMultiLayerTopicsMap(); Mockito.when(pulsar.getLocalMetadataStore()).thenReturn(Mockito.mock(ZKMetadataStore.class)); ServiceConfiguration mockConfig = Mockito.mock(ServiceConfiguration.class); doReturn(mockConfig).when(pulsar).getConfiguration(); @@ -70,8 +69,8 @@ public void testGenerateSubscriptionsStats() { final String namespace = "tenant/cluster/ns"; // prepare multi-layer topic map - ConcurrentOpenHashMap bundlesMap = ConcurrentOpenHashMap.newBuilder().build(); - ConcurrentOpenHashMap topicsMap = ConcurrentOpenHashMap.newBuilder().build(); + final var bundlesMap = new ConcurrentHashMap>(); + final var topicsMap = new ConcurrentHashMap(); ConcurrentOpenHashMap subscriptionsMaps = ConcurrentOpenHashMap.newBuilder().build(); bundlesMap.put("my-bundle", topicsMap); multiLayerTopicsMap.put(namespace, bundlesMap); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 3924281c094b1..f21e11b980209 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -93,7 +93,6 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -201,20 +200,14 @@ private void recoverTest(String testTopic) throws Exception { Awaitility.await().until(() -> { for (int i = 0; i < getPulsarServiceList().size(); i++) { - Field field = BrokerService.class.getDeclaredField("topics"); - field.setAccessible(true); - ConcurrentOpenHashMap>> topics = - (ConcurrentOpenHashMap>>) field - .get(getPulsarServiceList().get(i).getBrokerService()); + final var topics = getPulsarServiceList().get(i).getBrokerService().getTopics(); CompletableFuture> completableFuture = topics.get("persistent://" + testTopic); if (completableFuture != null) { Optional topic = completableFuture.get(); if (topic.isPresent()) { PersistentTopic persistentTopic = (PersistentTopic) topic.get(); - field = PersistentTopic.class.getDeclaredField("transactionBuffer"); - field.setAccessible(true); TopicTransactionBuffer topicTransactionBuffer = - (TopicTransactionBuffer) field.get(persistentTopic); + (TopicTransactionBuffer) persistentTopic.getTransactionBuffer(); if (topicTransactionBuffer.checkIfReady()) { return true; } else { @@ -455,17 +448,13 @@ private void testTopicTransactionBufferDeleteAbort(Boolean enableSnapshotSegment assertTrue(((MessageIdImpl) messageId2).getLedgerId() != ((MessageIdImpl) messageId1).getLedgerId()); boolean exist = false; for (int i = 0; i < getPulsarServiceList().size(); i++) { - Field field = BrokerService.class.getDeclaredField("topics"); - field.setAccessible(true); - ConcurrentOpenHashMap>> topics = - (ConcurrentOpenHashMap>>) field - .get(getPulsarServiceList().get(i).getBrokerService()); + final var topics = getPulsarServiceList().get(i).getBrokerService().getTopics(); CompletableFuture> completableFuture = topics.get("persistent://" + ABORT_DELETE); if (completableFuture != null) { Optional topic = completableFuture.get(); if (topic.isPresent()) { PersistentTopic persistentTopic = (PersistentTopic) topic.get(); - field = ManagedLedgerImpl.class.getDeclaredField("ledgers"); + var field = ManagedLedgerImpl.class.getDeclaredField("ledgers"); field.setAccessible(true); NavigableMap ledgers = (NavigableMap) field.get(persistentTopic.getManagedLedger()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java index 39f36f4d38c65..14b1d563c11ec 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java @@ -390,7 +390,7 @@ private int getPendingAckCount(String topic, String subscriptionName) throws Exc int pendingAckCount = 0; for (PulsarService pulsarService : getPulsarServiceList()) { - for (String key : pulsarService.getBrokerService().getTopics().keys()) { + for (String key : pulsarService.getBrokerService().getTopics().keySet()) { if (key.contains(topic)) { Field field = clazz.getDeclaredField("pendingAckHandle"); field.setAccessible(true); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index cc09fa212198d..5480b1a21d5a0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -145,7 +145,6 @@ import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.schema.SchemaInfo; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.compaction.CompactionServiceFactory; import org.apache.pulsar.compaction.PulsarCompactionServiceFactory; import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; @@ -385,8 +384,7 @@ public void brokerNotInitTxnManagedLedgerTopic() throws Exception { return true; }); - ConcurrentOpenHashMap>> topics = - getPulsarServiceList().get(0).getBrokerService().getTopics(); + final var topics = getPulsarServiceList().get(0).getBrokerService().getTopics(); Assert.assertNull(topics.get(TopicName.get(TopicDomain.persistent.value(), NamespaceName.SYSTEM_NAMESPACE, TRANSACTION_LOG_PREFIX).toString() + 0)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java index 2e6d9c61bde79..818b854ffe941 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java @@ -33,7 +33,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; import org.apache.commons.collections4.map.LinkedMap; -import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -54,7 +53,6 @@ import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStore; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStoreState; @@ -215,18 +213,14 @@ public void testPendingAckLowWaterMark() throws Exception { LinkedMap> individualAckOfTransaction = null; for (int i = 0; i < getPulsarServiceList().size(); i++) { - Field field = BrokerService.class.getDeclaredField("topics"); - field.setAccessible(true); - ConcurrentOpenHashMap>> topics = - (ConcurrentOpenHashMap>>) field - .get(getPulsarServiceList().get(i).getBrokerService()); + final var topics = getPulsarServiceList().get(i).getBrokerService().getTopics(); CompletableFuture> completableFuture = topics.get(TOPIC); if (completableFuture != null) { Optional topic = completableFuture.get(); if (topic.isPresent()) { PersistentSubscription persistentSubscription = (PersistentSubscription) topic.get() .getSubscription(subName); - field = PersistentSubscription.class.getDeclaredField("pendingAckHandle"); + var field = PersistentSubscription.class.getDeclaredField("pendingAckHandle"); field.setAccessible(true); PendingAckHandleImpl pendingAckHandle = (PendingAckHandleImpl) field.get(persistentSubscription); field = PendingAckHandleImpl.class.getDeclaredField("individualAckOfTransaction"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java index fd4e984b6c1fc..58cf59aa6b3b9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckInMemoryDeleteTest.java @@ -21,7 +21,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; -import java.lang.reflect.Field; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -35,7 +34,6 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.commons.lang3.tuple.MutablePair; -import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.transaction.TransactionTestBase; @@ -48,7 +46,6 @@ import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.common.util.collections.BitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -120,18 +117,14 @@ public void txnAckTestNoBatchAndSharedSubMemoryDeleteTest() throws Exception { int count = 0; for (int i = 0; i < getPulsarServiceList().size(); i++) { - Field field = BrokerService.class.getDeclaredField("topics"); - field.setAccessible(true); - ConcurrentOpenHashMap>> topics = - (ConcurrentOpenHashMap>>) field - .get(getPulsarServiceList().get(i).getBrokerService()); + final var topics = getPulsarServiceList().get(i).getBrokerService().getTopics(); CompletableFuture> completableFuture = topics.get("persistent://" + normalTopic); if (completableFuture != null) { Optional topic = completableFuture.get(); if (topic.isPresent()) { PersistentSubscription persistentSubscription = (PersistentSubscription) topic.get() .getSubscription(subscriptionName); - field = PersistentSubscription.class.getDeclaredField("pendingAckHandle"); + var field = PersistentSubscription.class.getDeclaredField("pendingAckHandle"); field.setAccessible(true); PendingAckHandleImpl pendingAckHandle = (PendingAckHandleImpl) field.get(persistentSubscription); field = PendingAckHandleImpl.class.getDeclaredField("individualAckOfTransaction"); @@ -214,18 +207,14 @@ public void txnAckTestBatchAndSharedSubMemoryDeleteTest() throws Exception { commitTxn.commit().get(); int count = 0; for (int i = 0; i < getPulsarServiceList().size(); i++) { - Field field = BrokerService.class.getDeclaredField("topics"); - field.setAccessible(true); - ConcurrentOpenHashMap>> topics = - (ConcurrentOpenHashMap>>) field - .get(getPulsarServiceList().get(i).getBrokerService()); + final var topics = getPulsarServiceList().get(i).getBrokerService().getTopics(); CompletableFuture> completableFuture = topics.get("persistent://" + normalTopic); if (completableFuture != null) { Optional topic = completableFuture.get(); if (topic.isPresent()) { PersistentSubscription testPersistentSubscription = (PersistentSubscription) topic.get().getSubscription(subscriptionName); - field = PersistentSubscription.class.getDeclaredField("pendingAckHandle"); + var field = PersistentSubscription.class.getDeclaredField("pendingAckHandle"); field.setAccessible(true); pendingAckHandle = (PendingAckHandleImpl) field.get(testPersistentSubscription); field = PendingAckHandleImpl.class.getDeclaredField("individualAckOfTransaction"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java index 424081b904c81..e04dde65fa872 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java @@ -996,7 +996,7 @@ public void testMultiReaderMessageAvailableAfterRestart() throws Exception { } // cause broker to drop topic. Will be loaded next time we access it - pulsar.getBrokerService().getTopics().keys().forEach(topicName -> { + pulsar.getBrokerService().getTopics().keySet().forEach(topicName -> { try { pulsar.getBrokerService().getTopicReference(topicName).get().close(false).get(); } catch (Exception e) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java index 06c6069ebae71..1e8754a2d675c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java @@ -48,7 +48,6 @@ import java.util.HashSet; import java.util.List; import java.util.NavigableMap; -import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentSkipListMap; @@ -101,7 +100,6 @@ import org.apache.pulsar.common.protocol.PulsarHandler; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentLongHashMap; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; @@ -697,8 +695,7 @@ public void testCleanProducer() throws Exception { @Test(expectedExceptions = PulsarClientException.TimeoutException.class) public void testOperationTimeout() throws PulsarClientException { final String topicName = "persistent://my-property/my-ns/my-topic1"; - ConcurrentOpenHashMap>> topics = pulsar.getBrokerService() - .getTopics(); + final var topics = pulsar.getBrokerService().getTopics(); // non-complete topic future so, create topic should timeout topics.put(topicName, new CompletableFuture<>()); try (PulsarClient pulsarClient = PulsarClient.builder().serviceUrl(lookupUrl.toString()) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java index 812f8fd571cac..f490fa70539ea 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java @@ -73,7 +73,6 @@ import org.apache.pulsar.common.api.proto.CommandAck; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.transaction.coordinator.TransactionMetadataStore; import org.apache.pulsar.transaction.coordinator.TransactionSubscription; @@ -415,11 +414,7 @@ public void produceAbortTest() throws Exception { boolean exist = false; for (int i = 0; i < getPulsarServiceList().size(); i++) { - Field field = BrokerService.class.getDeclaredField("topics"); - field.setAccessible(true); - ConcurrentOpenHashMap>> topics = - (ConcurrentOpenHashMap>>) field - .get(getPulsarServiceList().get(i).getBrokerService()); + final var topics = getPulsarServiceList().get(i).getBrokerService().getTopics(); CompletableFuture> topicFuture = topics.get(topic); if (topicFuture != null) { @@ -722,9 +717,7 @@ public void txnMessageAckTest() throws Exception { Field field = BrokerService.class.getDeclaredField("topics"); field.setAccessible(true); - ConcurrentOpenHashMap>> topics = - (ConcurrentOpenHashMap>>) field - .get(getPulsarServiceList().get(i).getBrokerService()); + final var topics = getPulsarServiceList().get(i).getBrokerService().getTopics(); CompletableFuture> topicFuture = topics.get(topic); if (topicFuture != null) { @@ -1193,9 +1186,7 @@ public void txnTransactionRedeliverNullDispatcher(CommandAck.AckType ackType) th Field field = BrokerService.class.getDeclaredField("topics"); field.setAccessible(true); - ConcurrentOpenHashMap>> topics = - (ConcurrentOpenHashMap>>) field - .get(getPulsarServiceList().get(i).getBrokerService()); + final var topics = getPulsarServiceList().get(i).getBrokerService().getTopics(); CompletableFuture> topicFuture = topics.get(topic); if (topicFuture != null) { From 1ce7855c9424b23ac357cfd1cfe89bdb6e22ea57 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Sat, 21 Sep 2024 19:48:21 +0800 Subject: [PATCH 516/580] [improve][broker] Replace ConcurrentOpenHashMap with ConcurrentHashMap in Topic classes (#23322) --- .../admin/impl/PersistentTopicsBase.java | 6 +- .../pulsar/broker/service/AbstractTopic.java | 3 +- .../pulsar/broker/service/BrokerService.java | 4 +- .../apache/pulsar/broker/service/Topic.java | 7 +- .../nonpersistent/NonPersistentTopic.java | 30 ++--- .../service/persistent/PersistentTopic.java | 114 ++++++------------ .../ReplicatedSubscriptionsController.java | 3 +- ...eplicatedSubscriptionsSnapshotBuilder.java | 5 +- .../pulsar/broker/admin/AdminApiTest.java | 4 +- .../broker/admin/PersistentTopicsTest.java | 10 +- .../broker/admin/TopicPoliciesTest.java | 5 +- .../broker/admin/v1/V1_AdminApiTest.java | 4 +- .../broker/service/AbstractTopicTest.java | 8 +- .../broker/service/ClusterMigrationTest.java | 9 +- .../PersistentTopicConcurrentTest.java | 9 +- .../broker/service/PersistentTopicTest.java | 30 ++--- .../service/ReplicatedSubscriptionTest.java | 5 +- .../service/ReplicatorRateLimiterTest.java | 84 +++++++------ .../pulsar/broker/service/ReplicatorTest.java | 23 ++-- .../service/ReplicatorTopicPoliciesTest.java | 5 +- ...ransactionalReplicateSubscriptionTest.java | 3 +- .../nonpersistent/NonPersistentTopicTest.java | 3 +- ...catedSubscriptionsSnapshotBuilderTest.java | 15 +-- .../NamespaceStatsAggregatorTest.java | 10 +- .../api/DispatcherBlockConsumerTest.java | 19 +-- 25 files changed, 162 insertions(+), 256 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index bdbd70afbaeac..8860c9bb06d4d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -1265,14 +1265,14 @@ private void resumeAsyncResponse(AsyncResponse asyncResponse, Set subscr return; } } else { - asyncResponse.resume(new ArrayList<>(subscriptions)); + asyncResponse.resume(subscriptions); } }); } private void internalGetSubscriptionsForNonPartitionedTopic(AsyncResponse asyncResponse) { getTopicReferenceAsync(topicName) - .thenAccept(topic -> asyncResponse.resume(new ArrayList<>(topic.getSubscriptions().keys()))) + .thenAccept(topic -> asyncResponse.resume(topic.getSubscriptions().keySet())) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. if (isNot307And404Exception(ex)) { @@ -2024,7 +2024,7 @@ private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(Asy new ArrayList<>((int) topic.getReplicators().size()); List subNames = new ArrayList<>((int) topic.getSubscriptions().size()); - subNames.addAll(topic.getSubscriptions().keys().stream().filter( + subNames.addAll(topic.getSubscriptions().keySet().stream().filter( subName -> !subName.equals(Compactor.COMPACTION_SUBSCRIPTION)).toList()); for (int i = 0; i < subNames.size(); i++) { try { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index 3fdfeeee6e152..dce50a54db1f6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -26,6 +26,7 @@ import java.time.Clock; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.List; @@ -576,7 +577,7 @@ && getNumberOfSameAddressConsumers(consumer.getClientAddress()) >= maxSameAddres public abstract int getNumberOfSameAddressConsumers(String clientAddress); protected int getNumberOfSameAddressConsumers(final String clientAddress, - final List subscriptions) { + final Collection subscriptions) { int count = 0; if (clientAddress != null) { for (Subscription subscription : subscriptions) { 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 6b0be07c8f7a8..09f04d878c4e5 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 @@ -1176,7 +1176,7 @@ private CompletableFuture deleteTopicInternal(String topic, boolean forceD // v2 topics have a global name so check if the topic is replicated. if (t.isReplicated()) { // Delete is disallowed on global topic - final List clusters = t.getReplicators().keys(); + final var clusters = t.getReplicators().keySet(); log.error("Delete forbidden topic {} is replicated on clusters {}", topic, clusters); return FutureUtil.failedFuture( new IllegalStateException("Delete forbidden topic is replicated on clusters " + clusters)); @@ -1184,7 +1184,7 @@ private CompletableFuture deleteTopicInternal(String topic, boolean forceD // shadow topic should be deleted first. if (t.isShadowReplicated()) { - final List shadowTopics = t.getShadowReplicators().keys(); + final var shadowTopics = t.getShadowReplicators().keySet(); log.error("Delete forbidden. Topic {} is replicated to shadow topics: {}", topic, shadowTopics); return FutureUtil.failedFuture(new IllegalStateException( "Delete forbidden. Topic " + topic + " is replicated to shadow topics.")); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java index 3ec09e9bfcd28..ec7889af6bbbe 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java @@ -44,7 +44,6 @@ import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.protocol.schema.SchemaData; import org.apache.pulsar.common.protocol.schema.SchemaVersion; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; import org.apache.pulsar.utils.StatsOutputStream; @@ -183,7 +182,7 @@ CompletableFuture createSubscription(String subscriptionName, Init CompletableFuture unsubscribe(String subName); - ConcurrentOpenHashMap getSubscriptions(); + Map getSubscriptions(); CompletableFuture delete(); @@ -265,9 +264,9 @@ void updateRates(NamespaceStats nsStats, NamespaceBundleStats currentBundleStats Subscription getSubscription(String subscription); - ConcurrentOpenHashMap getReplicators(); + Map getReplicators(); - ConcurrentOpenHashMap getShadowReplicators(); + Map getShadowReplicators(); TopicStatsImpl getStats(boolean getPreciseBacklog, boolean subscriptionBacklogSize, boolean getEarliestTimeInBacklog); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 2abd505d527cc..34c2678f847a5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -33,6 +33,7 @@ import java.util.Set; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLongFieldUpdater; @@ -96,7 +97,6 @@ import org.apache.pulsar.common.protocol.schema.SchemaData; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; import org.apache.pulsar.utils.StatsOutputStream; import org.slf4j.Logger; @@ -105,9 +105,9 @@ public class NonPersistentTopic extends AbstractTopic implements Topic, TopicPolicyListener { // Subscriptions to this topic - private final ConcurrentOpenHashMap subscriptions; + private final Map subscriptions = new ConcurrentHashMap<>(); - private final ConcurrentOpenHashMap replicators; + private final Map replicators = new ConcurrentHashMap<>(); // Ever increasing counter of entries added private static final AtomicLongFieldUpdater ENTRIES_ADDED_COUNTER_UPDATER = @@ -152,17 +152,6 @@ public void reset() { public NonPersistentTopic(String topic, BrokerService brokerService) { super(topic, brokerService); - - this.subscriptions = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); - this.replicators = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); this.isFenced = false; registerTopicPolicyListener(); } @@ -446,8 +435,8 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, boolean c if (failIfHasSubscriptions) { if (!subscriptions.isEmpty()) { isFenced = false; - deleteFuture.completeExceptionally( - new TopicBusyException("Topic has subscriptions:" + subscriptions.keys())); + deleteFuture.completeExceptionally(new TopicBusyException("Topic has subscriptions:" + + subscriptions.keySet().stream().toList())); return; } } else { @@ -714,18 +703,18 @@ public int getNumberOfSameAddressConsumers(final String clientAddress) { } @Override - public ConcurrentOpenHashMap getSubscriptions() { + public Map getSubscriptions() { return subscriptions; } @Override - public ConcurrentOpenHashMap getReplicators() { + public Map getReplicators() { return replicators; } @Override - public ConcurrentOpenHashMap getShadowReplicators() { - return ConcurrentOpenHashMap.emptyMap(); + public Map getShadowReplicators() { + return Map.of(); } @Override @@ -1043,7 +1032,6 @@ private CompletableFuture checkAndUnsubscribeSubscriptions() { private CompletableFuture disconnectReplicators() { List> futures = new ArrayList<>(); - ConcurrentOpenHashMap replicators = getReplicators(); replicators.forEach((r, replicator) -> { futures.add(replicator.terminate()); }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index d664d6812adaa..f8581cfc79985 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -44,6 +44,7 @@ import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.RejectedExecutionException; @@ -187,7 +188,6 @@ import org.apache.pulsar.common.topics.TopicCompactionStrategy; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.compaction.CompactedTopicContext; import org.apache.pulsar.compaction.CompactedTopicImpl; import org.apache.pulsar.compaction.Compactor; @@ -207,10 +207,10 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal protected final ManagedLedger ledger; // Subscriptions to this topic - private final ConcurrentOpenHashMap subscriptions; + private final Map subscriptions = new ConcurrentHashMap<>(); - private final ConcurrentOpenHashMap replicators; - private final ConcurrentOpenHashMap shadowReplicators; + private final Map replicators = new ConcurrentHashMap<>(); + private final Map shadowReplicators = new ConcurrentHashMap<>(); @Getter private volatile List shadowTopics; private final TopicName shadowSourceTopic; @@ -392,18 +392,6 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS ? brokerService.getTopicOrderedExecutor().chooseThread(topic) : null; this.ledger = ledger; - this.subscriptions = ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); - this.replicators = ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); - this.shadowReplicators = ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); this.backloggedCursorThresholdEntries = brokerService.pulsar().getConfiguration().getManagedLedgerCursorBackloggedThreshold(); this.messageDeduplication = new MessageDeduplication(brokerService.pulsar(), this, ledger); @@ -429,6 +417,28 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS } } + @VisibleForTesting + PersistentTopic(String topic, BrokerService brokerService, ManagedLedger ledger, + MessageDeduplication messageDeduplication) { + super(topic, brokerService); + // null check for backwards compatibility with tests which mock the broker service + this.orderedExecutor = brokerService.getTopicOrderedExecutor() != null + ? brokerService.getTopicOrderedExecutor().chooseThread(topic) + : null; + this.ledger = ledger; + this.messageDeduplication = messageDeduplication; + this.backloggedCursorThresholdEntries = + brokerService.pulsar().getConfiguration().getManagedLedgerCursorBackloggedThreshold(); + + if (brokerService.pulsar().getConfiguration().isTransactionCoordinatorEnabled()) { + this.transactionBuffer = brokerService.getPulsar() + .getTransactionBufferProvider().newTransactionBuffer(this); + } else { + this.transactionBuffer = new TransactionBufferDisable(this); + } + shadowSourceTopic = null; + } + @Override public CompletableFuture initialize() { List> futures = new ArrayList<>(); @@ -476,41 +486,6 @@ public CompletableFuture initialize() { })); } - // for testing purposes - @VisibleForTesting - PersistentTopic(String topic, BrokerService brokerService, ManagedLedger ledger, - MessageDeduplication messageDeduplication) { - super(topic, brokerService); - // null check for backwards compatibility with tests which mock the broker service - this.orderedExecutor = brokerService.getTopicOrderedExecutor() != null - ? brokerService.getTopicOrderedExecutor().chooseThread(topic) - : null; - this.ledger = ledger; - this.messageDeduplication = messageDeduplication; - this.subscriptions = ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); - this.replicators = ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); - this.shadowReplicators = ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); - this.backloggedCursorThresholdEntries = - brokerService.pulsar().getConfiguration().getManagedLedgerCursorBackloggedThreshold(); - - if (brokerService.pulsar().getConfiguration().isTransactionCoordinatorEnabled()) { - this.transactionBuffer = brokerService.getPulsar() - .getTransactionBufferProvider().newTransactionBuffer(this); - } else { - this.transactionBuffer = new TransactionBufferDisable(this); - } - shadowSourceTopic = null; - } - private void initializeDispatchRateLimiterIfNeeded() { synchronized (dispatchRateLimiterLock) { // dispatch rate limiter for topic @@ -1455,8 +1430,8 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, // In this case, we shouldn't care if the usageCount is 0 or not, just proceed if (!closeIfClientsConnected) { if (failIfHasSubscriptions && !subscriptions.isEmpty()) { - return FutureUtil.failedFuture( - new TopicBusyException("Topic has subscriptions: " + subscriptions.keys())); + return FutureUtil.failedFuture(new TopicBusyException("Topic has subscriptions: " + + subscriptions.keySet().stream().toList())); } else if (failIfHasBacklogs) { if (hasBacklogs(false)) { List backlogSubs = @@ -2129,10 +2104,6 @@ protected CompletableFuture addReplicationCluster(String remoteCluster, Ma } return null; }); - // clean up replicator if startup is failed - if (replicator == null) { - replicators.removeNullValue(remoteCluster); - } } finally { lock.readLock().unlock(); } @@ -2210,11 +2181,6 @@ protected CompletableFuture addShadowReplicationCluster(String shadowTopic } return null; }); - - // clean up replicator if startup is failed - if (replicator == null) { - shadowReplicators.removeNullValue(shadowTopic); - } }); } @@ -2274,7 +2240,7 @@ protected String getSchemaId() { } @Override - public ConcurrentOpenHashMap getSubscriptions() { + public Map getSubscriptions() { return subscriptions; } @@ -2284,12 +2250,12 @@ public PersistentSubscription getSubscription(String subscriptionName) { } @Override - public ConcurrentOpenHashMap getReplicators() { + public Map getReplicators() { return replicators; } @Override - public ConcurrentOpenHashMap getShadowReplicators() { + public Map getShadowReplicators() { return shadowReplicators; } @@ -3091,7 +3057,6 @@ private CompletableFuture checkAndDisconnectProducers() { private CompletableFuture checkAndDisconnectReplicators() { List> futures = new ArrayList<>(); - ConcurrentOpenHashMap replicators = getReplicators(); replicators.forEach((r, replicator) -> { if (replicator.getNumberOfEntriesInBacklog() <= 0) { futures.add(replicator.terminate()); @@ -3106,12 +3071,9 @@ public boolean shouldProducerMigrate() { @Override public boolean isReplicationBacklogExist() { - ConcurrentOpenHashMap replicators = getReplicators(); - if (replicators != null) { - for (Replicator replicator : replicators.values()) { - if (replicator.getNumberOfEntriesInBacklog() > 0) { - return true; - } + for (Replicator replicator : replicators.values()) { + if (replicator.getNumberOfEntriesInBacklog() > 0) { + return true; } } return false; @@ -3759,9 +3721,9 @@ public boolean isOldestMessageExpired(ManagedCursor cursor, int messageTTLInSeco public CompletableFuture clearBacklog() { log.info("[{}] Clearing backlog on all cursors in the topic.", topic); List> futures = new ArrayList<>(); - List cursors = getSubscriptions().keys(); - cursors.addAll(getReplicators().keys()); - cursors.addAll(getShadowReplicators().keys()); + List cursors = new ArrayList<>(getSubscriptions().keySet()); + cursors.addAll(getReplicators().keySet()); + cursors.addAll(getShadowReplicators().keySet()); for (String cursor : cursors) { futures.add(clearBacklog(cursor)); } @@ -4161,7 +4123,7 @@ private void unfenceReplicatorsToResume() { checkShadowReplication(); } - private void removeTerminatedReplicators(ConcurrentOpenHashMap replicators) { + private void removeTerminatedReplicators(Map replicators) { Map terminatedReplicators = new HashMap<>(); replicators.forEach((cluster, replicator) -> { if (replicator.isTerminated()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java index b873bc93cd1e4..f56cf9de66b75 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java @@ -243,7 +243,8 @@ private void startNewSnapshot() { pendingSnapshotsMetric.inc(); stats.recordSnapshotStarted(); ReplicatedSubscriptionsSnapshotBuilder builder = new ReplicatedSubscriptionsSnapshotBuilder(this, - topic.getReplicators().keys(), topic.getBrokerService().pulsar().getConfiguration(), Clock.systemUTC()); + topic.getReplicators().keySet(), topic.getBrokerService().pulsar().getConfiguration(), + Clock.systemUTC()); pendingSnapshots.put(builder.getSnapshotId(), builder); builder.start(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java index 0dacade3eed1c..e08b549f8aec9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java @@ -20,7 +20,6 @@ import io.prometheus.client.Summary; import java.time.Clock; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeMap; @@ -43,7 +42,7 @@ public class ReplicatedSubscriptionsSnapshotBuilder { private final ReplicatedSubscriptionsController controller; private final Map responses = new TreeMap<>(); - private final List remoteClusters; + private final Set remoteClusters; private final Set missingClusters; private final boolean needTwoRounds; @@ -60,7 +59,7 @@ public class ReplicatedSubscriptionsSnapshotBuilder { "Time taken to create a consistent snapshot across clusters").register(); public ReplicatedSubscriptionsSnapshotBuilder(ReplicatedSubscriptionsController controller, - List remoteClusters, ServiceConfiguration conf, Clock clock) { + Set remoteClusters, ServiceConfiguration conf, Clock clock) { this.snapshotId = UUID.randomUUID().toString(); this.controller = controller; this.remoteClusters = remoteClusters; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index 5432b8a430d63..4a1dbface2c63 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -2315,8 +2315,8 @@ public void testUnsubscribeOnNamespace(Integer numBundles) throws Exception { admin.namespaces().unsubscribeNamespace("prop-xyz/ns1-bundles", "my-sub"); - assertEquals(admin.topics().getSubscriptions("persistent://prop-xyz/ns1-bundles/ds2"), - List.of("my-sub-1", "my-sub-2")); + assertEquals(admin.topics().getSubscriptions("persistent://prop-xyz/ns1-bundles/ds2").stream().sorted() + .toList(), List.of("my-sub-1", "my-sub-2")); assertEquals(admin.topics().getSubscriptions("persistent://prop-xyz/ns1-bundles/ds1"), List.of("my-sub-1")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index 55b4c6e1c6f59..18fd3dd1c8bb3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -251,7 +251,7 @@ public void testGetSubscriptions() { response = mock(AsyncResponse.class); persistentTopics.getSubscriptions(response, testTenant, testNamespace, testLocalTopicName + "-partition-0", true); - verify(response, timeout(5000).times(1)).resume(List.of("test")); + verify(response, timeout(5000).times(1)).resume(Set.of("test")); // 6) Delete the subscription response = mock(AsyncResponse.class); @@ -265,7 +265,7 @@ public void testGetSubscriptions() { response = mock(AsyncResponse.class); persistentTopics.getSubscriptions(response, testTenant, testNamespace, testLocalTopicName + "-partition-0", true); - verify(response, timeout(5000).times(1)).resume(new ArrayList<>()); + verify(response, timeout(5000).times(1)).resume(Set.of()); // 8) Create a sub of partitioned-topic response = mock(AsyncResponse.class); @@ -279,16 +279,16 @@ public void testGetSubscriptions() { response = mock(AsyncResponse.class); persistentTopics.getSubscriptions(response, testTenant, testNamespace, testLocalTopicName + "-partition-1", true); - verify(response, timeout(5000).times(1)).resume(List.of("test")); + verify(response, timeout(5000).times(1)).resume(Set.of("test")); // response = mock(AsyncResponse.class); persistentTopics.getSubscriptions(response, testTenant, testNamespace, testLocalTopicName + "-partition-0", true); - verify(response, timeout(5000).times(1)).resume(new ArrayList<>()); + verify(response, timeout(5000).times(1)).resume(Set.of()); // response = mock(AsyncResponse.class); persistentTopics.getSubscriptions(response, testTenant, testNamespace, testLocalTopicName, true); - verify(response, timeout(5000).times(1)).resume(List.of("test")); + verify(response, timeout(5000).times(1)).resume(Set.of("test")); // 9) Delete the partitioned topic response = mock(AsyncResponse.class); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java index 1351c41e4279e..dc9a7ec4429fc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java @@ -88,10 +88,8 @@ import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.policies.data.TopicStats; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -3199,8 +3197,7 @@ public void testUpdateRetentionWithPartialFailure() throws Exception { // Inject an error that makes dispatch rate update fail. PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(tpName, false).join().get(); - ConcurrentOpenHashMap subscriptions = - WhiteboxImpl.getInternalState(persistentTopic, "subscriptions"); + final var subscriptions = persistentTopic.getSubscriptions(); PersistentSubscription mockedSubscription = Mockito.mock(PersistentSubscription.class); Mockito.when(mockedSubscription.getDispatcher()).thenThrow(new RuntimeException("Mocked error: getDispatcher")); subscriptions.put("mockedSubscription", mockedSubscription); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java index f2faa98636ba2..d92c3126c5404 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java @@ -1380,8 +1380,8 @@ public void testUnsubscribeOnNamespace(Integer numBundles) throws Exception { admin.namespaces().unsubscribeNamespace("prop-xyz/use/ns1-bundles", "my-sub"); - assertEquals(admin.topics().getSubscriptions("persistent://prop-xyz/use/ns1-bundles/ds2"), - List.of("my-sub-1", "my-sub-2")); + assertEquals(admin.topics().getSubscriptions("persistent://prop-xyz/use/ns1-bundles/ds2").stream() + .sorted().toList(), List.of("my-sub-1", "my-sub-2")); assertEquals(admin.topics().getSubscriptions("persistent://prop-xyz/use/ns1-bundles/ds1"), List.of("my-sub-1")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractTopicTest.java index 39be56e3f41cf..337717ed97b1b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractTopicTest.java @@ -24,10 +24,10 @@ import static org.mockito.Mockito.when; import static org.mockito.Mockito.withSettings; import static org.testng.Assert.assertEquals; +import java.util.concurrent.ConcurrentHashMap; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.qos.AsyncTokenBucket; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -54,11 +54,7 @@ public void beforeMethod() { .useConstructor("topic", brokerService) .defaultAnswer(CALLS_REAL_METHODS)); - ConcurrentOpenHashMap subscriptions = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); + final var subscriptions = new ConcurrentHashMap(); subscriptions.put("subscription", subscription); when(topic.getSubscriptions()).thenAnswer(invocation -> subscriptions); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java index 8ec565f7d4566..e56a3495600f0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java @@ -49,7 +49,6 @@ import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterPolicies.ClusterUrl; import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.AfterMethod; @@ -344,7 +343,7 @@ public void testClusterMigration() throws Exception { assertFalse(topic2.getSubscriptions().isEmpty()); topic1.checkClusterMigration().get(); - ConcurrentOpenHashMap replicators = topic1.getReplicators(); + final var replicators = topic1.getReplicators(); replicators.forEach((r, replicator) -> { assertFalse(replicator.isConnected()); }); @@ -798,20 +797,20 @@ public void testNamespaceMigration(SubscriptionType subType, boolean isClusterMi blueTopicNs2_1.checkClusterMigration().get(); } - ConcurrentOpenHashMap replicators = blueTopicNs1_1.getReplicators(); + final var replicators = blueTopicNs1_1.getReplicators(); replicators.forEach((r, replicator) -> { assertFalse(replicator.isConnected()); }); assertTrue(blueTopicNs1_1.getSubscriptions().isEmpty()); if (isClusterMigrate) { - ConcurrentOpenHashMap replicatorsNm = blueTopicNs2_1.getReplicators(); + final var replicatorsNm = blueTopicNs2_1.getReplicators(); replicatorsNm.forEach((r, replicator) -> { assertFalse(replicator.isConnected()); }); assertTrue(blueTopicNs2_1.getSubscriptions().isEmpty()); } else { - ConcurrentOpenHashMap replicatorsNm = blueTopicNs2_1.getReplicators(); + final var replicatorsNm = blueTopicNs2_1.getReplicators(); replicatorsNm.forEach((r, replicator) -> { assertTrue(replicator.isConnected()); }); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java index cbbb8808f3d1a..85e0887465db2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java @@ -53,7 +53,6 @@ import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.BeforeMethod; @@ -154,7 +153,7 @@ public void testConcurrentTopicAndSubscriptionDelete() throws Exception { try { barrier.await(); // do subscription delete - ConcurrentOpenHashMap subscriptions = topic.getSubscriptions(); + final var subscriptions = topic.getSubscriptions(); PersistentSubscription ps = subscriptions.get(successSubName); // Thread.sleep(2,0); log.info("unsubscriber outcome is {}", ps.doUnsubscribe(ps.getConsumers().get(0)).get()); @@ -219,7 +218,7 @@ public void testConcurrentTopicGCAndSubscriptionDelete() throws Exception { try { barrier.await(); // do subscription delete - ConcurrentOpenHashMap subscriptions = topic.getSubscriptions(); + final var subscriptions = topic.getSubscriptions(); PersistentSubscription ps = subscriptions.get(successSubName); // Thread.sleep(2,0); log.info("unsubscriber outcome is {}", ps.doUnsubscribe(ps.getConsumers().get(0)).get()); @@ -278,7 +277,7 @@ public void testConcurrentTopicDeleteAndUnsubscribe() throws Exception { barrier.await(); // Thread.sleep(2,0); // assertTrue(topic.unsubscribe(successSubName).isDone()); - ConcurrentOpenHashMap subscriptions = topic.getSubscriptions(); + final var subscriptions = topic.getSubscriptions(); PersistentSubscription ps = subscriptions.get(successSubName); log.info("unsubscribe result : {}", topic.unsubscribe(successSubName).get()); log.info("closing consumer.."); @@ -339,7 +338,7 @@ public void testConcurrentTopicDeleteAndSubsUnsubscribe() throws Exception { log.info("&&&&&&&&& UNSUBSCRIBER TH"); // Thread.sleep(2,0); // assertTrue(topic.unsubscribe(successSubName).isDone()); - ConcurrentOpenHashMap subscriptions = topic.getSubscriptions(); + final var subscriptions = topic.getSubscriptions(); PersistentSubscription ps = subscriptions.get(successSubName); log.info("unsubscribe result : " + ps.doUnsubscribe(ps.getConsumers().get(0)).get()); } catch (Exception e) { 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 b975041d04ee4..81c12df4f3918 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 @@ -64,6 +64,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutionException; @@ -132,7 +133,6 @@ import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.compaction.CompactedTopic; import org.apache.pulsar.compaction.CompactedTopicContext; import org.apache.pulsar.compaction.Compactor; @@ -777,11 +777,7 @@ private void testMaxConsumersShared() throws Exception { addConsumerToSubscription.setAccessible(true); // for count consumers on topic - ConcurrentOpenHashMap subscriptions = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); + final var subscriptions = new ConcurrentHashMap(); subscriptions.put("sub-1", sub); subscriptions.put("sub-2", sub2); Field field = topic.getClass().getDeclaredField("subscriptions"); @@ -873,11 +869,7 @@ private void testMaxConsumersFailover() throws Exception { addConsumerToSubscription.setAccessible(true); // for count consumers on topic - ConcurrentOpenHashMap subscriptions = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); + final var subscriptions = new ConcurrentHashMap(); subscriptions.put("sub-1", sub); subscriptions.put("sub-2", sub2); Field field = topic.getClass().getDeclaredField("subscriptions"); @@ -992,11 +984,7 @@ public void testMaxSameAddressConsumers() throws Exception { addConsumerToSubscription.setAccessible(true); // for count consumers on topic - ConcurrentOpenHashMap subscriptions = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); + final var subscriptions = new ConcurrentHashMap(); subscriptions.put("sub1", sub1); subscriptions.put("sub2", sub2); Field field = topic.getClass().getDeclaredField("subscriptions"); @@ -1299,7 +1287,7 @@ public void testConcurrentTopicAndSubscriptionDelete() throws Exception { try { barrier.await(); // do subscription delete - ConcurrentOpenHashMap subscriptions = topic.getSubscriptions(); + final var subscriptions = topic.getSubscriptions(); PersistentSubscription ps = subscriptions.get(successSubName); // Thread.sleep(5,0); log.info("unsubscriber outcome is {}", ps.doUnsubscribe(ps.getConsumers().get(0)).get()); @@ -1681,7 +1669,7 @@ public void testAtomicReplicationRemoval() throws Exception { PersistentTopic topic = new PersistentTopic(globalTopicName, ledgerMock, brokerService); topic.initialize().join(); String remoteReplicatorName = topic.getReplicatorPrefix() + "." + remoteCluster; - ConcurrentOpenHashMap replicatorMap = topic.getReplicators(); + final var replicatorMap = topic.getReplicators(); ManagedCursor cursor = mock(ManagedCursorImpl.class); doReturn(remoteCluster).when(cursor).getName(); @@ -2018,11 +2006,7 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { public void testCheckInactiveSubscriptions() throws Exception { PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); - ConcurrentOpenHashMap subscriptions = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); + final var subscriptions = new ConcurrentHashMap(); // This subscription is connected by consumer. PersistentSubscription nonDeletableSubscription1 = spyWithClassAndConstructorArgsRecordingInvocations(PersistentSubscription.class, topic, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatedSubscriptionTest.java index 4273e8bbaeb5b..5b896a22baa33 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatedSubscriptionTest.java @@ -68,7 +68,6 @@ import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicStats; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -292,7 +291,7 @@ public void testReplicatedSubscribeAndSwitchToStandbyCluster() throws Exception sentMessages.add(msg); } Awaitility.await().untilAsserted(() -> { - ConcurrentOpenHashMap replicators = topic1.getReplicators(); + final var replicators = topic1.getReplicators(); assertTrue(replicators != null && replicators.size() == 1, "Replicator should started"); assertTrue(replicators.values().iterator().next().isConnected(), "Replicator should be connected"); assertTrue(topic1.getReplicatedSubscriptionController().get().getLastCompletedSnapshotId().isPresent(), @@ -1072,7 +1071,7 @@ private void testReplicatedSubscriptionWhenEnableReplication(Producer pr Awaitility.await().untilAsserted(() -> { List keys = pulsar1.getBrokerService() .getTopic(topic, false).get().get() - .getReplicators().keys(); + .getReplicators().keySet().stream().toList(); assertEquals(keys.size(), 1); assertTrue(pulsar1.getBrokerService() .getTopic(topic, false).get().get() diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java index 90df16360614d..bec6b558ea401 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java @@ -25,9 +25,11 @@ import static org.testng.AssertJUnit.assertFalse; import com.google.common.collect.Sets; import java.lang.reflect.Method; +import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; +import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -101,7 +103,7 @@ public void testReplicatorRateLimiterWithOnlyTopicLevel() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getOrCreateTopic(topicName).get(); // rate limiter disable by default - assertFalse(topic.getReplicators().values().get(0).getRateLimiter().isPresent()); + assertFalse(getRateLimiter(topic).isPresent()); //set topic-level policy, which should take effect DispatchRate topicRate = DispatchRate.builder() @@ -112,16 +114,16 @@ public void testReplicatorRateLimiterWithOnlyTopicLevel() throws Exception { admin1.topics().setReplicatorDispatchRate(topicName, topicRate); Awaitility.await().untilAsserted(() -> assertEquals(admin1.topics().getReplicatorDispatchRate(topicName), topicRate)); - assertTrue(topic.getReplicators().values().get(0).getRateLimiter().isPresent()); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), 10); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), 20L); + assertTrue(getRateLimiter(topic).isPresent()); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), 10); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), 20L); //remove topic-level policy admin1.topics().removeReplicatorDispatchRate(topicName); Awaitility.await().untilAsserted(() -> assertNull(admin1.topics().getReplicatorDispatchRate(topicName))); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), -1); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), + assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), -1); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), -1L); } @@ -145,7 +147,7 @@ public void testReplicatorRateLimiterWithOnlyNamespaceLevel() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getOrCreateTopic(topicName).get(); // rate limiter disable by default - assertFalse(topic.getReplicators().values().get(0).getRateLimiter().isPresent()); + assertFalse(getRateLimiter(topic).isPresent()); //set namespace-level policy, which should take effect DispatchRate topicRate = DispatchRate.builder() @@ -156,16 +158,16 @@ public void testReplicatorRateLimiterWithOnlyNamespaceLevel() throws Exception { admin1.namespaces().setReplicatorDispatchRate(namespace, topicRate); Awaitility.await().untilAsserted(() -> assertEquals(admin1.namespaces().getReplicatorDispatchRate(namespace), topicRate)); - assertTrue(topic.getReplicators().values().get(0).getRateLimiter().isPresent()); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), 10); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), 20L); + assertTrue(getRateLimiter(topic).isPresent()); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), 10); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), 20L); //remove topic-level policy admin1.namespaces().removeReplicatorDispatchRate(namespace); Awaitility.await().untilAsserted(() -> assertNull(admin1.namespaces().getReplicatorDispatchRate(namespace))); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), -1); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), + assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), -1); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), -1L); } @@ -189,7 +191,7 @@ public void testReplicatorRateLimiterWithOnlyBrokerLevel() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getOrCreateTopic(topicName).get(); // rate limiter disable by default - assertFalse(topic.getReplicators().values().get(0).getRateLimiter().isPresent()); + assertFalse(getRateLimiter(topic).isPresent()); //set broker-level policy, which should take effect admin1.brokers().updateDynamicConfiguration("dispatchThrottlingRatePerReplicatorInMsg", "10"); @@ -203,9 +205,9 @@ public void testReplicatorRateLimiterWithOnlyBrokerLevel() throws Exception { .getAllDynamicConfigurations().get("dispatchThrottlingRatePerReplicatorInByte"), "20"); }); - assertTrue(topic.getReplicators().values().get(0).getRateLimiter().isPresent()); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), 10); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), 20L); + assertTrue(getRateLimiter(topic).isPresent()); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), 10); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), 20L); } @Test @@ -228,9 +230,9 @@ public void testReplicatorRatePriority() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getOrCreateTopic(topicName).get(); //use broker-level by default - assertTrue(topic.getReplicators().values().get(0).getRateLimiter().isPresent()); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), 100); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), 200L); + assertTrue(getRateLimiter(topic).isPresent()); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), 100); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), 200L); //set namespace-level policy, which should take effect DispatchRate nsDispatchRate = DispatchRate.builder() @@ -241,8 +243,8 @@ public void testReplicatorRatePriority() throws Exception { admin1.namespaces().setReplicatorDispatchRate(namespace, nsDispatchRate); Awaitility.await() .untilAsserted(() -> assertEquals(admin1.namespaces().getReplicatorDispatchRate(namespace), nsDispatchRate)); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), 50); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), 60L); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), 50); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), 60L); //set topic-level policy, which should take effect DispatchRate topicRate = DispatchRate.builder() @@ -253,8 +255,8 @@ public void testReplicatorRatePriority() throws Exception { admin1.topics().setReplicatorDispatchRate(topicName, topicRate); Awaitility.await().untilAsserted(() -> assertEquals(admin1.topics().getReplicatorDispatchRate(topicName), topicRate)); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), 10); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), 20L); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), 10); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), 20L); //Set the namespace-level policy, which should not take effect DispatchRate nsDispatchRate2 = DispatchRate.builder() @@ -265,21 +267,21 @@ public void testReplicatorRatePriority() throws Exception { admin1.namespaces().setReplicatorDispatchRate(namespace, nsDispatchRate2); Awaitility.await() .untilAsserted(() -> assertEquals(admin1.namespaces().getReplicatorDispatchRate(namespace), nsDispatchRate2)); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), 20L); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), 20L); //remove topic-level policy, namespace-level should take effect admin1.topics().removeReplicatorDispatchRate(topicName); Awaitility.await().untilAsserted(() -> assertNull(admin1.topics().getReplicatorDispatchRate(topicName))); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), 500); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), + assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), 500); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), 600L); //remove namespace-level policy, broker-level should take effect admin1.namespaces().setReplicatorDispatchRate(namespace, null); Awaitility.await().untilAsserted(() -> - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), 100)); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), + assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), 100)); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), 200L); } @@ -315,7 +317,7 @@ public void testReplicatorRateLimiterDynamicallyChange() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getOrCreateTopic(topicName).get(); // 1. default replicator throttling not configured - Assert.assertFalse(topic.getReplicators().values().get(0).getRateLimiter().isPresent()); + Assert.assertFalse(getRateLimiter(topic).isPresent()); // 2. change namespace setting of replicator dispatchRateMsg, verify topic changed. int messageRate = 100; @@ -329,7 +331,7 @@ public void testReplicatorRateLimiterDynamicallyChange() throws Exception { boolean replicatorUpdated = false; int retry = 5; for (int i = 0; i < retry; i++) { - if (topic.getReplicators().values().get(0).getRateLimiter().isPresent()) { + if (getRateLimiter(topic).isPresent()) { replicatorUpdated = true; break; } else { @@ -339,7 +341,7 @@ public void testReplicatorRateLimiterDynamicallyChange() throws Exception { } } Assert.assertTrue(replicatorUpdated); - Assert.assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), messageRate); + Assert.assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), messageRate); // 3. change namespace setting of replicator dispatchRateByte, verify topic changed. messageRate = 500; @@ -351,7 +353,7 @@ public void testReplicatorRateLimiterDynamicallyChange() throws Exception { admin1.namespaces().setReplicatorDispatchRate(namespace, dispatchRateByte); replicatorUpdated = false; for (int i = 0; i < retry; i++) { - if (topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte() == messageRate) { + if (getRateLimiter(topic).get().getDispatchRateOnByte() == messageRate) { replicatorUpdated = true; break; } else { @@ -414,7 +416,7 @@ public void testReplicatorRateLimiterMessageNotReceivedAllMessages(DispatchRateT boolean replicatorUpdated = false; int retry = 5; for (int i = 0; i < retry; i++) { - if (topic.getReplicators().values().get(0).getRateLimiter().isPresent()) { + if (getRateLimiter(topic).isPresent()) { replicatorUpdated = true; break; } else { @@ -425,9 +427,9 @@ public void testReplicatorRateLimiterMessageNotReceivedAllMessages(DispatchRateT } Assert.assertTrue(replicatorUpdated); if (DispatchRateType.messageRate.equals(dispatchRateType)) { - Assert.assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), messageRate); + Assert.assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), messageRate); } else { - Assert.assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), messageRate); + Assert.assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), messageRate); } @Cleanup @@ -499,7 +501,7 @@ public void testReplicatorRateLimiterMessageReceivedAllMessages() throws Excepti boolean replicatorUpdated = false; int retry = 5; for (int i = 0; i < retry; i++) { - if (topic.getReplicators().values().get(0).getRateLimiter().isPresent()) { + if (getRateLimiter(topic).isPresent()) { replicatorUpdated = true; break; } else { @@ -509,7 +511,7 @@ public void testReplicatorRateLimiterMessageReceivedAllMessages() throws Excepti } } Assert.assertTrue(replicatorUpdated); - Assert.assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnMsg(), messageRate); + Assert.assertEquals(getRateLimiter(topic).get().getDispatchRateOnMsg(), messageRate); @Cleanup PulsarClient client2 = PulsarClient.builder().serviceUrl(url2.toString()).statsInterval(0, TimeUnit.SECONDS) @@ -578,8 +580,8 @@ public void testReplicatorRateLimiterByBytes() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getOrCreateTopic(topicName).get(); Awaitility.await() - .untilAsserted(() -> assertTrue(topic.getReplicators().values().get(0).getRateLimiter().isPresent())); - assertEquals(topic.getReplicators().values().get(0).getRateLimiter().get().getDispatchRateOnByte(), byteRate); + .untilAsserted(() -> assertTrue(getRateLimiter(topic).isPresent())); + assertEquals(getRateLimiter(topic).get().getDispatchRateOnByte(), byteRate); @Cleanup PulsarClient client2 = PulsarClient.builder().serviceUrl(url2.toString()) @@ -608,5 +610,9 @@ public void testReplicatorRateLimiterByBytes() throws Exception { }); } + private static Optional getRateLimiter(PersistentTopic topic) { + return getRateLimiter(topic); + } + private static final Logger log = LoggerFactory.getLogger(ReplicatorRateLimiterTest.class); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index 8e115e14b3770..aac7a85f477c5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -109,7 +109,6 @@ import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.protocol.Commands; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.apache.pulsar.schema.Schemas; import org.awaitility.Awaitility; @@ -637,7 +636,7 @@ public void testReplicatePeekAndSkip() throws Exception { producer1.produce(2); PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getTopicReference(dest.toString()).get(); PersistentReplicator replicator = (PersistentReplicator) topic.getReplicators() - .get(topic.getReplicators().keys().get(0)); + .get(topic.getReplicators().keySet().stream().toList().get(0)); replicator.skipMessages(2); CompletableFuture result = replicator.peekNthMessage(1); Entry entry = result.get(50, TimeUnit.MILLISECONDS); @@ -664,7 +663,7 @@ public void testReplicatorClearBacklog() throws Exception { producer1.produce(2); PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getTopicReference(dest.toString()).get(); PersistentReplicator replicator = (PersistentReplicator) spy( - topic.getReplicators().get(topic.getReplicators().keys().get(0))); + topic.getReplicators().get(topic.getReplicators().keySet().stream().toList().get(0))); replicator.readEntriesFailed(new ManagedLedgerException.InvalidCursorPositionException("failed"), null); replicator.clearBacklog().get(); Thread.sleep(100); @@ -691,7 +690,7 @@ public void testResetReplicatorSubscriptionPosition() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getTopicReference(dest.toString()).get(); PersistentReplicator replicator = (PersistentReplicator) spy( - topic.getReplicators().get(topic.getReplicators().keys().get(0))); + topic.getReplicators().get(topic.getReplicators().keySet().stream().toList().get(0))); MessageId id = topic.getLastMessageId().get(); admin1.topics().expireMessages(dest.getPartitionedTopicName(), @@ -795,7 +794,7 @@ public void testDeleteReplicatorFailure() throws Exception { @Cleanup MessageProducer producer1 = new MessageProducer(url1, dest); PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getTopicReference(topicName).get(); - final String replicatorClusterName = topic.getReplicators().keys().get(0); + final String replicatorClusterName = topic.getReplicators().keySet().stream().toList().get(0); ManagedLedgerImpl ledger = (ManagedLedgerImpl) topic.getManagedLedger(); CountDownLatch latch = new CountDownLatch(1); // delete cursor already : so next time if topic.removeReplicator will get exception but then it should @@ -836,7 +835,7 @@ public void testReplicatorProducerClosing() throws Exception { @Cleanup MessageProducer producer1 = new MessageProducer(url1, dest); PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService().getTopicReference(topicName).get(); - final String replicatorClusterName = topic.getReplicators().keys().get(0); + final String replicatorClusterName = topic.getReplicators().keySet().stream().toList().get(0); Replicator replicator = topic.getPersistentReplicator(replicatorClusterName); pulsar2.close(); pulsar2 = null; @@ -1675,7 +1674,7 @@ public void testReplicatorWithFailedAck() throws Exception { Awaitility.await().pollInterval(1, TimeUnit.SECONDS).timeout(30, TimeUnit.SECONDS) .ignoreExceptions() .untilAsserted(() -> { - ConcurrentOpenHashMap replicators = topic.getReplicators(); + final var replicators = topic.getReplicators(); PersistentReplicator replicator = (PersistentReplicator) replicators.get("r2"); assertEquals(org.apache.pulsar.broker.service.AbstractReplicator.State.Started, replicator.getState()); @@ -1928,9 +1927,9 @@ public void testEnableReplicationWithNamespaceAllowedClustersPolices() throws Ex // Verify the replication from cluster1 to cluster2 is ready, but the replication form the cluster2 to cluster1 // is not ready. Awaitility.await().untilAsserted(() -> { - ConcurrentOpenHashMap replicatorMap = persistentTopic1.getReplicators(); + final var replicatorMap = persistentTopic1.getReplicators(); assertEquals(replicatorMap.size(), 1); - Replicator replicator = replicatorMap.get(replicatorMap.keys().get(0)); + Replicator replicator = replicatorMap.get(replicatorMap.keySet().stream().toList().get(0)); assertTrue(replicator.isConnected()); }); @@ -1940,16 +1939,16 @@ public void testEnableReplicationWithNamespaceAllowedClustersPolices() throws Ex .get(); Awaitility.await().untilAsserted(() -> { - ConcurrentOpenHashMap replicatorMap = persistentTopic2.getReplicators(); + final var replicatorMap = persistentTopic2.getReplicators(); assertEquals(replicatorMap.size(), 0); }); // Enable replication at the topic level in the cluster2. admin2.topics().setReplicationClusters(topicName.toString(), List.of("r1", "r2")); // Verify the replication between cluster1 and cluster2 is ready. Awaitility.await().untilAsserted(() -> { - ConcurrentOpenHashMap replicatorMap = persistentTopic2.getReplicators(); + final var replicatorMap = persistentTopic2.getReplicators(); assertEquals(replicatorMap.size(), 1); - Replicator replicator = replicatorMap.get(replicatorMap.keys().get(0)); + Replicator replicator = replicatorMap.get(replicatorMap.keySet().stream().toList().get(0)); assertTrue(replicator.isConnected()); }); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTopicPoliciesTest.java index f89ca2bdebb91..ab1f0c0ece2e2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTopicPoliciesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTopicPoliciesTest.java @@ -29,7 +29,6 @@ import java.util.List; import java.util.Set; import java.util.UUID; -import java.util.stream.Collectors; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -748,7 +747,7 @@ public void testRemoveReplicationClusters() throws Exception { assertNotNull(topicRef); Awaitility.await().untilAsserted(() -> { - List replicaClusters = topicRef.getReplicators().keys().stream().sorted().collect(Collectors.toList()); + List replicaClusters = topicRef.getReplicators().keySet().stream().sorted().toList(); assertEquals(replicaClusters.size(), 1); assertEquals(replicaClusters.toString(), "[r2]"); }); @@ -756,7 +755,7 @@ public void testRemoveReplicationClusters() throws Exception { // removing topic replica cluster policy, so namespace policy should take effect admin1.topics().removeReplicationClusters(persistentTopicName); Awaitility.await().untilAsserted(() -> { - List replicaClusters = topicRef.getReplicators().keys().stream().sorted().collect(Collectors.toList()); + List replicaClusters = topicRef.getReplicators().keySet().stream().sorted().toList(); assertEquals(replicaClusters.size(), 2); assertEquals(replicaClusters.toString(), "[r2, r3]"); }); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionalReplicateSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionalReplicateSubscriptionTest.java index 2d348f8259746..aa39e859a8c3d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionalReplicateSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionalReplicateSubscriptionTest.java @@ -32,7 +32,6 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; @@ -118,7 +117,7 @@ public void testReplicatedSubscribeAndSwitchToStandbyClusterWithTransaction() th } txn1.commit().get(); Awaitility.await().untilAsserted(() -> { - ConcurrentOpenHashMap replicators = topic1.getReplicators(); + final var replicators = topic1.getReplicators(); assertTrue(replicators != null && replicators.size() == 1, "Replicator should started"); assertTrue(replicators.values().iterator().next().isConnected(), "Replicator should be connected"); assertTrue(topic1.getReplicatedSubscriptionController().get().getLastCompletedSnapshotId().isPresent(), diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java index e2aec70fb114e..e0d6a432bdad2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java @@ -37,7 +37,6 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicStats; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.mockito.Mockito; import org.testng.annotations.AfterMethod; @@ -212,7 +211,7 @@ public void testSubscriptionsOnNonPersistentTopic() throws Exception { .subscriptionMode(SubscriptionMode.Durable) .subscribe(); - ConcurrentOpenHashMap subscriptionMap = mockTopic.getSubscriptions(); + final var subscriptionMap = mockTopic.getSubscriptions(); assertEquals(subscriptionMap.size(), 4); // Check exclusive subscription diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilderTest.java index 562c5eda58109..fa409832fc17b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilderTest.java @@ -28,9 +28,8 @@ import io.netty.buffer.ByteBuf; import java.time.Clock; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.List; +import java.util.Set; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshot; @@ -75,10 +74,8 @@ public void setup() { @Test public void testBuildSnapshotWith2Clusters() throws Exception { - List remoteClusters = Collections.singletonList("b"); - ReplicatedSubscriptionsSnapshotBuilder builder = new ReplicatedSubscriptionsSnapshotBuilder(controller, - remoteClusters, + Set.of("b"), conf, clock); assertTrue(markers.isEmpty()); @@ -115,10 +112,8 @@ public void testBuildSnapshotWith2Clusters() throws Exception { @Test public void testBuildSnapshotWith3Clusters() throws Exception { - List remoteClusters = Arrays.asList("b", "c"); - ReplicatedSubscriptionsSnapshotBuilder builder = new ReplicatedSubscriptionsSnapshotBuilder(controller, - remoteClusters, + Set.of("b", "c"), conf, clock); assertTrue(markers.isEmpty()); @@ -198,10 +193,8 @@ public void testBuildSnapshotWith3Clusters() throws Exception { @Test public void testBuildTimeout() { - List remoteClusters = Collections.singletonList("b"); - ReplicatedSubscriptionsSnapshotBuilder builder = new ReplicatedSubscriptionsSnapshotBuilder(controller, - remoteClusters, + Set.of("b"), conf, clock); assertFalse(builder.isTimedOut()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java index 45e3fb253bf11..e091eee178d8c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java @@ -30,16 +30,14 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Consumer; -import org.apache.pulsar.broker.service.Replicator; -import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.policies.data.stats.SubscriptionStatsImpl; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.mockito.Mockito; import org.testng.annotations.BeforeMethod; @@ -71,7 +69,7 @@ public void testGenerateSubscriptionsStats() { // prepare multi-layer topic map final var bundlesMap = new ConcurrentHashMap>(); final var topicsMap = new ConcurrentHashMap(); - ConcurrentOpenHashMap subscriptionsMaps = ConcurrentOpenHashMap.newBuilder().build(); + final var subscriptionsMaps = new ConcurrentHashMap(); bundlesMap.put("my-bundle", topicsMap); multiLayerTopicsMap.put(namespace, bundlesMap); @@ -87,7 +85,7 @@ public void testGenerateSubscriptionsStats() { // Prepare topic and subscription PersistentTopic topic = Mockito.mock(PersistentTopic.class); - Subscription subscription = Mockito.mock(Subscription.class); + PersistentSubscription subscription = Mockito.mock(PersistentSubscription.class); Consumer consumer = Mockito.mock(Consumer.class); ConsumerStatsImpl consumerStats = new ConsumerStatsImpl(); when(consumer.getStats()).thenReturn(consumerStats); @@ -99,7 +97,7 @@ public void testGenerateSubscriptionsStats() { when(topic.getStats(false, false, false)).thenReturn(topicStats); when(topic.getBrokerService()).thenReturn(broker); when(topic.getSubscriptions()).thenReturn(subscriptionsMaps); - when(topic.getReplicators()).thenReturn(ConcurrentOpenHashMap.newBuilder().build()); + when(topic.getReplicators()).thenReturn(new ConcurrentHashMap<>()); when(topic.getManagedLedger()).thenReturn(ml); when(topic.getBacklogQuota(Mockito.any())).thenReturn(Mockito.mock(BacklogQuota.class)); PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java index bd0119823fd95..88286af98ae5f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java @@ -29,7 +29,6 @@ import com.google.common.collect.Multimap; import com.google.common.collect.Queues; import com.google.common.collect.Sets; -import java.lang.reflect.Field; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -49,14 +48,11 @@ import java.util.stream.Collectors; import lombok.Cleanup; import org.apache.pulsar.broker.namespace.NamespaceService; -import org.apache.pulsar.broker.service.BrokerService; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TopicStats; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -703,11 +699,7 @@ public void testBlockBrokerDispatching() { stopBroker(); startBroker(); - Field field = BrokerService.class.getDeclaredField("blockedDispatchers"); - field.setAccessible(true); - @SuppressWarnings("unchecked") - ConcurrentOpenHashSet blockedDispatchers = - (ConcurrentOpenHashSet) field.get(pulsar.getBrokerService()); + final var blockedDispatchers = pulsar.getBrokerService().getBlockedDispatchers(); final int receiverQueueSize = 10; final int totalProducedMsgs = maxUnAckPerBroker * 3; @@ -783,7 +775,7 @@ public void testBlockBrokerDispatching() { consumer2Sub1.close(); // (1.c) verify that dispatcher is part of blocked dispatcher assertEquals(blockedDispatchers.size(), 1); - String dispatcherName = blockedDispatchers.values().get(0).getName(); + String dispatcherName = blockedDispatchers.stream().findFirst().orElseThrow().getName(); String subName = dispatcherName.substring(dispatcherName.lastIndexOf("/") + 2, dispatcherName.length()); assertEquals(subName, subscriberName1); timestamps.add(System.currentTimeMillis()); @@ -918,10 +910,7 @@ public void testBrokerDispatchBlockAndSubAckBackRequiredMsgs() { stopBroker(); startBroker(); - Field field = BrokerService.class.getDeclaredField("blockedDispatchers"); - field.setAccessible(true); - ConcurrentOpenHashSet blockedDispatchers = - (ConcurrentOpenHashSet) field.get(pulsar.getBrokerService()); + final var blockedDispatchers = pulsar.getBrokerService().getBlockedDispatchers(); final int receiverQueueSize = 10; final int totalProducedMsgs = maxUnAckPerBroker * 3; @@ -992,7 +981,7 @@ public void testBrokerDispatchBlockAndSubAckBackRequiredMsgs() { consumer2Sub1.close(); // (1.c) verify that dispatcher is part of blocked dispatcher assertEquals(blockedDispatchers.size(), 1); - String dispatcherName = blockedDispatchers.values().get(0).getName(); + String dispatcherName = blockedDispatchers.stream().findFirst().orElseThrow().getName(); String subName = dispatcherName.substring(dispatcherName.lastIndexOf("/") + 2, dispatcherName.length()); assertEquals(subName, subscriberName1); From 9012422bcbaac7b38820ce545cd5a3b4f8b586d0 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 23 Sep 2024 10:44:48 +0800 Subject: [PATCH 517/580] [improve][broker] Remove ConcurrentOpenHashMap and ConcurrentOpenHashSet (#23329) --- .../impl/ManagedLedgerFactoryImpl.java | 4 +- .../broker/namespace/NamespaceService.java | 21 +- .../persistent/MessageDeduplication.java | 15 +- .../stats/ClusterReplicationMetrics.java | 7 +- .../persistent/MessageDuplicationTest.java | 5 +- .../pulsar/client/impl/ConsumerBase.java | 6 +- .../pulsar/client/impl/ConsumerImpl.java | 4 +- .../client/impl/PartitionedProducerImpl.java | 7 +- .../pulsar/client/impl/ProducerBase.java | 7 +- .../AcknowledgementsGroupingTrackerTest.java | 4 +- .../impl/UnAckedMessageTrackerTest.java | 7 +- .../collections/ConcurrentOpenHashMap.java | 658 ---------------- .../collections/ConcurrentOpenHashSet.java | 622 ---------------- .../ConcurrentOpenHashMapTest.java | 700 ------------------ .../ConcurrentOpenHashSetTest.java | 503 ------------- .../pulsar/websocket/WebSocketService.java | 36 +- .../pulsar/websocket/stats/ProxyStats.java | 7 +- 17 files changed, 40 insertions(+), 2573 deletions(-) delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMap.java delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSet.java delete mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMapTest.java delete mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSetTest.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 586beb412d297..34dd3610d4ec9 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -104,7 +104,6 @@ import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.Runnables; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.Stat; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @@ -1207,8 +1206,7 @@ public void calculateCursorBacklogs(final TopicName topicName, BookKeeper bk = getBookKeeper().get(); final CountDownLatch allCursorsCounter = new CountDownLatch(1); final long errorInReadingCursor = -1; - ConcurrentOpenHashMap ledgerRetryMap = - ConcurrentOpenHashMap.newBuilder().build(); + final var ledgerRetryMap = new ConcurrentHashMap(); final MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo = ledgers.lastEntry().getValue(); final Position lastLedgerPosition = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 0b1661fb9540a..b2ee299bb030e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -109,7 +109,6 @@ import org.apache.pulsar.common.stats.MetricsUtil; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.opentelemetry.annotations.PulsarDeprecatedMetric; @@ -150,7 +149,7 @@ public class NamespaceService implements AutoCloseable { public static final String HEARTBEAT_NAMESPACE_FMT_V2 = "pulsar/%s"; public static final String SLA_NAMESPACE_FMT = SLA_NAMESPACE_PROPERTY + "/%s/%s"; - private final ConcurrentOpenHashMap namespaceClients; + private final Map namespaceClients = new ConcurrentHashMap<>(); private final List bundleOwnershipListeners; @@ -204,8 +203,6 @@ public NamespaceService(PulsarService pulsar) { this.loadManager = pulsar.getLoadManager(); this.bundleFactory = new NamespaceBundleFactory(pulsar, Hashing.crc32()); this.ownershipCache = new OwnershipCache(pulsar, this); - this.namespaceClients = - ConcurrentOpenHashMap.newBuilder().build(); this.bundleOwnershipListeners = new CopyOnWriteArrayList<>(); this.bundleSplitListeners = new CopyOnWriteArrayList<>(); this.localBrokerDataCache = pulsar.getLocalMetadataStore().getMetadataCache(LocalBrokerData.class); @@ -461,16 +458,10 @@ public boolean registerNamespace(NamespaceName nsname, boolean ensureOwned) thro } } - private final ConcurrentOpenHashMap>> - findingBundlesAuthoritative = - ConcurrentOpenHashMap.>>newBuilder() - .build(); - private final ConcurrentOpenHashMap>> - findingBundlesNotAuthoritative = - ConcurrentOpenHashMap.>>newBuilder() - .build(); + private final Map>> + findingBundlesAuthoritative = new ConcurrentHashMap<>(); + private final Map>> + findingBundlesNotAuthoritative = new ConcurrentHashMap<>(); /** * Main internal method to lookup and setup ownership of service unit to a broker. @@ -485,7 +476,7 @@ private CompletableFuture> findBrokerServiceUrl( LOG.debug("findBrokerServiceUrl: {} - options: {}", bundle, options); } - ConcurrentOpenHashMap>> targetMap; + Map>> targetMap; if (options.isAuthoritative()) { targetMap = findingBundlesAuthoritative; } else { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java index e8d19d2e2eca1..dfb8b9d2edb12 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageDeduplication.java @@ -42,7 +42,6 @@ import org.apache.pulsar.broker.service.Topic.PublishContext; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.protocol.Commands; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -101,20 +100,12 @@ public MessageDupUnknownException() { // Map that contains the highest sequenceId that have been sent by each producers. The map will be updated before // the messages are persisted @VisibleForTesting - final ConcurrentOpenHashMap highestSequencedPushed = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); + final Map highestSequencedPushed = new ConcurrentHashMap<>(); // Map that contains the highest sequenceId that have been persistent by each producers. The map will be updated // after the messages are persisted @VisibleForTesting - final ConcurrentOpenHashMap highestSequencedPersisted = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); + final Map highestSequencedPersisted = new ConcurrentHashMap<>(); // Number of persisted entries after which to store a snapshot of the sequence ids map private final int snapshotInterval; @@ -434,7 +425,7 @@ public void resetHighestSequenceIdPushed() { } highestSequencedPushed.clear(); - for (String producer : highestSequencedPersisted.keys()) { + for (String producer : highestSequencedPersisted.keySet()) { highestSequencedPushed.put(producer, highestSequencedPersisted.get(producer)); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/ClusterReplicationMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/ClusterReplicationMetrics.java index 6b274b26b57fb..828cb48be429d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/ClusterReplicationMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/ClusterReplicationMetrics.java @@ -20,23 +20,22 @@ import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; /** */ public class ClusterReplicationMetrics { private final List metricsList; private final String localCluster; - private final ConcurrentOpenHashMap metricsMap; + private final Map metricsMap = new ConcurrentHashMap<>(); public static final String SEPARATOR = "_"; public final boolean metricsEnabled; public ClusterReplicationMetrics(String localCluster, boolean metricsEnabled) { metricsList = new ArrayList<>(); this.localCluster = localCluster; - metricsMap = ConcurrentOpenHashMap.newBuilder() - .build(); this.metricsEnabled = metricsEnabled; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java index e7dcbc602134c..5b1c78574b462 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/MessageDuplicationTest.java @@ -60,7 +60,6 @@ import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.broker.qos.AsyncTokenBucket; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.compaction.CompactionServiceFactory; import org.awaitility.Awaitility; import org.testng.Assert; @@ -230,9 +229,7 @@ public void testInactiveProducerRemove() throws Exception { messageDeduplication.purgeInactiveProducers(); assertFalse(inactiveProducers.containsKey(producerName2)); assertFalse(inactiveProducers.containsKey(producerName3)); - field = MessageDeduplication.class.getDeclaredField("highestSequencedPushed"); - field.setAccessible(true); - ConcurrentOpenHashMap highestSequencedPushed = (ConcurrentOpenHashMap) field.get(messageDeduplication); + final var highestSequencedPushed = messageDeduplication.highestSequencedPushed; assertEquals((long) highestSequencedPushed.get(producerName1), 2L); assertFalse(highestSequencedPushed.containsKey(producerName2)); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 03256a3e139b6..111cbdb8a8ef3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -31,6 +31,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -67,7 +68,6 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.BitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.util.collections.GrowableArrayBlockingQueue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,7 +88,7 @@ public abstract class ConsumerBase extends HandlerState implements Consumer> incomingMessages; - protected ConcurrentOpenHashMap unAckedChunkedMessageIdSequenceMap; + protected Map unAckedChunkedMessageIdSequenceMap = new ConcurrentHashMap<>(); protected final ConcurrentLinkedQueue>> pendingReceives; protected final int maxReceiverQueueSize; private volatile int currentReceiverQueueSize; @@ -138,8 +138,6 @@ protected ConsumerBase(PulsarClientImpl client, String topic, ConsumerConfigurat this.consumerEventListener = conf.getConsumerEventListener(); // Always use growable queue since items can exceed the advertised size this.incomingMessages = new GrowableArrayBlockingQueue<>(); - this.unAckedChunkedMessageIdSequenceMap = - ConcurrentOpenHashMap.newBuilder().build(); this.executorProvider = executorProvider; this.messageListenerExecutor = conf.getMessageListenerExecutor() == null ? (conf.getSubscriptionType() == SubscriptionType.Key_Shared diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 60b9d145c4897..03ccbae01c276 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -130,7 +130,6 @@ import org.apache.pulsar.common.util.SafeCollectionUtils; import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.util.collections.GrowableArrayBlockingQueue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -207,8 +206,7 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle protected volatile boolean paused; - protected ConcurrentOpenHashMap chunkedMessagesMap = - ConcurrentOpenHashMap.newBuilder().build(); + protected Map chunkedMessagesMap = new ConcurrentHashMap<>(); private int pendingChunkedMessageCount = 0; protected long expireTimeOfIncompleteChunkedMessageMillis = 0; private final AtomicBoolean expireChunkMessageTaskScheduled = new AtomicBoolean(false); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java index bf7f1066173f6..2dc826d9e3af3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java @@ -27,9 +27,11 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -52,7 +54,6 @@ import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,7 +61,7 @@ public class PartitionedProducerImpl extends ProducerBase { private static final Logger log = LoggerFactory.getLogger(PartitionedProducerImpl.class); - private final ConcurrentOpenHashMap> producers; + private final Map> producers = new ConcurrentHashMap<>(); private final MessageRouter routerPolicy; private final PartitionedTopicProducerStatsRecorderImpl stats; private TopicMetadata topicMetadata; @@ -76,8 +77,6 @@ public PartitionedProducerImpl(PulsarClientImpl client, String topic, ProducerCo int numPartitions, CompletableFuture> producerCreatedFuture, Schema schema, ProducerInterceptors interceptors) { super(client, topic, conf, producerCreatedFuture, schema, interceptors); - this.producers = - ConcurrentOpenHashMap.>newBuilder().build(); this.topicMetadata = new TopicMetadataImpl(numPartitions); this.routerPolicy = getMessageRouter(); stats = client.getConfiguration().getStatsIntervalSeconds() > 0 diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerBase.java index 7dc5f78398434..12e380fdd510c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerBase.java @@ -19,7 +19,9 @@ package org.apache.pulsar.client.impl; import static com.google.common.base.Preconditions.checkArgument; +import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -32,7 +34,6 @@ import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.common.protocol.schema.SchemaHash; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; public abstract class ProducerBase extends HandlerState implements Producer { @@ -40,7 +41,7 @@ public abstract class ProducerBase extends HandlerState implements Producer schema; protected final ProducerInterceptors interceptors; - protected final ConcurrentOpenHashMap schemaCache; + protected final Map schemaCache = new ConcurrentHashMap<>(); protected volatile MultiSchemaMode multiSchemaMode = MultiSchemaMode.Auto; protected ProducerBase(PulsarClientImpl client, String topic, ProducerConfigurationData conf, @@ -50,8 +51,6 @@ protected ProducerBase(PulsarClientImpl client, String topic, ProducerConfigurat this.conf = conf; this.schema = schema; this.interceptors = interceptors; - this.schemaCache = - ConcurrentOpenHashMap.newBuilder().build(); if (!conf.isMultiSchema()) { multiSchemaMode = MultiSchemaMode.Disabled; } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java index 514e3dde14070..a62d9e7479852 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java @@ -45,7 +45,6 @@ import org.apache.pulsar.client.util.TimedCompletableFuture; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.api.proto.ProtocolVersion; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -62,8 +61,7 @@ public class AcknowledgementsGroupingTrackerTest { public void setup() throws NoSuchFieldException, IllegalAccessException { eventLoopGroup = new NioEventLoopGroup(1); consumer = mock(ConsumerImpl.class); - consumer.unAckedChunkedMessageIdSequenceMap = - ConcurrentOpenHashMap.newBuilder().build(); + consumer.unAckedChunkedMessageIdSequenceMap = new ConcurrentHashMap<>(); cnx = spy(new ClientCnxTest(new ClientConfigurationData(), eventLoopGroup)); PulsarClientImpl client = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java index b01fbcb879f80..eaac165818a56 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/UnAckedMessageTrackerTest.java @@ -31,13 +31,11 @@ import io.netty.util.concurrent.DefaultThreadFactory; import java.time.Duration; import java.util.HashSet; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; - import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.metrics.InstrumentProvider; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.testng.annotations.Test; @@ -113,8 +111,7 @@ public void testTrackChunkedMessageId() { ChunkMessageIdImpl chunkedMessageId = new ChunkMessageIdImpl(chunkMsgIds[0], chunkMsgIds[chunkMsgIds.length - 1]); - consumer.unAckedChunkedMessageIdSequenceMap = - ConcurrentOpenHashMap.newBuilder().build(); + consumer.unAckedChunkedMessageIdSequenceMap = new ConcurrentHashMap<>(); consumer.unAckedChunkedMessageIdSequenceMap.put(chunkedMessageId, chunkMsgIds); // Redeliver chunked message diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMap.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMap.java deleted file mode 100644 index 7f0dbb4379265..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMap.java +++ /dev/null @@ -1,658 +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.collections; - -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Objects.requireNonNull; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; -import java.util.concurrent.locks.StampedLock; -import java.util.function.BiConsumer; -import java.util.function.Function; - -/** - * Concurrent hash map. - * - *

Provides similar methods as a {@code ConcurrentMap} but since it's an open hash map with linear probing, - * no node allocations are required to store the values. - * - *
- * WARN: method forEach do not guarantee thread safety, nor do the keys and values method. - *
- * The forEach method is specifically designed for single-threaded usage. When iterating over a map - * with concurrent writes, it becomes possible for new values to be either observed or not observed. - * There is no guarantee that if we write value1 and value2, and are able to see value2, then we will also see value1. - * In some cases, it is even possible to encounter two mappings with the same key, - * leading the keys method to return a List containing two identical keys. - * - *
- * It is crucial to understand that the results obtained from aggregate status methods such as keys and values - * are typically reliable only when the map is not undergoing concurrent updates from other threads. - * When concurrent updates are involved, the results of these methods reflect transient states - * that may be suitable for monitoring or estimation purposes, but not for program control. - * @param - */ -@SuppressWarnings("unchecked") -public class ConcurrentOpenHashMap { - - private static final Object EmptyKey = null; - private static final Object DeletedKey = new Object(); - private static final ConcurrentOpenHashMap EmptyMap = new ConcurrentOpenHashMap<>(1, 1); - - /** - * This object is used to delete empty value in this map. - * EmptyValue.equals(null) = true. - */ - private static final Object EmptyValue = new Object() { - - @SuppressFBWarnings - @Override - public boolean equals(Object obj) { - return obj == null; - } - - /** - * This is just for avoiding spotbugs errors - */ - @Override - public int hashCode() { - return super.hashCode(); - } - }; - - private static final int DefaultExpectedItems = 256; - private static final int DefaultConcurrencyLevel = 16; - - private static final float DefaultMapFillFactor = 0.66f; - private static final float DefaultMapIdleFactor = 0.15f; - - private static final float DefaultExpandFactor = 2; - private static final float DefaultShrinkFactor = 2; - - private static final boolean DefaultAutoShrink = false; - - private final Section[] sections; - - public static Builder newBuilder() { - return new Builder<>(); - } - - /** - * Builder of ConcurrentOpenHashMap. - */ - public static class Builder { - int expectedItems = DefaultExpectedItems; - int concurrencyLevel = DefaultConcurrencyLevel; - float mapFillFactor = DefaultMapFillFactor; - float mapIdleFactor = DefaultMapIdleFactor; - float expandFactor = DefaultExpandFactor; - float shrinkFactor = DefaultShrinkFactor; - boolean autoShrink = DefaultAutoShrink; - - public Builder expectedItems(int expectedItems) { - this.expectedItems = expectedItems; - return this; - } - - public Builder concurrencyLevel(int concurrencyLevel) { - this.concurrencyLevel = concurrencyLevel; - return this; - } - - public Builder mapFillFactor(float mapFillFactor) { - this.mapFillFactor = mapFillFactor; - return this; - } - - public Builder mapIdleFactor(float mapIdleFactor) { - this.mapIdleFactor = mapIdleFactor; - return this; - } - - public Builder expandFactor(float expandFactor) { - this.expandFactor = expandFactor; - return this; - } - - public Builder shrinkFactor(float shrinkFactor) { - this.shrinkFactor = shrinkFactor; - return this; - } - - public Builder autoShrink(boolean autoShrink) { - this.autoShrink = autoShrink; - return this; - } - - public ConcurrentOpenHashMap build() { - return new ConcurrentOpenHashMap<>(expectedItems, concurrencyLevel, - mapFillFactor, mapIdleFactor, autoShrink, expandFactor, shrinkFactor); - } - } - - @Deprecated - public ConcurrentOpenHashMap() { - this(DefaultExpectedItems); - } - - @Deprecated - public ConcurrentOpenHashMap(int expectedItems) { - this(expectedItems, DefaultConcurrencyLevel); - } - - @Deprecated - public ConcurrentOpenHashMap(int expectedItems, int concurrencyLevel) { - this(expectedItems, concurrencyLevel, DefaultMapFillFactor, DefaultMapIdleFactor, - DefaultAutoShrink, DefaultExpandFactor, DefaultShrinkFactor); - } - - public ConcurrentOpenHashMap(int expectedItems, int concurrencyLevel, - float mapFillFactor, float mapIdleFactor, - boolean autoShrink, float expandFactor, float shrinkFactor) { - checkArgument(expectedItems > 0); - checkArgument(concurrencyLevel > 0); - checkArgument(expectedItems >= concurrencyLevel); - checkArgument(mapFillFactor > 0 && mapFillFactor < 1); - checkArgument(mapIdleFactor > 0 && mapIdleFactor < 1); - checkArgument(mapFillFactor > mapIdleFactor); - checkArgument(expandFactor > 1); - checkArgument(shrinkFactor > 1); - - int numSections = concurrencyLevel; - int perSectionExpectedItems = expectedItems / numSections; - int perSectionCapacity = (int) (perSectionExpectedItems / mapFillFactor); - this.sections = (Section[]) new Section[numSections]; - - for (int i = 0; i < numSections; i++) { - sections[i] = new Section<>(perSectionCapacity, mapFillFactor, mapIdleFactor, - autoShrink, expandFactor, shrinkFactor); - } - } - - public static ConcurrentOpenHashMap emptyMap() { - return (ConcurrentOpenHashMap) EmptyMap; - } - - long getUsedBucketCount() { - long usedBucketCount = 0; - for (Section s : sections) { - usedBucketCount += s.usedBuckets; - } - return usedBucketCount; - } - - public long size() { - long size = 0; - for (Section s : sections) { - size += s.size; - } - return size; - } - - public long capacity() { - long capacity = 0; - for (Section s : sections) { - capacity += s.capacity; - } - return capacity; - } - - public boolean isEmpty() { - for (Section s : sections) { - if (s.size != 0) { - return false; - } - } - - return true; - } - - public V get(K key) { - requireNonNull(key); - long h = hash(key); - return getSection(h).get(key, (int) h); - } - - public boolean containsKey(K key) { - return get(key) != null; - } - - public V put(K key, V value) { - requireNonNull(key); - requireNonNull(value); - long h = hash(key); - return getSection(h).put(key, value, (int) h, false, null); - } - - public V putIfAbsent(K key, V value) { - requireNonNull(key); - requireNonNull(value); - long h = hash(key); - return getSection(h).put(key, value, (int) h, true, null); - } - - public V computeIfAbsent(K key, Function provider) { - requireNonNull(key); - requireNonNull(provider); - long h = hash(key); - return getSection(h).put(key, null, (int) h, true, provider); - } - - public V remove(K key) { - requireNonNull(key); - long h = hash(key); - return getSection(h).remove(key, null, (int) h); - } - - public boolean remove(K key, Object value) { - requireNonNull(key); - requireNonNull(value); - long h = hash(key); - return getSection(h).remove(key, value, (int) h) != null; - } - - public void removeNullValue(K key) { - remove(key, EmptyValue); - } - - private Section getSection(long hash) { - // Use 32 msb out of long to get the section - final int sectionIdx = (int) (hash >>> 32) & (sections.length - 1); - return sections[sectionIdx]; - } - - public void clear() { - for (int i = 0; i < sections.length; i++) { - sections[i].clear(); - } - } - - /** - * Iterate over all the entries in the map and apply the processor function to each of them. - *

- * Warning: Do Not Guarantee Thread-Safety. - * @param processor the function to apply to each entry - */ - public void forEach(BiConsumer processor) { - for (int i = 0; i < sections.length; i++) { - sections[i].forEach(processor); - } - } - - /** - * @return a new list of all keys (makes a copy) - */ - public List keys() { - List keys = new ArrayList<>((int) size()); - forEach((key, value) -> keys.add(key)); - return keys; - } - - public List values() { - List values = new ArrayList<>((int) size()); - forEach((key, value) -> values.add(value)); - return values; - } - - // A section is a portion of the hash map that is covered by a single - @SuppressWarnings("serial") - private static final class Section extends StampedLock { - // Each item take up 2 continuous array space. - private static final int ITEM_SIZE = 2; - - // Keys and values are stored interleaved in the table array - private volatile Object[] table; - - private volatile int capacity; - private final int initCapacity; - private static final AtomicIntegerFieldUpdater

SIZE_UPDATER = - AtomicIntegerFieldUpdater.newUpdater(Section.class, "size"); - private volatile int size; - private int usedBuckets; - private int resizeThresholdUp; - private int resizeThresholdBelow; - private final float mapFillFactor; - private final float mapIdleFactor; - private final float expandFactor; - private final float shrinkFactor; - private final boolean autoShrink; - - Section(int capacity, float mapFillFactor, float mapIdleFactor, boolean autoShrink, - float expandFactor, float shrinkFactor) { - this.capacity = alignToPowerOfTwo(capacity); - this.initCapacity = this.capacity; - this.table = new Object[ITEM_SIZE * this.capacity]; - this.size = 0; - this.usedBuckets = 0; - this.autoShrink = autoShrink; - this.mapFillFactor = mapFillFactor; - this.mapIdleFactor = mapIdleFactor; - this.expandFactor = expandFactor; - this.shrinkFactor = shrinkFactor; - this.resizeThresholdUp = (int) (this.capacity * mapFillFactor); - this.resizeThresholdBelow = (int) (this.capacity * mapIdleFactor); - } - - V get(K key, int keyHash) { - long stamp = tryOptimisticRead(); - boolean acquiredLock = false; - - // add local variable here, so OutOfBound won't happen - Object[] table = this.table; - // calculate table.length / 2 as capacity to avoid rehash changing capacity - int bucket = signSafeMod(keyHash, table.length / ITEM_SIZE); - - try { - while (true) { - // First try optimistic locking - K storedKey = (K) table[bucket]; - V storedValue = (V) table[bucket + 1]; - - if (!acquiredLock && validate(stamp)) { - // The values we have read are consistent - if (key.equals(storedKey)) { - return storedValue; - } else if (storedKey == EmptyKey) { - // Not found - return null; - } - } else { - // Fallback to acquiring read lock - if (!acquiredLock) { - stamp = readLock(); - acquiredLock = true; - - // update local variable - table = this.table; - bucket = signSafeMod(keyHash, table.length / ITEM_SIZE); - storedKey = (K) table[bucket]; - storedValue = (V) table[bucket + 1]; - } - - if (key.equals(storedKey)) { - return storedValue; - } else if (storedKey == EmptyKey) { - // Not found - return null; - } - } - - bucket = (bucket + ITEM_SIZE) & (table.length - 1); - } - } finally { - if (acquiredLock) { - unlockRead(stamp); - } - } - } - - V put(K key, V value, int keyHash, boolean onlyIfAbsent, Function valueProvider) { - long stamp = writeLock(); - int bucket = signSafeMod(keyHash, capacity); - - // Remember where we find the first available spot - int firstDeletedKey = -1; - - try { - while (true) { - K storedKey = (K) table[bucket]; - V storedValue = (V) table[bucket + 1]; - - if (key.equals(storedKey)) { - if (!onlyIfAbsent) { - // Over written an old value for same key - table[bucket + 1] = value; - return storedValue; - } else { - return storedValue; - } - } else if (storedKey == EmptyKey) { - // Found an empty bucket. This means the key is not in the map. If we've already seen a deleted - // key, we should write at that position - if (firstDeletedKey != -1) { - bucket = firstDeletedKey; - } else { - ++usedBuckets; - } - - if (value == null) { - value = valueProvider.apply(key); - } - - table[bucket] = key; - table[bucket + 1] = value; - SIZE_UPDATER.incrementAndGet(this); - return valueProvider != null ? value : null; - } else if (storedKey == DeletedKey) { - // The bucket contained a different deleted key - if (firstDeletedKey == -1) { - firstDeletedKey = bucket; - } - } - - bucket = (bucket + ITEM_SIZE) & (table.length - 1); - } - } finally { - if (usedBuckets > resizeThresholdUp) { - try { - // Expand the hashmap - int newCapacity = alignToPowerOfTwo((int) (capacity * expandFactor)); - rehash(newCapacity); - } finally { - unlockWrite(stamp); - } - } else { - unlockWrite(stamp); - } - } - } - - private V remove(K key, Object value, int keyHash) { - long stamp = writeLock(); - int bucket = signSafeMod(keyHash, capacity); - - try { - while (true) { - K storedKey = (K) table[bucket]; - V storedValue = (V) table[bucket + 1]; - if (key.equals(storedKey)) { - if (value == null || value.equals(storedValue)) { - SIZE_UPDATER.decrementAndGet(this); - - int nextInArray = (bucket + ITEM_SIZE) & (table.length - 1); - if (table[nextInArray] == EmptyKey) { - table[bucket] = EmptyKey; - table[bucket + 1] = null; - --usedBuckets; - - // Cleanup all the buckets that were in `DeletedKey` state, - // so that we can reduce unnecessary expansions - int lastBucket = (bucket - ITEM_SIZE) & (table.length - 1); - while (table[lastBucket] == DeletedKey) { - table[lastBucket] = EmptyKey; - table[lastBucket + 1] = null; - --usedBuckets; - - lastBucket = (lastBucket - ITEM_SIZE) & (table.length - 1); - } - } else { - table[bucket] = DeletedKey; - table[bucket + 1] = null; - } - - return storedValue; - } else { - return null; - } - } else if (storedKey == EmptyKey) { - // Key wasn't found - return null; - } - - bucket = (bucket + ITEM_SIZE) & (table.length - 1); - } - - } finally { - if (autoShrink && size < resizeThresholdBelow) { - try { - // Shrinking must at least ensure initCapacity, - // so as to avoid frequent shrinking and expansion near initCapacity, - // frequent shrinking and expansion, - // additionally opened arrays will consume more memory and affect GC - int newCapacity = Math.max(alignToPowerOfTwo((int) (capacity / shrinkFactor)), initCapacity); - int newResizeThresholdUp = (int) (newCapacity * mapFillFactor); - if (newCapacity < capacity && newResizeThresholdUp > size) { - // shrink the hashmap - rehash(newCapacity); - } - } finally { - unlockWrite(stamp); - } - } else { - unlockWrite(stamp); - } - } - } - - void clear() { - long stamp = writeLock(); - - try { - if (autoShrink && capacity > initCapacity) { - shrinkToInitCapacity(); - } else { - Arrays.fill(table, EmptyKey); - this.size = 0; - this.usedBuckets = 0; - } - } finally { - unlockWrite(stamp); - } - } - - public void forEach(BiConsumer processor) { - // Take a reference to the data table, if there is a rehashing event, we'll be - // simply iterating over a snapshot of the data. - Object[] table = this.table; - - // Go through all the buckets for this section. We try to renew the stamp only after a validation - // error, otherwise we keep going with the same. - long stamp = 0; - for (int bucket = 0; bucket < table.length; bucket += ITEM_SIZE) { - if (stamp == 0) { - stamp = tryOptimisticRead(); - } - - K storedKey = (K) table[bucket]; - V storedValue = (V) table[bucket + 1]; - - if (!validate(stamp)) { - // Fallback to acquiring read lock - stamp = readLock(); - - try { - storedKey = (K) table[bucket]; - storedValue = (V) table[bucket + 1]; - } finally { - unlockRead(stamp); - } - - stamp = 0; - } - - if (storedKey != DeletedKey && storedKey != EmptyKey) { - processor.accept(storedKey, storedValue); - } - } - } - - private void rehash(int newCapacity) { - // Expand the hashmap - Object[] newTable = new Object[ITEM_SIZE * newCapacity]; - - // Re-hash table - for (int i = 0; i < table.length; i += ITEM_SIZE) { - K storedKey = (K) table[i]; - V storedValue = (V) table[i + 1]; - if (storedKey != EmptyKey && storedKey != DeletedKey) { - insertKeyValueNoLock(newTable, newCapacity, storedKey, storedValue); - } - } - - table = newTable; - capacity = newCapacity; - usedBuckets = size; - resizeThresholdUp = (int) (capacity * mapFillFactor); - resizeThresholdBelow = (int) (capacity * mapIdleFactor); - } - - private void shrinkToInitCapacity() { - Object[] newTable = new Object[ITEM_SIZE * initCapacity]; - - table = newTable; - size = 0; - usedBuckets = 0; - // Capacity needs to be updated after the values, so that we won't see - // a capacity value bigger than the actual array size - capacity = initCapacity; - resizeThresholdUp = (int) (capacity * mapFillFactor); - resizeThresholdBelow = (int) (capacity * mapIdleFactor); - } - - private static void insertKeyValueNoLock(Object[] table, int capacity, K key, V value) { - int bucket = signSafeMod(hash(key), capacity); - - while (true) { - K storedKey = (K) table[bucket]; - - if (storedKey == EmptyKey) { - // The bucket is empty, so we can use it - table[bucket] = key; - table[bucket + 1] = value; - return; - } - - bucket = (bucket + ITEM_SIZE) & (table.length - 1); - } - } - } - - private static final long HashMixer = 0xc6a4a7935bd1e995L; - private static final int R = 47; - - static final long hash(K key) { - long hash = key.hashCode() * HashMixer; - hash ^= hash >>> R; - hash *= HashMixer; - return hash; - } - - static final int signSafeMod(long n, int max) { - // as the ITEM_SIZE of Section is 2, so the index is the multiple of 2 - // that is to left shift 1 bit - return (int) (n & (max - 1)) << 1; - } - - private static int alignToPowerOfTwo(int n) { - return (int) Math.pow(2, 32 - Integer.numberOfLeadingZeros(n - 1)); - } -} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSet.java deleted file mode 100644 index 0a9f802037bce..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSet.java +++ /dev/null @@ -1,622 +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.collections; - -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Objects.requireNonNull; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; -import java.util.concurrent.locks.StampedLock; -import java.util.function.Consumer; -import java.util.function.Predicate; - -/** - * Concurrent hash set. - * - *

Provides similar methods as a {@code ConcurrentMap} but since it's an open hash map with linear probing, - * no node allocations are required to store the values. - * - *
- * WARN: method forEach do not guarantee thread safety, nor does the values method. - *
- * The forEach method is specifically designed for single-threaded usage. When iterating over a set - * with concurrent writes, it becomes possible for new values to be either observed or not observed. - * There is no guarantee that if we write value1 and value2, and are able to see value2, then we will also see value1. - * - *
- * It is crucial to understand that the results obtained from aggregate status methods such as values - * are typically reliable only when the map is not undergoing concurrent updates from other threads. - * When concurrent updates are involved, the results of these methods reflect transient states - * that may be suitable for monitoring or estimation purposes, but not for program control. - * @param - */ -@SuppressWarnings("unchecked") -public class ConcurrentOpenHashSet { - - private static final Object EmptyValue = null; - private static final Object DeletedValue = new Object(); - - private static final int DefaultExpectedItems = 256; - private static final int DefaultConcurrencyLevel = 16; - - private static final float DefaultMapFillFactor = 0.66f; - private static final float DefaultMapIdleFactor = 0.15f; - - private static final float DefaultExpandFactor = 2; - private static final float DefaultShrinkFactor = 2; - - private static final boolean DefaultAutoShrink = false; - - private final Section[] sections; - - public static Builder newBuilder() { - return new Builder<>(); - } - - /** - * Builder of ConcurrentOpenHashSet. - */ - public static class Builder { - int expectedItems = DefaultExpectedItems; - int concurrencyLevel = DefaultConcurrencyLevel; - float mapFillFactor = DefaultMapFillFactor; - float mapIdleFactor = DefaultMapIdleFactor; - float expandFactor = DefaultExpandFactor; - float shrinkFactor = DefaultShrinkFactor; - boolean autoShrink = DefaultAutoShrink; - - public Builder expectedItems(int expectedItems) { - this.expectedItems = expectedItems; - return this; - } - - public Builder concurrencyLevel(int concurrencyLevel) { - this.concurrencyLevel = concurrencyLevel; - return this; - } - - public Builder mapFillFactor(float mapFillFactor) { - this.mapFillFactor = mapFillFactor; - return this; - } - - public Builder mapIdleFactor(float mapIdleFactor) { - this.mapIdleFactor = mapIdleFactor; - return this; - } - - public Builder expandFactor(float expandFactor) { - this.expandFactor = expandFactor; - return this; - } - - public Builder shrinkFactor(float shrinkFactor) { - this.shrinkFactor = shrinkFactor; - return this; - } - - public Builder autoShrink(boolean autoShrink) { - this.autoShrink = autoShrink; - return this; - } - - public ConcurrentOpenHashSet build() { - return new ConcurrentOpenHashSet<>(expectedItems, concurrencyLevel, - mapFillFactor, mapIdleFactor, autoShrink, expandFactor, shrinkFactor); - } - } - - @Deprecated - public ConcurrentOpenHashSet() { - this(DefaultExpectedItems); - } - - @Deprecated - public ConcurrentOpenHashSet(int expectedItems) { - this(expectedItems, DefaultConcurrencyLevel); - } - - @Deprecated - public ConcurrentOpenHashSet(int expectedItems, int concurrencyLevel) { - this(expectedItems, concurrencyLevel, DefaultMapFillFactor, DefaultMapIdleFactor, - DefaultAutoShrink, DefaultExpandFactor, DefaultShrinkFactor); - } - - public ConcurrentOpenHashSet(int expectedItems, int concurrencyLevel, - float mapFillFactor, float mapIdleFactor, - boolean autoShrink, float expandFactor, float shrinkFactor) { - checkArgument(expectedItems > 0); - checkArgument(concurrencyLevel > 0); - checkArgument(expectedItems >= concurrencyLevel); - checkArgument(mapFillFactor > 0 && mapFillFactor < 1); - checkArgument(mapIdleFactor > 0 && mapIdleFactor < 1); - checkArgument(mapFillFactor > mapIdleFactor); - checkArgument(expandFactor > 1); - checkArgument(shrinkFactor > 1); - - int numSections = concurrencyLevel; - int perSectionExpectedItems = expectedItems / numSections; - int perSectionCapacity = (int) (perSectionExpectedItems / mapFillFactor); - this.sections = (Section[]) new Section[numSections]; - - for (int i = 0; i < numSections; i++) { - sections[i] = new Section<>(perSectionCapacity, mapFillFactor, mapIdleFactor, - autoShrink, expandFactor, shrinkFactor); - } - } - - long getUsedBucketCount() { - long usedBucketCount = 0; - for (Section s : sections) { - usedBucketCount += s.usedBuckets; - } - return usedBucketCount; - } - - public long size() { - long size = 0; - for (int i = 0; i < sections.length; i++) { - size += sections[i].size; - } - return size; - } - - public long capacity() { - long capacity = 0; - for (int i = 0; i < sections.length; i++) { - capacity += sections[i].capacity; - } - return capacity; - } - - public boolean isEmpty() { - for (int i = 0; i < sections.length; i++) { - if (sections[i].size != 0) { - return false; - } - } - - return true; - } - - public boolean contains(V value) { - requireNonNull(value); - long h = hash(value); - return getSection(h).contains(value, (int) h); - } - - public boolean add(V value) { - requireNonNull(value); - long h = hash(value); - return getSection(h).add(value, (int) h); - } - - public boolean remove(V value) { - requireNonNull(value); - long h = hash(value); - return getSection(h).remove(value, (int) h); - } - - private Section getSection(long hash) { - // Use 32 msb out of long to get the section - final int sectionIdx = (int) (hash >>> 32) & (sections.length - 1); - return sections[sectionIdx]; - } - - public void clear() { - for (int i = 0; i < sections.length; i++) { - sections[i].clear(); - } - } - - /** - * Iterate over all the elements in the set and apply the provided function. - *

- * Warning: Do Not Guarantee Thread-Safety. - * @param processor the function to apply to each element - */ - public void forEach(Consumer processor) { - for (int i = 0; i < sections.length; i++) { - sections[i].forEach(processor); - } - } - - public int removeIf(Predicate filter) { - requireNonNull(filter); - - int removedCount = 0; - for (int i = 0; i < sections.length; i++) { - removedCount += sections[i].removeIf(filter); - } - - return removedCount; - } - - /** - * @return a new list of all values (makes a copy) - */ - public List values() { - List values = new ArrayList<>(); - forEach(value -> values.add(value)); - return values; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append('{'); - final AtomicBoolean first = new AtomicBoolean(true); - forEach(value -> { - if (!first.getAndSet(false)) { - sb.append(", "); - } - - sb.append(value.toString()); - }); - sb.append('}'); - return sb.toString(); - } - - // A section is a portion of the hash map that is covered by a single - @SuppressWarnings("serial") - private static final class Section extends StampedLock { - private volatile V[] values; - - private volatile int capacity; - private final int initCapacity; - private static final AtomicIntegerFieldUpdater

SIZE_UPDATER = - AtomicIntegerFieldUpdater.newUpdater(Section.class, "size"); - private volatile int size; - private int usedBuckets; - private int resizeThresholdUp; - private int resizeThresholdBelow; - private final float mapFillFactor; - private final float mapIdleFactor; - private final float expandFactor; - private final float shrinkFactor; - private final boolean autoShrink; - - Section(int capacity, float mapFillFactor, float mapIdleFactor, boolean autoShrink, - float expandFactor, float shrinkFactor) { - this.capacity = alignToPowerOfTwo(capacity); - this.initCapacity = this.capacity; - this.values = (V[]) new Object[this.capacity]; - this.size = 0; - this.usedBuckets = 0; - this.autoShrink = autoShrink; - this.mapFillFactor = mapFillFactor; - this.mapIdleFactor = mapIdleFactor; - this.expandFactor = expandFactor; - this.shrinkFactor = shrinkFactor; - this.resizeThresholdUp = (int) (this.capacity * mapFillFactor); - this.resizeThresholdBelow = (int) (this.capacity * mapIdleFactor); - } - - boolean contains(V value, int keyHash) { - long stamp = tryOptimisticRead(); - boolean acquiredLock = false; - - // add local variable here, so OutOfBound won't happen - V[] values = this.values; - // calculate table.length as capacity to avoid rehash changing capacity - int bucket = signSafeMod(keyHash, values.length); - - try { - while (true) { - // First try optimistic locking - V storedValue = values[bucket]; - - if (!acquiredLock && validate(stamp)) { - // The values we have read are consistent - if (value.equals(storedValue)) { - return true; - } else if (storedValue == EmptyValue) { - // Not found - return false; - } - } else { - // Fallback to acquiring read lock - if (!acquiredLock) { - stamp = readLock(); - acquiredLock = true; - - // update local variable - values = this.values; - bucket = signSafeMod(keyHash, values.length); - storedValue = values[bucket]; - } - - if (value.equals(storedValue)) { - return true; - } else if (storedValue == EmptyValue) { - // Not found - return false; - } - } - bucket = (bucket + 1) & (values.length - 1); - } - } finally { - if (acquiredLock) { - unlockRead(stamp); - } - } - } - - boolean add(V value, int keyHash) { - int bucket = keyHash; - - long stamp = writeLock(); - int capacity = this.capacity; - - // Remember where we find the first available spot - int firstDeletedValue = -1; - - try { - while (true) { - bucket = signSafeMod(bucket, capacity); - - V storedValue = values[bucket]; - - if (value.equals(storedValue)) { - return false; - } else if (storedValue == EmptyValue) { - // Found an empty bucket. This means the value is not in the set. If we've already seen a - // deleted value, we should write at that position - if (firstDeletedValue != -1) { - bucket = firstDeletedValue; - } else { - ++usedBuckets; - } - - values[bucket] = value; - SIZE_UPDATER.incrementAndGet(this); - return true; - } else if (storedValue == DeletedValue) { - // The bucket contained a different deleted key - if (firstDeletedValue == -1) { - firstDeletedValue = bucket; - } - } - - ++bucket; - } - } finally { - if (usedBuckets > resizeThresholdUp) { - try { - // Expand the hashmap - int newCapacity = alignToPowerOfTwo((int) (capacity * expandFactor)); - rehash(newCapacity); - } finally { - unlockWrite(stamp); - } - } else { - unlockWrite(stamp); - } - } - } - - private boolean remove(V value, int keyHash) { - int bucket = keyHash; - long stamp = writeLock(); - - try { - while (true) { - int capacity = this.capacity; - bucket = signSafeMod(bucket, capacity); - - V storedValue = values[bucket]; - if (value.equals(storedValue)) { - SIZE_UPDATER.decrementAndGet(this); - cleanBucket(bucket); - return true; - } else if (storedValue == EmptyValue) { - // Value wasn't found - return false; - } - - ++bucket; - } - - } finally { - if (autoShrink && size < resizeThresholdBelow) { - try { - // Shrinking must at least ensure initCapacity, - // so as to avoid frequent shrinking and expansion near initCapacity, - // frequent shrinking and expansion, - // additionally opened arrays will consume more memory and affect GC - int newCapacity = Math.max(alignToPowerOfTwo((int) (capacity / shrinkFactor)), initCapacity); - int newResizeThresholdUp = (int) (newCapacity * mapFillFactor); - if (newCapacity < capacity && newResizeThresholdUp > size) { - // shrink the hashmap - rehash(newCapacity); - } - } finally { - unlockWrite(stamp); - } - } else { - unlockWrite(stamp); - } - } - } - - void clear() { - long stamp = writeLock(); - - try { - if (autoShrink && capacity > initCapacity) { - shrinkToInitCapacity(); - } else { - Arrays.fill(values, EmptyValue); - this.size = 0; - this.usedBuckets = 0; - } - } finally { - unlockWrite(stamp); - } - } - - int removeIf(Predicate filter) { - long stamp = writeLock(); - - int removedCount = 0; - try { - // Go through all the buckets for this section - for (int bucket = capacity - 1; bucket >= 0; bucket--) { - V storedValue = values[bucket]; - - if (storedValue != DeletedValue && storedValue != EmptyValue) { - if (filter.test(storedValue)) { - // Removing item - SIZE_UPDATER.decrementAndGet(this); - ++removedCount; - cleanBucket(bucket); - } - } - } - - return removedCount; - } finally { - unlockWrite(stamp); - } - } - - private void cleanBucket(int bucket) { - int nextInArray = signSafeMod(bucket + 1, capacity); - if (values[nextInArray] == EmptyValue) { - values[bucket] = (V) EmptyValue; - --usedBuckets; - - // Cleanup all the buckets that were in `DeletedValue` state, - // so that we can reduce unnecessary expansions - int lastBucket = signSafeMod(bucket - 1, capacity); - while (values[lastBucket] == DeletedValue) { - values[lastBucket] = (V) EmptyValue; - --usedBuckets; - - lastBucket = signSafeMod(lastBucket - 1, capacity); - } - } else { - values[bucket] = (V) DeletedValue; - } - } - - public void forEach(Consumer processor) { - V[] values = this.values; - - // Go through all the buckets for this section. We try to renew the stamp only after a validation - // error, otherwise we keep going with the same. - long stamp = 0; - for (int bucket = 0; bucket < capacity; bucket++) { - if (stamp == 0) { - stamp = tryOptimisticRead(); - } - - V storedValue = values[bucket]; - - if (!validate(stamp)) { - // Fallback to acquiring read lock - stamp = readLock(); - - try { - storedValue = values[bucket]; - } finally { - unlockRead(stamp); - } - - stamp = 0; - } - - if (storedValue != DeletedValue && storedValue != EmptyValue) { - processor.accept(storedValue); - } - } - } - - private void rehash(int newCapacity) { - // Expand the hashmap - V[] newValues = (V[]) new Object[newCapacity]; - - // Re-hash table - for (int i = 0; i < values.length; i++) { - V storedValue = values[i]; - if (storedValue != EmptyValue && storedValue != DeletedValue) { - insertValueNoLock(newValues, storedValue); - } - } - - values = newValues; - capacity = newCapacity; - usedBuckets = size; - resizeThresholdUp = (int) (capacity * mapFillFactor); - resizeThresholdBelow = (int) (capacity * mapIdleFactor); - } - - private void shrinkToInitCapacity() { - V[] newValues = (V[]) new Object[initCapacity]; - - values = newValues; - size = 0; - usedBuckets = 0; - // Capacity needs to be updated after the values, so that we won't see - // a capacity value bigger than the actual array size - capacity = initCapacity; - resizeThresholdUp = (int) (capacity * mapFillFactor); - resizeThresholdBelow = (int) (capacity * mapIdleFactor); - } - - private static void insertValueNoLock(V[] values, V value) { - int bucket = (int) hash(value); - - while (true) { - bucket = signSafeMod(bucket, values.length); - - V storedValue = values[bucket]; - - if (storedValue == EmptyValue) { - // The bucket is empty, so we can use it - values[bucket] = value; - return; - } - - ++bucket; - } - } - } - - private static final long HashMixer = 0xc6a4a7935bd1e995L; - private static final int R = 47; - - static final long hash(K key) { - long hash = key.hashCode() * HashMixer; - hash ^= hash >>> R; - hash *= HashMixer; - return hash; - } - - static final int signSafeMod(long n, int max) { - return (int) n & (max - 1); - } - - private static int alignToPowerOfTwo(int n) { - return (int) Math.pow(2, 32 - Integer.numberOfLeadingZeros(n - 1)); - } -} diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMapTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMapTest.java deleted file mode 100644 index 48a1a705a3202..0000000000000 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashMapTest.java +++ /dev/null @@ -1,700 +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.collections; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNotEquals; -import static org.testng.Assert.assertNull; -import static org.testng.Assert.assertThrows; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Random; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; - -import lombok.Cleanup; -import org.testng.annotations.Test; - -import com.google.common.collect.Lists; - -public class ConcurrentOpenHashMapTest { - - @Test - public void testConstructor() { - try { - ConcurrentOpenHashMap.newBuilder() - .expectedItems(0) - .build(); - fail("should have thrown exception"); - } catch (IllegalArgumentException e) { - // ok - } - - try { - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(0) - .build(); - fail("should have thrown exception"); - } catch (IllegalArgumentException e) { - // ok - } - - try { - ConcurrentOpenHashMap.newBuilder() - .expectedItems(4) - .concurrencyLevel(8) - .build(); - fail("should have thrown exception"); - } catch (IllegalArgumentException e) { - // ok - } - } - - @Test - public void simpleInsertions() { - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .build(); - - assertTrue(map.isEmpty()); - assertNull(map.put("1", "one")); - assertFalse(map.isEmpty()); - - assertNull(map.put("2", "two")); - assertNull(map.put("3", "three")); - - assertEquals(map.size(), 3); - - assertEquals(map.get("1"), "one"); - assertEquals(map.size(), 3); - - assertEquals(map.remove("1"), "one"); - assertEquals(map.size(), 2); - assertNull(map.get("1")); - assertNull(map.get("5")); - assertEquals(map.size(), 2); - - assertNull(map.put("1", "one")); - assertEquals(map.size(), 3); - assertEquals(map.put("1", "uno"), "one"); - assertEquals(map.size(), 3); - } - - @Test - public void testReduceUnnecessaryExpansions() { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .build(); - assertNull(map.put("1", "1")); - assertNull(map.put("2", "2")); - assertNull(map.put("3", "3")); - assertNull(map.put("4", "4")); - - assertEquals(map.remove("1"), "1"); - assertEquals(map.remove("2"), "2"); - assertEquals(map.remove("3"), "3"); - assertEquals(map.remove("4"), "4"); - - assertEquals(0, map.getUsedBucketCount()); - } - - @Test - public void testClear() { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - assertTrue(map.capacity() == 4); - - assertNull(map.put("k1", "v1")); - assertNull(map.put("k2", "v2")); - assertNull(map.put("k3", "v3")); - - assertTrue(map.capacity() == 8); - map.clear(); - assertTrue(map.capacity() == 4); - } - - @Test - public void testExpandAndShrink() { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - assertTrue(map.capacity() == 4); - - assertNull(map.put("k1", "v1")); - assertNull(map.put("k2", "v2")); - assertNull(map.put("k3", "v3")); - - // expand hashmap - assertTrue(map.capacity() == 8); - - assertTrue(map.remove("k1", "v1")); - // not shrink - assertTrue(map.capacity() == 8); - assertTrue(map.remove("k2", "v2")); - // shrink hashmap - assertTrue(map.capacity() == 4); - - // expand hashmap - assertNull(map.put("k4", "v4")); - assertNull(map.put("k5", "v5")); - assertTrue(map.capacity() == 8); - - //verify that the map does not keep shrinking at every remove() operation - assertNull(map.put("k6", "v6")); - assertTrue(map.remove("k6", "v6")); - assertTrue(map.capacity() == 8); - } - - @Test - public void testExpandShrinkAndClear() { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - final long initCapacity = map.capacity(); - assertTrue(map.capacity() == 4); - assertNull(map.put("k1", "v1")); - assertNull(map.put("k2", "v2")); - assertNull(map.put("k3", "v3")); - - // expand hashmap - assertTrue(map.capacity() == 8); - - assertTrue(map.remove("k1", "v1")); - // not shrink - assertTrue(map.capacity() == 8); - assertTrue(map.remove("k2", "v2")); - // shrink hashmap - assertTrue(map.capacity() == 4); - - assertTrue(map.remove("k3", "v3")); - // Will not shrink the hashmap again because shrink capacity is less than initCapacity - // current capacity is equal than the initial capacity - assertTrue(map.capacity() == initCapacity); - map.clear(); - // after clear, because current capacity is equal than the initial capacity, so not shrinkToInitCapacity - assertTrue(map.capacity() == initCapacity); - } - - @Test - public void testConcurrentExpandAndShrinkAndGet() throws Throwable { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - assertEquals(map.capacity(), 4); - - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - final int readThreads = 16; - final int writeThreads = 1; - final int n = 1_000; - CyclicBarrier barrier = new CyclicBarrier(writeThreads + readThreads); - Future future = null; - AtomicReference ex = new AtomicReference<>(); - - for (int i = 0; i < readThreads; i++) { - executor.submit(() -> { - try { - barrier.await(); - } catch (Exception e) { - throw new RuntimeException(e); - } - while (!Thread.currentThread().isInterrupted()) { - try { - map.get("k2"); - } catch (Exception e) { - ex.set(e); - } - } - }); - } - - assertNull(map.put("k1","v1")); - future = executor.submit(() -> { - try { - barrier.await(); - } catch (Exception e) { - throw new RuntimeException(e); - } - - for (int i = 0; i < n; i++) { - // expand hashmap - assertNull(map.put("k2", "v2")); - assertNull(map.put("k3", "v3")); - assertEquals(map.capacity(), 8); - - // shrink hashmap - assertTrue(map.remove("k2", "v2")); - assertTrue(map.remove("k3", "v3")); - assertEquals(map.capacity(), 4); - } - }); - - future.get(); - assertTrue(ex.get() == null); - // shut down pool - executor.shutdown(); - } - - @Test - public void testRemove() { - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder().build(); - - assertTrue(map.isEmpty()); - assertNull(map.put("1", "one")); - assertFalse(map.isEmpty()); - - assertFalse(map.remove("0", "zero")); - assertFalse(map.remove("1", "uno")); - - assertFalse(map.isEmpty()); - assertTrue(map.remove("1", "one")); - assertTrue(map.isEmpty()); - } - - @Test - public void testRehashing() { - int n = 16; - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(n / 2) - .concurrencyLevel(1) - .build(); - assertEquals(map.capacity(), n); - assertEquals(map.size(), 0); - - for (int i = 0; i < n; i++) { - map.put(Integer.toString(i), i); - } - - assertEquals(map.capacity(), 2 * n); - assertEquals(map.size(), n); - } - - @Test - public void testRehashingWithDeletes() { - int n = 16; - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(n / 2) - .concurrencyLevel(1) - .build(); - assertEquals(map.capacity(), n); - assertEquals(map.size(), 0); - - for (int i = 0; i < n / 2; i++) { - map.put(i, i); - } - - for (int i = 0; i < n / 2; i++) { - map.remove(i); - } - - for (int i = n; i < (2 * n); i++) { - map.put(i, i); - } - - assertEquals(map.capacity(), 2 * n); - assertEquals(map.size(), n); - } - - @Test - public void concurrentInsertions() throws Throwable { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - - final int nThreads = 16; - final int N = 100_000; - String value = "value"; - - List> futures = new ArrayList<>(); - for (int i = 0; i < nThreads; i++) { - final int threadIdx = i; - - futures.add(executor.submit(() -> { - Random random = new Random(); - - for (int j = 0; j < N; j++) { - long key = random.nextLong(); - // Ensure keys are uniques - key -= key % (threadIdx + 1); - - map.put(key, value); - } - })); - } - - for (Future future : futures) { - future.get(); - } - - assertEquals(map.size(), N * nThreads); - } - - @Test - public void concurrentInsertionsAndReads() throws Throwable { - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder().build(); - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - - final int nThreads = 16; - final int N = 100_000; - String value = "value"; - - List> futures = new ArrayList<>(); - for (int i = 0; i < nThreads; i++) { - final int threadIdx = i; - - futures.add(executor.submit(() -> { - Random random = new Random(); - - for (int j = 0; j < N; j++) { - long key = random.nextLong(); - // Ensure keys are uniques - key -= key % (threadIdx + 1); - - map.put(key, value); - } - })); - } - - for (Future future : futures) { - future.get(); - } - - assertEquals(map.size(), N * nThreads); - } - - @Test - public void testIteration() { - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder().build(); - - assertEquals(map.keys(), Collections.emptyList()); - assertEquals(map.values(), Collections.emptyList()); - - map.put(0l, "zero"); - - assertEquals(map.keys(), Lists.newArrayList(0l)); - assertEquals(map.values(), Lists.newArrayList("zero")); - - map.remove(0l); - - assertEquals(map.keys(), Collections.emptyList()); - assertEquals(map.values(), Collections.emptyList()); - - map.put(0l, "zero"); - map.put(1l, "one"); - map.put(2l, "two"); - - List keys = map.keys(); - keys.sort(null); - assertEquals(keys, Lists.newArrayList(0l, 1l, 2l)); - - List values = map.values(); - values.sort(null); - assertEquals(values, Lists.newArrayList("one", "two", "zero")); - - map.put(1l, "uno"); - - keys = map.keys(); - keys.sort(null); - assertEquals(keys, Lists.newArrayList(0l, 1l, 2l)); - - values = map.values(); - values.sort(null); - assertEquals(values, Lists.newArrayList("two", "uno", "zero")); - - map.clear(); - assertTrue(map.isEmpty()); - } - - @Test - public void testHashConflictWithDeletion() { - final int Buckets = 16; - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(Buckets) - .concurrencyLevel(1) - .build(); - - // Pick 2 keys that fall into the same bucket - long key1 = 1; - long key2 = 27; - - int bucket1 = ConcurrentOpenHashMap.signSafeMod(ConcurrentOpenHashMap.hash(key1), Buckets); - int bucket2 = ConcurrentOpenHashMap.signSafeMod(ConcurrentOpenHashMap.hash(key2), Buckets); - assertEquals(bucket1, bucket2); - - assertNull(map.put(key1, "value-1")); - assertNull(map.put(key2, "value-2")); - assertEquals(map.size(), 2); - - assertEquals(map.remove(key1), "value-1"); - assertEquals(map.size(), 1); - - assertNull(map.put(key1, "value-1-overwrite")); - assertEquals(map.size(), 2); - - assertEquals(map.remove(key1), "value-1-overwrite"); - assertEquals(map.size(), 1); - - assertEquals(map.put(key2, "value-2-overwrite"), "value-2"); - assertEquals(map.get(key2), "value-2-overwrite"); - - assertEquals(map.size(), 1); - assertEquals(map.remove(key2), "value-2-overwrite"); - assertTrue(map.isEmpty()); - } - - @Test - public void testPutIfAbsent() { - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder().build(); - assertNull(map.putIfAbsent(1l, "one")); - assertEquals(map.get(1l), "one"); - - assertEquals(map.putIfAbsent(1l, "uno"), "one"); - assertEquals(map.get(1l), "one"); - } - - @Test - public void testComputeIfAbsent() { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); - AtomicInteger counter = new AtomicInteger(); - Function provider = key -> counter.getAndIncrement(); - - assertEquals(map.computeIfAbsent(0, provider).intValue(), 0); - assertEquals(map.get(0).intValue(), 0); - - assertEquals(map.computeIfAbsent(1, provider).intValue(), 1); - assertEquals(map.get(1).intValue(), 1); - - assertEquals(map.computeIfAbsent(1, provider).intValue(), 1); - assertEquals(map.get(1).intValue(), 1); - - assertEquals(map.computeIfAbsent(2, provider).intValue(), 2); - assertEquals(map.get(2).intValue(), 2); - } - - @Test - public void testEqualsKeys() { - class T { - int value; - - T(int value) { - this.value = value; - } - - @Override - public int hashCode() { - return Integer.hashCode(value); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof T) { - return value == ((T) obj).value; - } - - return false; - } - } - - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder().build(); - - T t1 = new T(1); - T t1_b = new T(1); - T t2 = new T(2); - - assertEquals(t1, t1_b); - assertNotEquals(t2, t1); - assertNotEquals(t2, t1_b); - - assertNull(map.put(t1, "t1")); - assertEquals(map.get(t1), "t1"); - assertEquals(map.get(t1_b), "t1"); - assertNull(map.get(t2)); - - assertEquals(map.remove(t1_b), "t1"); - assertNull(map.get(t1)); - assertNull(map.get(t1_b)); - } - - @Test - public void testNullValue() { - ConcurrentOpenHashMap map = - ConcurrentOpenHashMap.newBuilder() - .expectedItems(16) - .concurrencyLevel(1) - .build(); - String key = "a"; - assertThrows(NullPointerException.class, () -> map.put(key, null)); - - //put a null value. - assertNull(map.computeIfAbsent(key, k -> null)); - assertEquals(1, map.size()); - assertEquals(1, map.keys().size()); - assertEquals(1, map.values().size()); - assertNull(map.get(key)); - assertFalse(map.containsKey(key)); - - //test remove null value - map.removeNullValue(key); - assertTrue(map.isEmpty()); - assertEquals(0, map.keys().size()); - assertEquals(0, map.values().size()); - assertNull(map.get(key)); - assertFalse(map.containsKey(key)); - - - //test not remove non-null value - map.put(key, "V"); - assertEquals(1, map.size()); - map.removeNullValue(key); - assertEquals(1, map.size()); - - } - - static final int Iterations = 1; - static final int ReadIterations = 1000; - static final int N = 1_000_000; - - public void benchConcurrentOpenHashMap() throws Exception { - ConcurrentOpenHashMap map = ConcurrentOpenHashMap.newBuilder() - .expectedItems(N) - .concurrencyLevel(1) - .build(); - - for (long i = 0; i < Iterations; i++) { - for (int j = 0; j < N; j++) { - map.put(i, "value"); - } - - for (long h = 0; h < ReadIterations; h++) { - for (int j = 0; j < N; j++) { - map.get(i); - } - } - - for (long j = 0; j < N; j++) { - map.remove(i); - } - } - } - - public void benchConcurrentHashMap() throws Exception { - ConcurrentHashMap map = new ConcurrentHashMap(N, 0.66f, 1); - - for (long i = 0; i < Iterations; i++) { - for (int j = 0; j < N; j++) { - map.put(i, "value"); - } - - for (long h = 0; h < ReadIterations; h++) { - for (int j = 0; j < N; j++) { - map.get(i); - } - } - - for (int j = 0; j < N; j++) { - map.remove(i); - } - } - } - - void benchHashMap() { - HashMap map = new HashMap<>(N, 0.66f); - - for (long i = 0; i < Iterations; i++) { - for (int j = 0; j < N; j++) { - map.put(i, "value"); - } - - for (long h = 0; h < ReadIterations; h++) { - for (int j = 0; j < N; j++) { - map.get(i); - } - } - - for (int j = 0; j < N; j++) { - map.remove(i); - } - } - } - - public static void main(String[] args) throws Exception { - ConcurrentOpenHashMapTest t = new ConcurrentOpenHashMapTest(); - - long start = System.nanoTime(); - t.benchHashMap(); - long end = System.nanoTime(); - - System.out.println("HM: " + TimeUnit.NANOSECONDS.toMillis(end - start) + " ms"); - - start = System.nanoTime(); - t.benchConcurrentHashMap(); - end = System.nanoTime(); - - System.out.println("CHM: " + TimeUnit.NANOSECONDS.toMillis(end - start) + " ms"); - - start = System.nanoTime(); - t.benchConcurrentOpenHashMap(); - end = System.nanoTime(); - - System.out.println("CLHM: " + TimeUnit.NANOSECONDS.toMillis(end - start) + " ms"); - - } -} diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSetTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSetTest.java deleted file mode 100644 index d509002e21998..0000000000000 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenHashSetTest.java +++ /dev/null @@ -1,503 +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.collections; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNotEquals; -import static org.testng.Assert.assertThrows; -import static org.testng.Assert.assertTrue; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Random; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicReference; - -import lombok.Cleanup; -import org.testng.annotations.Test; - -import com.google.common.collect.Lists; - -// Deprecation warning suppressed as this test targets deprecated class -@SuppressWarnings("deprecation") -public class ConcurrentOpenHashSetTest { - - @Test - public void testConstructor() { - assertThrows(IllegalArgumentException.class, () -> new ConcurrentOpenHashSet(0)); - assertThrows(IllegalArgumentException.class, () -> new ConcurrentOpenHashSet(16, 0)); - assertThrows(IllegalArgumentException.class, () -> new ConcurrentOpenHashSet(4, 8)); - } - - @Test - public void simpleInsertions() { - ConcurrentOpenHashSet set = new ConcurrentOpenHashSet<>(16); - - assertTrue(set.isEmpty()); - assertTrue(set.add("1")); - assertFalse(set.isEmpty()); - - assertTrue(set.add("2")); - assertTrue(set.add("3")); - - assertEquals(set.size(), 3); - - assertTrue(set.contains("1")); - assertEquals(set.size(), 3); - - assertTrue(set.remove("1")); - assertEquals(set.size(), 2); - assertFalse(set.contains("1")); - assertFalse(set.contains("5")); - assertEquals(set.size(), 2); - - assertTrue(set.add("1")); - assertEquals(set.size(), 3); - assertFalse(set.add("1")); - assertEquals(set.size(), 3); - } - - @Test - public void testReduceUnnecessaryExpansions() { - ConcurrentOpenHashSet set = - ConcurrentOpenHashSet.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .build(); - - assertTrue(set.add("1")); - assertTrue(set.add("2")); - assertTrue(set.add("3")); - assertTrue(set.add("4")); - - assertTrue(set.remove("1")); - assertTrue(set.remove("2")); - assertTrue(set.remove("3")); - assertTrue(set.remove("4")); - assertEquals(0, set.getUsedBucketCount()); - } - - @Test - public void testClear() { - ConcurrentOpenHashSet set = - ConcurrentOpenHashSet.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - assertEquals(set.capacity(), 4); - - assertTrue(set.add("k1")); - assertTrue(set.add("k2")); - assertTrue(set.add("k3")); - - assertEquals(set.capacity(), 8); - set.clear(); - assertEquals(set.capacity(), 4); - } - - @Test - public void testExpandAndShrink() { - ConcurrentOpenHashSet map = - ConcurrentOpenHashSet.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - assertEquals(map.capacity(), 4); - - assertTrue(map.add("k1")); - assertTrue(map.add("k2")); - assertTrue(map.add("k3")); - - // expand hashmap - assertEquals(map.capacity(), 8); - - assertTrue(map.remove("k1")); - // not shrink - assertEquals(map.capacity(), 8); - assertTrue(map.remove("k2")); - // shrink hashmap - assertEquals(map.capacity(), 4); - - // expand hashmap - assertTrue(map.add("k4")); - assertTrue(map.add("k5")); - assertEquals(map.capacity(), 8); - - //verify that the map does not keep shrinking at every remove() operation - assertTrue(map.add("k6")); - assertTrue(map.remove("k6")); - assertEquals(map.capacity(), 8); - } - - @Test - public void testExpandShrinkAndClear() { - ConcurrentOpenHashSet map = ConcurrentOpenHashSet.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - final long initCapacity = map.capacity(); - assertTrue(map.capacity() == 4); - - assertTrue(map.add("k1")); - assertTrue(map.add("k2")); - assertTrue(map.add("k3")); - - // expand hashmap - assertTrue(map.capacity() == 8); - - assertTrue(map.remove("k1")); - // not shrink - assertTrue(map.capacity() == 8); - assertTrue(map.remove("k2")); - // shrink hashmap - assertTrue(map.capacity() == 4); - - assertTrue(map.remove("k3")); - // Will not shrink the hashmap again because shrink capacity is less than initCapacity - // current capacity is equal than the initial capacity - assertTrue(map.capacity() == initCapacity); - map.clear(); - // after clear, because current capacity is equal than the initial capacity, so not shrinkToInitCapacity - assertTrue(map.capacity() == initCapacity); - } - - @Test - public void testConcurrentExpandAndShrinkAndGet() throws Throwable { - ConcurrentOpenHashSet set = ConcurrentOpenHashSet.newBuilder() - .expectedItems(2) - .concurrencyLevel(1) - .autoShrink(true) - .mapIdleFactor(0.25f) - .build(); - assertEquals(set.capacity(), 4); - - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - final int readThreads = 16; - final int writeThreads = 1; - final int n = 1_000; - CyclicBarrier barrier = new CyclicBarrier(writeThreads + readThreads); - Future future = null; - AtomicReference ex = new AtomicReference<>(); - - for (int i = 0; i < readThreads; i++) { - executor.submit(() -> { - try { - barrier.await(); - } catch (Exception e) { - throw new RuntimeException(e); - } - while (!Thread.currentThread().isInterrupted()) { - try { - set.contains("k2"); - } catch (Exception e) { - ex.set(e); - } - } - }); - } - - assertTrue(set.add("k1")); - future = executor.submit(() -> { - try { - barrier.await(); - } catch (Exception e) { - throw new RuntimeException(e); - } - - for (int i = 0; i < n; i++) { - // expand hashmap - assertTrue(set.add("k2")); - assertTrue(set.add("k3")); - assertEquals(set.capacity(), 8); - - // shrink hashmap - assertTrue(set.remove("k2")); - assertTrue(set.remove("k3")); - assertEquals(set.capacity(), 4); - } - }); - - future.get(); - assertTrue(ex.get() == null); - // shut down pool - executor.shutdown(); - } - - @Test - public void testRemove() { - ConcurrentOpenHashSet set = - ConcurrentOpenHashSet.newBuilder().build(); - - assertTrue(set.isEmpty()); - assertTrue(set.add("1")); - assertFalse(set.isEmpty()); - - assertFalse(set.remove("0")); - assertFalse(set.isEmpty()); - assertTrue(set.remove("1")); - assertTrue(set.isEmpty()); - } - - @Test - public void testRehashing() { - int n = 16; - ConcurrentOpenHashSet set = new ConcurrentOpenHashSet<>(n / 2, 1); - assertEquals(set.capacity(), n); - assertEquals(set.size(), 0); - - for (int i = 0; i < n; i++) { - set.add(i); - } - - assertEquals(set.capacity(), 2 * n); - assertEquals(set.size(), n); - } - - @Test - public void testRehashingWithDeletes() { - int n = 16; - ConcurrentOpenHashSet set = new ConcurrentOpenHashSet<>(n / 2, 1); - assertEquals(set.capacity(), n); - assertEquals(set.size(), 0); - - for (int i = 0; i < n / 2; i++) { - set.add(i); - } - - for (int i = 0; i < n / 2; i++) { - set.remove(i); - } - - for (int i = n; i < (2 * n); i++) { - set.add(i); - } - - assertEquals(set.capacity(), 2 * n); - assertEquals(set.size(), n); - } - - @Test - public void concurrentInsertions() throws Throwable { - ConcurrentOpenHashSet set = - ConcurrentOpenHashSet.newBuilder().build(); - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - - final int nThreads = 16; - final int N = 100_000; - - List> futures = new ArrayList<>(); - for (int i = 0; i < nThreads; i++) { - final int threadIdx = i; - - futures.add(executor.submit(() -> { - Random random = new Random(); - - for (int j = 0; j < N; j++) { - long key = random.nextLong(); - // Ensure keys are unique - key -= key % (threadIdx + 1); - - set.add(key); - } - })); - } - - for (Future future : futures) { - future.get(); - } - - assertEquals(set.size(), N * nThreads); - } - - @Test - public void concurrentInsertionsAndReads() throws Throwable { - ConcurrentOpenHashSet map = - ConcurrentOpenHashSet.newBuilder().build(); - @Cleanup("shutdownNow") - ExecutorService executor = Executors.newCachedThreadPool(); - - final int nThreads = 16; - final int N = 100_000; - - List> futures = new ArrayList<>(); - for (int i = 0; i < nThreads; i++) { - final int threadIdx = i; - - futures.add(executor.submit(() -> { - Random random = new Random(); - - for (int j = 0; j < N; j++) { - long key = random.nextLong(); - // Ensure keys are unique - key -= key % (threadIdx + 1); - - map.add(key); - } - })); - } - - for (Future future : futures) { - future.get(); - } - - assertEquals(map.size(), N * nThreads); - } - - @Test - public void testIteration() { - ConcurrentOpenHashSet set = ConcurrentOpenHashSet.newBuilder().build(); - - assertEquals(set.values(), Collections.emptyList()); - - set.add(0l); - - assertEquals(set.values(), Lists.newArrayList(0l)); - - set.remove(0l); - - assertEquals(set.values(), Collections.emptyList()); - - set.add(0l); - set.add(1l); - set.add(2l); - - List values = set.values(); - values.sort(null); - assertEquals(values, Lists.newArrayList(0l, 1l, 2l)); - - set.clear(); - assertTrue(set.isEmpty()); - } - - @Test - public void testRemoval() { - ConcurrentOpenHashSet set = - ConcurrentOpenHashSet.newBuilder().build(); - - set.add(0); - set.add(1); - set.add(3); - set.add(6); - set.add(7); - - List values = set.values(); - values.sort(null); - assertEquals(values, Lists.newArrayList(0, 1, 3, 6, 7)); - - int numOfItemsDeleted = set.removeIf(i -> i < 5); - assertEquals(numOfItemsDeleted, 3); - assertEquals(set.size(), values.size() - numOfItemsDeleted); - values = set.values(); - values.sort(null); - assertEquals(values, Lists.newArrayList(6, 7)); - } - - @Test - public void testHashConflictWithDeletion() { - final int Buckets = 16; - ConcurrentOpenHashSet set = new ConcurrentOpenHashSet<>(Buckets, 1); - - // Pick 2 keys that fall into the same bucket - long key1 = 1; - long key2 = 27; - - int bucket1 = ConcurrentOpenHashSet.signSafeMod(ConcurrentOpenHashSet.hash(key1), Buckets); - int bucket2 = ConcurrentOpenHashSet.signSafeMod(ConcurrentOpenHashSet.hash(key2), Buckets); - assertEquals(bucket1, bucket2); - - assertTrue(set.add(key1)); - assertTrue(set.add(key2)); - assertEquals(set.size(), 2); - - assertTrue(set.remove(key1)); - assertEquals(set.size(), 1); - - assertTrue(set.add(key1)); - assertEquals(set.size(), 2); - - assertTrue(set.remove(key1)); - assertEquals(set.size(), 1); - - assertFalse(set.add(key2)); - assertTrue(set.contains(key2)); - - assertEquals(set.size(), 1); - assertTrue(set.remove(key2)); - assertTrue(set.isEmpty()); - } - - @Test - public void testEqualsObjects() { - class T { - int value; - - T(int value) { - this.value = value; - } - - @Override - public int hashCode() { - return Integer.hashCode(value); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof T) { - return value == ((T) obj).value; - } - - return false; - } - } - - ConcurrentOpenHashSet set = - ConcurrentOpenHashSet.newBuilder().build(); - - T t1 = new T(1); - T t1_b = new T(1); - T t2 = new T(2); - - assertEquals(t1, t1_b); - assertNotEquals(t2, t1); - assertNotEquals(t2, t1_b); - - set.add(t1); - assertTrue(set.contains(t1)); - assertTrue(set.contains(t1_b)); - assertFalse(set.contains(t2)); - - assertTrue(set.remove(t1_b)); - assertFalse(set.contains(t1)); - assertFalse(set.contains(t1_b)); - } - -} diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java index 889f4431cc35b..7bb4df7baa533 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/WebSocketService.java @@ -23,7 +23,10 @@ import java.io.Closeable; import java.io.IOException; import java.net.MalformedURLException; +import java.util.Map; import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -44,8 +47,6 @@ import org.apache.pulsar.client.internal.PropertiesUtils; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @@ -73,9 +74,9 @@ public class WebSocketService implements Closeable { private Optional cryptoKeyReader = Optional.empty(); private ClusterData localCluster; - private final ConcurrentOpenHashMap> topicProducerMap; - private final ConcurrentOpenHashMap> topicConsumerMap; - private final ConcurrentOpenHashMap> topicReaderMap; + private final Map> topicProducerMap = new ConcurrentHashMap<>(); + private final Map> topicConsumerMap = new ConcurrentHashMap<>(); + private final Map> topicReaderMap = new ConcurrentHashMap<>(); private final ProxyStats proxyStats; public WebSocketService(WebSocketProxyConfiguration config) { @@ -88,17 +89,6 @@ public WebSocketService(ClusterData localCluster, ServiceConfiguration config) { .newScheduledThreadPool(config.getWebSocketNumServiceThreads(), new DefaultThreadFactory("pulsar-websocket")); this.localCluster = localCluster; - this.topicProducerMap = - ConcurrentOpenHashMap.>newBuilder() - .build(); - this.topicConsumerMap = - ConcurrentOpenHashMap.>newBuilder() - .build(); - this.topicReaderMap = - ConcurrentOpenHashMap.>newBuilder() - .build(); this.proxyStats = new ProxyStats(this); } @@ -288,11 +278,11 @@ public boolean isAuthorizationEnabled() { public boolean addProducer(ProducerHandler producer) { return topicProducerMap .computeIfAbsent(producer.getProducer().getTopic(), - topic -> ConcurrentOpenHashSet.newBuilder().build()) + topic -> ConcurrentHashMap.newKeySet()) .add(producer); } - public ConcurrentOpenHashMap> getProducers() { + public Map> getProducers() { return topicProducerMap; } @@ -306,12 +296,11 @@ public boolean removeProducer(ProducerHandler producer) { public boolean addConsumer(ConsumerHandler consumer) { return topicConsumerMap - .computeIfAbsent(consumer.getConsumer().getTopic(), topic -> - ConcurrentOpenHashSet.newBuilder().build()) + .computeIfAbsent(consumer.getConsumer().getTopic(), topic -> ConcurrentHashMap.newKeySet()) .add(consumer); } - public ConcurrentOpenHashMap> getConsumers() { + public Map> getConsumers() { return topicConsumerMap; } @@ -324,12 +313,11 @@ public boolean removeConsumer(ConsumerHandler consumer) { } public boolean addReader(ReaderHandler reader) { - return topicReaderMap.computeIfAbsent(reader.getConsumer().getTopic(), topic -> - ConcurrentOpenHashSet.newBuilder().build()) + return topicReaderMap.computeIfAbsent(reader.getConsumer().getTopic(), topic -> ConcurrentHashMap.newKeySet()) .add(reader); } - public ConcurrentOpenHashMap> getReaders() { + public Map> getReaders() { return topicReaderMap; } diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/stats/ProxyStats.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/stats/ProxyStats.java index eb1566ef7d412..4660340e9cc54 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/stats/ProxyStats.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/stats/ProxyStats.java @@ -24,11 +24,11 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.stats.JvmMetrics; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.websocket.WebSocketService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,7 +41,7 @@ public class ProxyStats { private final WebSocketService service; private final JvmMetrics jvmMetrics; - private ConcurrentOpenHashMap topicStats; + private final Map topicStats = new ConcurrentHashMap<>(); private List metricsCollection; private List tempMetricsCollection; @@ -50,9 +50,6 @@ public ProxyStats(WebSocketService service) { this.service = service; this.jvmMetrics = JvmMetrics.create( service.getExecutor(), "prx", service.getConfig().getJvmGCMetricsLoggerClassName()); - this.topicStats = - ConcurrentOpenHashMap.newBuilder() - .build(); this.metricsCollection = new ArrayList<>(); this.tempMetricsCollection = new ArrayList<>(); // schedule stat generation task every 1 minute From 216b83008deb469e0fc55ed8117f0c393ebcb0ac Mon Sep 17 00:00:00 2001 From: maheshnikam <55378196+nikam14@users.noreply.github.com> Date: Mon, 23 Sep 2024 14:17:31 +0530 Subject: [PATCH 518/580] [fix][test] Fix flaky test LeaderElectionTest.revalidateLeaderWithinSameSession (#22383) --- .../java/org/apache/pulsar/metadata/api/MetadataCache.java | 2 +- .../org/apache/pulsar/metadata/BaseMetadataStoreTest.java | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java index 8e153b23d3087..4af712d33571e 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java @@ -59,7 +59,7 @@ public interface MetadataCache { * * @param path * the path of the object in the metadata store - * @return the cached object or an empty {@link Optional} is the cache doesn't have the object + * @return the cached object or an empty {@link Optional} is the cache does not have the object */ Optional getIfCached(String path); diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java index c77de92ae3c4c..d0265e3ca44ee 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/BaseMetadataStoreTest.java @@ -145,10 +145,11 @@ public static void assertEqualsAndRetry(Supplier actual, int retryCount, long intSleepTimeInMillis) throws Exception { assertTrue(retryStrategically((__) -> { - if (actual.get().equals(expectedAndRetry)) { + Object actualObject = actual.get(); + if (actualObject.equals(expectedAndRetry)) { return false; } - assertEquals(actual.get(), expected); + assertEquals(actualObject, expected); return true; }, retryCount, intSleepTimeInMillis)); } From 7d4ac9dc542ff1e840f4e520836b6a3c49c6338d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 23 Sep 2024 15:00:52 +0300 Subject: [PATCH 519/580] [fix][build] Fix problem where git.commit.id.abbrev is missing in image tagging (#23337) --- docker/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/pom.xml b/docker/pom.xml index a5ea238241c6a..ffcaec3ffdc30 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -68,7 +68,6 @@ false true - true false From 501dfdeace9ef321acbdc5ce32d98eb3e56e083a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 23 Sep 2024 16:37:33 +0300 Subject: [PATCH 520/580] [fix][sec] Upgrade vertx to 4.5.10 to address CVE-2024-8391 (#23338) --- distribution/server/src/assemble/LICENSE.bin.txt | 10 +++++----- pom.xml | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 579613b0d8f2f..1d78913849bda 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -491,11 +491,11 @@ The Apache Software License, Version 2.0 * JCTools - Java Concurrency Tools for the JVM - org.jctools-jctools-core-2.1.2.jar * Vertx - - io.vertx-vertx-auth-common-4.5.8.jar - - io.vertx-vertx-bridge-common-4.5.8.jar - - io.vertx-vertx-core-4.5.8.jar - - io.vertx-vertx-web-4.5.8.jar - - io.vertx-vertx-web-common-4.5.8.jar + - io.vertx-vertx-auth-common-4.5.10.jar + - io.vertx-vertx-bridge-common-4.5.10.jar + - io.vertx-vertx-core-4.5.10.jar + - io.vertx-vertx-web-4.5.10.jar + - io.vertx-vertx-web-common-4.5.10.jar * Apache ZooKeeper - org.apache.zookeeper-zookeeper-3.9.2.jar - org.apache.zookeeper-zookeeper-jute-3.9.2.jar diff --git a/pom.xml b/pom.xml index dda7f316acc6f..1f7ecd3b90c1e 100644 --- a/pom.xml +++ b/pom.xml @@ -155,7 +155,7 @@ flexible messaging model and an intuitive client API. 2.42 1.10.50 0.16.0 - 4.5.8 + 4.5.10 7.9.2 2.0.13 4.4 From 4ce0c752cc4b2d6dccb818ab0ffa854e82e42b85 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 24 Sep 2024 03:50:01 +0800 Subject: [PATCH 521/580] [fix] Key_Shared mode consumption latency when low traffic (#23340) Co-authored-by: Lari Hotari --- conf/broker.conf | 4 ++-- conf/standalone.conf | 4 ++-- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 4 ++-- .../persistent/PersistentDispatcherMultipleConsumers.java | 6 ++++-- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 125b2aa8c1b39..617e202e5ec65 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -489,12 +489,12 @@ dispatcherReadFailureBackoffMandatoryStopTimeInMs=0 # On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered # out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff # delay. This parameter sets the initial backoff delay in milliseconds. -dispatcherRetryBackoffInitialTimeInMs=100 +dispatcherRetryBackoffInitialTimeInMs=1 # On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered # out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff # delay. This parameter sets the maximum backoff delay in milliseconds. -dispatcherRetryBackoffMaxTimeInMs=1000 +dispatcherRetryBackoffMaxTimeInMs=10 # Precise dispatcher flow control according to history message number of each entry preciseDispatcherFlowControl=false diff --git a/conf/standalone.conf b/conf/standalone.conf index 622949bf6c325..535800a43f3e0 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -305,12 +305,12 @@ dispatcherReadFailureBackoffMandatoryStopTimeInMs=0 # On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered # out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff # delay. This parameter sets the initial backoff delay in milliseconds. -dispatcherRetryBackoffInitialTimeInMs=100 +dispatcherRetryBackoffInitialTimeInMs=1 # On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered # out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff # delay. This parameter sets the maximum backoff delay in milliseconds. -dispatcherRetryBackoffMaxTimeInMs=1000 +dispatcherRetryBackoffMaxTimeInMs=10 # Precise dispatcher flow control according to history message number of each entry preciseDispatcherFlowControl=false 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 486587ec174a0..33b4fbff5f5bb 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 @@ -1231,14 +1231,14 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece doc = "On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered " + "out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff " + "delay. This parameter sets the initial backoff delay in milliseconds.") - private int dispatcherRetryBackoffInitialTimeInMs = 100; + private int dispatcherRetryBackoffInitialTimeInMs = 1; @FieldContext( category = CATEGORY_POLICIES, doc = "On Shared and KeyShared subscriptions, if all available messages in the subscription are filtered " + "out and not dispatched to any consumer, message dispatching will be rescheduled with a backoff " + "delay. This parameter sets the maximum backoff delay in milliseconds.") - private int dispatcherRetryBackoffMaxTimeInMs = 1000; + private int dispatcherRetryBackoffMaxTimeInMs = 10; @FieldContext( dynamic = true, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 450a446c85a78..8fdb65e7b3076 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -729,11 +729,13 @@ private synchronized void handleSendingMessagesAndReadingMore(ReadType readType, boolean triggerReadingMore = sendMessagesToConsumers(readType, entries, needAcquireSendInProgress); int entriesDispatched = lastNumberOfEntriesDispatched; updatePendingBytesToDispatch(-totalBytesSize); + if (entriesDispatched > 0) { + // Reset the backoff when we successfully dispatched messages + retryBackoff.reset(); + } if (triggerReadingMore) { if (entriesDispatched > 0 || skipNextBackoff) { skipNextBackoff = false; - // Reset the backoff when we successfully dispatched messages - retryBackoff.reset(); // Call readMoreEntries in the same thread to trigger the next read readMoreEntries(); } else if (entriesDispatched == 0) { From 5ea4252a493c5d93046cfc3aeb1977814bc64a41 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 25 Sep 2024 08:28:21 +0300 Subject: [PATCH 522/580] [fix][broker] Avoid introducing delay when there are delayed messages or marker messages (#23343) --- ...PersistentDispatcherMultipleConsumers.java | 34 +++++--- ...tStickyKeyDispatcherMultipleConsumers.java | 11 ++- ...ckyKeyDispatcherMultipleConsumersTest.java | 82 +++++++++++++++++++ 3 files changed, 112 insertions(+), 15 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 8fdb65e7b3076..73ad2cf0a3dee 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -134,7 +134,11 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul private AtomicBoolean isRescheduleReadInProgress = new AtomicBoolean(false); protected final ExecutorService dispatchMessagesThread; private final SharedConsumerAssignor assignor; - protected int lastNumberOfEntriesDispatched; + // tracks how many entries were processed by consumers in the last trySendMessagesToConsumers call + // the number includes also delayed messages, marker messages, aborted txn messages and filtered messages + // When no messages were processed, the value is 0. This is also an indication that the dispatcher didn't + // make progress in the last trySendMessagesToConsumers call. + protected int lastNumberOfEntriesProcessed; protected boolean skipNextBackoff; private final Backoff retryBackoff; protected enum ReadType { @@ -727,19 +731,22 @@ private synchronized void handleSendingMessagesAndReadingMore(ReadType readType, boolean needAcquireSendInProgress, long totalBytesSize) { boolean triggerReadingMore = sendMessagesToConsumers(readType, entries, needAcquireSendInProgress); - int entriesDispatched = lastNumberOfEntriesDispatched; + int entriesProcessed = lastNumberOfEntriesProcessed; updatePendingBytesToDispatch(-totalBytesSize); - if (entriesDispatched > 0) { - // Reset the backoff when we successfully dispatched messages + boolean canReadMoreImmediately = false; + if (entriesProcessed > 0 || skipNextBackoff) { + // Reset the backoff when messages were processed retryBackoff.reset(); + // Reset the possible flag to skip the backoff delay + skipNextBackoff = false; + canReadMoreImmediately = true; } if (triggerReadingMore) { - if (entriesDispatched > 0 || skipNextBackoff) { - skipNextBackoff = false; + if (canReadMoreImmediately) { // Call readMoreEntries in the same thread to trigger the next read readMoreEntries(); - } else if (entriesDispatched == 0) { - // If no messages were dispatched, we need to reschedule a new read with an increasing backoff delay + } else { + // reschedule a new read with an increasing backoff delay reScheduleReadWithBackoff(); } } @@ -779,7 +786,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis if (needTrimAckedMessages()) { cursor.trimDeletedEntries(entries); } - lastNumberOfEntriesDispatched = 0; + lastNumberOfEntriesProcessed = 0; int entriesToDispatch = entries.size(); // Trigger read more messages @@ -809,6 +816,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis long totalMessagesSent = 0; long totalBytesSent = 0; long totalEntries = 0; + long totalEntriesProcessed = 0; int avgBatchSizePerMsg = remainingMessages > 0 ? Math.max(remainingMessages / entries.size(), 1) : 1; // If the dispatcher is closed, firstAvailableConsumerPermits will be 0, which skips dispatching the @@ -820,6 +828,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis log.info("[{}] rewind because no available consumer found from total {}", name, consumerList.size()); entries.subList(start, entries.size()).forEach(Entry::release); cursor.rewind(); + lastNumberOfEntriesProcessed = (int) totalEntriesProcessed; return false; } @@ -863,6 +872,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis totalEntries += filterEntriesForConsumer(metadataArray, start, entriesForThisConsumer, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, readType == ReadType.Replay, c); + totalEntriesProcessed += entriesForThisConsumer.size(); c.sendMessages(entriesForThisConsumer, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), redeliveryTracker); @@ -882,7 +892,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis totalBytesSent += sendMessageInfo.getTotalBytes(); } - lastNumberOfEntriesDispatched = (int) totalEntries; + lastNumberOfEntriesProcessed = (int) totalEntriesProcessed; acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); if (entriesToDispatch > 0) { @@ -917,6 +927,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, long totalMessagesSent = 0; long totalBytesSent = 0; long totalEntries = 0; + long totalEntriesProcessed = 0; final AtomicInteger numConsumers = new AtomicInteger(assignResult.size()); for (Map.Entry> current : assignResult.entrySet()) { final Consumer consumer = current.getKey(); @@ -947,6 +958,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, totalEntries += filterEntriesForConsumer(entryAndMetadataList, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); + totalEntriesProcessed += entryAndMetadataList.size(); consumer.sendMessages(entryAndMetadataList, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), getRedeliveryTracker() @@ -962,7 +974,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, totalBytesSent += sendMessageInfo.getTotalBytes(); } - lastNumberOfEntriesDispatched = (int) totalEntries; + lastNumberOfEntriesProcessed = (int) totalEntriesProcessed; acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); return numConsumers.get() == 0; // trigger a new readMoreEntries() call diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 26463ba902c58..ecd3f19a14028 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -190,10 +190,11 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE @Override protected synchronized boolean trySendMessagesToConsumers(ReadType readType, List entries) { - lastNumberOfEntriesDispatched = 0; + lastNumberOfEntriesProcessed = 0; long totalMessagesSent = 0; long totalBytesSent = 0; long totalEntries = 0; + long totalEntriesProcessed = 0; int entriesCount = entries.size(); // Trigger read more messages @@ -233,6 +234,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } else if (readType == ReadType.Replay) { entries.forEach(Entry::release); } + skipNextBackoff = true; return true; } } @@ -298,6 +300,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(entriesForConsumer.size()); totalEntries += filterEntriesForConsumer(entriesForConsumer, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); + totalEntriesProcessed += entriesForConsumer.size(); consumer.sendMessages(entriesForConsumer, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), @@ -368,7 +371,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } } - lastNumberOfEntriesDispatched = (int) totalEntries; + lastNumberOfEntriesProcessed = (int) totalEntriesProcessed; // acquire message-dispatch permits for already delivered messages acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); @@ -387,8 +390,8 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis return true; } - // if no messages were sent, we should retry after a backoff delay - if (entriesByConsumerForDispatching.size() == 0) { + // if no messages were sent to consumers, we should retry + if (totalEntries == 0) { return true; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index b78d1e554c32d..dcd852f409dbb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -46,6 +46,8 @@ import io.netty.buffer.Unpooled; import io.netty.channel.ChannelPromise; import io.netty.channel.EventLoopGroup; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Arrays; @@ -996,6 +998,86 @@ protected void reScheduleReadInMs(long readAfterMs) { ); } + + @Test(dataProvider = "testBackoffDelayWhenNoMessagesDispatched") + public void testNoBackoffDelayWhenDelayedMessages(boolean dispatchMessagesInSubscriptionThread, boolean isKeyShared) + throws Exception { + persistentDispatcher.close(); + + doReturn(dispatchMessagesInSubscriptionThread).when(configMock) + .isDispatcherDispatchMessagesInSubscriptionThread(); + + AtomicInteger readMoreEntriesCalled = new AtomicInteger(0); + AtomicInteger reScheduleReadInMsCalled = new AtomicInteger(0); + AtomicBoolean delayAllMessages = new AtomicBoolean(true); + + PersistentDispatcherMultipleConsumers dispatcher; + if (isKeyShared) { + dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers( + topicMock, cursorMock, subscriptionMock, configMock, + new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)) { + @Override + protected void reScheduleReadInMs(long readAfterMs) { + reScheduleReadInMsCalled.incrementAndGet(); + } + + @Override + public synchronized void readMoreEntries() { + readMoreEntriesCalled.incrementAndGet(); + } + + @Override + public boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata msgMetadata) { + if (delayAllMessages.get()) { + // simulate delayed message + return true; + } + return super.trackDelayedDelivery(ledgerId, entryId, msgMetadata); + } + }; + } else { + dispatcher = new PersistentDispatcherMultipleConsumers(topicMock, cursorMock, subscriptionMock) { + @Override + protected void reScheduleReadInMs(long readAfterMs) { + reScheduleReadInMsCalled.incrementAndGet(); + } + + @Override + public synchronized void readMoreEntries() { + readMoreEntriesCalled.incrementAndGet(); + } + + @Override + public boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata msgMetadata) { + if (delayAllMessages.get()) { + // simulate delayed message + return true; + } + return super.trackDelayedDelivery(ledgerId, entryId, msgMetadata); + } + }; + } + + doAnswer(invocationOnMock -> { + GenericFutureListener> listener = invocationOnMock.getArgument(0); + Future future = mock(Future.class); + when(future.isDone()).thenReturn(true); + listener.operationComplete(future); + return channelMock; + }).when(channelMock).addListener(any()); + + // add a consumer with permits + consumerMockAvailablePermits.set(1000); + dispatcher.addConsumer(consumerMock); + + List entries = new ArrayList<>(List.of(EntryImpl.create(1, 1, createMessage("message1", 1)))); + dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + Awaitility.await().untilAsserted(() -> { + assertEquals(reScheduleReadInMsCalled.get(), 0, "reScheduleReadInMs should not be called"); + assertTrue(readMoreEntriesCalled.get() >= 1); + }); + } + private ByteBuf createMessage(String message, int sequenceId) { return createMessage(message, sequenceId, "testKey"); } From cefa72c7d18ab05d413e8737672f746629874f59 Mon Sep 17 00:00:00 2001 From: maheshnikam <55378196+nikam14@users.noreply.github.com> Date: Wed, 25 Sep 2024 20:15:12 +0530 Subject: [PATCH 523/580] [improve][schema] Improve Incompatible Schema Exception error message in ServerCnx.java (#23344) --- .../java/org/apache/pulsar/broker/service/ServerCnx.java | 5 +++-- 1 file changed, 3 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 7d196ad30235b..5b67b01115e7c 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 @@ -3052,7 +3052,8 @@ private CompletableFuture tryAddSchema(Topic topic, SchemaData sc CompletableFuture result = new CompletableFuture<>(); if (hasSchema && (schemaValidationEnforced || topic.getSchemaValidationEnforced())) { result.completeExceptionally(new IncompatibleSchemaException( - "Producers cannot connect or send message without a schema to topics with a schema")); + "Producers cannot connect or send message without a schema to topics with a schema" + + "when SchemaValidationEnforced is enabled")); } else { result.complete(SchemaVersion.Empty); } @@ -3727,4 +3728,4 @@ public void incrementThrottleCount() { public void decrementThrottleCount() { throttleTracker.decrementThrottleCount(); } -} \ No newline at end of file +} From 31f27a091920dfcdd9ae44a1c738d701294b318b Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Wed, 25 Sep 2024 14:01:08 -0700 Subject: [PATCH 524/580] [fix][cli] Fix Pulsar-Client to allow consume encrypted messages with appropriate crypto-failure-action (#23346) --- .../main/java/org/apache/pulsar/client/cli/CmdConsume.java | 5 +++++ .../src/main/java/org/apache/pulsar/client/cli/CmdRead.java | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java index 71c172b633713..46adc45e2ea4d 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.api.AuthenticationDataProvider; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; @@ -111,6 +112,9 @@ public class CmdConsume extends AbstractCmdConsume { @Option(names = {"-rs", "--replicated" }, description = "Whether the subscription status should be replicated") private boolean replicateSubscriptionState = false; + @Option(names = { "-ca", "--crypto-failure-action" }, description = "Crypto Failure Action") + private ConsumerCryptoFailureAction cryptoFailureAction = ConsumerCryptoFailureAction.FAIL; + public CmdConsume() { // Do nothing super(); @@ -174,6 +178,7 @@ private int consume(String topic) { } builder.autoAckOldestChunkedMessageOnQueueFull(this.autoAckOldestChunkedMessageOnQueueFull); + builder.cryptoFailureAction(cryptoFailureAction); if (isNotBlank(this.encKeyValue)) { builder.defaultCryptoKeyReader(this.encKeyValue); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java index daab436499219..51bf2d6898b6b 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java @@ -30,6 +30,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.pulsar.client.api.AuthenticationDataProvider; +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; @@ -101,6 +102,9 @@ public class CmdRead extends AbstractCmdConsume { @Option(names = { "-pm", "--pool-messages" }, description = "Use the pooled message", arity = "1") private boolean poolMessages = true; + @Option(names = { "-ca", "--crypto-failure-action" }, description = "Crypto Failure Action") + private ConsumerCryptoFailureAction cryptoFailureAction = ConsumerCryptoFailureAction.FAIL; + public CmdRead() { // Do nothing super(); @@ -153,6 +157,7 @@ private int read(String topic) { } builder.autoAckOldestChunkedMessageOnQueueFull(this.autoAckOldestChunkedMessageOnQueueFull); + builder.cryptoFailureAction(cryptoFailureAction); if (isNotBlank(this.encKeyValue)) { builder.defaultCryptoKeyReader(this.encKeyValue); From b1c5d96ab480ba1b8ad8cbe2077cbe4c467dfc61 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Wed, 25 Sep 2024 15:30:55 -0700 Subject: [PATCH 525/580] [fix][cli] Fix Pulsar-Client CLI to print metadata of message including encryption metadata (#23347) --- .../pulsar/client/cli/AbstractCmdConsume.java | 45 ++++++++++++++++++- .../apache/pulsar/client/cli/CmdConsume.java | 5 ++- .../org/apache/pulsar/client/cli/CmdRead.java | 5 ++- 3 files changed, 52 insertions(+), 3 deletions(-) diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/AbstractCmdConsume.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/AbstractCmdConsume.java index 658b34767b594..33df4aca96d2d 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/AbstractCmdConsume.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/AbstractCmdConsume.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Base64; import java.util.HashMap; import java.util.Map; import java.util.concurrent.BlockingQueue; @@ -40,7 +41,9 @@ import org.apache.pulsar.client.api.schema.Field; import org.apache.pulsar.client.api.schema.GenericObject; import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.common.api.EncryptionContext; import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.collections.GrowableArrayBlockingQueue; import org.eclipse.jetty.websocket.api.RemoteEndpoint; import org.eclipse.jetty.websocket.api.Session; @@ -87,7 +90,8 @@ public void updateConfig(ClientBuilder clientBuilder, Authentication authenticat * Whether to display BytesMessages in hexdump style, ignored for simple text messages * @return String representation of the message */ - protected String interpretMessage(Message message, boolean displayHex) throws IOException { + protected String interpretMessage(Message message, boolean displayHex, boolean printMetadata) + throws IOException { StringBuilder sb = new StringBuilder(); String properties = Arrays.toString(message.getProperties().entrySet().toArray()); @@ -122,6 +126,45 @@ protected String interpretMessage(Message message, boolean displayHex) throws } sb.append("content:").append(data); + if (printMetadata) { + if (message.getEncryptionCtx().isPresent()) { + EncryptionContext encContext = message.getEncryptionCtx().get(); + if (encContext.getKeys() != null && !encContext.getKeys().isEmpty()) { + sb.append(", "); + sb.append("encryption-keys:").append(", "); + encContext.getKeys().forEach((keyName, keyInfo) -> { + String metadata = Arrays.toString(keyInfo.getMetadata().entrySet().toArray()); + sb.append("name:").append(keyName).append(", ").append("key-value:") + .append(Base64.getEncoder().encode(keyInfo.getKeyValue())).append(", ") + .append("metadata:").append(metadata).append(", "); + + }); + sb.append(", ").append("param:").append(Base64.getEncoder().encode(encContext.getParam())) + .append(", ").append("algorithm:").append(encContext.getAlgorithm()).append(", ") + .append("compression-type:").append(encContext.getCompressionType()).append(", ") + .append("uncompressed-size").append(encContext.getUncompressedMessageSize()).append(", ") + .append("batch-size") + .append(encContext.getBatchSize().isPresent() ? encContext.getBatchSize().get() : 1); + } + } + if (message.hasBrokerPublishTime()) { + sb.append(", ").append("publish-time:").append(DateFormatter.format(message.getPublishTime())); + } + sb.append(", ").append("event-time:").append(DateFormatter.format(message.getEventTime())); + sb.append(", ").append("message-id:").append(message.getMessageId()); + sb.append(", ").append("producer-name:").append(message.getProducerName()); + sb.append(", ").append("sequence-id:").append(message.getSequenceId()); + sb.append(", ").append("replicated-from:").append(message.getReplicatedFrom()); + sb.append(", ").append("redelivery-count:").append(message.getRedeliveryCount()); + sb.append(", ").append("ordering-key:") + .append(message.getOrderingKey() != null ? new String(message.getOrderingKey()) : ""); + sb.append(", ").append("schema-version:") + .append(message.getSchemaVersion() != null ? new String(message.getSchemaVersion()) : ""); + if (message.hasIndex()) { + sb.append(", ").append("index:").append(message.getIndex()); + } + } + return sb.toString(); } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java index 46adc45e2ea4d..0f0e2f0a9c813 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java @@ -115,6 +115,9 @@ public class CmdConsume extends AbstractCmdConsume { @Option(names = { "-ca", "--crypto-failure-action" }, description = "Crypto Failure Action") private ConsumerCryptoFailureAction cryptoFailureAction = ConsumerCryptoFailureAction.FAIL; + @Option(names = { "-mp", "--print-metadata" }, description = "Message metadata") + private boolean printMetadata = false; + public CmdConsume() { // Do nothing super(); @@ -199,7 +202,7 @@ private int consume(String topic) { numMessagesConsumed += 1; if (!hideContent) { System.out.println(MESSAGE_BOUNDARY); - String output = this.interpretMessage(msg, displayHex); + String output = this.interpretMessage(msg, displayHex, printMetadata); System.out.println(output); } else if (numMessagesConsumed % 1000 == 0) { System.out.println("Received " + numMessagesConsumed + " messages"); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java index 51bf2d6898b6b..529d1d9c41272 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdRead.java @@ -105,6 +105,9 @@ public class CmdRead extends AbstractCmdConsume { @Option(names = { "-ca", "--crypto-failure-action" }, description = "Crypto Failure Action") private ConsumerCryptoFailureAction cryptoFailureAction = ConsumerCryptoFailureAction.FAIL; + @Option(names = { "-mp", "--print-metadata" }, description = "Message metadata") + private boolean printMetadata = false; + public CmdRead() { // Do nothing super(); @@ -178,7 +181,7 @@ private int read(String topic) { numMessagesRead += 1; if (!hideContent) { System.out.println(MESSAGE_BOUNDARY); - String output = this.interpretMessage(msg, displayHex); + String output = this.interpretMessage(msg, displayHex, printMetadata); System.out.println(output); } else if (numMessagesRead % 1000 == 0) { System.out.println("Received " + numMessagesRead + " messages"); From 5583102aae135f5f62884f83e1ddd927b24ee737 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 27 Sep 2024 16:34:19 +0800 Subject: [PATCH 526/580] [fix] [log] Do not print error log if tenant/namespace does not exist when calling get topic metadata (#23291) --- .../pulsar/broker/service/ServerCnx.java | 41 ++++++++++++--- .../admin/GetPartitionMetadataTest.java | 51 +++++++++++++++++++ 2 files changed, 85 insertions(+), 7 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 5b67b01115e7c..aedd68d416fe7 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 @@ -63,6 +63,8 @@ import java.util.stream.Collectors; import javax.naming.AuthenticationException; import javax.net.ssl.SSLSession; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Response; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedger; @@ -672,8 +674,6 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa commandSender.sendPartitionMetadataResponse(ServerError.AuthorizationError, ex.getMessage(), requestId); } else { - log.warn("Failed to get Partitioned Metadata [{}] {}: {}", remoteAddress, - topicName, ex.getMessage(), ex); ServerError error = ServerError.ServiceNotReady; if (ex instanceof MetadataStoreException) { error = ServerError.MetadataError; @@ -685,6 +685,14 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa error = ServerError.MetadataError; } } + if (error == ServerError.TopicNotFound) { + log.info("Trying to get Partitioned Metadata for a resource not exist" + + "[{}] {}: {}", remoteAddress, + topicName, ex.getMessage()); + } else { + log.warn("Failed to get Partitioned Metadata [{}] {}: {}", + remoteAddress, topicName, ex.getMessage(), ex); + } commandSender.sendPartitionMetadataResponse(error, ex.getMessage(), requestId); } @@ -702,6 +710,16 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa return null; }).exceptionally(ex -> { logAuthException(remoteAddress, "partition-metadata", getPrincipal(), Optional.of(topicName), ex); + Throwable actEx = FutureUtil.unwrapCompletionException(ex); + if (actEx instanceof WebApplicationException restException) { + if (restException.getResponse().getStatus() == Response.Status.NOT_FOUND.getStatusCode()) { + writeAndFlush(Commands.newPartitionMetadataResponse(ServerError.MetadataError, + "Tenant or namespace or topic does not exist: " + topicName.getNamespace() , + requestId)); + lookupSemaphore.release(); + return null; + } + } final String msg = "Exception occurred while trying to authorize get Partition Metadata"; writeAndFlush(Commands.newPartitionMetadataResponse(ServerError.AuthorizationError, msg, requestId)); @@ -3663,13 +3681,22 @@ protected void messageReceived() { private static void logAuthException(SocketAddress remoteAddress, String operation, String principal, Optional topic, Throwable ex) { String topicString = topic.map(t -> ", topic=" + t.toString()).orElse(""); - if (ex instanceof AuthenticationException) { + Throwable actEx = FutureUtil.unwrapCompletionException(ex); + if (actEx instanceof AuthenticationException) { log.info("[{}] Failed to authenticate: operation={}, principal={}{}, reason={}", - remoteAddress, operation, principal, topicString, ex.getMessage()); - } else { - log.error("[{}] Error trying to authenticate: operation={}, principal={}{}", - remoteAddress, operation, principal, topicString, ex); + remoteAddress, operation, principal, topicString, actEx.getMessage()); + return; + } else if (actEx instanceof WebApplicationException restException){ + // Do not print error log if users tries to access a not found resource. + if (restException.getResponse().getStatus() == Response.Status.NOT_FOUND.getStatusCode()) { + log.info("[{}] Trying to authenticate for a topic which under a namespace not exists: operation={}," + + " principal={}{}, reason: {}", + remoteAddress, operation, principal, topicString, actEx.getMessage()); + return; + } } + log.error("[{}] Error trying to authenticate: operation={}, principal={}{}", + remoteAddress, operation, principal, topicString, ex); } private static void logNamespaceNameAuthException(SocketAddress remoteAddress, String operation, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java index 87bc4267b48a3..e9a639697d9ff 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java @@ -578,4 +578,55 @@ public void testGetMetadataIfNotAllowedCreateOfNonPersistentTopic(boolean config assertEquals(getLookupRequestPermits(), lookupPermitsBefore); }); } + + @Test(dataProvider = "topicDomains") + public void testNamespaceNotExist(TopicDomain topicDomain) throws Exception { + int lookupPermitsBefore = getLookupRequestPermits(); + final String namespaceNotExist = BrokerTestUtil.newUniqueName("public/ns"); + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.toString() + "://" + namespaceNotExist + "/tp"); + PulsarClientImpl[] clientArray = getClientsToTest(false); + for (PulsarClientImpl client : clientArray) { + try { + PartitionedTopicMetadata topicMetadata = client + .getPartitionedTopicMetadata(topicNameStr, true, true) + .join(); + log.info("Get topic metadata: {}", topicMetadata.partitions); + fail("Expected a not found ex"); + } catch (Exception ex) { + Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex); + assertTrue(unwrapEx instanceof PulsarClientException.BrokerMetadataException || + unwrapEx instanceof PulsarClientException.TopicDoesNotExistException); + } + } + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + } + + @Test(dataProvider = "topicDomains") + public void testTenantNotExist(TopicDomain topicDomain) throws Exception { + int lookupPermitsBefore = getLookupRequestPermits(); + final String tenantNotExist = BrokerTestUtil.newUniqueName("tenant"); + final String namespaceNotExist = BrokerTestUtil.newUniqueName(tenantNotExist + "/default"); + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.toString() + "://" + namespaceNotExist + "/tp"); + PulsarClientImpl[] clientArray = getClientsToTest(false); + for (PulsarClientImpl client : clientArray) { + try { + PartitionedTopicMetadata topicMetadata = client + .getPartitionedTopicMetadata(topicNameStr, true, true) + .join(); + log.info("Get topic metadata: {}", topicMetadata.partitions); + fail("Expected a not found ex"); + } catch (Exception ex) { + Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex); + assertTrue(unwrapEx instanceof PulsarClientException.BrokerMetadataException || + unwrapEx instanceof PulsarClientException.TopicDoesNotExistException); + } + } + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + } } From e91574ac7b44348a05f1ae812c5aae3abb26fe64 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 27 Sep 2024 17:33:28 +0800 Subject: [PATCH 527/580] [fix][broker] Fix unloadNamespaceBundlesGracefully can be stuck with extensible load manager (#23349) --- .../apache/pulsar/broker/PulsarService.java | 7 +- .../extensions/ExtensibleLoadManagerImpl.java | 130 +++++++++++++----- .../ExtensibleLoadManagerWrapper.java | 2 +- .../channel/ServiceUnitStateChannelImpl.java | 43 ++++-- .../channel/ServiceUnitStateData.java | 2 +- .../ServiceUnitStateDataConflictResolver.java | 2 +- .../ServiceUnitStateTableViewImpl.java | 19 ++- .../filter/BrokerMaxTopicCountFilter.java | 7 +- .../store/TableViewLoadDataStoreImpl.java | 10 +- .../pulsar/broker/service/BrokerService.java | 7 +- .../SystemTopicBasedTopicPoliciesService.java | 32 +++-- .../ExtensibleLoadManagerCloseTest.java | 50 +++++-- .../extensions/store/LoadDataStoreTest.java | 3 +- .../pulsar/client/impl/TableViewImpl.java | 7 +- 14 files changed, 240 insertions(+), 81 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index a2f6fb9e9773b..6c768a078974f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -513,6 +513,9 @@ public CompletableFuture closeAsync() { return closeFuture; } LOG.info("Closing PulsarService"); + if (topicPoliciesService != null) { + topicPoliciesService.close(); + } if (brokerService != null) { brokerService.unloadNamespaceBundlesGracefully(); } @@ -633,10 +636,6 @@ public CompletableFuture closeAsync() { transactionBufferClient.close(); } - if (topicPoliciesService != null) { - topicPoliciesService.close(); - topicPoliciesService = null; - } if (client != null) { client.close(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 98ef6bf36edac..841f9bfb669d4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -181,7 +181,14 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager, BrokerS private SplitManager splitManager; - volatile boolean started = false; + enum State { + INIT, + RUNNING, + // It's removing visibility of the current broker from other brokers. In this state, it cannot play as a leader + // or follower. + DISABLED, + } + private final AtomicReference state = new AtomicReference<>(State.INIT); private boolean configuredSystemTopics = false; @@ -214,7 +221,7 @@ public CompletableFuture> getOwnedServiceUnitsAsync() { } public Set getOwnedServiceUnits() { - if (!started) { + if (state.get() == State.INIT) { log.warn("Failed to get owned service units, load manager is not started."); return Collections.emptySet(); } @@ -344,7 +351,7 @@ public static CompletableFuture> getAssignedBrokerLoo @Override public void start() throws PulsarServerException { - if (this.started) { + if (state.get() != State.INIT) { return; } try { @@ -443,7 +450,9 @@ public void start() throws PulsarServerException { this.splitScheduler.start(); this.initWaiter.complete(true); - this.started = true; + if (!state.compareAndSet(State.INIT, State.RUNNING)) { + failForUnexpectedState("start"); + } log.info("Started load manager."); } catch (Throwable e) { failStarting(e); @@ -615,21 +624,17 @@ public CompletableFuture> selectAsync(ServiceUnitId bundle, filter.filterAsync(availableBrokerCandidates, bundle, context); futures.add(future); } - CompletableFuture> result = new CompletableFuture<>(); - FutureUtil.waitForAll(futures).whenComplete((__, ex) -> { - if (ex != null) { - // TODO: We may need to revisit this error case. - log.error("Failed to filter out brokers when select bundle: {}", bundle, ex); - } + return FutureUtil.waitForAll(futures).exceptionally(e -> { + // TODO: We may need to revisit this error case. + log.error("Failed to filter out brokers when select bundle: {}", bundle, e); + return null; + }).thenApply(__ -> { if (availableBrokerCandidates.isEmpty()) { - result.complete(Optional.empty()); - return; + return Optional.empty(); } Set candidateBrokers = availableBrokerCandidates.keySet(); - - result.complete(getBrokerSelectionStrategy().select(candidateBrokers, bundle, context)); + return getBrokerSelectionStrategy().select(candidateBrokers, bundle, context); }); - return result; }); } @@ -667,6 +672,9 @@ public CompletableFuture unloadNamespaceBundleAsync(ServiceUnitId bundle, boolean force, long timeout, TimeUnit timeoutUnit) { + if (state.get() == State.INIT) { + return CompletableFuture.completedFuture(null); + } if (NamespaceService.isSLAOrHeartbeatNamespace(bundle.getNamespaceObject().toString())) { log.info("Skip unloading namespace bundle: {}.", bundle); return CompletableFuture.completedFuture(null); @@ -755,24 +763,11 @@ private CompletableFuture splitAsync(SplitDecision decision, @Override public void close() throws PulsarServerException { - if (!this.started) { + if (state.get() == State.INIT) { return; } try { - if (brokerLoadDataReportTask != null) { - brokerLoadDataReportTask.cancel(true); - } - - if (topBundlesLoadDataReportTask != null) { - topBundlesLoadDataReportTask.cancel(true); - } - - if (monitorTask != null) { - monitorTask.cancel(true); - } - - this.brokerLoadDataStore.shutdown(); - this.topBundlesLoadDataStore.shutdown(); + stopLoadDataReportTasks(); this.unloadScheduler.close(); this.splitScheduler.close(); this.serviceUnitStateTableViewSyncer.close(); @@ -791,7 +786,7 @@ public void close() throws PulsarServerException { } catch (Exception e) { throw new PulsarServerException(e); } finally { - this.started = false; + state.set(State.INIT); } } @@ -799,6 +794,28 @@ public void close() throws PulsarServerException { } } + private void stopLoadDataReportTasks() { + if (brokerLoadDataReportTask != null) { + brokerLoadDataReportTask.cancel(true); + } + if (topBundlesLoadDataReportTask != null) { + topBundlesLoadDataReportTask.cancel(true); + } + if (monitorTask != null) { + monitorTask.cancel(true); + } + try { + brokerLoadDataStore.shutdown(); + } catch (IOException e) { + log.warn("Failed to shutdown brokerLoadDataStore", e); + } + try { + topBundlesLoadDataStore.shutdown(); + } catch (IOException e) { + log.warn("Failed to shutdown topBundlesLoadDataStore", e); + } + } + public static boolean isInternalTopic(String topic) { return INTERNAL_TOPICS.contains(topic) || topic.startsWith(TOPIC) @@ -814,13 +831,16 @@ synchronized void playLeader() { boolean becameFollower = false; while (!Thread.currentThread().isInterrupted()) { try { - if (!initWaiter.get()) { + if (!initWaiter.get() || disabled()) { return; } if (!serviceUnitStateChannel.isChannelOwner()) { becameFollower = true; break; } + if (disabled()) { + return; + } // Confirm the system topics have been created or create them if they do not exist. // If the leader has changed, the new leader need to reset // the local brokerService.topics (by this topic creations). @@ -835,6 +855,11 @@ synchronized void playLeader() { } break; } catch (Throwable e) { + if (disabled()) { + log.warn("The broker:{} failed to set the role but exit because it's disabled", + pulsar.getBrokerId(), e); + return; + } log.warn("The broker:{} failed to set the role. Retrying {} th ...", pulsar.getBrokerId(), ++retry, e); try { @@ -846,6 +871,9 @@ synchronized void playLeader() { } } } + if (disabled()) { + return; + } if (becameFollower) { log.warn("The broker:{} became follower while initializing leader role.", pulsar.getBrokerId()); @@ -869,13 +897,16 @@ synchronized void playFollower() { boolean becameLeader = false; while (!Thread.currentThread().isInterrupted()) { try { - if (!initWaiter.get()) { + if (!initWaiter.get() || disabled()) { return; } if (serviceUnitStateChannel.isChannelOwner()) { becameLeader = true; break; } + if (disabled()) { + return; + } unloadScheduler.close(); serviceUnitStateChannel.cancelOwnershipMonitor(); closeInternalTopics(); @@ -885,6 +916,11 @@ synchronized void playFollower() { serviceUnitStateTableViewSyncer.close(); break; } catch (Throwable e) { + if (disabled()) { + log.warn("The broker:{} failed to set the role but exit because it's disabled", + pulsar.getBrokerId(), e); + return; + } log.warn("The broker:{} failed to set the role. Retrying {} th ...", pulsar.getBrokerId(), ++retry, e); try { @@ -896,6 +932,9 @@ synchronized void playFollower() { } } } + if (disabled()) { + return; + } if (becameLeader) { log.warn("This broker:{} became leader while initializing follower role.", pulsar.getBrokerId()); @@ -982,9 +1021,20 @@ protected void monitor() { } public void disableBroker() throws Exception { + // TopicDoesNotExistException might be thrown and it's not recoverable. Enable this flag to exit playFollower() + // or playLeader() quickly. + if (!state.compareAndSet(State.RUNNING, State.DISABLED)) { + failForUnexpectedState("disableBroker"); + } + stopLoadDataReportTasks(); serviceUnitStateChannel.cleanOwnerships(); - leaderElectionService.close(); brokerRegistry.unregister(); + leaderElectionService.close(); + final var availableBrokers = brokerRegistry.getAvailableBrokersAsync() + .get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); + if (availableBrokers.isEmpty()) { + close(); + } // Close the internal topics (if owned any) after giving up the possible leader role, // so that the subsequent lookups could hit the next leader. closeInternalTopics(); @@ -1018,4 +1068,16 @@ protected BrokerRegistry createBrokerRegistry(PulsarService pulsar) { protected ServiceUnitStateChannel createServiceUnitStateChannel(PulsarService pulsar) { return new ServiceUnitStateChannelImpl(pulsar); } + + private void failForUnexpectedState(String msg) { + throw new IllegalStateException("Failed to " + msg + ", state: " + state.get()); + } + + boolean running() { + return state.get() == State.RUNNING; + } + + private boolean disabled() { + return state.get() == State.DISABLED; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java index 6a48607977ba9..35f6cfcbcf549 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerWrapper.java @@ -51,7 +51,7 @@ public void start() throws PulsarServerException { } public boolean started() { - return loadManager.started && loadManager.getServiceUnitStateChannel().started(); + return loadManager.running() && loadManager.getServiceUnitStateChannel().started(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index ddbc9eacac921..ce975495feb2a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -255,6 +255,7 @@ public void cancelOwnershipMonitor() { @Override public void cleanOwnerships() { + disable(); doCleanup(brokerId, true); } @@ -412,9 +413,7 @@ public CompletableFuture isChannelOwnerAsync() { if (owner.isPresent()) { return isTargetBroker(owner.get()); } else { - String msg = "There is no channel owner now."; - log.error(msg); - throw new IllegalStateException(msg); + throw new IllegalStateException("There is no channel owner now."); } }); } @@ -679,11 +678,15 @@ private void handleEvent(String serviceUnit, ServiceUnitStateData data) { brokerId, serviceUnit, data, totalHandledRequests); } - if (channelState == Disabled) { + ServiceUnitState state = state(data); + if (channelState == Disabled && (data == null || !data.force())) { + final var request = getOwnerRequests.remove(serviceUnit); + if (request != null) { + request.completeExceptionally(new BrokerServiceException.ServiceUnitNotReadyException( + "cancel the lookup request for " + serviceUnit + " when receiving " + state)); + } return; } - - ServiceUnitState state = state(data); try { switch (state) { case Owned -> handleOwnEvent(serviceUnit, data); @@ -851,7 +854,7 @@ private void handleSplitEvent(String serviceUnit, ServiceUnitStateData data) { } } - private void handleFreeEvent(String serviceUnit, ServiceUnitStateData data) { + private CompletableFuture handleFreeEvent(String serviceUnit, ServiceUnitStateData data) { var getOwnerRequest = getOwnerRequests.remove(serviceUnit); if (getOwnerRequest != null) { getOwnerRequest.complete(null); @@ -865,8 +868,10 @@ private void handleFreeEvent(String serviceUnit, ServiceUnitStateData data) { : CompletableFuture.completedFuture(0)).thenApply(__ -> null); stateChangeListeners.notifyOnCompletion(future, serviceUnit, data) .whenComplete((__, e) -> log(e, serviceUnit, data, null)); + return future; } else { stateChangeListeners.notify(serviceUnit, data, null); + return CompletableFuture.completedFuture(null); } } @@ -1273,7 +1278,11 @@ private void handleBrokerDeletionEvent(String broker) { return; } } catch (Exception e) { - log.error("Failed to handle broker deletion event.", e); + if (e instanceof ExecutionException && e.getCause() instanceof IllegalStateException) { + log.warn("Failed to handle broker deletion event due to {}", e.getMessage()); + } else { + log.error("Failed to handle broker deletion event.", e); + } return; } MetadataState state = getMetadataState(); @@ -1293,6 +1302,11 @@ private void handleBrokerDeletionEvent(String broker) { private void scheduleCleanup(String broker, long delayInSecs) { var scheduled = new MutableObject>(); try { + final var channelState = this.channelState; + if (channelState == Disabled || channelState == Closed) { + log.warn("[{}] Skip scheduleCleanup because the state is {} now", brokerId, channelState); + return; + } cleanupJobs.computeIfAbsent(broker, k -> { Executor delayed = CompletableFuture .delayedExecutor(delayInSecs, TimeUnit.SECONDS, pulsar.getLoadManagerExecutor()); @@ -1393,6 +1407,7 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max if (data.state() == Owned && broker.equals(data.dstBroker())) { cleaned = false; + log.info("[{}] bundle {} is still owned by this, data: {}", broker, serviceUnit, data); break; } } @@ -1400,10 +1415,15 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max break; } else { try { - MILLISECONDS.sleep(OWNERSHIP_CLEAN_UP_WAIT_RETRY_DELAY_IN_MILLIS); + tableview.flush(OWNERSHIP_CLEAN_UP_WAIT_RETRY_DELAY_IN_MILLIS / 2); + Thread.sleep(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS / 2); } catch (InterruptedException e) { log.warn("Interrupted while delaying the next service unit clean-up. Cleaning broker:{}", brokerId); + } catch (ExecutionException e) { + log.error("Failed to flush table view", e.getCause()); + } catch (TimeoutException e) { + log.warn("Failed to flush the table view in {} ms", OWNERSHIP_CLEAN_UP_WAIT_RETRY_DELAY_IN_MILLIS); } } } @@ -1428,6 +1448,11 @@ private synchronized void doCleanup(String broker, boolean gracefully) { log.info("Started ownership cleanup for the inactive broker:{}", broker); int orphanServiceUnitCleanupCnt = 0; long totalCleanupErrorCntStart = totalCleanupErrorCnt.get(); + try { + tableview.flush(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS); + } catch (Exception e) { + log.error("Failed to flush", e); + } Map orphanSystemServiceUnits = new HashMap<>(); for (var etr : tableview.entrySet()) { var stateData = etr.getValue(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java index e85134e611632..4a990ddbc9b21 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateData.java @@ -34,7 +34,7 @@ public record ServiceUnitStateData( public ServiceUnitStateData { Objects.requireNonNull(state); - if (StringUtils.isBlank(dstBroker) && StringUtils.isBlank(sourceBroker)) { + if (state != ServiceUnitState.Free && StringUtils.isBlank(dstBroker) && StringUtils.isBlank(sourceBroker)) { throw new IllegalArgumentException("Empty broker"); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java index b1dbb6fac8709..3e43237f4c00e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateDataConflictResolver.java @@ -145,4 +145,4 @@ private boolean invalidUnload(ServiceUnitStateData from, ServiceUnitStateData to || !from.dstBroker().equals(to.sourceBroker()) || from.dstBroker().equals(to.dstBroker()); } -} \ No newline at end of file +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java index 8dfaddcdabca1..12cf87445a3dd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateTableViewImpl.java @@ -33,6 +33,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.common.naming.TopicDomain; @@ -144,8 +145,13 @@ public CompletableFuture put(String key, ServiceUnitStateData value) { .sendAsync() .whenComplete((messageId, e) -> { if (e != null) { - log.error("Failed to publish the message: serviceUnit:{}, data:{}", - key, value, e); + if (e instanceof PulsarClientException.AlreadyClosedException) { + log.info("Skip publishing the message since the producer is closed, serviceUnit: {}, data: " + + "{}", key, value); + } else { + log.error("Failed to publish the message: serviceUnit:{}, data:{}", + key, value, e); + } future.completeExceptionally(e); } else { future.complete(null); @@ -159,7 +165,14 @@ public void flush(long waitDurationInMillis) throws InterruptedException, Timeou if (!isValidState()) { throw new IllegalStateException(INVALID_STATE_ERROR_MSG); } - producer.flushAsync().get(waitDurationInMillis, MILLISECONDS); + final var deadline = System.currentTimeMillis() + waitDurationInMillis; + var waitTimeMs = waitDurationInMillis; + producer.flushAsync().get(waitTimeMs, MILLISECONDS); + waitTimeMs = deadline - System.currentTimeMillis(); + if (waitTimeMs < 0) { + waitTimeMs = 0; + } + tableview.refreshAsync().get(waitTimeMs, MILLISECONDS); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerMaxTopicCountFilter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerMaxTopicCountFilter.java index 48213c18e6376..9863d05ee751e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerMaxTopicCountFilter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerMaxTopicCountFilter.java @@ -41,7 +41,12 @@ public CompletableFuture> filterAsync(Map { - Optional brokerLoadDataOpt = context.brokerLoadDataStore().get(broker); + final Optional brokerLoadDataOpt; + try { + brokerLoadDataOpt = context.brokerLoadDataStore().get(broker); + } catch (IllegalStateException ignored) { + return false; + } long topics = brokerLoadDataOpt.map(BrokerLoadData::getTopics).orElse(0L); // TODO: The broker load data might be delayed, so the max topic check might not accurate. return topics >= loadBalancerBrokerMaxTopics; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java index c9d18676cfa99..3ce44a1e65a73 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/TableViewLoadDataStoreImpl.java @@ -92,7 +92,11 @@ public synchronized CompletableFuture removeAsync(String key) { public synchronized Optional get(String key) { String msg = validateTableView(); if (StringUtils.isNotBlank(msg)) { - throw new IllegalStateException(msg); + if (msg.equals(SHUTDOWN_ERR_MSG)) { + return Optional.empty(); + } else { + throw new IllegalStateException(msg); + } } return Optional.ofNullable(tableView.get(key)); } @@ -193,7 +197,9 @@ public synchronized void startProducer() throws LoadDataStoreException { @Override public synchronized void close() throws IOException { - validateState(); + if (isShutdown) { + return; + } closeProducer(); closeTableView(); } 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 09f04d878c4e5..bfa99eedcadce 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 @@ -973,7 +973,12 @@ public void unloadNamespaceBundlesGracefully(int maxConcurrentUnload, boolean cl pulsar.getNamespaceService().unloadNamespaceBundle(su, timeout, MILLISECONDS, closeWithoutWaitingClientDisconnect).get(timeout, MILLISECONDS); } catch (Exception e) { - log.warn("Failed to unload namespace bundle {}", su, e); + if (e instanceof ExecutionException + && e.getCause() instanceof ServiceUnitNotReadyException) { + log.warn("Failed to unload namespace bundle {}: {}", su, e.getMessage()); + } else { + log.warn("Failed to unload namespace bundle {}", su, e); + } } } }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index 18b4c610a5c9b..6ff6408916b1c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -254,7 +254,7 @@ public CompletableFuture> getTopicPoliciesAsync(TopicNam // initialization by calling this method. At the moment, the load manager does not start so the lookup // for "__change_events" will fail. In this case, just return an empty policies to avoid deadlock. final var loadManager = pulsarService.getLoadManager().get(); - if (loadManager == null || !loadManager.started()) { + if (loadManager == null || !loadManager.started() || closed.get()) { return CompletableFuture.completedFuture(Optional.empty()); } final CompletableFuture preparedFuture = prepareInitPoliciesCacheAsync(topicName.getNamespaceObject()); @@ -308,6 +308,9 @@ public void addOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) { @VisibleForTesting @Nonnull CompletableFuture prepareInitPoliciesCacheAsync(@Nonnull NamespaceName namespace) { requireNonNull(namespace); + if (closed.get()) { + return CompletableFuture.completedFuture(false); + } return pulsarService.getPulsarResources().getNamespaceResources().getPoliciesAsync(namespace) .thenCompose(namespacePolicies -> { if (namespacePolicies.isEmpty() || namespacePolicies.get().deleted) { @@ -331,6 +334,9 @@ public void addOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) { }); initFuture.exceptionally(ex -> { try { + if (closed.get()) { + return null; + } log.error("[{}] Failed to create reader on __change_events topic", namespace, ex); cleanCacheAndCloseReader(namespace, false); @@ -681,14 +687,22 @@ public void close() throws Exception { if (closed.compareAndSet(false, true)) { writerCaches.synchronous().invalidateAll(); readerCaches.values().forEach(future -> { - if (future != null && !future.isCompletedExceptionally()) { - future.thenAccept(reader -> { - try { - reader.close(); - } catch (Exception e) { - log.error("Failed to close reader.", e); - } - }); + try { + final var reader = future.getNow(null); + if (reader != null) { + reader.close(); + log.info("Closed the reader for topic policies"); + } else { + // Avoid blocking the thread that the reader is created + future.thenAccept(SystemTopicClient.Reader::closeAsync).whenComplete((__, e) -> { + if (e == null) { + log.info("Closed the reader for topic policies"); + } else { + log.error("Failed to close the reader for topic policies", e); + } + }); + } + } catch (Throwable ignored) { } }); readerCaches.clear(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java index 41413f3e3a913..fa63ce566c603 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java @@ -22,13 +22,15 @@ import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -36,26 +38,33 @@ import org.testng.annotations.Test; @Slf4j +@Test(groups = "broker") public class ExtensibleLoadManagerCloseTest { private static final String clusterName = "test"; - private final LocalBookkeeperEnsemble bk = new LocalBookkeeperEnsemble(1, 0, () -> 0); private final List brokers = new ArrayList<>(); - private PulsarAdmin admin; + private LocalBookkeeperEnsemble bk; @BeforeClass(alwaysRun = true) public void setup() throws Exception { + bk = new LocalBookkeeperEnsemble(1, 0, () -> 0); bk.start(); - for (int i = 0; i < 3; i++) { + } + + private void setupBrokers(int numBrokers) throws Exception { + brokers.clear(); + for (int i = 0; i < numBrokers; i++) { final var broker = new PulsarService(brokerConfig()); broker.start(); brokers.add(broker); } - admin = brokers.get(0).getAdminClient(); - admin.clusters().createCluster(clusterName, ClusterData.builder().build()); - admin.tenants().createTenant("public", TenantInfo.builder() - .allowedClusters(Collections.singleton(clusterName)).build()); - admin.namespaces().createNamespace("public/default"); + final var admin = brokers.get(0).getAdminClient(); + if (!admin.clusters().getClusters().contains(clusterName)) { + admin.clusters().createCluster(clusterName, ClusterData.builder().build()); + admin.tenants().createTenant("public", TenantInfo.builder() + .allowedClusters(Collections.singleton(clusterName)).build()); + admin.namespaces().createNamespace("public/default"); + } } @@ -85,7 +94,9 @@ private ServiceConfiguration brokerConfig() { @Test public void testCloseAfterLoadingBundles() throws Exception { + setupBrokers(3); final var topic = "test"; + final var admin = brokers.get(0).getAdminClient(); admin.topics().createPartitionedTopic(topic, 20); admin.lookups().lookupPartitionedTopic(topic); final var client = PulsarClient.builder().serviceUrl(brokers.get(0).getBrokerServiceUrl()).build(); @@ -104,4 +115,25 @@ public void testCloseAfterLoadingBundles() throws Exception { Assert.assertTrue(closeTimeMs < 5000L); } } + + @Test + public void testLookup() throws Exception { + setupBrokers(1); + final var topic = "test-lookup"; + final var numPartitions = 16; + final var admin = brokers.get(0).getAdminClient(); + admin.topics().createPartitionedTopic(topic, numPartitions); + + final var futures = new ArrayList>(); + for (int i = 0; i < numPartitions; i++) { + futures.add(admin.lookups().lookupTopicAsync(topic + TopicName.PARTITIONED_TOPIC_SUFFIX + i)); + } + FutureUtil.waitForAll(futures).get(); + + final var start = System.currentTimeMillis(); + brokers.get(0).close(); + final var closeTimeMs = System.currentTimeMillis() - start; + log.info("Broker close time: {}", closeTimeMs); + Assert.assertTrue(closeTimeMs < 5000L); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java index 3267e67ad2c3e..820307637be67 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreTest.java @@ -198,7 +198,7 @@ public void testShutdown() throws Exception { Assert.assertTrue(loadDataStore.pushAsync("2", 2).isCompletedExceptionally()); Assert.assertTrue(loadDataStore.removeAsync("2").isCompletedExceptionally()); - assertThrows(IllegalStateException.class, () -> loadDataStore.get("2")); + assertTrue(loadDataStore.get("2").isEmpty()); assertThrows(IllegalStateException.class, loadDataStore::size); assertThrows(IllegalStateException.class, loadDataStore::entrySet); assertThrows(IllegalStateException.class, () -> loadDataStore.forEach((k, v) -> {})); @@ -206,7 +206,6 @@ public void testShutdown() throws Exception { assertThrows(IllegalStateException.class, loadDataStore::start); assertThrows(IllegalStateException.class, loadDataStore::startProducer); assertThrows(IllegalStateException.class, loadDataStore::startTableView); - assertThrows(IllegalStateException.class, loadDataStore::close); assertThrows(IllegalStateException.class, loadDataStore::closeTableView); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java index d5d4174ee10a9..4f52060497864 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java @@ -364,8 +364,8 @@ private void readAllExistingMessages(Reader reader, CompletableFuture f } }).exceptionally(ex -> { if (ex.getCause() instanceof PulsarClientException.AlreadyClosedException) { - log.error("Reader {} was closed while reading existing messages.", - reader.getTopic(), ex); + log.info("Reader {} was closed while reading existing messages.", + reader.getTopic()); } else { log.warn("Reader {} was interrupted while reading existing messages. ", reader.getTopic(), ex); @@ -393,8 +393,7 @@ private void readTailMessages(Reader reader) { readTailMessages(reader); }).exceptionally(ex -> { if (ex.getCause() instanceof PulsarClientException.AlreadyClosedException) { - log.error("Reader {} was closed while reading tail messages.", - reader.getTopic(), ex); + log.info("Reader {} was closed while reading tail messages.", reader.getTopic()); // Fail all refresh request when no more messages can be read. pendingRefreshRequests.keySet().forEach(future -> { pendingRefreshRequests.remove(future); From 95bd1d1dd3d447f0705a96092afbc9d6bd6cd1dc Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 27 Sep 2024 19:49:02 +0800 Subject: [PATCH 528/580] [fix][broker] Fix the broker registery cannot recover from the metadata node deletion (#23359) --- .../extensions/BrokerRegistry.java | 6 +- .../extensions/BrokerRegistryImpl.java | 57 ++++---- .../BrokerRegistryIntegrationTest.java | 124 ++++++++++++++++++ .../extensions/BrokerRegistryTest.java | 6 +- .../ExtensibleLoadManagerImplTest.java | 4 +- 5 files changed, 168 insertions(+), 29 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java index 8133d4c482752..79dba9c63342e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java @@ -25,6 +25,8 @@ import java.util.function.BiConsumer; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; +import org.apache.pulsar.common.classification.InterfaceAudience; +import org.apache.pulsar.common.classification.InterfaceStability; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.NotificationType; @@ -32,6 +34,8 @@ * Responsible for registering the current Broker lookup info to * the distributed store (e.g. Zookeeper) for broker discovery. */ +@InterfaceAudience.LimitedPrivate +@InterfaceStability.Unstable public interface BrokerRegistry extends AutoCloseable { /** @@ -47,7 +51,7 @@ public interface BrokerRegistry extends AutoCloseable { /** * Register local broker to metadata store. */ - void register() throws MetadataStoreException; + CompletableFuture registerAsync(); /** * Unregister the broker. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java index f34d377990b68..9fd0518a054cc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java @@ -32,6 +32,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarServerException; @@ -69,10 +70,12 @@ protected enum State { Init, Started, Registered, + Unregistering, Closed } - private State state; + @VisibleForTesting + final AtomicReference state = new AtomicReference<>(State.Init); public BrokerRegistryImpl(PulsarService pulsar) { this.pulsar = pulsar; @@ -80,6 +83,13 @@ public BrokerRegistryImpl(PulsarService pulsar) { this.brokerLookupDataMetadataCache = pulsar.getLocalMetadataStore().getMetadataCache(BrokerLookupData.class); this.scheduler = pulsar.getLoadManagerExecutor(); this.listeners = new ArrayList<>(); + // The registered node is an ephemeral node that could be deleted when the metadata store client's session + // is expired. In this case, we should register again. + this.listeners.add((broker, notificationType) -> { + if (notificationType == NotificationType.Deleted && getBrokerId().equals(broker)) { + registerAsync(); + } + }); this.brokerIdKeyPath = keyPath(pulsar.getBrokerId()); this.brokerLookupData = new BrokerLookupData( pulsar.getWebServiceAddress(), @@ -94,44 +104,45 @@ public BrokerRegistryImpl(PulsarService pulsar) { System.currentTimeMillis(), pulsar.getBrokerVersion(), pulsar.getConfig().lookupProperties()); - this.state = State.Init; } @Override public synchronized void start() throws PulsarServerException { - if (this.state != State.Init) { - return; + if (!this.state.compareAndSet(State.Init, State.Started)) { + throw new PulsarServerException("Cannot start the broker registry in state " + state.get()); } pulsar.getLocalMetadataStore().registerListener(this::handleMetadataStoreNotification); try { - this.state = State.Started; - this.register(); - } catch (MetadataStoreException e) { - throw new PulsarServerException(e); + this.registerAsync().get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + throw PulsarServerException.from(e); } } @Override public boolean isStarted() { - return this.state == State.Started || this.state == State.Registered; + final var state = this.state.get(); + return state == State.Started || state == State.Registered; } @Override - public synchronized void register() throws MetadataStoreException { - if (this.state == State.Started) { - try { - brokerLookupDataMetadataCache.put(brokerIdKeyPath, brokerLookupData, EnumSet.of(CreateOption.Ephemeral)) - .get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); - this.state = State.Registered; - } catch (InterruptedException | ExecutionException | TimeoutException e) { - throw MetadataStoreException.unwrap(e); - } + public CompletableFuture registerAsync() { + final var state = this.state.get(); + if (state != State.Started && state != State.Registered) { + log.info("[{}] Skip registering self because the state is {}", getBrokerId(), state); + return CompletableFuture.completedFuture(null); } + log.info("[{}] Started registering self to {} (state: {})", getBrokerId(), brokerIdKeyPath, state); + return brokerLookupDataMetadataCache.put(brokerIdKeyPath, brokerLookupData, EnumSet.of(CreateOption.Ephemeral)) + .thenAccept(__ -> { + this.state.set(State.Registered); + log.info("[{}] Finished registering self", getBrokerId()); + }); } @Override public synchronized void unregister() throws MetadataStoreException { - if (this.state == State.Registered) { + if (state.compareAndSet(State.Registered, State.Unregistering)) { try { brokerLookupDataMetadataCache.delete(brokerIdKeyPath) .get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS); @@ -144,7 +155,7 @@ public synchronized void unregister() throws MetadataStoreException { } catch (InterruptedException | TimeoutException e) { throw MetadataStoreException.unwrap(e); } finally { - this.state = State.Started; + state.set(State.Started); } } } @@ -191,7 +202,7 @@ public synchronized void addListener(BiConsumer listen @Override public synchronized void close() throws PulsarServerException { - if (this.state == State.Closed) { + if (this.state.get() == State.Closed) { return; } try { @@ -200,7 +211,7 @@ public synchronized void close() throws PulsarServerException { } catch (Exception ex) { log.error("Unexpected error when unregistering the broker registry", ex); } finally { - this.state = State.Closed; + this.state.set(State.Closed); } } @@ -238,7 +249,7 @@ protected static String keyPath(String brokerId) { } private void checkState() throws IllegalStateException { - if (this.state == State.Closed) { + if (this.state.get() == State.Closed) { throw new IllegalStateException("The registry already closed."); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java new file mode 100644 index 0000000000000..162ea50829d40 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java @@ -0,0 +1,124 @@ +/* + * 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.loadbalance.extensions; + +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.util.PortManager; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.LoadManager; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.awaitility.Awaitility; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class BrokerRegistryIntegrationTest { + + private static final String clusterName = "test"; + private final int zkPort = PortManager.nextFreePort(); + private final LocalBookkeeperEnsemble bk = new LocalBookkeeperEnsemble(2, zkPort, PortManager::nextFreePort); + private PulsarService pulsar; + private BrokerRegistry brokerRegistry; + private String brokerMetadataPath; + + @BeforeClass + protected void setup() throws Exception { + bk.start(); + pulsar = new PulsarService(brokerConfig()); + pulsar.start(); + final var admin = pulsar.getAdminClient(); + admin.clusters().createCluster(clusterName, ClusterData.builder().build()); + admin.tenants().createTenant("public", TenantInfo.builder() + .allowedClusters(Collections.singleton(clusterName)).build()); + admin.namespaces().createNamespace("public/default"); + brokerRegistry = ((ExtensibleLoadManagerWrapper) pulsar.getLoadManager().get()).get().getBrokerRegistry(); + brokerMetadataPath = LoadManager.LOADBALANCE_BROKERS_ROOT + "/" + pulsar.getBrokerId(); + } + + @AfterClass(alwaysRun = true) + protected void cleanup() throws Exception { + if (pulsar != null) { + pulsar.close(); + } + bk.stop(); + } + + @Test + public void testRecoverFromNodeDeletion() throws Exception { + // Simulate the case that the node was somehow deleted (e.g. by session timeout) + Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( + brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); + pulsar.getLocalMetadataStore().delete(brokerMetadataPath, Optional.empty()); + Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( + brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); + + // If the node is deleted by unregister(), it should not recreate the path + brokerRegistry.unregister(); + Thread.sleep(3000); + Assert.assertTrue(brokerRegistry.getAvailableBrokersAsync().get().isEmpty()); + + // Restore the normal state + brokerRegistry.registerAsync().get(); + Assert.assertEquals(brokerRegistry.getAvailableBrokersAsync().get(), List.of(pulsar.getBrokerId())); + } + + @Test + public void testRegisterAgain() throws Exception { + Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( + brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); + final var metadataStore = pulsar.getLocalMetadataStore(); + final var oldResult = metadataStore.get(brokerMetadataPath).get().orElseThrow(); + log.info("Old result: {} {}", new String(oldResult.getValue()), oldResult.getStat().getVersion()); + brokerRegistry.registerAsync().get(); + + Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> { + final var newResult = metadataStore.get(brokerMetadataPath).get().orElseThrow(); + log.info("New result: {} {}", new String(newResult.getValue()), newResult.getStat().getVersion()); + Assert.assertTrue(newResult.getStat().getVersion() > oldResult.getStat().getVersion()); + Assert.assertEquals(newResult.getValue(), oldResult.getValue()); + }); + } + + private ServiceConfiguration brokerConfig() { + final var config = new ServiceConfiguration(); + config.setClusterName(clusterName); + config.setAdvertisedAddress("localhost"); + config.setBrokerServicePort(Optional.of(0)); + config.setWebServicePort(Optional.of(0)); + config.setMetadataStoreUrl("zk:127.0.0.1:" + bk.getZookeeperPort()); + config.setManagedLedgerDefaultWriteQuorum(1); + config.setManagedLedgerDefaultAckQuorum(1); + config.setManagedLedgerDefaultEnsembleSize(1); + config.setDefaultNumberOfNamespaceBundles(16); + config.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + config.setLoadBalancerDebugModeEnabled(true); + config.setBrokerShutdownTimeoutMs(100); + return config; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java index 91ada90dda690..28a2a18500f5f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java @@ -332,7 +332,7 @@ public void testCloseRegister() throws Exception { assertEquals(getState(brokerRegistry), BrokerRegistryImpl.State.Started); // Check state after re-register. - brokerRegistry.register(); + brokerRegistry.registerAsync().get(); assertEquals(getState(brokerRegistry), BrokerRegistryImpl.State.Registered); // Check state after close. @@ -396,8 +396,8 @@ public void testKeyPath() { assertEquals(keyPath, LOADBALANCE_BROKERS_ROOT + "/brokerId"); } - public BrokerRegistryImpl.State getState(BrokerRegistryImpl brokerRegistry) { - return WhiteboxImpl.getInternalState(brokerRegistry, BrokerRegistryImpl.State.class); + private static BrokerRegistryImpl.State getState(BrokerRegistryImpl brokerRegistry) { + return brokerRegistry.state.get(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index 4f6a006918318..7871e612c847a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -1211,7 +1211,7 @@ public void testDeployAndRollbackLoadManager() throws Exception { producer.send("t1"); // Test re-register broker and check the lookup result - loadManager4.getBrokerRegistry().register(); + loadManager4.getBrokerRegistry().registerAsync().get(); result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); assertNotNull(result); @@ -1423,7 +1423,7 @@ public void testLoadBalancerServiceUnitTableViewSyncer() throws Exception { producer.send("t1"); // Test re-register broker and check the lookup result - loadManager4.getBrokerRegistry().register(); + loadManager4.getBrokerRegistry().registerAsync().get(); result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); assertNotNull(result); From ca4a7435db6a99560284324b470ff66cae9d84f5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 28 Sep 2024 20:10:54 +0300 Subject: [PATCH 529/580] [fix][build] Disable flaky test BrokerRegistryIntegrationTest (#23367) --- .../extensions/BrokerRegistryIntegrationTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java index 162ea50829d40..d6615a8a5b49b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java @@ -37,7 +37,7 @@ import org.testng.annotations.Test; @Slf4j -@Test(groups = "broker") +@Test(groups = "flaky") public class BrokerRegistryIntegrationTest { private static final String clusterName = "test"; @@ -69,7 +69,7 @@ protected void cleanup() throws Exception { bk.stop(); } - @Test + @Test(enabled = false) public void testRecoverFromNodeDeletion() throws Exception { // Simulate the case that the node was somehow deleted (e.g. by session timeout) Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( @@ -88,7 +88,7 @@ public void testRecoverFromNodeDeletion() throws Exception { Assert.assertEquals(brokerRegistry.getAvailableBrokersAsync().get(), List.of(pulsar.getBrokerId())); } - @Test + @Test(enabled = false) public void testRegisterAgain() throws Exception { Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); From 77cb67a8c05210b9af0deb719cd24e3c3f5521b1 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Sun, 29 Sep 2024 11:16:33 +0800 Subject: [PATCH 530/580] [fix][ml] Managed ledger should recover after open ledger failed (#23368) --- .../impl/ManagedLedgerFactoryImpl.java | 1 + .../mledger/impl/ManagedLedgerErrorsTest.java | 31 +++++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index 34dd3610d4ec9..f546a487f84be 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -440,6 +440,7 @@ public void initializeFailed(ManagedLedgerException e) { // Clean the map if initialization fails ledgers.remove(name, future); + entryCacheManager.removeEntryCache(name); if (pendingInitializeLedgers.remove(name, pendingLedger)) { pendingLedger.ledger.asyncClose(new CloseCallback() { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java index 7b2f8228ad722..d72bffa27d30a 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java @@ -31,12 +31,14 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; import org.apache.bookkeeper.mledger.Entry; @@ -509,6 +511,35 @@ public void recoverAfterWriteError() throws Exception { entries.forEach(Entry::release); } + @Test + public void recoverAfterOpenManagedLedgerFail() throws Exception { + ManagedLedger ledger = factory.open("recoverAfterOpenManagedLedgerFail"); + Position position = ledger.addEntry("entry".getBytes()); + ledger.close(); + bkc.failAfter(0, BKException.Code.BookieHandleNotAvailableException); + try { + factory.open("recoverAfterOpenManagedLedgerFail"); + } catch (Exception e) { + // ok + } + + ledger = factory.open("recoverAfterOpenManagedLedgerFail"); + CompletableFuture future = new CompletableFuture<>(); + ledger.asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { + @Override + public void readEntryComplete(Entry entry, Object ctx) { + future.complete(entry.getData()); + } + + @Override + public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + }, null); + byte[] bytes = future.get(30, TimeUnit.SECONDS); + assertEquals(new String(bytes), "entry"); + } + @Test public void recoverLongTimeAfterMultipleWriteErrors() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("recoverLongTimeAfterMultipleWriteErrors"); From 950309b57a3674c43e2168168cdb9ef670ac6274 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sun, 29 Sep 2024 06:28:06 +0300 Subject: [PATCH 531/580] [fix][misc] Log Conscrypt security provider initialization warnings at debug level (#23364) --- .../org/apache/pulsar/common/util/SecurityUtility.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java index 8c1f1f5d8b39c..2b7b1a984634f 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java @@ -133,12 +133,12 @@ private static Provider loadConscryptProvider() { conscryptClazz.getMethod("checkAvailability").invoke(null); } catch (Throwable e) { if (e instanceof ClassNotFoundException) { - log.warn("Conscrypt isn't available in the classpath. Using JDK default security provider."); + log.debug("Conscrypt isn't available in the classpath. Using JDK default security provider."); } else if (e.getCause() instanceof UnsatisfiedLinkError) { - log.warn("Conscrypt isn't available for {} {}. Using JDK default security provider.", + log.debug("Conscrypt isn't available for {} {}. Using JDK default security provider.", System.getProperty("os.name"), System.getProperty("os.arch")); } else { - log.warn("Conscrypt isn't available. Using JDK default security provider." + log.debug("Conscrypt isn't available. Using JDK default security provider." + " Cause : {}, Reason : {}", e.getCause(), e.getMessage()); } return null; @@ -148,7 +148,7 @@ private static Provider loadConscryptProvider() { try { provider = (Provider) Class.forName(CONSCRYPT_PROVIDER_CLASS).getDeclaredConstructor().newInstance(); } catch (ReflectiveOperationException e) { - log.warn("Unable to get security provider for class {}", CONSCRYPT_PROVIDER_CLASS, e); + log.debug("Unable to get security provider for class {}", CONSCRYPT_PROVIDER_CLASS, e); return null; } From 5e832a1cc1441eaf8d64fe72c1a2af8829030d3d Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Sun, 29 Sep 2024 15:13:38 +0800 Subject: [PATCH 532/580] [fix][test] Fix ReplicatorRateLimiterTest (#23369) --- .../apache/pulsar/broker/service/ReplicatorRateLimiterTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java index bec6b558ea401..2e0dd0a90e8a6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorRateLimiterTest.java @@ -611,7 +611,7 @@ public void testReplicatorRateLimiterByBytes() throws Exception { } private static Optional getRateLimiter(PersistentTopic topic) { - return getRateLimiter(topic); + return topic.getReplicators().values().stream().findFirst().map(Replicator::getRateLimiter).orElseThrow(); } private static final Logger log = LoggerFactory.getLogger(ReplicatorRateLimiterTest.class); From f071a898775de3b527b538477d94a326f4b9c7e8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sun, 29 Sep 2024 14:16:54 +0300 Subject: [PATCH 533/580] [fix][cli] Remove deprecated "-client" JVM arg (#23370) --- conf/pulsar_tools_env.sh | 3 --- 1 file changed, 3 deletions(-) diff --git a/conf/pulsar_tools_env.sh b/conf/pulsar_tools_env.sh index 9d22b73905df3..96ee304bf0b3a 100755 --- a/conf/pulsar_tools_env.sh +++ b/conf/pulsar_tools_env.sh @@ -57,9 +57,6 @@ if [ -n "$PULSAR_MEM" ]; then fi PULSAR_MEM=${PULSAR_MEM:-"-Xmx128m -XX:MaxDirectMemorySize=128m"} -# Garbage collection options -PULSAR_GC=${PULSAR_GC:-" -client "} - # Extra options to be passed to the jvm PULSAR_EXTRA_OPTS="${PULSAR_MEM} ${PULSAR_GC} ${PULSAR_GC_LOG} -Dio.netty.leakDetectionLevel=disabled ${PULSAR_EXTRA_OPTS}" From 7e59bdeb9d142430c7119346a34c488045271c19 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sun, 29 Sep 2024 20:09:54 +0300 Subject: [PATCH 534/580] [improve] Install openssl in the docker image to fix compatibility with Apache Pulsar Helm chart (#23362) --- docker/pulsar/Dockerfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 81446ae5ee5ce..38b74e8503d51 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -92,7 +92,8 @@ RUN apk add --no-cache \ ca-certificates \ procps \ curl \ - bind-tools + bind-tools \ + openssl # Upgrade all packages to get latest versions with security fixes RUN apk upgrade --no-cache From b24285029b1113840ded42404229bc0eb344d5bd Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 30 Sep 2024 07:09:04 +0300 Subject: [PATCH 535/580] [improve][ci] Switch to Java 21 as default JVM version for CI (#23373) --- .github/workflows/pulsar-ci.yaml | 14 +++++++------- bin/function-localrunner | 9 ++++++++- conf/bkenv.sh | 14 +++++++++++--- conf/pulsar_env.sh | 13 ++++++++++--- docker/pulsar/Dockerfile | 3 ++- docker/pulsar/pom.xml | 1 + pom.xml | 28 +--------------------------- 7 files changed, 40 insertions(+), 42 deletions(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index dd93003eecce6..ad017674ac6ee 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -25,9 +25,9 @@ on: - branch-* - pulsar-* schedule: - # scheduled job with JDK 17 - - cron: '0 12 * * *' # scheduled job with JDK 21 + - cron: '0 12 * * *' + # scheduled job with JDK 17 # if cron expression is changed, make sure to update the expression in jdk_major_version step in preconditions job - cron: '0 6 * * *' workflow_dispatch: @@ -44,7 +44,7 @@ on: options: - '17' - '21' - default: '17' + default: '21' trace_test_resource_cleanup: description: 'Collect thread & heap information before exiting a test JVM. When set to "on", thread dump and heap histogram will be collected. When set to "full", a heap dump will also be collected.' required: true @@ -95,13 +95,13 @@ jobs: - name: Select JDK major version id: jdk_major_version run: | - # use JDK 21 for the scheduled build with cron expression '0 6 * * *' + # use JDK 17 for the scheduled build with cron expression '0 6 * * *' if [[ "${{ github.event_name == 'schedule' && github.event.schedule == '0 6 * * *' && 'true' || 'false' }}" == "true" ]]; then - echo "jdk_major_version=21" >> $GITHUB_OUTPUT + echo "jdk_major_version=17" >> $GITHUB_OUTPUT exit 0 fi - # use JDK 17 for build unless overridden with workflow_dispatch input - echo "jdk_major_version=${{ github.event_name == 'workflow_dispatch' && github.event.inputs.jdk_major_version || '17'}}" >> $GITHUB_OUTPUT + # use JDK 21 for build unless overridden with workflow_dispatch input + echo "jdk_major_version=${{ github.event_name == 'workflow_dispatch' && github.event.inputs.jdk_major_version || '21'}}" >> $GITHUB_OUTPUT - name: checkout if: ${{ github.event_name == 'pull_request' }} diff --git a/bin/function-localrunner b/bin/function-localrunner index b2405db724e72..a47f3efa48609 100755 --- a/bin/function-localrunner +++ b/bin/function-localrunner @@ -52,7 +52,14 @@ done PULSAR_MEM=${PULSAR_MEM:-"-Xmx128m -XX:MaxDirectMemorySize=128m"} # Garbage collection options -PULSAR_GC=${PULSAR_GC:-"-XX:+UseZGC -XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch"} +if [ -z "$PULSAR_GC" ]; then + PULSAR_GC="-XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch" + if [[ $JAVA_MAJOR_VERSION -ge 21 ]]; then + PULSAR_GC="-XX:+UseZGC -XX:+ZGenerational ${PULSAR_GC}" + else + PULSAR_GC="-XX:+UseZGC ${PULSAR_GC}" + fi +fi # Garbage collection log. PULSAR_GC_LOG_DIR=${PULSAR_GC_LOG_DIR:-logs} diff --git a/conf/bkenv.sh b/conf/bkenv.sh index b41532d3a0c91..8beea47cee312 100644 --- a/conf/bkenv.sh +++ b/conf/bkenv.sh @@ -37,9 +37,6 @@ BOOKIE_LOG_DIR=${BOOKIE_LOG_DIR:-"${PULSAR_LOG_DIR}"} # Memory size options BOOKIE_MEM=${BOOKIE_MEM:-${PULSAR_MEM:-"-Xms2g -Xmx2g -XX:MaxDirectMemorySize=2g"}} -# Garbage collection options -BOOKIE_GC=${BOOKIE_GC:-${PULSAR_GC:-"-XX:+UseZGC -XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch"}} - if [ -z "$JAVA_HOME" ]; then JAVA_BIN=java else @@ -60,6 +57,17 @@ for token in $("$JAVA_BIN" -version 2>&1 | grep 'version "'); do fi done +# Garbage collection options +BOOKIE_GC="${BOOKIE_GC:-${PULSAR_GC}}" +if [ -z "$BOOKIE_GC" ]; then + BOOKIE_GC="-XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch" + if [[ $JAVA_MAJOR_VERSION -ge 21 ]]; then + BOOKIE_GC="-XX:+UseZGC -XX:+ZGenerational ${BOOKIE_GC}" + else + BOOKIE_GC="-XX:+UseZGC ${BOOKIE_GC}" + fi +fi + if [[ -z "$BOOKIE_GC_LOG" ]]; then # fallback to PULSAR_GC_LOG if it is set BOOKIE_GC_LOG="$PULSAR_GC_LOG" diff --git a/conf/pulsar_env.sh b/conf/pulsar_env.sh index 3a069e31fdc90..f95d0ac83c13a 100755 --- a/conf/pulsar_env.sh +++ b/conf/pulsar_env.sh @@ -44,9 +44,6 @@ # Extra options to be passed to the jvm PULSAR_MEM=${PULSAR_MEM:-"-Xms2g -Xmx2g -XX:MaxDirectMemorySize=4g"} -# Garbage collection options -PULSAR_GC=${PULSAR_GC:-"-XX:+UseZGC -XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch"} - if [ -z "$JAVA_HOME" ]; then JAVA_BIN=java else @@ -67,6 +64,16 @@ for token in $("$JAVA_BIN" -version 2>&1 | grep 'version "'); do fi done +# Garbage collection options +if [ -z "$PULSAR_GC" ]; then + PULSAR_GC="-XX:+PerfDisableSharedMem -XX:+AlwaysPreTouch" + if [[ $JAVA_MAJOR_VERSION -ge 21 ]]; then + PULSAR_GC="-XX:+UseZGC -XX:+ZGenerational ${PULSAR_GC}" + else + PULSAR_GC="-XX:+UseZGC ${PULSAR_GC}" + fi +fi + PULSAR_GC_LOG_DIR=${PULSAR_GC_LOG_DIR:-"${PULSAR_LOG_DIR}"} if [[ -z "$PULSAR_GC_LOG" ]]; then diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 38b74e8503d51..f3b0f3d944bdc 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -18,6 +18,7 @@ # ARG ALPINE_VERSION=3.20 +ARG IMAGE_JDK_MAJOR_VERSION=21 # First create a stage with just the Pulsar tarball and scripts FROM alpine:$ALPINE_VERSION as pulsar @@ -54,7 +55,7 @@ RUN chmod -R o+rx /pulsar RUN echo 'OPTS="$OPTS -Dorg.xerial.snappy.use.systemlib=true"' >> /pulsar/conf/bkenv.sh ### Create one stage to include JVM distribution -FROM amazoncorretto:21-alpine AS jvm +FROM amazoncorretto:${IMAGE_JDK_MAJOR_VERSION}-alpine AS jvm RUN apk add --no-cache binutils diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 68d82ae552825..481fc319be732 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -83,6 +83,7 @@ target/pulsar-server-distribution-${project.version}-bin.tar.gz ${pulsar.client.python.version} ${snappy.version} + ${IMAGE_JDK_MAJOR_VERSION} ${project.basedir} diff --git a/pom.xml b/pom.xml index 1f7ecd3b90c1e..9f2330d7c75e1 100644 --- a/pom.xml +++ b/pom.xml @@ -84,9 +84,7 @@ flexible messaging model and an intuitive client API. 3.4.0 - http://archive.ubuntu.com/ubuntu/ - http://security.ubuntu.com/ubuntu/ - 17 + 21 **/Test*.java,**/*Test.java,**/*Tests.java,**/*TestCase.java @@ -2748,30 +2746,6 @@ flexible messaging model and an intuitive client API. - - ubuntu-mirror-set - - - env.UBUNTU_MIRROR - - - - - ${env.UBUNTU_MIRROR} - - - - ubuntu-security-mirror-set - - - env.UBUNTU_SECURITY_MIRROR - - - - - ${env.UBUNTU_SECURITY_MIRROR} - - jdk-major-version-set From 7d7dc80f0ee9e5926c8ac53a155de98bc6ffa3fa Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Mon, 30 Sep 2024 14:20:33 +0800 Subject: [PATCH 536/580] [fix][broker] Fix the broker registering might be blocked for long time (#23371) --- .../extensions/BrokerRegistryImpl.java | 14 ++++---- .../BrokerRegistryIntegrationTest.java | 13 ++++--- ...rRegistryMetadataStoreIntegrationTest.java | 35 +++++++++++++++++++ .../pulsar/client/impl/TableViewTest.java | 3 ++ .../pulsar/client/impl/TableViewImpl.java | 25 +++++++++---- 5 files changed, 71 insertions(+), 19 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java index 9fd0518a054cc..a13b332e6eb5f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java @@ -83,13 +83,6 @@ public BrokerRegistryImpl(PulsarService pulsar) { this.brokerLookupDataMetadataCache = pulsar.getLocalMetadataStore().getMetadataCache(BrokerLookupData.class); this.scheduler = pulsar.getLoadManagerExecutor(); this.listeners = new ArrayList<>(); - // The registered node is an ephemeral node that could be deleted when the metadata store client's session - // is expired. In this case, we should register again. - this.listeners.add((broker, notificationType) -> { - if (notificationType == NotificationType.Deleted && getBrokerId().equals(broker)) { - registerAsync(); - } - }); this.brokerIdKeyPath = keyPath(pulsar.getBrokerId()); this.brokerLookupData = new BrokerLookupData( pulsar.getWebServiceAddress(), @@ -223,11 +216,16 @@ private void handleMetadataStoreNotification(Notification t) { if (log.isDebugEnabled()) { log.debug("Handle notification: [{}]", t); } + // The registered node is an ephemeral node that could be deleted when the metadata store client's session + // is expired. In this case, we should register again. + final var brokerId = t.getPath().substring(LOADBALANCE_BROKERS_ROOT.length() + 1); + if (t.getType() == NotificationType.Deleted && getBrokerId().equals(brokerId)) { + registerAsync(); + } if (listeners.isEmpty()) { return; } this.scheduler.submit(() -> { - String brokerId = t.getPath().substring(LOADBALANCE_BROKERS_ROOT.length() + 1); for (BiConsumer listener : listeners) { listener.accept(brokerId, t.getType()); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java index d6615a8a5b49b..232088afb94fe 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java @@ -37,7 +37,7 @@ import org.testng.annotations.Test; @Slf4j -@Test(groups = "flaky") +@Test(groups = "broker") public class BrokerRegistryIntegrationTest { private static final String clusterName = "test"; @@ -63,13 +63,18 @@ protected void setup() throws Exception { @AfterClass(alwaysRun = true) protected void cleanup() throws Exception { + final var startMs = System.currentTimeMillis(); if (pulsar != null) { pulsar.close(); } + final var elapsedMs = System.currentTimeMillis() - startMs; bk.stop(); + if (elapsedMs > 5000) { + throw new RuntimeException("Broker took " + elapsedMs + "ms to close"); + } } - @Test(enabled = false) + @Test public void testRecoverFromNodeDeletion() throws Exception { // Simulate the case that the node was somehow deleted (e.g. by session timeout) Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( @@ -88,7 +93,7 @@ public void testRecoverFromNodeDeletion() throws Exception { Assert.assertEquals(brokerRegistry.getAvailableBrokersAsync().get(), List.of(pulsar.getBrokerId())); } - @Test(enabled = false) + @Test public void testRegisterAgain() throws Exception { Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); @@ -105,7 +110,7 @@ public void testRegisterAgain() throws Exception { }); } - private ServiceConfiguration brokerConfig() { + protected ServiceConfiguration brokerConfig() { final var config = new ServiceConfiguration(); config.setClusterName(clusterName); config.setAdvertisedAddress("localhost"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java new file mode 100644 index 0000000000000..3e01b1fad0f21 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java @@ -0,0 +1,35 @@ +/* + * 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.loadbalance.extensions; + +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateMetadataStoreTableViewImpl; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class BrokerRegistryMetadataStoreIntegrationTest extends BrokerRegistryIntegrationTest { + + @Override + protected ServiceConfiguration brokerConfig() { + final var config = super.brokerConfig(); + config.setLoadManagerServiceUnitStateTableViewClassName( + ServiceUnitStateMetadataStoreTableViewImpl.class.getName()); + return config; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TableViewTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TableViewTest.java index 61ab4de8a3294..5448751160a9c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TableViewTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TableViewTest.java @@ -173,6 +173,9 @@ public void testRefreshAPI(int partition) throws Exception { TableView tv = pulsarClient.newTableView(Schema.BYTES) .topic(topic) .create(); + // Verify refresh can handle the case when the topic is empty + tv.refreshAsync().get(3, TimeUnit.SECONDS); + // 2. Add a listen action to provide the test environment. // The listen action will be triggered when there are incoming messages every time. // This is a sync operation, so sleep in the listen action can slow down the reading rate of messages. diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java index 4f52060497864..17b49828eeced 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TableViewImpl.java @@ -36,6 +36,7 @@ import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.ReaderBuilder; @@ -259,7 +260,11 @@ private void handleMessage(Message msg) { @Override public CompletableFuture refreshAsync() { CompletableFuture completableFuture = new CompletableFuture<>(); - reader.thenCompose(reader -> getLastMessageIds(reader).thenAccept(lastMessageIds -> { + reader.thenCompose(reader -> getLastMessageIdOfNonEmptyTopics(reader).thenAccept(lastMessageIds -> { + if (lastMessageIds.isEmpty()) { + completableFuture.complete(null); + return; + } // After get the response of lastMessageIds, put the future and result into `refreshMap` // and then filter out partitions that has been read to the lastMessageID. pendingRefreshRequests.put(completableFuture, lastMessageIds); @@ -291,8 +296,12 @@ private CompletableFuture readAllExistingMessages(Reader reader) { AtomicLong messagesRead = new AtomicLong(); CompletableFuture future = new CompletableFuture<>(); - getLastMessageIds(reader).thenAccept(maxMessageIds -> { - readAllExistingMessages(reader, future, startTime, messagesRead, maxMessageIds); + getLastMessageIdOfNonEmptyTopics(reader).thenAccept(lastMessageIds -> { + if (lastMessageIds.isEmpty()) { + future.complete(null); + return; + } + readAllExistingMessages(reader, future, startTime, messagesRead, lastMessageIds); }).exceptionally(ex -> { future.completeExceptionally(ex); return null; @@ -300,13 +309,15 @@ private CompletableFuture readAllExistingMessages(Reader reader) { return future; } - private CompletableFuture> getLastMessageIds(Reader reader) { + private CompletableFuture> getLastMessageIdOfNonEmptyTopics(Reader reader) { return reader.getLastMessageIdsAsync().thenApply(lastMessageIds -> { - Map maxMessageIds = new ConcurrentHashMap<>(); + Map lastMessageIdMap = new ConcurrentHashMap<>(); lastMessageIds.forEach(topicMessageId -> { - maxMessageIds.put(topicMessageId.getOwnerTopic(), topicMessageId); + if (((MessageIdAdv) topicMessageId).getEntryId() >= 0) { + lastMessageIdMap.put(topicMessageId.getOwnerTopic(), topicMessageId); + } // else: a negative entry id represents an empty topic so that we don't have to read messages from it }); - return maxMessageIds; + return lastMessageIdMap; }); } From 9980967d777040706f15cc4a976af81d85c7faa6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 30 Sep 2024 17:10:15 +0300 Subject: [PATCH 537/580] [improve] Upgrade Pulsar Python client in docker image to 3.5.0 (#23377) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9f2330d7c75e1..881a1541c5eaf 100644 --- a/pom.xml +++ b/pom.xml @@ -82,7 +82,7 @@ flexible messaging model and an intuitive client API. ${maven.compiler.target} 8 - 3.4.0 + 3.5.0 21 From e0b754dd3938a2d142623001dbb15c92cc2f5cb4 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Mon, 30 Sep 2024 18:45:22 -0700 Subject: [PATCH 538/580] [improve][broker] check system topic is used before configuring system topic (ExtensibleLoadManagerImpl only) (#23381) --- .../extensions/ExtensibleLoadManagerImpl.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index 841f9bfb669d4..d8a279b854576 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -50,6 +50,7 @@ import org.apache.pulsar.broker.loadbalance.LoadManager; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewImpl; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateTableViewSyncer; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; @@ -992,7 +993,7 @@ protected void monitor() { if (isChannelOwner) { // System topic config might fail due to the race condition // with topic policy init(Topic policies cache have not init). - if (!configuredSystemTopics) { + if (isPersistentSystemTopicUsed() && !configuredSystemTopics) { configuredSystemTopics = configureSystemTopics(pulsar, COMPACTION_THRESHOLD); } if (role != Leader) { @@ -1080,4 +1081,11 @@ boolean running() { private boolean disabled() { return state.get() == State.DISABLED; } + + private boolean isPersistentSystemTopicUsed() { + return ServiceUnitStateTableViewImpl.class.getName() + .equals(pulsar.getConfiguration().getLoadManagerServiceUnitStateTableViewClassName()); + } + + } From 5b98d371922832b78c596c33042932c660bea0c4 Mon Sep 17 00:00:00 2001 From: Philipp Dolif <52791955+phil-cd@users.noreply.github.com> Date: Tue, 1 Oct 2024 13:32:02 +0200 Subject: [PATCH 539/580] [feat] Use producer name and sequence number as fallback key in Key_Shared implementation (#23219) --- .../broker/service/EntryAndMetadata.java | 3 + .../client/api/KeySharedSubscriptionTest.java | 66 +++++++++++-------- .../pulsar/client/impl/ConsumerBase.java | 8 ++- .../pulsar/common/protocol/Commands.java | 3 + .../common/compression/CommandsTest.java | 24 +++++-- .../testclient/PerformanceProducerTest.java | 42 +++++------- .../integration/messaging/MessagingBase.java | 7 +- 7 files changed, 88 insertions(+), 65 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java index 70643d5de2a3f..efa89a8ff16f6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java @@ -55,6 +55,9 @@ public byte[] getStickyKey() { return metadata.getOrderingKey(); } else if (metadata.hasPartitionKey()) { return metadata.getPartitionKey().getBytes(StandardCharsets.UTF_8); + } else if (metadata.hasProducerName() && metadata.hasSequenceId()) { + String fallbackKey = metadata.getProducerName() + "-" + metadata.getSequenceId(); + return fallbackKey.getBytes(StandardCharsets.UTF_8); } } return "NONE_KEY".getBytes(StandardCharsets.UTF_8); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index ddf7b0f1d5ee2..c08c37b413f4f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -44,6 +44,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Random; import java.util.Set; @@ -338,11 +339,11 @@ public void testConsumerCrashSendAndReceiveWithHashRangeAutoSplitStickyKeyConsum } @Test(dataProvider = "data") - public void testNonKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector( + public void testNoKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector( String topicType, boolean enableBatch ) throws PulsarClientException { - String topic = topicType + "://public/default/key_shared_none_key-" + UUID.randomUUID(); + String topic = topicType + "://public/default/key_shared_no_key-" + UUID.randomUUID(); @Cleanup Consumer consumer1 = createConsumer(topic); @@ -362,13 +363,13 @@ public void testNonKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelec .send(); } - receive(Lists.newArrayList(consumer1, consumer2, consumer3)); + receiveAndCheckDistribution(Lists.newArrayList(consumer1, consumer2, consumer3), 100); } @Test(dataProvider = "batch") - public void testNonKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) + public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) throws PulsarClientException { - String topic = "persistent://public/default/key_shared_none_key_exclusive-" + UUID.randomUUID(); + String topic = "persistent://public/default/key_shared_no_key_exclusive-" + UUID.randomUUID(); @Cleanup Consumer consumer1 = createConsumer(topic, KeySharedPolicy.stickyHashRange() @@ -385,21 +386,32 @@ public void testNonKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelec @Cleanup Producer producer = createProducer(topic, enableBatch); + int consumer1ExpectMessages = 0; + int consumer2ExpectMessages = 0; + int consumer3ExpectMessages = 0; + for (int i = 0; i < 100; i++) { producer.newMessage() .value(i) .send(); + + String fallbackKey = producer.getProducerName() + "-" + producer.getLastSequenceId(); + int slot = Murmur3_32Hash.getInstance().makeHash(fallbackKey.getBytes()) + % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; + if (slot <= 20000) { + consumer1ExpectMessages++; + } else if (slot <= 40000) { + consumer2ExpectMessages++; + } else { + consumer3ExpectMessages++; + } } - int slot = Murmur3_32Hash.getInstance().makeHash("NONE_KEY".getBytes()) - % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; + List, Integer>> checkList = new ArrayList<>(); - if (slot <= 20000) { - checkList.add(new KeyValue<>(consumer1, 100)); - } else if (slot <= 40000) { - checkList.add(new KeyValue<>(consumer2, 100)); - } else { - checkList.add(new KeyValue<>(consumer3, 100)); - } + checkList.add(new KeyValue<>(consumer1, consumer1ExpectMessages)); + checkList.add(new KeyValue<>(consumer2, consumer2ExpectMessages)); + checkList.add(new KeyValue<>(consumer3, consumer3ExpectMessages)); + receiveAndCheck(checkList); } @@ -1740,19 +1752,17 @@ private void receiveAndCheckDistribution(List> consumers, int expect private void receiveAndCheck(List, Integer>> checkList) throws PulsarClientException { Map> consumerKeys = new HashMap<>(); for (KeyValue, Integer> check : checkList) { - if (check.getValue() % 2 != 0) { - throw new IllegalArgumentException(); - } + Consumer consumer = check.getKey(); int received = 0; Map> lastMessageForKey = new HashMap<>(); for (Integer i = 0; i < check.getValue(); i++) { - Message message = check.getKey().receive(); + Message message = consumer.receive(); if (i % 2 == 0) { - check.getKey().acknowledge(message); + consumer.acknowledge(message); } String key = message.hasOrderingKey() ? new String(message.getOrderingKey()) : message.getKey(); log.info("[{}] Receive message key: {} value: {} messageId: {}", - check.getKey().getConsumerName(), key, message.getValue(), message.getMessageId()); + consumer.getConsumerName(), key, message.getValue(), message.getMessageId()); // check messages is order by key if (lastMessageForKey.get(key) == null) { Assert.assertNotNull(message); @@ -1761,8 +1771,8 @@ private void receiveAndCheck(List, Integer>> checkLis .compareTo(lastMessageForKey.get(key).getValue()) > 0); } lastMessageForKey.put(key, message); - consumerKeys.putIfAbsent(check.getKey(), new HashSet<>()); - consumerKeys.get(check.getKey()).add(key); + consumerKeys.putIfAbsent(consumer, new HashSet<>()); + consumerKeys.get(consumer).add(key); received++; } Assert.assertEquals(check.getValue().intValue(), received); @@ -1771,12 +1781,12 @@ private void receiveAndCheck(List, Integer>> checkLis // messages not acked, test redelivery lastMessageForKey = new HashMap<>(); for (int i = 0; i < redeliveryCount; i++) { - Message message = check.getKey().receive(); + Message message = consumer.receive(); received++; - check.getKey().acknowledge(message); + consumer.acknowledge(message); String key = message.hasOrderingKey() ? new String(message.getOrderingKey()) : message.getKey(); log.info("[{}] Receive redeliver message key: {} value: {} messageId: {}", - check.getKey().getConsumerName(), key, message.getValue(), message.getMessageId()); + consumer.getConsumerName(), key, message.getValue(), message.getMessageId()); // check redelivery messages is order by key if (lastMessageForKey.get(key) == null) { Assert.assertNotNull(message); @@ -1788,16 +1798,16 @@ private void receiveAndCheck(List, Integer>> checkLis } Message noMessages = null; try { - noMessages = check.getKey().receive(100, TimeUnit.MILLISECONDS); + noMessages = consumer.receive(100, TimeUnit.MILLISECONDS); } catch (PulsarClientException ignore) { } Assert.assertNull(noMessages, "redeliver too many messages."); Assert.assertEquals((check.getValue() + redeliveryCount), received); } Set allKeys = new HashSet<>(); - consumerKeys.forEach((k, v) -> v.forEach(key -> { + consumerKeys.forEach((k, v) -> v.stream().filter(Objects::nonNull).forEach(key -> { assertTrue(allKeys.add(key), - "Key "+ key + "is distributed to multiple consumers." ); + "Key " + key + " is distributed to multiple consumers." ); })); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 111cbdb8a8ef3..3073f3a833487 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -1192,11 +1192,13 @@ protected void callMessageListener(Message msg) { static final byte[] NONE_KEY = "NONE_KEY".getBytes(StandardCharsets.UTF_8); protected byte[] peekMessageKey(Message msg) { byte[] key = NONE_KEY; - if (msg.hasKey()) { - key = msg.getKeyBytes(); - } if (msg.hasOrderingKey()) { key = msg.getOrderingKey(); + } else if (msg.hasKey()) { + key = msg.getKeyBytes(); + } else if (msg.getProducerName() != null) { + String fallbackKey = msg.getProducerName() + "-" + msg.getSequenceId(); + key = fallbackKey.getBytes(StandardCharsets.UTF_8); } return key; } 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 3fb2fd5ad3d25..15b5676094ec1 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 @@ -1983,6 +1983,9 @@ public static byte[] peekStickyKey(ByteBuf metadataAndPayload, String topic, Str return Base64.getDecoder().decode(metadata.getPartitionKey()); } return metadata.getPartitionKey().getBytes(StandardCharsets.UTF_8); + } else if (metadata.hasProducerName() && metadata.hasSequenceId()) { + String fallbackKey = metadata.getProducerName() + "-" + metadata.getSequenceId(); + return fallbackKey.getBytes(StandardCharsets.UTF_8); } } catch (Throwable t) { log.error("[{}] [{}] Failed to peek sticky key from the message metadata", topic, subscription, t); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/compression/CommandsTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/compression/CommandsTest.java index 42f1a58100283..a1f79b7ae7faf 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/compression/CommandsTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/compression/CommandsTest.java @@ -98,9 +98,11 @@ private int computeChecksum(MessageMetadata msgMetadata, ByteBuf compressedPaylo public void testPeekStickyKey() { String message = "msg-1"; String partitionedKey = "key1"; + String producerName = "testProducer"; + int sequenceId = 1; MessageMetadata messageMetadata2 = new MessageMetadata() - .setSequenceId(1) - .setProducerName("testProducer") + .setSequenceId(sequenceId) + .setProducerName(producerName) .setPartitionKey(partitionedKey) .setPartitionKeyB64Encoded(false) .setPublishTime(System.currentTimeMillis()); @@ -113,16 +115,28 @@ public void testPeekStickyKey() { // test 64 encoded String partitionedKey2 = Base64.getEncoder().encodeToString("key2".getBytes(UTF_8)); MessageMetadata messageMetadata = new MessageMetadata() - .setSequenceId(1) - .setProducerName("testProducer") + .setSequenceId(sequenceId) + .setProducerName(producerName) .setPartitionKey(partitionedKey2) .setPartitionKeyB64Encoded(true) .setPublishTime(System.currentTimeMillis()); ByteBuf byteBuf2 = serializeMetadataAndPayload(Commands.ChecksumType.Crc32c, messageMetadata, Unpooled.copiedBuffer(message.getBytes(UTF_8))); byte[] bytes2 = Commands.peekStickyKey(byteBuf2, "topic-2", "sub-2"); - String key2 = Base64.getEncoder().encodeToString(bytes2);; + String key2 = Base64.getEncoder().encodeToString(bytes2); Assert.assertEquals(partitionedKey2, key2); ReferenceCountUtil.safeRelease(byteBuf2); + // test fallback key if no key given in message metadata + String fallbackPartitionedKey = producerName + "-" + sequenceId; + MessageMetadata messageMetadataWithoutKey = new MessageMetadata() + .setSequenceId(sequenceId) + .setProducerName(producerName) + .setPublishTime(System.currentTimeMillis()); + ByteBuf byteBuf3 = serializeMetadataAndPayload(Commands.ChecksumType.Crc32c, messageMetadataWithoutKey, + Unpooled.copiedBuffer(message.getBytes(UTF_8))); + byte[] bytes3 = Commands.peekStickyKey(byteBuf3, "topic-3", "sub-3"); + String key3 = new String(bytes3); + Assert.assertEquals(fallbackPartitionedKey, key3); + ReferenceCountUtil.safeRelease(byteBuf3); } } diff --git a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java index d0b25c6971697..519bed6cdb5ae 100644 --- a/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java +++ b/pulsar-testclient/src/test/java/org/apache/pulsar/testclient/PerformanceProducerTest.java @@ -98,26 +98,20 @@ public void testMsgKey() throws Exception { thread.start(); - int count1 = 0; - int count2 = 0; - for (int i = 0; i < 10; i++) { - Message message = consumer1.receive(1, TimeUnit.SECONDS); - if (message == null) { - break; - } - count1++; - consumer1.acknowledge(message); - } - for (int i = 0; i < 10; i++) { - Message message = consumer2.receive(1, TimeUnit.SECONDS); - if (message == null) { - break; - } - count2++; - consumer2.acknowledge(message); - } - //in key_share mode, only one consumer can get msg - Assert.assertTrue(count1 == 0 || count2 == 0); + // in key_shared mode if no message key is set, both consumers should receive messages + Awaitility.await() + .untilAsserted(() -> { + Message message = consumer1.receive(1, TimeUnit.SECONDS); + assertNotNull(message); + consumer1.acknowledge(message); + }); + + Awaitility.await() + .untilAsserted(() -> { + Message message = consumer2.receive(1, TimeUnit.SECONDS); + assertNotNull(message); + consumer2.acknowledge(message); + }); consumer1.close(); consumer2.close(); @@ -149,19 +143,15 @@ public void testMsgKey() throws Exception { Awaitility.await() .untilAsserted(() -> { Message message = newConsumer1.receive(1, TimeUnit.SECONDS); - if (message != null) { - newConsumer1.acknowledge(message); - } assertNotNull(message); + newConsumer1.acknowledge(message); }); Awaitility.await() .untilAsserted(() -> { Message message = newConsumer2.receive(1, TimeUnit.SECONDS); - if (message != null) { - newConsumer2.acknowledge(message); - } assertNotNull(message); + newConsumer2.acknowledge(message); }); thread2.interrupt(); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/MessagingBase.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/MessagingBase.java index 0e7106ef65ea1..ddedacc531a7c 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/MessagingBase.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/MessagingBase.java @@ -36,6 +36,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -150,11 +151,11 @@ protected String getPartitionedTopic(String topicPrefix, boolean isPersistent, i } } } - // Make sure key will not be distributed to multiple consumers + // Make sure key will not be distributed to multiple consumers (except null key) Set allKeys = Sets.newHashSet(); - consumerKeys.forEach((k, v) -> v.forEach(key -> { + consumerKeys.forEach((k, v) -> v.stream().filter(Objects::nonNull).forEach(key -> { assertTrue(allKeys.add(key), - "Key "+ key + "is distributed to multiple consumers" ); + "Key " + key + " is distributed to multiple consumers" ); })); assertEquals(messagesReceived.size(), messagesToReceive); } From 9eeffe595b6c2312b1b92eb8b9606639f25ab276 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 1 Oct 2024 12:53:45 -0700 Subject: [PATCH 540/580] [fix][broker] Support large number of unack message store for cursor recovery (#9292) --- .../mledger/ManagedLedgerConfig.java | 4 +- .../mledger/impl/ManagedCursorImpl.java | 64 +++++++++++++++++-- .../mledger/impl/RangeSetWrapper.java | 29 +++++++++ .../src/main/proto/MLDataFormats.proto | 6 ++ .../mledger/impl/ManagedCursorTest.java | 11 ++-- .../mledger/impl/ManagedLedgerBkTest.java | 47 +++++++++++++- .../BrokerRegistryIntegrationTest.java | 2 +- .../ConcurrentOpenLongPairRangeSet.java | 41 ++++++++++++ .../util/collections/LongPairRangeSet.java | 14 ++++ .../collections/OpenLongPairRangeSet.java | 46 +++++++++++++ 10 files changed, 249 insertions(+), 15 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 03439f93ccad8..a24251450b4f4 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -505,8 +505,10 @@ public int getMaxUnackedRangesToPersistInMetadataStore() { return maxUnackedRangesToPersistInMetadataStore; } - public void setMaxUnackedRangesToPersistInMetadataStore(int maxUnackedRangesToPersistInMetadataStore) { + public ManagedLedgerConfig setMaxUnackedRangesToPersistInMetadataStore( + int maxUnackedRangesToPersistInMetadataStore) { this.maxUnackedRangesToPersistInMetadataStore = maxUnackedRangesToPersistInMetadataStore; + return this; } /** diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index e27814eadd0b5..b39fd231cdc06 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -59,6 +59,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; import java.util.function.Predicate; +import java.util.stream.Collectors; import java.util.stream.LongStream; import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; @@ -91,12 +92,15 @@ import org.apache.bookkeeper.mledger.ScanOutcome; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongListMap; import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongProperty; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; +import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo.Builder; import org.apache.bookkeeper.mledger.proto.MLDataFormats.StringProperty; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; import org.apache.pulsar.common.util.DateFormatter; @@ -606,9 +610,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac } Position position = PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId()); - if (positionInfo.getIndividualDeletedMessagesCount() > 0) { - recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); - } + recoverIndividualDeletedMessages(positionInfo); if (getConfig().isDeletionAtBatchIndexLevelEnabled() && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) { recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList()); @@ -627,6 +629,45 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac } } + public void recoverIndividualDeletedMessages(PositionInfo positionInfo) { + if (positionInfo.getIndividualDeletedMessagesCount() > 0) { + recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); + } else if (positionInfo.getIndividualDeletedMessageRangesCount() > 0) { + List rangeList = positionInfo.getIndividualDeletedMessageRangesList(); + try { + Map rangeMap = rangeList.stream().collect(Collectors.toMap(LongListMap::getKey, + list -> list.getValuesList().stream().mapToLong(i -> i).toArray())); + individualDeletedMessages.build(rangeMap); + } catch (Exception e) { + log.warn("[{}]-{} Failed to recover individualDeletedMessages from serialized data", ledger.getName(), + name, e); + } + } + } + + private List buildLongPropertiesMap(Map properties) { + if (properties.isEmpty()) { + return Collections.emptyList(); + } + List longListMap = new ArrayList<>(); + MutableInt serializedSize = new MutableInt(); + properties.forEach((id, ranges) -> { + if (ranges == null || ranges.length <= 0) { + return; + } + org.apache.bookkeeper.mledger.proto.MLDataFormats.LongListMap.Builder lmBuilder = LongListMap.newBuilder() + .setKey(id); + for (long range : ranges) { + lmBuilder.addValues(range); + } + LongListMap lm = lmBuilder.build(); + longListMap.add(lm); + serializedSize.add(lm.getSerializedSize()); + }); + individualDeletedMessagesSerializedSize = serializedSize.toInteger(); + return longListMap; + } + private void recoverIndividualDeletedMessages(List individualDeletedMessagesList) { lock.writeLock().lock(); try { @@ -3125,12 +3166,23 @@ private List buildBatchEntryDeletio void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, final VoidCallback callback) { Position position = mdEntry.newPosition; - PositionInfo pi = PositionInfo.newBuilder().setLedgerId(position.getLedgerId()) + Builder piBuilder = PositionInfo.newBuilder().setLedgerId(position.getLedgerId()) .setEntryId(position.getEntryId()) - .addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()) .addAllBatchedEntryDeletionIndexInfo(buildBatchEntryDeletionIndexInfoList()) - .addAllProperties(buildPropertiesMap(mdEntry.properties)).build(); + .addAllProperties(buildPropertiesMap(mdEntry.properties)); + Map internalRanges = null; + try { + internalRanges = individualDeletedMessages.toRanges(getConfig().getMaxUnackedRangesToPersist()); + } catch (Exception e) { + log.warn("[{}]-{} Failed to serialize individualDeletedMessages", ledger.getName(), name, e); + } + if (internalRanges != null && !internalRanges.isEmpty()) { + piBuilder.addAllIndividualDeletedMessageRanges(buildLongPropertiesMap(internalRanges)); + } else { + piBuilder.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()); + } + PositionInfo pi = piBuilder.build(); if (log.isDebugEnabled()) { log.debug("[{}] Cursor {} Appending to ledger={} position={}", ledger.getName(), name, lh.getId(), diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java index a55e6444b2fd9..11cce409bec54 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/RangeSetWrapper.java @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.common.util.collections.OpenLongPairRangeSet; @@ -142,6 +143,16 @@ public Range lastRange() { return rangeSet.lastRange(); } + @Override + public Map toRanges(int maxRanges) { + return rangeSet.toRanges(maxRanges); + } + + @Override + public void build(Map internalRange) { + rangeSet.build(internalRange); + } + @Override public int cardinality(long lowerKey, long lowerValue, long upperKey, long upperValue) { return rangeSet.cardinality(lowerKey, lowerValue, upperKey, upperValue); @@ -176,4 +187,22 @@ public boolean isDirtyLedgers(long ledgerId) { public String toString() { return rangeSet.toString(); } + + @Override + public int hashCode() { + return rangeSet.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof RangeSetWrapper)) { + return false; + } + if (this == obj) { + return true; + } + @SuppressWarnings("rawtypes") + RangeSetWrapper set = (RangeSetWrapper) obj; + return this.rangeSet.equals(set.rangeSet); + } } diff --git a/managed-ledger/src/main/proto/MLDataFormats.proto b/managed-ledger/src/main/proto/MLDataFormats.proto index fdffed6762db7..f196649df0fdf 100644 --- a/managed-ledger/src/main/proto/MLDataFormats.proto +++ b/managed-ledger/src/main/proto/MLDataFormats.proto @@ -82,6 +82,7 @@ message PositionInfo { // Store which index in the batch message has been deleted repeated BatchedEntryDeletionIndexInfo batchedEntryDeletionIndexInfo = 5; + repeated LongListMap individualDeletedMessageRanges = 6; } message NestedPositionInfo { @@ -89,6 +90,11 @@ message NestedPositionInfo { required int64 entryId = 2; } +message LongListMap { + required int64 key = 1; + repeated int64 values = 2; +} + message MessageRange { required NestedPositionInfo lowerEndpoint = 1; required NestedPositionInfo upperEndpoint = 2; diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 8913c4013b4ab..1067cda441f6a 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -3223,7 +3223,7 @@ public void testOutOfOrderDeletePersistenceIntoLedgerWithClose() throws Exceptio managedLedgerConfig.setMaxUnackedRangesToPersistInMetadataStore(10); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(ledgerName, managedLedgerConfig); - ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(cursorName); + final ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(cursorName); List addedPositions = new ArrayList<>(); for (int i = 0; i < totalAddEntries; i++) { @@ -3269,7 +3269,8 @@ public void operationFailed(MetaStoreException e) { LedgerEntry entry = seq.nextElement(); PositionInfo positionInfo; positionInfo = PositionInfo.parseFrom(entry.getEntry()); - individualDeletedMessagesCount.set(positionInfo.getIndividualDeletedMessagesCount()); + c1.recoverIndividualDeletedMessages(positionInfo); + individualDeletedMessagesCount.set(c1.getIndividuallyDeletedMessagesSet().asRanges().size()); } catch (Exception e) { } latch.countDown(); @@ -3286,12 +3287,12 @@ public void operationFailed(MetaStoreException e) { @Cleanup("shutdown") ManagedLedgerFactory factory2 = new ManagedLedgerFactoryImpl(metadataStore, bkc); ledger = (ManagedLedgerImpl) factory2.open(ledgerName, managedLedgerConfig); - c1 = (ManagedCursorImpl) ledger.openCursor("c1"); + ManagedCursorImpl reopenCursor = (ManagedCursorImpl) ledger.openCursor("c1"); // verify cursor has been recovered - assertEquals(c1.getNumberOfEntriesInBacklog(false), totalAddEntries / 2); + assertEquals(reopenCursor.getNumberOfEntriesInBacklog(false), totalAddEntries / 2); // try to read entries which should only read non-deleted positions - List entries = c1.readEntries(totalAddEntries); + List entries = reopenCursor.readEntries(totalAddEntries); assertEquals(entries.size(), totalAddEntries / 2); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java index cd1dcf05c3708..9635376a782d3 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java @@ -23,7 +23,6 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; -import io.netty.buffer.ByteBuf; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -34,7 +33,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -import lombok.Cleanup; + import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperTestClient; import org.apache.bookkeeper.client.api.DigestType; @@ -53,9 +52,13 @@ import org.apache.bookkeeper.mledger.util.ThrowableToStringUtil; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; +import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.awaitility.Awaitility; import org.testng.annotations.Test; +import io.netty.buffer.ByteBuf; +import lombok.Cleanup; + public class ManagedLedgerBkTest extends BookKeeperClusterTestCase { public ManagedLedgerBkTest() { @@ -587,4 +590,44 @@ public void testPeriodicRollover() throws Exception { Awaitility.await().until(() -> cursorImpl.getCursorLedger() != currentLedgerId); } + /** + * This test validates that cursor serializes and deserializes individual-ack list from the bk-ledger. + * + * @throws Exception + */ + @Test + public void testUnackmessagesAndRecovery() throws Exception { + ManagedLedgerFactoryConfig factoryConf = new ManagedLedgerFactoryConfig(); + factoryConf.setMaxCacheSize(0); + + ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConf); + + ManagedLedgerConfig config = new ManagedLedgerConfig().setEnsembleSize(1).setWriteQuorumSize(1) + .setAckQuorumSize(1).setMetadataEnsembleSize(1).setMetadataWriteQuorumSize(1) + .setMaxUnackedRangesToPersistInMetadataStore(1).setMaxEntriesPerLedger(5).setMetadataAckQuorumSize(1); + ManagedLedger ledger = factory.open("my_test_unack_messages", config); + ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor("c1"); + + int totalEntries = 100; + for (int i = 0; i < totalEntries; i++) { + Position p = ledger.addEntry("entry".getBytes()); + if (i % 2 == 0) { + cursor.delete(p); + } + } + + LongPairRangeSet unackMessagesBefore = cursor.getIndividuallyDeletedMessagesSet(); + + ledger.close(); + + // open and recover cursor + ledger = factory.open("my_test_unack_messages", config); + cursor = (ManagedCursorImpl) ledger.openCursor("c1"); + + LongPairRangeSet unackMessagesAfter = cursor.getIndividuallyDeletedMessagesSet(); + assertTrue(unackMessagesBefore.equals(unackMessagesAfter)); + + ledger.close(); + factory.shutdown(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java index 232088afb94fe..e975671fa12e8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java @@ -80,7 +80,7 @@ public void testRecoverFromNodeDeletion() throws Exception { Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); pulsar.getLocalMetadataStore().delete(brokerMetadataPath, Optional.empty()); - Awaitility.await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> Assert.assertEquals( + Awaitility.await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> Assert.assertEquals( brokerRegistry.getAvailableBrokersAsync().join(), List.of(pulsar.getBrokerId()))); // If the node is deleted by unregister(), it should not recreate the path diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java index 6e45401978546..51f4a9ac51c90 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java @@ -18,16 +18,21 @@ */ package org.apache.pulsar.common.util.collections; +import static java.util.BitSet.valueOf; import static java.util.Objects.requireNonNull; import com.google.common.collect.BoundType; import com.google.common.collect.Range; import java.util.ArrayList; import java.util.BitSet; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Map.Entry; import java.util.NavigableMap; +import java.util.Objects; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.lang.mutable.MutableInt; /** @@ -253,6 +258,42 @@ public Range lastRange() { return Range.openClosed(consumer.apply(lastSet.getKey(), lower), consumer.apply(lastSet.getKey(), upper)); } + @Override + public Map toRanges(int maxRanges) { + Map internalBitSetMap = new HashMap<>(); + AtomicInteger rangeCount = new AtomicInteger(); + rangeBitSetMap.forEach((id, bmap) -> { + if (rangeCount.getAndAdd(bmap.cardinality()) > maxRanges) { + return; + } + internalBitSetMap.put(id, bmap.toLongArray()); + }); + return internalBitSetMap; + } + + @Override + public void build(Map internalRange) { + internalRange.forEach((id, ranges) -> rangeBitSetMap.put(id, valueOf(ranges))); + } + + @Override + public int hashCode() { + return Objects.hashCode(rangeBitSetMap); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof ConcurrentOpenLongPairRangeSet)) { + return false; + } + if (this == obj) { + return true; + } + @SuppressWarnings("rawtypes") + ConcurrentOpenLongPairRangeSet set = (ConcurrentOpenLongPairRangeSet) obj; + return this.rangeBitSetMap.equals(set.rangeBitSetMap); + } + @Override public int cardinality(long lowerKey, long lowerValue, long upperKey, long upperValue) { NavigableMap subMap = rangeBitSetMap.subMap(lowerKey, true, upperKey, true); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java index 8aad5587dfd38..df74857245bb3 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairRangeSet.java @@ -25,6 +25,7 @@ import java.util.Collection; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.NoSuchElementException; import java.util.Set; import lombok.EqualsAndHashCode; @@ -136,6 +137,19 @@ public interface LongPairRangeSet> { */ Range lastRange(); + default Map toRanges(int maxRanges) { + throw new UnsupportedOperationException(); + } + + /** + * Build {@link LongPairRangeSet} using internal ranges returned by {@link #toRanges(int)} . + * + * @param ranges + */ + default void build(Map ranges) { + throw new UnsupportedOperationException(); + } + /** * Return the number bit sets to true from lower (inclusive) to upper (inclusive). */ diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java index 6df6d414871ec..3076c6c5c5fa1 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java @@ -18,16 +18,21 @@ */ package org.apache.pulsar.common.util.collections; +import static java.util.BitSet.valueOf; import static java.util.Objects.requireNonNull; import com.google.common.collect.BoundType; import com.google.common.collect.Range; import java.util.ArrayList; import java.util.BitSet; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Map.Entry; import java.util.NavigableMap; +import java.util.Objects; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; import javax.annotation.concurrent.NotThreadSafe; import org.apache.commons.lang.mutable.MutableInt; @@ -250,6 +255,47 @@ public Range lastRange() { return Range.openClosed(consumer.apply(lastSet.getKey(), lower), consumer.apply(lastSet.getKey(), upper)); } + @Override + public Map toRanges(int maxRanges) { + Map internalBitSetMap = new HashMap<>(); + AtomicInteger rangeCount = new AtomicInteger(); + rangeBitSetMap.forEach((id, bmap) -> { + if (rangeCount.getAndAdd(bmap.cardinality()) > maxRanges) { + return; + } + internalBitSetMap.put(id, bmap.toLongArray()); + }); + return internalBitSetMap; + } + + @Override + public void build(Map internalRange) { + internalRange.forEach((id, ranges) -> { + BitSet bitset = createNewBitSet(); + bitset.or(valueOf(ranges)); + rangeBitSetMap.put(id, bitset); + }); + } + + + @Override + public int hashCode() { + return Objects.hashCode(rangeBitSetMap); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof OpenLongPairRangeSet)) { + return false; + } + if (this == obj) { + return true; + } + @SuppressWarnings("rawtypes") + OpenLongPairRangeSet set = (OpenLongPairRangeSet) obj; + return this.rangeBitSetMap.equals(set.rangeBitSetMap); + } + @Override public int cardinality(long lowerKey, long lowerValue, long upperKey, long upperValue) { NavigableMap subMap = rangeBitSetMap.subMap(lowerKey, true, upperKey, true); From d2c91b1e1a8fc2fb233eb2856ddb6f53511ba201 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 1 Oct 2024 22:54:32 +0300 Subject: [PATCH 541/580] [fix][broker] Cancel possible pending replay read in cancelPendingRead (#23384) --- .../persistent/PersistentDispatcherMultipleConsumers.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 73ad2cf0a3dee..d479d8f384ee9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -650,8 +650,9 @@ public synchronized CompletableFuture disconnectAllConsumers( @Override protected void cancelPendingRead() { - if (havePendingRead && cursor.cancelPendingReadRequest()) { + if ((havePendingRead || havePendingReplayRead) && cursor.cancelPendingReadRequest()) { havePendingRead = false; + havePendingReplayRead = false; } } From 50802bea7288f34b39ee19a47ed31b3629a9ddda Mon Sep 17 00:00:00 2001 From: Andrey Yegorov <8622884+dlg99@users.noreply.github.com> Date: Tue, 1 Oct 2024 15:37:43 -0700 Subject: [PATCH 542/580] [improve][pip] PIP-381: Handle large PositionInfo state (#23328) --- pip/pip-381-large-positioninfo.md | 153 ++++++++++++++++++++++++++++++ 1 file changed, 153 insertions(+) create mode 100644 pip/pip-381-large-positioninfo.md diff --git a/pip/pip-381-large-positioninfo.md b/pip/pip-381-large-positioninfo.md new file mode 100644 index 0000000000000..9dbe1cc7935e3 --- /dev/null +++ b/pip/pip-381-large-positioninfo.md @@ -0,0 +1,153 @@ +# PIP-381: Handle large PositionInfo state + +# Background knowledge + +In case of KEY_SHARED subscription and out-of-order acknowledgments, +the PositionInfo state can be persisted to preserve the state, +with configurable maximum number of ranges to persist: + +``` +# Max number of "acknowledgment holes" that are going to be persistently stored. +# When acknowledging out of order, a consumer will leave holes that are supposed +# to be quickly filled by acking all the messages. The information of which +# messages are acknowledged is persisted by compressing in "ranges" of messages +# that were acknowledged. After the max number of ranges is reached, the information +# will only be tracked in memory and messages will be redelivered in case of +# crashes. +managedLedgerMaxUnackedRangesToPersist=10000 +``` + +The PositionInfo state is stored to the BookKeeper as a single entry, and it can grow large if the number of ranges is large. +Currently, this means that BookKeeper can fail persisting too large PositionInfo state, e.g. over 1MB +by default and the ManagedCursor recovery on topic reload might not succeed. + +There is an abandoned PIP-81 for similar problem, this PIP takes over. + +# Motivation + +While keeping the number of ranges low to prevent such problems is a common sense solution, there are cases +where the higher number of ranges is required. For example, in case of the JMS protocol handler, +JMS consumers with filters may end up processing data out of order and/or at different speed, +and the number of ranges can grow large. + +# Goals + +Store the PositionInfo state in a BookKeeper ledger as multiple entries if the state grows too large to be stored as a single entry. + +## In Scope + +Transparent backwards compatibility if the PositionInfo state is small enough. + +## Out of Scope + +Backwards compatibility in case of the PositionInfo state is too large to be stored as a single entry. + +# High Level Design + +Cursor state writes and reads are happening at the same cases as currently, without changes. + +Write path: + +1. serialize the PositionInfo state to a byte array. +2. if the byte array is smaller than the threshold, store it as a single entry, as now. Done. +3. if the byte array is larger than the threshold, split it to smaller chunks and store the chunks in a BookKeeper ledger. +4. write the "footer" into the metadata store as a last entry. + +See `persistPositionToLedger()` in `ManagedCursorImpl` for the implementation. + +The footer is a JSON representation of + +```java + public static final class ChunkSequenceFooter { + private int numParts; + private int length; + } +``` + +Read path: + +1. read the last entry from the metadata store. +2. if the entry does not appear to be a JSON, treat it as serialized PositionInfo state and use it as is. Done. +3. if the footer is a JSON, parse number of chunks and length from the json. +4. read the chunks from the BookKeeper ledger (entries from `startPos = footerPosition - chunkSequenceFooter.numParts` to `footerPosition - 1`) and merge them. +5. parse the merged byte array as a PositionInfo state. + +See `recoverFromLedgerByEntryId()` in `ManagedCursorImpl` for the implementation. + +## Design & Implementation Details + +Proposed implementation: https://github.com/apache/pulsar/pull/22799 + +## Public-facing Changes + +Nothing + +### Public API + +None + +### Binary protocol + +No public-facing changes + +### Configuration + +* **managedLedgerMaxUnackedRangesToPersist**: int, default 10000 (existing parameter). Controls number of unacked ranges to store. +* **persistentUnackedRangesWithMultipleEntriesEnabled**: boolean, default false. If true, the PositionInfo state is stored as multiple entries in BookKeeper if it grows too large. +* **persistentUnackedRangesMaxEntrySize**: int, default 1MB. Maximum size of a single entry in BookKeeper, in bytes. +* **cursorInfoCompressionType**: string, default "NONE". Compression type to use for the PositionInfo state. + +### CLI + +None + +### Metrics + + + + +# Monitoring + +Existing monitoring should be sufficient. + +# Security Considerations + +N/A + +# Backward & Forward Compatibility + +## Upgrade + +Not affected, just upgrade. + +## Downgrade / Rollback + +Not affected, just downgrade **as long as the managedLedgerMaxUnackedRangesToPersist was in the range to fit it into a single entry in BK**. + +## Pulsar Geo-Replication Upgrade & Downgrade/Rollback Considerations + +Not affected AFAIK. + +# Alternatives + +1. Do nothing. Keep the number of ranges low. This does not fit some use cases. +2. Come up with an extremely efficient storage format for the unacked ranges to fit them into a single entry all the time for e.g. 10mil ranges. This breaks backwards compatibility and the feasibility is unclear. + +# General Notes + +# Links + +* Proposed implementation: https://github.com/apache/pulsar/pull/22799 +* PIP-81: https://github.com/apache/pulsar/wiki/PIP-81:-Split-the-individual-acknowledgments-into-multiple-entries +* PR that implements better storage format for the unacked ranges (alternative 2): https://github.com/apache/pulsar/pull/9292 + +ML discussion and voting threads: + +* Mailing List discussion thread: https://lists.apache.org/thread/8sm0h804v5914zowghrqxr92fp7c255d +* Mailing List voting thread: https://lists.apache.org/thread/q31fx0rox9tdt34xsmo1ol1l76q8vk99 From 1dad0788c0e3bdb2a2d76ba2908ddb30441460c3 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Tue, 1 Oct 2024 17:03:52 -0700 Subject: [PATCH 543/580] [improve][pip] Improve PIP process and voting timeline (#23387) --- pip/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pip/README.md b/pip/README.md index f386647e8c5c2..216cdd56298c6 100644 --- a/pip/README.md +++ b/pip/README.md @@ -77,7 +77,7 @@ The process works in the following way: sending a message using subject `[VOTE] PIP-xxx: {PIP TITLE}`. Make sure to include a link to the PIP PR in the body of the message. Make sure to update the PIP with a link to the vote. You can obtain it from [Apache Pony Mail](https://lists.apache.org/list.html?dev@pulsar.apache.org). Everyone is welcome to vote on the proposal, though only the vote of the PMC members will be considered binding. - It is required to have a lazy majority of at least 3 binding +1s votes. + The requirement is to have at least one binding +1 vote from a lazy majority if no binding -1 votes have been cast on the PIP. The vote should stay open for at least 48 hours. 9. When the vote is closed, if the outcome is positive, ask a PMC member (using voting thread on mailing list) to merge the PR. 10. If the outcome is negative, please close the PR (with a small comment that the close is a result of a vote). From c41c7e944d9a556dc02710314310457df82da502 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 2 Oct 2024 06:46:52 +0300 Subject: [PATCH 544/580] [improve] Configure Rocksdb to use musl libc flavor of the native library (#23375) --- docker/pulsar/Dockerfile | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index f3b0f3d944bdc..f8c22dc14a821 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -141,6 +141,8 @@ COPY --from=pulsar /pulsar /pulsar WORKDIR /pulsar ENV PATH=$PATH:$JAVA_HOME/bin:/pulsar/bin +# Use musl libc library for RocksDB +ENV ROCKSDB_MUSL_LIBC=true # The UID must be non-zero. Otherwise, it is arbitrary. No logic should rely on its specific value. ARG DEFAULT_USERNAME=pulsar From adb9014dbac21afdfb5fc252ac38e07ed2d6b19c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 2 Oct 2024 11:13:36 +0300 Subject: [PATCH 545/580] [fix][broker] Fix out-of-order issues with ConsistentHashingStickyKeyConsumerSelector (#23327) --- ...stentHashingStickyKeyConsumerSelector.java | 104 ++--- .../service/ConsumerIdentityWrapper.java | 70 ++++ .../service/ConsumerNameIndexTracker.java | 136 +++++++ ...tHashingStickyKeyConsumerSelectorTest.java | 366 +++++++++++++++++- .../service/ConsumerIdentityWrapperTest.java | 68 ++++ .../service/ConsumerNameIndexTrackerTest.java | 157 ++++++++ ...ckyKeyDispatcherMultipleConsumersTest.java | 9 +- .../org/apache/pulsar/client/api/Range.java | 11 +- 8 files changed, 853 insertions(+), 68 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapper.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerNameIndexTracker.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapperTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerNameIndexTrackerTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java index b2b2b512c8cfc..1ae9a6ff96b7d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java @@ -18,10 +18,8 @@ */ package org.apache.pulsar.broker.service; -import com.google.common.collect.Lists; import java.util.ArrayList; -import java.util.Comparator; -import java.util.LinkedHashMap; +import java.util.IdentityHashMap; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -44,7 +42,9 @@ public class ConsistentHashingStickyKeyConsumerSelector implements StickyKeyCons private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); // Consistent-Hash ring - private final NavigableMap> hashRing; + private final NavigableMap hashRing; + // Tracks the used consumer name indexes for each consumer name + private final ConsumerNameIndexTracker consumerNameIndexTracker = new ConsumerNameIndexTracker(); private final int numberOfPoints; @@ -57,21 +57,20 @@ public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints) { public CompletableFuture addConsumer(Consumer consumer) { rwLock.writeLock().lock(); try { + ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer); // Insert multiple points on the hash ring for every consumer // The points are deterministically added based on the hash of the consumer name for (int i = 0; i < numberOfPoints; i++) { - int hash = calculateHashForConsumerAndIndex(consumer, i); - hashRing.compute(hash, (k, v) -> { - if (v == null) { - return Lists.newArrayList(consumer); - } else { - if (!v.contains(consumer)) { - v.add(consumer); - v.sort(Comparator.comparing(Consumer::consumerName, String::compareTo)); - } - return v; - } - }); + int consumerNameIndex = + consumerNameIndexTracker.increaseConsumerRefCountAndReturnIndex(consumerIdentityWrapper); + int hash = calculateHashForConsumerAndIndex(consumer, consumerNameIndex, i); + // When there's a collision, the new consumer will replace the old one. + // This is a rare case, and it is acceptable to replace the old consumer since there + // are multiple points for each consumer. This won't affect the overall distribution significantly. + ConsumerIdentityWrapper removed = hashRing.put(hash, consumerIdentityWrapper); + if (removed != null) { + consumerNameIndexTracker.decreaseConsumerRefCount(removed); + } } return CompletableFuture.completedFuture(null); } finally { @@ -79,8 +78,19 @@ public CompletableFuture addConsumer(Consumer consumer) { } } - private static int calculateHashForConsumerAndIndex(Consumer consumer, int index) { - String key = consumer.consumerName() + KEY_SEPARATOR + index; + /** + * Calculate the hash for a consumer and hash ring point. + * The hash is calculated based on the consumer name, consumer name index, and hash ring point index. + * The resulting hash is used as the key to insert the consumer into the hash ring. + * + * @param consumer the consumer + * @param consumerNameIndex the index of the consumer name + * @param hashRingPointIndex the index of the hash ring point + * @return the hash value + */ + private static int calculateHashForConsumerAndIndex(Consumer consumer, int consumerNameIndex, + int hashRingPointIndex) { + String key = consumer.consumerName() + KEY_SEPARATOR + consumerNameIndex + KEY_SEPARATOR + hashRingPointIndex; return Murmur3_32Hash.getInstance().makeHash(key.getBytes()); } @@ -88,20 +98,16 @@ private static int calculateHashForConsumerAndIndex(Consumer consumer, int index public void removeConsumer(Consumer consumer) { rwLock.writeLock().lock(); try { - // Remove all the points that were added for this consumer - for (int i = 0; i < numberOfPoints; i++) { - int hash = calculateHashForConsumerAndIndex(consumer, i); - hashRing.compute(hash, (k, v) -> { - if (v == null) { - return null; - } else { - v.removeIf(c -> c.equals(consumer)); - if (v.isEmpty()) { - v = null; - } - return v; + ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer); + int consumerNameIndex = consumerNameIndexTracker.getTrackedIndex(consumerIdentityWrapper); + if (consumerNameIndex > -1) { + // Remove all the points that were added for this consumer + for (int i = 0; i < numberOfPoints; i++) { + int hash = calculateHashForConsumerAndIndex(consumer, consumerNameIndex, i); + if (hashRing.remove(hash, consumerIdentityWrapper)) { + consumerNameIndexTracker.decreaseConsumerRefCount(consumerIdentityWrapper); } - }); + } } } finally { rwLock.writeLock().unlock(); @@ -115,16 +121,13 @@ public Consumer select(int hash) { if (hashRing.isEmpty()) { return null; } - - List consumerList; - Map.Entry> ceilingEntry = hashRing.ceilingEntry(hash); + Map.Entry ceilingEntry = hashRing.ceilingEntry(hash); if (ceilingEntry != null) { - consumerList = ceilingEntry.getValue(); + return ceilingEntry.getValue().consumer; } else { - consumerList = hashRing.firstEntry().getValue(); + // Handle wrap-around in the hash ring, return the first consumer + return hashRing.firstEntry().getValue().consumer; } - - return consumerList.get(hash % consumerList.size()); } finally { rwLock.readLock().unlock(); } @@ -132,16 +135,27 @@ public Consumer select(int hash) { @Override public Map> getConsumerKeyHashRanges() { - Map> result = new LinkedHashMap<>(); + Map> result = new IdentityHashMap<>(); rwLock.readLock().lock(); try { + if (hashRing.isEmpty()) { + return result; + } int start = 0; - for (Map.Entry> entry: hashRing.entrySet()) { - for (Consumer consumer: entry.getValue()) { - result.computeIfAbsent(consumer, key -> new ArrayList<>()) - .add(Range.of(start, entry.getKey())); - } - start = entry.getKey() + 1; + int lastKey = 0; + for (Map.Entry entry: hashRing.entrySet()) { + Consumer consumer = entry.getValue().consumer; + result.computeIfAbsent(consumer, key -> new ArrayList<>()) + .add(Range.of(start, entry.getKey())); + lastKey = entry.getKey(); + start = lastKey + 1; + } + // Handle wrap-around in the hash ring, the first consumer will also contain the range from the last key + // to the maximum value of the hash range + Consumer firstConsumer = hashRing.firstEntry().getValue().consumer; + List ranges = result.get(firstConsumer); + if (lastKey != Integer.MAX_VALUE - 1) { + ranges.add(Range.of(lastKey + 1, Integer.MAX_VALUE - 1)); } } finally { rwLock.readLock().unlock(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapper.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapper.java new file mode 100644 index 0000000000000..2aae1d9b0622e --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapper.java @@ -0,0 +1,70 @@ +/* + * 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.service; + +/** + * A wrapper class for a Consumer instance that provides custom implementations + * of equals and hashCode methods. The equals method returns true if and only if + * the compared instance is the same instance. + * + *

The reason for this class is the custom implementation of {@link Consumer#equals(Object)}. + * Using this wrapper class will be useful in use cases where it's necessary to match a key + * in a map by instance or a value in a set by instance.

+ */ +class ConsumerIdentityWrapper { + final Consumer consumer; + + public ConsumerIdentityWrapper(Consumer consumer) { + this.consumer = consumer; + } + + /** + * Compares this wrapper to the specified object. The result is true if and only if + * the argument is not null and is a ConsumerIdentityWrapper object that wraps + * the same Consumer instance. + * + * @param obj the object to compare this ConsumerIdentityWrapper against + * @return true if the given object represents a ConsumerIdentityWrapper + * equivalent to this wrapper, false otherwise + */ + @Override + public boolean equals(Object obj) { + if (obj instanceof ConsumerIdentityWrapper) { + ConsumerIdentityWrapper other = (ConsumerIdentityWrapper) obj; + return consumer == other.consumer; + } + return false; + } + + /** + * Returns a hash code for this wrapper. The hash code is computed based on + * the wrapped Consumer instance. + * + * @return a hash code value for this object + */ + @Override + public int hashCode() { + return consumer.hashCode(); + } + + @Override + public String toString() { + return consumer.toString(); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerNameIndexTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerNameIndexTracker.java new file mode 100644 index 0000000000000..1f93313ab1b71 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerNameIndexTracker.java @@ -0,0 +1,136 @@ +/* + * 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.service; + +import java.util.HashMap; +import java.util.Map; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.commons.lang3.mutable.MutableInt; +import org.roaringbitmap.RoaringBitmap; + +/** + * Tracks the used consumer name indexes for each consumer name. + * This is used by {@link ConsistentHashingStickyKeyConsumerSelector} to get a unique "consumer name index" + * for each consumer name. It is useful when there are multiple consumers with the same name, but they are + * different consumers. The purpose of the index is to prevent collisions in the hash ring. + * + * The consumer name index serves as an additional key for the hash ring assignment. The logic keeps track of + * used "index slots" for each consumer name and assigns the first unused index when a new consumer is added. + * This approach minimizes hash collisions due to using the same consumer name. + * + * An added benefit of this tracking approach is that a consumer that leaves and then rejoins immediately will get the + * same index and therefore the same assignments in the hash ring. This improves stability since the hash assignment + * changes are minimized over time, although a better solution would be to avoid reusing the same consumer name + * in the first place. + * + * When a consumer is removed, the index is deallocated. RoaringBitmap is used to keep track of the used indexes. + * The data structure to track a consumer name is removed when the reference count of the consumer name is zero. + * + * This class is not thread-safe and should be used in a synchronized context in the caller. + */ +@NotThreadSafe +class ConsumerNameIndexTracker { + // tracks the used index slots for each consumer name + private final Map consumerNameIndexSlotsMap = new HashMap<>(); + // tracks the active consumer entries + private final Map consumerEntries = new HashMap<>(); + + // Represents a consumer entry in the tracker, including the consumer name, index, and reference count. + record ConsumerEntry(String consumerName, int nameIndex, MutableInt refCount) { + } + + /* + * Tracks the used indexes for a consumer name using a RoaringBitmap. + * A specific index slot is used when the bit is set. + * When all bits are cleared, the customer name can be removed from tracking. + */ + static class ConsumerNameIndexSlots { + private RoaringBitmap indexSlots = new RoaringBitmap(); + + public int allocateIndexSlot() { + // find the first index that is not set, if there is no such index, add a new one + int index = (int) indexSlots.nextAbsentValue(0); + if (index == -1) { + index = indexSlots.getCardinality(); + } + indexSlots.add(index); + return index; + } + + public boolean deallocateIndexSlot(int index) { + indexSlots.remove(index); + return indexSlots.isEmpty(); + } + } + + /* + * Adds a reference to the consumer and returns the index assigned to this consumer. + */ + public int increaseConsumerRefCountAndReturnIndex(ConsumerIdentityWrapper wrapper) { + ConsumerEntry entry = consumerEntries.computeIfAbsent(wrapper, k -> { + String consumerName = wrapper.consumer.consumerName(); + return new ConsumerEntry(consumerName, allocateConsumerNameIndex(consumerName), new MutableInt(0)); + }); + entry.refCount.increment(); + return entry.nameIndex; + } + + private int allocateConsumerNameIndex(String consumerName) { + return getConsumerNameIndexBitmap(consumerName).allocateIndexSlot(); + } + + private ConsumerNameIndexSlots getConsumerNameIndexBitmap(String consumerName) { + return consumerNameIndexSlotsMap.computeIfAbsent(consumerName, k -> new ConsumerNameIndexSlots()); + } + + /* + * Decreases the reference count of the consumer and removes the consumer name from tracking if the ref count is + * zero. + */ + public void decreaseConsumerRefCount(ConsumerIdentityWrapper removed) { + ConsumerEntry consumerEntry = consumerEntries.get(removed); + int refCount = consumerEntry.refCount.decrementAndGet(); + if (refCount == 0) { + deallocateConsumerNameIndex(consumerEntry.consumerName, consumerEntry.nameIndex); + consumerEntries.remove(removed, consumerEntry); + } + } + + private void deallocateConsumerNameIndex(String consumerName, int index) { + if (getConsumerNameIndexBitmap(consumerName).deallocateIndexSlot(index)) { + consumerNameIndexSlotsMap.remove(consumerName); + } + } + + /* + * Returns the currently tracked index for the consumer. + */ + public int getTrackedIndex(ConsumerIdentityWrapper wrapper) { + ConsumerEntry consumerEntry = consumerEntries.get(wrapper); + return consumerEntry != null ? consumerEntry.nameIndex : -1; + } + + int getTrackedConsumerNamesCount() { + return consumerNameIndexSlotsMap.size(); + } + + int getTrackedConsumersCount() { + return consumerEntries.size(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java index 48311c57338b5..04aafc49b47e6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java @@ -18,19 +18,27 @@ */ package org.apache.pulsar.broker.service; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; - +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException; import org.apache.pulsar.client.api.Range; +import org.assertj.core.data.Offset; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.Test; @@ -40,7 +48,7 @@ public class ConsistentHashingStickyKeyConsumerSelectorTest { @Test public void testConsumerSelect() throws ConsumerAssignException { - ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(200); String key1 = "anyKey"; Assert.assertNull(selector.select(key1.getBytes())); @@ -146,31 +154,115 @@ public void testGetConsumerKeyHashRanges() throws BrokerServiceException.Consume ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(3); List consumerName = Arrays.asList("consumer1", "consumer2", "consumer3"); List consumers = new ArrayList<>(); + long id=0; for (String s : consumerName) { - Consumer consumer = mock(Consumer.class); - when(consumer.consumerName()).thenReturn(s); + Consumer consumer = createMockConsumer(s, s, id++); selector.addConsumer(consumer); consumers.add(consumer); } + + // check that results are the same when called multiple times + assertThat(selector.getConsumerKeyHashRanges()) + .containsExactlyEntriesOf(selector.getConsumerKeyHashRanges()); + Map> expectedResult = new HashMap<>(); + assertThat(consumers.get(0).consumerName()).isEqualTo("consumer1"); expectedResult.put(consumers.get(0), Arrays.asList( - Range.of(119056335, 242013991), - Range.of(722195657, 1656011842), - Range.of(1707482098, 1914695766))); + Range.of(95615213, 440020355), + Range.of(440020356, 455987436), + Range.of(1189794593, 1264144431))); + assertThat(consumers.get(1).consumerName()).isEqualTo("consumer2"); expectedResult.put(consumers.get(1), Arrays.asList( - Range.of(0, 90164503), - Range.of(90164504, 119056334), - Range.of(382436668, 722195656))); + Range.of(939655188, 1189794592), + Range.of(1314727625, 1977451233), + Range.of(1977451234, 2016237253))); + assertThat(consumers.get(2).consumerName()).isEqualTo("consumer3"); expectedResult.put(consumers.get(2), Arrays.asList( - Range.of(242013992, 242377547), - Range.of(242377548, 382436667), - Range.of(1656011843, 1707482097))); - for (Map.Entry> entry : selector.getConsumerKeyHashRanges().entrySet()) { - System.out.println(entry.getValue()); - Assert.assertEquals(entry.getValue(), expectedResult.get(entry.getKey())); - expectedResult.remove(entry.getKey()); + Range.of(0, 95615212), + Range.of(455987437, 939655187), + Range.of(1264144432, 1314727624), + Range.of(2016237254, 2147483646))); + Map> consumerKeyHashRanges = selector.getConsumerKeyHashRanges(); + assertThat(consumerKeyHashRanges).containsExactlyInAnyOrderEntriesOf(expectedResult); + + // check that ranges are continuous and cover the whole range + List allRanges = + consumerKeyHashRanges.values().stream().flatMap(List::stream).sorted().collect(Collectors.toList()); + Range previousRange = null; + for (Range range : allRanges) { + if (previousRange != null) { + assertThat(range.getStart()).isEqualTo(previousRange.getEnd() + 1); + } + previousRange = range; + } + assertThat(allRanges.stream().mapToInt(r -> r.getEnd() - r.getStart() + 1).sum()).isEqualTo(Integer.MAX_VALUE); + } + + @Test + public void testConsumersGetSufficientlyAccuratelyEvenlyMapped() + throws BrokerServiceException.ConsumerAssignException { + ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(200); + List consumers = new ArrayList<>(); + for (int i = 0; i < 20; i++) { + // use the same name for all consumers, use toString to distinguish them + Consumer consumer = createMockConsumer("consumer", String.format("index %02d", i), i); + selector.addConsumer(consumer); + consumers.add(consumer); } - Assert.assertEquals(expectedResult.size(), 0); + printConsumerRangesStats(selector); + + int totalSelections = 10000; + + Map consumerSelectionCount = new HashMap<>(); + for (int i = 0; i < totalSelections; i++) { + Consumer selectedConsumer = selector.select(("key " + i).getBytes(StandardCharsets.UTF_8)); + consumerSelectionCount.computeIfAbsent(selectedConsumer, c -> new MutableInt()).increment(); + } + + printSelectionCountStats(consumerSelectionCount); + + int averageCount = totalSelections / consumers.size(); + int allowedVariance = (int) (0.2d * averageCount); + System.out.println("averageCount: " + averageCount + " allowedVariance: " + allowedVariance); + + for (Map.Entry entry : consumerSelectionCount.entrySet()) { + assertThat(entry.getValue().intValue()).describedAs("consumer: %s", entry.getKey()) + .isCloseTo(averageCount, Offset.offset(allowedVariance)); + } + + consumers.forEach(selector::removeConsumer); + assertThat(selector.getConsumerKeyHashRanges()).isEmpty(); + } + + private static void printSelectionCountStats(Map consumerSelectionCount) { + int totalSelections = consumerSelectionCount.values().stream().mapToInt(MutableInt::intValue).sum(); + consumerSelectionCount.entrySet().stream() + .sorted(Map.Entry.comparingByKey(Comparator.comparing(Consumer::toString))) + .forEach(entry -> System.out.println( + String.format("consumer: %s got selected %d times. ratio: %.2f%%", entry.getKey(), + entry.getValue().intValue(), + ((double) entry.getValue().intValue() / totalSelections) * 100.0d))); + } + + private static void printConsumerRangesStats(ConsistentHashingStickyKeyConsumerSelector selector) { + selector.getConsumerKeyHashRanges().entrySet().stream() + .map(entry -> Map.entry(entry.getKey(), + entry.getValue().stream().mapToInt(r -> r.getEnd() - r.getStart() + 1).sum())) + .sorted(Map.Entry.comparingByKey(Comparator.comparing(Consumer::toString))) + .forEach(entry -> System.out.println( + String.format("consumer: %s total ranges size: %d ratio: %.2f%%", entry.getKey(), + entry.getValue(), + ((double) entry.getValue() / (Integer.MAX_VALUE - 1)) * 100.0d))); + } + + private static Consumer createMockConsumer(String consumerName, String toString, long id) { + // without stubOnly, the mock will record method invocations and run into OOME + Consumer consumer = mock(Consumer.class, Mockito.withSettings().stubOnly()); + when(consumer.consumerName()).thenReturn(consumerName); + when(consumer.getPriorityLevel()).thenReturn(0); + when(consumer.toString()).thenReturn(toString); + when(consumer.consumerId()).thenReturn(id); + return consumer; } // reproduces https://github.com/apache/pulsar/issues/22050 @@ -215,5 +307,243 @@ public void shouldRemoveConsumersFromConsumerKeyHashRanges() { consumers.forEach(selector::removeConsumer); // then there should be no mapping remaining Assert.assertEquals(selector.getConsumerKeyHashRanges().size(), 0); + // when consumers are removed again, should not fail + consumers.forEach(selector::removeConsumer); + } + + @Test + public void testShouldNotChangeSelectedConsumerWhenConsumerIsRemoved() { + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final String consumerName = "consumer"; + final int numOfInitialConsumers = 100; + List consumers = new ArrayList<>(); + for (int i = 0; i < numOfInitialConsumers; i++) { + final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + + int hashRangeSize = Integer.MAX_VALUE; + int validationPointCount = 200; + int increment = hashRangeSize / (validationPointCount + 1); + List selectedConsumerBeforeRemoval = new ArrayList<>(); + + for (int i = 0; i < validationPointCount; i++) { + selectedConsumerBeforeRemoval.add(selector.select(i * increment)); + } + + for (int i = 0; i < validationPointCount; i++) { + Consumer selected = selector.select(i * increment); + Consumer expected = selectedConsumerBeforeRemoval.get(i); + assertThat(selected.consumerId()).as("validationPoint %d", i).isEqualTo(expected.consumerId()); + } + + Set removedConsumers = new HashSet<>(); + for (Consumer removedConsumer : consumers) { + selector.removeConsumer(removedConsumer); + removedConsumers.add(removedConsumer); + for (int i = 0; i < validationPointCount; i++) { + int hash = i * increment; + Consumer selected = selector.select(hash); + Consumer expected = selectedConsumerBeforeRemoval.get(i); + if (!removedConsumers.contains(expected)) { + assertThat(selected.consumerId()).as("validationPoint %d, removed %s, hash %d ranges %s", i, + removedConsumer.toString(), hash, selector.getConsumerKeyHashRanges()).isEqualTo(expected.consumerId()); + } + } + } + } + + @Test + public void testShouldNotChangeSelectedConsumerWhenConsumerIsRemovedCheckHashRanges() { + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final String consumerName = "consumer"; + final int numOfInitialConsumers = 25; + List consumers = new ArrayList<>(); + for (int i = 0; i < numOfInitialConsumers; i++) { + final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + + Map> expected = selector.getConsumerKeyHashRanges(); + assertThat(selector.getConsumerKeyHashRanges()).as("sanity check").containsExactlyInAnyOrderEntriesOf(expected); + System.out.println(expected); + + for (Consumer removedConsumer : consumers) { + selector.removeConsumer(removedConsumer); + for (Map.Entry> entry : expected.entrySet()) { + if (entry.getKey() == removedConsumer) { + continue; + } + for (Range range : entry.getValue()) { + Consumer rangeStartConsumer = selector.select(range.getStart()); + assertThat(rangeStartConsumer).as("removed %s, range %s", removedConsumer, range) + .isEqualTo(entry.getKey()); + Consumer rangeEndConsumer = selector.select(range.getEnd()); + assertThat(rangeEndConsumer).as("removed %s, range %s", removedConsumer, range) + .isEqualTo(entry.getKey()); + assertThat(rangeStartConsumer).isSameAs(rangeEndConsumer); + } + } + expected = selector.getConsumerKeyHashRanges(); + } + } + + @Test + public void testShouldNotChangeSelectedConsumerUnnecessarilyWhenConsumerIsAddedCheckHashRanges() { + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final String consumerName = "consumer"; + final int numOfInitialConsumers = 25; + List consumers = new ArrayList<>(); + for (int i = 0; i < numOfInitialConsumers; i++) { + final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + + Map> expected = selector.getConsumerKeyHashRanges(); + assertThat(selector.getConsumerKeyHashRanges()).as("sanity check").containsExactlyInAnyOrderEntriesOf(expected); + + for (int i = numOfInitialConsumers; i < numOfInitialConsumers * 2; i++) { + final Consumer addedConsumer = createMockConsumer(consumerName, "index " + i, i); + selector.addConsumer(addedConsumer); + for (Map.Entry> entry : expected.entrySet()) { + if (entry.getKey() == addedConsumer) { + continue; + } + for (Range range : entry.getValue()) { + Consumer rangeStartConsumer = selector.select(range.getStart()); + if (rangeStartConsumer != addedConsumer) { + assertThat(rangeStartConsumer).as("added %s, range start %s", addedConsumer, range) + .isEqualTo(entry.getKey()); + } + Consumer rangeEndConsumer = selector.select(range.getStart()); + if (rangeEndConsumer != addedConsumer) { + assertThat(rangeEndConsumer).as("added %s, range end %s", addedConsumer, range) + .isEqualTo(entry.getKey()); + } + } + } + expected = selector.getConsumerKeyHashRanges(); + } + } + + @Test + public void testShouldNotChangeSelectedConsumerWhenConsumerIsAdded() { + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final String consumerName = "consumer"; + final int numOfInitialConsumers = 50; + List consumers = new ArrayList<>(); + for (int i = 0; i < numOfInitialConsumers; i++) { + final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + + int hashRangeSize = Integer.MAX_VALUE; + int validationPointCount = 200; + int increment = hashRangeSize / (validationPointCount + 1); + List selectedConsumerBeforeRemoval = new ArrayList<>(); + + for (int i = 0; i < validationPointCount; i++) { + selectedConsumerBeforeRemoval.add(selector.select(i * increment)); + } + + for (int i = 0; i < validationPointCount; i++) { + Consumer selected = selector.select(i * increment); + Consumer expected = selectedConsumerBeforeRemoval.get(i); + assertThat(selected.consumerId()).as("validationPoint %d", i).isEqualTo(expected.consumerId()); + } + + Set addedConsumers = new HashSet<>(); + for (int i = numOfInitialConsumers; i < numOfInitialConsumers * 2; i++) { + final Consumer addedConsumer = createMockConsumer(consumerName, "index " + i, i); + selector.addConsumer(addedConsumer); + addedConsumers.add(addedConsumer); + for (int j = 0; j < validationPointCount; j++) { + int hash = j * increment; + Consumer selected = selector.select(hash); + Consumer expected = selectedConsumerBeforeRemoval.get(j); + if (!addedConsumers.contains(addedConsumer)) { + assertThat(selected.consumerId()).as("validationPoint %d, hash %d", j, hash).isEqualTo(expected.consumerId()); + } + } + } + } + + @Test + public void testShouldNotChangeMappingWhenConsumerLeavesAndRejoins() { + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final String consumerName = "consumer"; + final int numOfInitialConsumers = 25; + List consumers = new ArrayList<>(); + for (int i = 0; i < numOfInitialConsumers; i++) { + final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + + Map> expected = selector.getConsumerKeyHashRanges(); + assertThat(selector.getConsumerKeyHashRanges()).as("sanity check").containsExactlyInAnyOrderEntriesOf(expected); + + selector.removeConsumer(consumers.get(0)); + selector.removeConsumer(consumers.get(numOfInitialConsumers / 2)); + selector.addConsumer(consumers.get(0)); + selector.addConsumer(consumers.get(numOfInitialConsumers / 2)); + + assertThat(selector.getConsumerKeyHashRanges()).as("ranges shouldn't change").containsExactlyInAnyOrderEntriesOf(expected); + } + + @Test + public void testConsumersReconnect() { + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final String consumerName = "consumer"; + final int numOfInitialConsumers = 50; + final int validationPointCount = 200; + final List pointsToTest = pointsToTest(validationPointCount); + List consumers = new ArrayList<>(); + for (int i = 0; i < numOfInitialConsumers; i++) { + final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); + consumers.add(consumer); + selector.addConsumer(consumer); + } + + // Mark original results. + List selectedConsumersBeforeRemove = new ArrayList<>(); + for (int i = 0; i < validationPointCount; i++) { + int point = pointsToTest.get(i); + selectedConsumersBeforeRemove.add(selector.select(point)); + } + + // All consumers leave (in any order) + List randomOrderConsumers = new ArrayList<>(consumers); + Collections.shuffle(randomOrderConsumers); + for (Consumer c : randomOrderConsumers) { + selector.removeConsumer(c); + } + + // All consumers reconnect in the same order as originally + for (Consumer c : consumers) { + selector.addConsumer(c); + } + + // Check that the same consumers are selected as before + for (int j = 0; j < validationPointCount; j++) { + int point = pointsToTest.get(j); + Consumer selected = selector.select(point); + Consumer expected = selectedConsumersBeforeRemove.get(j); + assertThat(selected.consumerId()).as("validationPoint %d, hash %d", j, point).isEqualTo(expected.consumerId()); + } + } + + private List pointsToTest(int validationPointCount) { + List res = new ArrayList<>(); + int hashRangeSize = Integer.MAX_VALUE; + final int increment = hashRangeSize / (validationPointCount + 1); + for (int i = 0; i < validationPointCount; i++) { + res.add(i * increment); + } + return res; } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapperTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapperTest.java new file mode 100644 index 0000000000000..75c8e6db5d2a0 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerIdentityWrapperTest.java @@ -0,0 +1,68 @@ +/* + * 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.service; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class ConsumerIdentityWrapperTest { + private static Consumer mockConsumer() { + return mockConsumer("consumer"); + } + + private static Consumer mockConsumer(String consumerName) { + Consumer consumer = mock(Consumer.class); + when(consumer.consumerName()).thenReturn(consumerName); + return consumer; + } + + @Test + public void testEquals() { + Consumer consumer = mockConsumer(); + assertEquals(new ConsumerIdentityWrapper(consumer), new ConsumerIdentityWrapper(consumer)); + } + + @Test + public void testHashCode() { + Consumer consumer = mockConsumer(); + assertEquals(new ConsumerIdentityWrapper(consumer).hashCode(), + new ConsumerIdentityWrapper(consumer).hashCode()); + } + + @Test + public void testEqualsAndHashCode() { + Consumer consumer1 = mockConsumer(); + Consumer consumer2 = mockConsumer(); + ConsumerIdentityWrapper wrapper1 = new ConsumerIdentityWrapper(consumer1); + ConsumerIdentityWrapper wrapper2 = new ConsumerIdentityWrapper(consumer1); + ConsumerIdentityWrapper wrapper3 = new ConsumerIdentityWrapper(consumer2); + + // Test equality + assertEquals(wrapper1, wrapper2); + assertNotEquals(wrapper1, wrapper3); + + // Test hash code + assertEquals(wrapper1.hashCode(), wrapper2.hashCode()); + assertNotEquals(wrapper1.hashCode(), wrapper3.hashCode()); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerNameIndexTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerNameIndexTrackerTest.java new file mode 100644 index 0000000000000..0f18ecce2ffb4 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerNameIndexTrackerTest.java @@ -0,0 +1,157 @@ +/* + * 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.service; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class ConsumerNameIndexTrackerTest { + private ConsumerNameIndexTracker tracker; + + @BeforeMethod + public void setUp() { + tracker = new ConsumerNameIndexTracker(); + } + + private static Consumer mockConsumer() { + return mockConsumer("consumer"); + } + + + private static Consumer mockConsumer(String consumerName) { + Consumer consumer = mock(Consumer.class); + when(consumer.consumerName()).thenReturn(consumerName); + return consumer; + } + + @Test + public void testIncreaseConsumerRefCountAndReturnIndex() { + Consumer consumer1 = mockConsumer(); + Consumer consumer2 = mockConsumer(); + ConsumerIdentityWrapper wrapper1 = new ConsumerIdentityWrapper(consumer1); + ConsumerIdentityWrapper wrapper2 = new ConsumerIdentityWrapper(consumer2); + int index1 = tracker.increaseConsumerRefCountAndReturnIndex(wrapper1); + int index2 = tracker.increaseConsumerRefCountAndReturnIndex(wrapper2); + assertNotEquals(index1, index2); + assertEquals(index1, tracker.getTrackedIndex(wrapper1)); + assertEquals(index2, tracker.getTrackedIndex(wrapper2)); + } + + @Test + public void testTrackingReturnsStableIndexWhenRemovedAndAddedInSameOrder() { + List consumerIdentityWrappers = + IntStream.range(0, 100).mapToObj(i -> mockConsumer()).map(ConsumerIdentityWrapper::new).toList(); + Map trackedIndexes = + consumerIdentityWrappers.stream().collect(Collectors.toMap( + wrapper -> wrapper, wrapper -> tracker.increaseConsumerRefCountAndReturnIndex(wrapper))); + // stop tracking every other consumer + for (int i = 0; i < consumerIdentityWrappers.size(); i++) { + if (i % 2 == 0) { + tracker.decreaseConsumerRefCount(consumerIdentityWrappers.get(i)); + } + } + // check that others are tracked + for (int i = 0; i < consumerIdentityWrappers.size(); i++) { + ConsumerIdentityWrapper wrapper = consumerIdentityWrappers.get(i); + int trackedIndex = tracker.getTrackedIndex(wrapper); + assertEquals(trackedIndex, i % 2 == 0 ? -1 : trackedIndexes.get(wrapper)); + } + // check that new consumers are tracked with the same index + for (int i = 0; i < consumerIdentityWrappers.size(); i++) { + ConsumerIdentityWrapper wrapper = consumerIdentityWrappers.get(i); + if (i % 2 == 0) { + int trackedIndex = tracker.increaseConsumerRefCountAndReturnIndex(wrapper); + assertEquals(trackedIndex, trackedIndexes.get(wrapper)); + } + } + // check that all consumers are tracked with the original indexes + for (int i = 0; i < consumerIdentityWrappers.size(); i++) { + ConsumerIdentityWrapper wrapper = consumerIdentityWrappers.get(i); + int trackedIndex = tracker.getTrackedIndex(wrapper); + assertEquals(trackedIndex, trackedIndexes.get(wrapper)); + } + } + + @Test + public void testTrackingMultipleTimes() { + List consumerIdentityWrappers = + IntStream.range(0, 100).mapToObj(i -> mockConsumer()).map(ConsumerIdentityWrapper::new).toList(); + Map trackedIndexes = + consumerIdentityWrappers.stream().collect(Collectors.toMap( + wrapper -> wrapper, wrapper -> tracker.increaseConsumerRefCountAndReturnIndex(wrapper))); + Map trackedIndexes2 = + consumerIdentityWrappers.stream().collect(Collectors.toMap( + wrapper -> wrapper, wrapper -> tracker.increaseConsumerRefCountAndReturnIndex(wrapper))); + assertThat(tracker.getTrackedConsumerNamesCount()).isEqualTo(1); + assertThat(trackedIndexes).containsExactlyInAnyOrderEntriesOf(trackedIndexes2); + consumerIdentityWrappers.forEach(wrapper -> tracker.decreaseConsumerRefCount(wrapper)); + for (ConsumerIdentityWrapper wrapper : consumerIdentityWrappers) { + int trackedIndex = tracker.getTrackedIndex(wrapper); + assertEquals(trackedIndex, trackedIndexes.get(wrapper)); + } + consumerIdentityWrappers.forEach(wrapper -> tracker.decreaseConsumerRefCount(wrapper)); + assertThat(tracker.getTrackedConsumersCount()).isEqualTo(0); + assertThat(tracker.getTrackedConsumerNamesCount()).isEqualTo(0); + } + + @Test + public void testDecreaseConsumerRefCount() { + Consumer consumer1 = mockConsumer(); + ConsumerIdentityWrapper wrapper1 = new ConsumerIdentityWrapper(consumer1); + int index1 = tracker.increaseConsumerRefCountAndReturnIndex(wrapper1); + assertNotEquals(index1, -1); + tracker.decreaseConsumerRefCount(wrapper1); + assertEquals(tracker.getTrackedIndex(wrapper1), -1); + } + + @Test + public void testGetTrackedIndex() { + Consumer consumer1 = mockConsumer(); + Consumer consumer2 = mockConsumer(); + ConsumerIdentityWrapper wrapper1 = new ConsumerIdentityWrapper(consumer1); + ConsumerIdentityWrapper wrapper2 = new ConsumerIdentityWrapper(consumer2); + int index1 = tracker.increaseConsumerRefCountAndReturnIndex(wrapper1); + int index2 = tracker.increaseConsumerRefCountAndReturnIndex(wrapper2); + assertEquals(index1, tracker.getTrackedIndex(wrapper1)); + assertEquals(index2, tracker.getTrackedIndex(wrapper2)); + } + + @Test + public void testTrackingMultipleNames() { + List consumerIdentityWrappers = + IntStream.range(0, 100).mapToObj(i -> mockConsumer("consumer" + i)).map(ConsumerIdentityWrapper::new) + .toList(); + consumerIdentityWrappers.forEach(wrapper -> tracker.increaseConsumerRefCountAndReturnIndex(wrapper)); + assertThat(tracker.getTrackedConsumerNamesCount()).isEqualTo(100); + assertThat(tracker.getTrackedConsumersCount()).isEqualTo(100); + consumerIdentityWrappers.forEach(wrapper -> tracker.decreaseConsumerRefCount(wrapper)); + assertThat(tracker.getTrackedConsumersCount()).isEqualTo(0); + assertThat(tracker.getTrackedConsumerNamesCount()).isEqualTo(0); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index dcd852f409dbb..a0054f7e71425 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -20,6 +20,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.pulsar.common.protocol.Commands.serializeMetadataAndPayload; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.anyInt; @@ -326,7 +327,7 @@ public void testSkipRedeliverTemporally() { redeliverEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key1"))); final List readEntries = new ArrayList<>(); readEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); - readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key22"))); + readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key2"))); try { Field totalAvailablePermitsField = PersistentDispatcherMultipleConsumers.class.getDeclaredField("totalAvailablePermits"); @@ -417,7 +418,7 @@ public void testMessageRedelivery() throws Exception { // Messages with key1 are routed to consumer1 and messages with key2 are routed to consumer2 final List allEntries = new ArrayList<>(); - allEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key22"))); + allEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key2"))); allEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); allEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key1"))); allEntries.forEach(entry -> ((EntryImpl) entry).retain()); @@ -518,8 +519,8 @@ public void testMessageRedelivery() throws Exception { persistentDispatcher.readMoreEntries(); } - assertEquals(actualEntriesToConsumer1, expectedEntriesToConsumer1); - assertEquals(actualEntriesToConsumer2, expectedEntriesToConsumer2); + assertThat(actualEntriesToConsumer1).containsExactlyElementsOf(expectedEntriesToConsumer1); + assertThat(actualEntriesToConsumer2).containsExactlyElementsOf(expectedEntriesToConsumer2); allEntries.forEach(entry -> entry.release()); } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java index 4437ffc4ac6a2..488083f484b76 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java @@ -27,7 +27,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public class Range { +public class Range implements Comparable { private final int start; private final int end; @@ -84,4 +84,13 @@ public int hashCode() { public String toString() { return "[" + start + ", " + end + "]"; } + + @Override + public int compareTo(Range o) { + int result = Integer.compare(start, o.start); + if (result == 0) { + result = Integer.compare(end, o.end); + } + return result; + } } From 53e996c43d48bc33f6b60fb007fc5b202733df1b Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Wed, 2 Oct 2024 10:12:49 -0700 Subject: [PATCH 546/580] [fix][client] Fix failover consumer-listener stuck with cumulative ack and epoch time (#23345) --- .../client/impl/MessageRedeliveryTest.java | 64 ++++++++++++++++++- .../pulsar/client/impl/ConsumerImpl.java | 3 + .../client/impl/MultiTopicsConsumerImpl.java | 7 +- 3 files changed, 70 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java index 29b06f68b64eb..e2895b1d01e9f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java @@ -18,11 +18,16 @@ */ package org.apache.pulsar.client.impl; -import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import com.google.common.collect.Sets; +import io.netty.util.concurrent.DefaultThreadFactory; +import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -37,6 +42,8 @@ import org.apache.pulsar.client.api.BatchReceivePolicy; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageListener; import org.apache.pulsar.client.api.Messages; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; @@ -49,8 +56,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import com.google.common.collect.Sets; -import io.netty.util.concurrent.DefaultThreadFactory; @Test(groups = "broker-impl") public class MessageRedeliveryTest extends ProducerConsumerBase { @@ -539,4 +544,57 @@ public void testMultiConsumerBatchRedeliveryAddEpoch(boolean enableBatch) throws // can't receive message again assertEquals(consumer.batchReceive().size(), 0); } + + /** + * This test validates that client lib correctly increases permits of individual consumer to retrieve data in case + * of incorrect epoch for partition-topic multi-consumer. + * + * @throws Exception + */ + @Test + public void testRedeliveryWithMultiConsumerAndListenerAddEpoch() throws Exception { + final String topic = "testRedeliveryWithMultiConsumerAndListenerAddEpoch"; + final String subName = "my-sub"; + int totalMessages = 100; + admin.topics().createPartitionedTopic(topic, 2); + + Map ids = new ConcurrentHashMap<>(); + CountDownLatch latch = new CountDownLatch(totalMessages); + MessageListener msgListener = (Consumer consumer, Message msg) -> { + String id = msg.getMessageId().toString(); + consumer.acknowledgeCumulativeAsync(msg); + if (ids.put(msg.getMessageId(), id) == null) { + latch.countDown(); + } + }; + @Cleanup + Consumer newConsumer = pulsarClient.newConsumer(Schema.STRING).topic(topic).subscriptionName(subName) + .messageListener(msgListener).subscriptionType(SubscriptionType.Failover) + .receiverQueueSize(totalMessages / 10).subscribe(); + + MultiTopicsConsumerImpl consumer = (MultiTopicsConsumerImpl) newConsumer; + long epoch = consumer.getConsumerEpoch() + 1; + consumer.setConsumerEpoch(epoch); + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).enableBatching(false) + .create(); + + for (int i = 0; i < totalMessages; i++) { + producer.sendAsync("test" + i); + } + producer.flush(); + + // make sure listener has not received any messages until + // we call redelivery with correct epoch + for (int i = 0; i < 2; i++) { + assertTrue(ids.isEmpty()); + Thread.sleep(1000); + } + // make epoch valid to consume redelivery message again + consumer.setConsumerEpoch(epoch - 1); + consumer.redeliverUnacknowledgedMessages(); + + latch.await(10, TimeUnit.SECONDS); + assertEquals(ids.size(), totalMessages); + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 03ccbae01c276..b7010a1ddc7b4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1842,6 +1842,9 @@ protected void increaseAvailablePermits(ClientCnx currentCnx, int delta) { int available = AVAILABLE_PERMITS_UPDATER.addAndGet(this, delta); while (available >= getCurrentReceiverQueueSize() / 2 && !paused) { if (AVAILABLE_PERMITS_UPDATER.compareAndSet(this, available, 0)) { + if (log.isDebugEnabled()) { + log.debug("[{}] Sending permit-cmd to broker with available permits = {}", topic, available); + } sendFlowPermitsToBroker(currentCnx, available); break; } else { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 513c0101ac6ac..ff293af230838 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -270,8 +270,13 @@ private void receiveMessageFromConsumer(ConsumerImpl consumer, boolean batchR // Process the message, add to the queue and trigger listener or async callback messages.forEach(msg -> { final boolean skipDueToSeek = duringSeek; - if (isValidConsumerEpoch((MessageImpl) msg) && !skipDueToSeek) { + MessageImpl msgImpl = (MessageImpl) msg; + ClientCnx cnx = msgImpl.getCnx(); + boolean isValidEpoch = isValidConsumerEpoch(msgImpl); + if (isValidEpoch && !skipDueToSeek) { messageReceived(consumer, msg); + } else if (!isValidEpoch) { + consumer.increaseAvailablePermits(cnx); } else if (skipDueToSeek) { log.info("[{}] [{}] Skip processing message {} received during seek", topic, subscription, msg.getMessageId()); From e49d9ad01c94deea36c30ce45be0b0fe26cba16b Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Wed, 2 Oct 2024 13:01:50 -0700 Subject: [PATCH 547/580] [improve][pip] PIP-360 Add admin API to display Schema metadata (#22913) --- pip/pip-360.md | 83 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 83 insertions(+) create mode 100644 pip/pip-360.md diff --git a/pip/pip-360.md b/pip/pip-360.md new file mode 100644 index 0000000000000..21e8e18dc0531 --- /dev/null +++ b/pip/pip-360.md @@ -0,0 +1,83 @@ +# PIP-360: Admin API to display Schema metadata + +# Background knowledge + +Broker loads and initializes Schema of the topic during the topic loading. However, we have seen large number of instances and issues when broker fails to load the topic when topic schema is broken due to missing or corrupt schema ledger, index ledger or even schema data. Therefore, if broker is not able to load the topic for any reason then it is not possible to fetch schema metadata and identify which schema ledger is causing the issue because broker is storing schema metadata into binary format and there is no such API exists which shows schema metadata into readable format. So, it is very important to have an API to read schema metadata with complete information to help system admin to understand topic unavailability issues. It is also very useful to get schema metadata to build various schema related external tools which can be used by system administrator. We already have APIs for managed-ledger and bookkeeper-ledgers which are used by external tools and CLI to read binary data from metadata store and display in readable format. + + +# Motivation + +Schema is one of the important part of the topic because it also plays important part in topic availability and required to successfully load the topic, and if schema initialization failure is causing issue in topic loading then it is very important to get schema metadata information to understand schema related issues and perform appropriate actions to mitigate that issue to successfully load the topic and make it available for users. Therefore, similar to ledger metadata and managed-ledger metadata, Pulsar should have API to show schema metadata and related ledger info which can be used by tools or users to perform appropriate actions during topic availability issues or any other troubleshooting. + +# Goals +Add an .admin API under schema resource which returns schema metadata into readable format + + +# High Level Design + +This PIP will introduce REST api which will accept the topic name and return schema metadata along with ledger information of schema-ledgers and index entries. It will also add CLI support to print schema metadata for users to see it in human readable format. + + +### Public API + + +This PIP will add a new REST endpoint under Schema resource path. +``` +Path: schema/{tenant}/{namespace}/{topic}/metadata +Response code: +307, message = Current broker doesn't serve the namespace of this topic +401, message = Client is not authorized or Don't have admin permission +403, message = Client is not authenticated +404, message = Tenant or Namespace or Topic doesn't exist; or Schema is not found for +412, message = Failed to find the ownership for the topic +``` +This admin API will return below schema metadata response. + +``` +@Data +public class SchemaMetadata { + public Entry info; + public List index; + @Data + @AllArgsConstructor + @NoArgsConstructor + static class Entry { + private long ledgerId; + private long entryId; + private long version; + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("ledgerId", ledgerId) + .add("entryId", entryId) + .add("version", version) + .toString(); + } + } +} +``` + +### CLI + +This PIP will also add appropriate CLI command under Schema command to get schema metadata. +``` +bin/pulsar-admin schemas get-metadata +``` + +# Links + +Sample PR: https://github.com/apache/pulsar/pull/22938 + +* Mailing List discussion thread: +* Mailing List voting thread: From ab684a0fb9d433ab3214b6e8baba828895c07999 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 3 Oct 2024 18:42:35 +0300 Subject: [PATCH 548/580] [fix][sec] Upgrade protobuf-java to 3.25.5 (#23356) --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- distribution/shell/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 1d78913849bda..61d4c2231adad 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -565,8 +565,8 @@ MIT License - com.auth0-jwks-rsa-0.22.0.jar Protocol Buffers License * Protocol Buffers - - com.google.protobuf-protobuf-java-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt - - com.google.protobuf-protobuf-java-util-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt + - com.google.protobuf-protobuf-java-3.25.5.jar -- ../licenses/LICENSE-protobuf.txt + - com.google.protobuf-protobuf-java-util-3.25.5.jar -- ../licenses/LICENSE-protobuf.txt CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 9ab22ae83e42e..aa3853c6dd926 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -431,7 +431,7 @@ MIT License Protocol Buffers License * Protocol Buffers - - protobuf-java-3.22.3.jar -- ../licenses/LICENSE-protobuf.txt + - protobuf-java-3.25.5.jar -- ../licenses/LICENSE-protobuf.txt CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API diff --git a/pom.xml b/pom.xml index 881a1541c5eaf..66009003aa110 100644 --- a/pom.xml +++ b/pom.xml @@ -170,7 +170,7 @@ flexible messaging model and an intuitive client API. 0.5.0 1.14.12 1.17 - 3.22.3 + 3.25.5 ${protobuf3.version} 1.56.1 1.41.0 From eee9283666cc9d84d0ddb998c279600898121d2b Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Thu, 3 Oct 2024 12:50:11 -0700 Subject: [PATCH 549/580] [fix][broker] timeout when broker registry hangs and monitor broker registry (ExtensibleLoadManagerImpl only) (#23382) --- .../pulsar/broker/admin/impl/BrokersBase.java | 11 ++- .../extensions/BrokerRegistry.java | 5 + .../extensions/BrokerRegistryImpl.java | 69 ++++++++++--- .../extensions/ExtensibleLoadManagerImpl.java | 18 +++- .../channel/ServiceUnitStateChannelImpl.java | 98 ++++++++++++++++--- .../extensions/BrokerRegistryTest.java | 33 +++++++ .../ExtensibleLoadManagerImplTest.java | 15 +++ .../channel/ServiceUnitStateChannelTest.java | 70 +++++++++---- .../apache/pulsar/client/admin/Brokers.java | 12 ++- .../client/admin/internal/BrokersImpl.java | 18 +++- 10 files changed, 299 insertions(+), 50 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index 4d0b598a8e4f1..e13cb1858f79d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -48,6 +48,7 @@ import javax.ws.rs.container.Suspended; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; +import org.apache.commons.lang.StringUtils; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService.State; @@ -368,20 +369,26 @@ public void isReady(@Suspended AsyncResponse asyncResponse) { @ApiOperation(value = "Run a healthCheck against the broker") @ApiResponses(value = { @ApiResponse(code = 200, message = "Everything is OK"), + @ApiResponse(code = 307, message = "Current broker is not the target broker"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Cluster doesn't exist"), @ApiResponse(code = 500, message = "Internal server error")}) public void healthCheck(@Suspended AsyncResponse asyncResponse, @ApiParam(value = "Topic Version") - @QueryParam("topicVersion") TopicVersion topicVersion) { + @QueryParam("topicVersion") TopicVersion topicVersion, + @QueryParam("brokerId") String brokerId) { validateSuperUserAccessAsync() .thenAccept(__ -> checkDeadlockedThreads()) + .thenCompose(__ -> maybeRedirectToBroker( + StringUtils.isBlank(brokerId) ? pulsar().getBrokerId() : brokerId)) .thenCompose(__ -> internalRunHealthCheck(topicVersion)) .thenAccept(__ -> { LOG.info("[{}] Successfully run health check.", clientAppId()); asyncResponse.resume(Response.ok("ok").build()); }).exceptionally(ex -> { - LOG.error("[{}] Fail to run health check.", clientAppId(), ex); + if (!isRedirectException(ex)) { + LOG.error("[{}] Fail to run health check.", clientAppId(), ex); + } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java index 79dba9c63342e..d154edfbb320e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistry.java @@ -48,6 +48,11 @@ public interface BrokerRegistry extends AutoCloseable { */ boolean isStarted(); + /** + * Return the broker has been registered. + */ + boolean isRegistered(); + /** * Register local broker to metadata store. */ diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java index a13b332e6eb5f..5a8307df27a63 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java @@ -52,6 +52,8 @@ @Slf4j public class BrokerRegistryImpl implements BrokerRegistry { + private static final int MAX_REGISTER_RETRY_DELAY_IN_MILLIS = 1000; + private final PulsarService pulsar; private final ServiceConfiguration conf; @@ -77,10 +79,11 @@ protected enum State { @VisibleForTesting final AtomicReference state = new AtomicReference<>(State.Init); - public BrokerRegistryImpl(PulsarService pulsar) { + @VisibleForTesting + BrokerRegistryImpl(PulsarService pulsar, MetadataCache brokerLookupDataMetadataCache) { this.pulsar = pulsar; this.conf = pulsar.getConfiguration(); - this.brokerLookupDataMetadataCache = pulsar.getLocalMetadataStore().getMetadataCache(BrokerLookupData.class); + this.brokerLookupDataMetadataCache = brokerLookupDataMetadataCache; this.scheduler = pulsar.getLoadManagerExecutor(); this.listeners = new ArrayList<>(); this.brokerIdKeyPath = keyPath(pulsar.getBrokerId()); @@ -99,6 +102,10 @@ public BrokerRegistryImpl(PulsarService pulsar) { pulsar.getConfig().lookupProperties()); } + public BrokerRegistryImpl(PulsarService pulsar) { + this(pulsar, pulsar.getLocalMetadataStore().getMetadataCache(BrokerLookupData.class)); + } + @Override public synchronized void start() throws PulsarServerException { if (!this.state.compareAndSet(State.Init, State.Started)) { @@ -118,6 +125,12 @@ public boolean isStarted() { return state == State.Started || state == State.Registered; } + @Override + public boolean isRegistered() { + final var state = this.state.get(); + return state == State.Registered; + } + @Override public CompletableFuture registerAsync() { final var state = this.state.get(); @@ -127,12 +140,35 @@ public CompletableFuture registerAsync() { } log.info("[{}] Started registering self to {} (state: {})", getBrokerId(), brokerIdKeyPath, state); return brokerLookupDataMetadataCache.put(brokerIdKeyPath, brokerLookupData, EnumSet.of(CreateOption.Ephemeral)) - .thenAccept(__ -> { - this.state.set(State.Registered); - log.info("[{}] Finished registering self", getBrokerId()); + .orTimeout(pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS) + .whenComplete((__, ex) -> { + if (ex == null) { + this.state.set(State.Registered); + log.info("[{}] Finished registering self", getBrokerId()); + } else { + log.error("[{}] Failed registering self", getBrokerId(), ex); + } }); } + private void doRegisterAsyncWithRetries(int retry, CompletableFuture future) { + pulsar.getExecutor().schedule(() -> { + registerAsync().whenComplete((__, e) -> { + if (e != null) { + doRegisterAsyncWithRetries(retry + 1, future); + } else { + future.complete(null); + } + }); + }, Math.min(MAX_REGISTER_RETRY_DELAY_IN_MILLIS, retry * retry * 50), TimeUnit.MILLISECONDS); + } + + private CompletableFuture registerAsyncWithRetries() { + var retryFuture = new CompletableFuture(); + doRegisterAsyncWithRetries(0, retryFuture); + return retryFuture; + } + @Override public synchronized void unregister() throws MetadataStoreException { if (state.compareAndSet(State.Registered, State.Unregistering)) { @@ -219,17 +255,26 @@ private void handleMetadataStoreNotification(Notification t) { // The registered node is an ephemeral node that could be deleted when the metadata store client's session // is expired. In this case, we should register again. final var brokerId = t.getPath().substring(LOADBALANCE_BROKERS_ROOT.length() + 1); + + CompletableFuture register; if (t.getType() == NotificationType.Deleted && getBrokerId().equals(brokerId)) { - registerAsync(); - } - if (listeners.isEmpty()) { - return; + this.state.set(State.Started); + register = registerAsyncWithRetries(); + } else { + register = CompletableFuture.completedFuture(null); } - this.scheduler.submit(() -> { - for (BiConsumer listener : listeners) { - listener.accept(brokerId, t.getType()); + // Make sure to run the listeners after re-registered. + register.thenAccept(__ -> { + if (listeners.isEmpty()) { + return; } + this.scheduler.submit(() -> { + for (BiConsumer listener : listeners) { + listener.accept(brokerId, t.getType()); + } + }); }); + } catch (RejectedExecutionException e) { // Executor is shutting down } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java index d8a279b854576..abca2bb398232 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImpl.java @@ -35,8 +35,10 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; @@ -987,8 +989,12 @@ protected void monitor() { return; } + // Monitor broker registry + // Periodically check the broker registry in case metadata store fails. + validateBrokerRegistry(); + // Monitor role - // Periodically check the role in case ZK watcher fails. + // Periodically check the role in case metadata store fails. var isChannelOwner = serviceUnitStateChannel.isChannelOwner(); if (isChannelOwner) { // System topic config might fail due to the race condition @@ -1087,5 +1093,15 @@ private boolean isPersistentSystemTopicUsed() { .equals(pulsar.getConfiguration().getLoadManagerServiceUnitStateTableViewClassName()); } + private void validateBrokerRegistry() + throws ExecutionException, InterruptedException, TimeoutException { + var timeout = pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(); + var lookup = brokerRegistry.lookupAsync(brokerRegistry.getBrokerId()).get(timeout, TimeUnit.SECONDS); + if (lookup.isEmpty()) { + log.warn("Found this broker:{} has not registered yet. Trying to register it", + brokerRegistry.getBrokerId()); + brokerRegistry.registerAsync().get(timeout, TimeUnit.SECONDS); + } + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java index ce975495feb2a..49d038d512e59 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java @@ -86,11 +86,13 @@ import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceBundleFactory; import org.apache.pulsar.common.naming.NamespaceBundleSplitAlgorithm; import org.apache.pulsar.common.naming.NamespaceBundles; +import org.apache.pulsar.common.naming.TopicVersion; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.Reflections; @@ -108,6 +110,8 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private static final long MIN_CLEAN_UP_DELAY_TIME_IN_SECS = 0; // 0 secs to clean immediately private static final long MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS = 10; private static final long MAX_OWNED_BUNDLE_COUNT_DELAY_TIME_IN_MILLIS = 10 * 60 * 1000; + private static final long MAX_BROKER_HEALTH_CHECK_RETRY = 3; + private static final long MAX_BROKER_HEALTH_CHECK_DELAY_IN_MILLIS = 1000; private final PulsarService pulsar; private final ServiceConfiguration config; private final Schema schema; @@ -115,6 +119,7 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private final String brokerId; private final Map> cleanupJobs; private final StateChangeListeners stateChangeListeners; + private BrokerRegistry brokerRegistry; private LeaderElectionService leaderElectionService; @@ -350,6 +355,11 @@ protected LeaderElectionService getLeaderElectionService() { .get().getLeaderElectionService(); } + @VisibleForTesting + protected PulsarAdmin getPulsarAdmin() throws PulsarServerException { + return pulsar.getAdminClient(); + } + @Override public synchronized void close() throws PulsarServerException { channelState = Closed; @@ -448,6 +458,14 @@ private CompletableFuture> getActiveOwnerAsync( String serviceUnit, ServiceUnitState state, Optional owner) { + + // If this broker's registry does not exist(possibly suffering from connecting to the metadata store), + // we return the owner without its activeness check. + // This broker tries to serve lookups on a best efforts basis when metadata store connection is unstable. + if (!brokerRegistry.isRegistered()) { + return CompletableFuture.completedFuture(owner); + } + return dedupeGetOwnerRequest(serviceUnit) .thenCompose(newOwner -> { if (newOwner == null) { @@ -1255,19 +1273,25 @@ private MetadataState getMetadataState() { } private void handleBrokerCreationEvent(String broker) { - CompletableFuture future = cleanupJobs.remove(broker); - if (future != null) { - future.cancel(false); - totalInactiveBrokerCleanupCancelledCnt++; - log.info("Successfully cancelled the ownership cleanup for broker:{}." - + " Active cleanup job count:{}", - broker, cleanupJobs.size()); - } else { - if (debug()) { - log.info("No needs to cancel the ownership cleanup for broker:{}." - + " There was no scheduled cleanup job. Active cleanup job count:{}", - broker, cleanupJobs.size()); - } + + if (!cleanupJobs.isEmpty() && cleanupJobs.containsKey(broker)) { + healthCheckBrokerAsync(broker) + .thenAccept(__ -> { + CompletableFuture future = cleanupJobs.remove(broker); + if (future != null) { + future.cancel(false); + totalInactiveBrokerCleanupCancelledCnt++; + log.info("Successfully cancelled the ownership cleanup for broker:{}." + + " Active cleanup job count:{}", + broker, cleanupJobs.size()); + } else { + if (debug()) { + log.info("No needs to cancel the ownership cleanup for broker:{}." + + " There was no scheduled cleanup job. Active cleanup job count:{}", + broker, cleanupJobs.size()); + } + } + }); } } @@ -1431,6 +1455,37 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max System.currentTimeMillis() - started); } + private CompletableFuture healthCheckBrokerAsync(String brokerId) { + CompletableFuture future = new CompletableFuture<>(); + doHealthCheckBrokerAsyncWithRetries(brokerId, 0, future); + return future; + } + + private void doHealthCheckBrokerAsyncWithRetries(String brokerId, int retry, CompletableFuture future) { + try { + var admin = getPulsarAdmin(); + admin.brokers().healthcheckAsync(TopicVersion.V2, Optional.of(brokerId)) + .whenComplete((__, e) -> { + if (e == null) { + log.info("Completed health-check broker :{}", brokerId, e); + future.complete(null); + return; + } + if (retry == MAX_BROKER_HEALTH_CHECK_RETRY) { + log.error("Failed health-check broker :{}", brokerId, e); + future.completeExceptionally(FutureUtil.unwrapCompletionException(e)); + } else { + pulsar.getExecutor() + .schedule(() -> doHealthCheckBrokerAsyncWithRetries(brokerId, retry + 1, future), + Math.min(MAX_BROKER_HEALTH_CHECK_DELAY_IN_MILLIS, retry * retry * 50), + MILLISECONDS); + } + }); + } catch (PulsarServerException e) { + future.completeExceptionally(e); + } + } + private synchronized void doCleanup(String broker, boolean gracefully) { try { if (getChannelOwnerAsync().get(MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS, TimeUnit.SECONDS) @@ -1444,6 +1499,23 @@ private synchronized void doCleanup(String broker, boolean gracefully) { return; } + // if not gracefully, verify the broker is inactive by health-check. + if (!gracefully) { + try { + healthCheckBrokerAsync(broker).get( + pulsar.getConfiguration().getMetadataStoreOperationTimeoutSeconds(), SECONDS); + log.warn("Found that the broker to clean is healthy. Skip the broker:{}'s orphan bundle cleanup", + broker); + return; + } catch (Exception e) { + if (debug()) { + log.info("Failed to check broker:{} health", broker, e); + } + log.info("Checked the broker:{} health. Continue the orphan bundle cleanup", broker); + } + } + + long startTime = System.nanoTime(); log.info("Started ownership cleanup for the inactive broker:{}", broker); int orphanServiceUnitCleanupCnt = 0; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java index 28a2a18500f5f..941d0e4cbc3a0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryTest.java @@ -19,7 +19,10 @@ package org.apache.pulsar.broker.loadbalance.extensions; import static org.apache.pulsar.broker.loadbalance.LoadManager.LOADBALANCE_BROKERS_ROOT; +import static org.mockito.ArgumentMatchers.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.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; @@ -36,6 +39,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -48,6 +52,7 @@ import org.apache.pulsar.common.naming.ServiceUnitId; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; @@ -396,6 +401,34 @@ public void testKeyPath() { assertEquals(keyPath, LOADBALANCE_BROKERS_ROOT + "/brokerId"); } + @Test + public void testRegisterAsyncTimeout() throws Exception { + var pulsar1 = createPulsarService(); + pulsar1.start(); + pulsar1.getConfiguration().setMetadataStoreOperationTimeoutSeconds(1); + var metadataCache = mock(MetadataCache.class); + var brokerRegistry = new BrokerRegistryImpl(pulsar1, metadataCache); + + // happy case + doReturn(CompletableFuture.completedFuture(null)).when(metadataCache).put(any(), any(), any()); + brokerRegistry.start(); + + // unhappy case (timeout) + doAnswer(invocationOnMock -> { + return CompletableFuture.supplyAsync(() -> null, CompletableFuture.delayedExecutor(5, TimeUnit.SECONDS)); + }).when(metadataCache).put(any(), any(), any()); + try { + brokerRegistry.registerAsync().join(); + } catch (Exception e) { + assertTrue(e.getCause() instanceof TimeoutException); + } + + // happy case again + doReturn(CompletableFuture.completedFuture(null)).when(metadataCache).put(any(), any(), any()); + brokerRegistry.registerAsync().join(); + } + + private static BrokerRegistryImpl.State getState(BrokerRegistryImpl brokerRegistry) { return brokerRegistry.state.get(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java index 7871e612c847a..d8d3e5bb44ffb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplTest.java @@ -132,6 +132,7 @@ import org.apache.pulsar.common.policies.data.NamespaceOwnershipStatus; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; import org.apache.pulsar.policies.data.loadbalancer.ResourceUsage; import org.apache.pulsar.policies.data.loadbalancer.SystemResourceUsage; @@ -2106,6 +2107,20 @@ public void compactionScheduleTest() { }); } + @Test(timeOut = 30 * 1000) + public void testMonitorBrokerRegistry() throws MetadataStoreException { + primaryLoadManager.getBrokerRegistry().unregister(); + assertFalse(primaryLoadManager.getBrokerRegistry().isRegistered()); + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(30, TimeUnit.SECONDS) + .ignoreExceptions() + .untilAsserted(() -> { // wait until true + primaryLoadManager.monitor(); + assertTrue(primaryLoadManager.getBrokerRegistry().isRegistered()); + }); + } + private static abstract class MockBrokerFilter implements BrokerFilter { @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java index 92cdf61f44269..b6e38d4f6956c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelTest.java @@ -44,6 +44,7 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -89,6 +90,8 @@ import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.client.admin.Brokers; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.TopicName; @@ -136,10 +139,14 @@ public class ServiceUnitStateChannelTest extends MockedPulsarServiceBaseTest { private BrokerRegistryImpl registry; + private PulsarAdmin pulsarAdmin; + private ExtensibleLoadManagerImpl loadManager; private final String serviceUnitStateTableViewClassName; + private Brokers brokers; + @DataProvider(name = "serviceUnitStateTableViewClassName") public static Object[][] serviceUnitStateTableViewClassName() { return new Object[][]{ @@ -174,7 +181,9 @@ protected void setup() throws Exception { admin.namespaces().createNamespace(namespaceName2); pulsar1 = pulsar; - registry = new BrokerRegistryImpl(pulsar); + registry = spy(new BrokerRegistryImpl(pulsar1)); + registry.start(); + pulsarAdmin = spy(pulsar.getAdminClient()); loadManagerContext = mock(LoadManagerContext.class); doReturn(mock(LoadDataStore.class)).when(loadManagerContext).brokerLoadDataStore(); doReturn(mock(LoadDataStore.class)).when(loadManagerContext).topBundleLoadDataStore(); @@ -207,6 +216,10 @@ protected void setup() throws Exception { childBundle31 = namespaceName2 + "/" + childBundle1Range; childBundle32 = namespaceName2 + "/" + childBundle2Range; + + brokers = mock(Brokers.class); + doReturn(CompletableFuture.failedFuture(new RuntimeException("failed"))).when(brokers) + .healthcheckAsync(any(), any()); } @BeforeMethod @@ -220,6 +233,7 @@ protected void initChannels() throws Exception { cleanMetadataState(channel1); cleanMetadataState(channel2); enableChannels(); + reset(pulsarAdmin); } @@ -719,17 +733,19 @@ public void handleMetadataSessionEventTest() throws IllegalAccessException { @Test(priority = 8) public void handleBrokerCreationEventTest() throws IllegalAccessException { var cleanupJobs = getCleanupJobs(channel1); - String broker = "broker-1"; + String broker = brokerId2; var future = new CompletableFuture(); cleanupJobs.put(broker, future); ((ServiceUnitStateChannelImpl) channel1).handleBrokerRegistrationEvent(broker, NotificationType.Created); - assertEquals(0, cleanupJobs.size()); - assertTrue(future.isCancelled()); + Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> { + assertEquals(0, cleanupJobs.size()); + assertTrue(future.isCancelled()); + }); + } @Test(priority = 9) - public void handleBrokerDeletionEventTest() - throws IllegalAccessException, ExecutionException, InterruptedException, TimeoutException { + public void handleBrokerDeletionEventTest() throws Exception { var cleanupJobs1 = getCleanupJobs(channel1); var cleanupJobs2 = getCleanupJobs(channel2); @@ -782,8 +798,12 @@ public void handleBrokerDeletionEventTest() System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); FieldUtils.writeDeclaredField(followerChannel, "lastMetadataSessionEventTimestamp", System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); + + doReturn(brokers).when(pulsarAdmin).brokers(); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); followerChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); + + leaderChannel.handleBrokerRegistrationEvent(brokerId2, NotificationType.Deleted); followerChannel.handleBrokerRegistrationEvent(brokerId2, @@ -841,6 +861,7 @@ public void handleBrokerDeletionEventTest() 3, 0, 0); + reset(pulsarAdmin); // broker is back online leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Created); @@ -865,6 +886,7 @@ public void handleBrokerDeletionEventTest() // broker is offline again + doReturn(brokers).when(pulsarAdmin).brokers(); FieldUtils.writeDeclaredField(leaderChannel, "maxCleanupDelayTimeInSecs", 3, true); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); followerChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); @@ -906,6 +928,7 @@ public void handleBrokerDeletionEventTest() 4, 0, 1); + reset(pulsarAdmin); // test unstable state channel1.publishUnloadEventAsync(new Unload(brokerId2, bundle1, Optional.of(broker))); @@ -1585,9 +1608,12 @@ public void testOverrideInactiveBrokerStateData() System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); FieldUtils.writeDeclaredField(followerChannel, "lastMetadataSessionEventTimestamp", System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); + + doReturn(brokers).when(pulsarAdmin).brokers(); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); followerChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); + waitUntilNewOwner(channel2, releasingBundle, brokerId2); waitUntilNewOwner(channel2, childBundle11, brokerId2); waitUntilNewOwner(channel2, childBundle12, brokerId2); @@ -1600,7 +1626,7 @@ public void testOverrideInactiveBrokerStateData() // clean-up FieldUtils.writeDeclaredField(leaderChannel, "maxCleanupDelayTimeInSecs", 3 * 60, true); cleanTableViews(); - + reset(pulsarAdmin); } @Test(priority = 19) @@ -1736,13 +1762,10 @@ public void testActiveGetOwner() throws Exception { } // case 5: the owner lookup gets delayed - var spyRegistry = spy(new BrokerRegistryImpl(pulsar)); - FieldUtils.writeDeclaredField(channel1, - "brokerRegistry", spyRegistry, true); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1000, true); var delayedFuture = new CompletableFuture(); - doReturn(delayedFuture).when(spyRegistry).lookupAsync(eq(broker)); + doReturn(delayedFuture).when(registry).lookupAsync(eq(broker)); CompletableFuture.runAsync(() -> { try { Thread.sleep(500); @@ -1760,7 +1783,7 @@ public void testActiveGetOwner() throws Exception { // case 6: the owner is inactive doReturn(CompletableFuture.completedFuture(Optional.empty())) - .when(spyRegistry).lookupAsync(eq(broker)); + .when(registry).lookupAsync(eq(broker)); // verify getOwnerAsync times out start = System.currentTimeMillis(); @@ -1768,6 +1791,18 @@ public void testActiveGetOwner() throws Exception { assertTrue(ex.getCause() instanceof IllegalStateException); assertTrue(System.currentTimeMillis() - start >= 1000); + try { + // verify getOwnerAsync returns immediately when not registered + registry.unregister(); + start = System.currentTimeMillis(); + assertEquals(broker, channel1.getOwnerAsync(bundle).get().get()); + elapsed = System.currentTimeMillis() - start; + assertTrue(elapsed < 1000); + } finally { + registry.registerAsync().join(); + } + + // case 7: the ownership cleanup(no new owner) by the leader channel doReturn(CompletableFuture.completedFuture(Optional.empty())) .when(loadManager).selectAsync(any(), any(), any()); @@ -1781,6 +1816,7 @@ public void testActiveGetOwner() throws Exception { leaderChannel.handleMetadataSessionEvent(SessionReestablished); FieldUtils.writeDeclaredField(leaderChannel, "lastMetadataSessionEventTimestamp", System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); + doReturn(brokers).when(pulsarAdmin).brokers(); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); // verify the ownership cleanup, and channel's getOwnerAsync returns empty result without timeout @@ -1792,7 +1828,7 @@ public void testActiveGetOwner() throws Exception { waitUntilState(channel2, bundle, Init); assertTrue(System.currentTimeMillis() - start < 20_000); - + reset(pulsarAdmin); // case 8: simulate ownership cleanup(brokerId1 as the new owner) by the leader channel try { disableChannels(); @@ -1807,6 +1843,7 @@ public void testActiveGetOwner() throws Exception { FieldUtils.writeDeclaredField(leaderChannel, "lastMetadataSessionEventTimestamp", System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); getCleanupJobs(leaderChannel).clear(); + doReturn(brokers).when(pulsarAdmin).brokers(); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); // verify the ownership cleanup, and channel's getOwnerAsync returns brokerId1 without timeout @@ -1817,10 +1854,8 @@ public void testActiveGetOwner() throws Exception { // test clean-up FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 30 * 1000, true); - FieldUtils.writeDeclaredField(channel1, - "brokerRegistry", registry, true); cleanTableViews(); - + reset(pulsarAdmin); } @Test(priority = 21) @@ -2253,7 +2288,7 @@ private static void validateMonitorCounters(ServiceUnitStateChannel channel, } ServiceUnitStateChannelImpl createChannel(PulsarService pulsar) - throws IllegalAccessException { + throws IllegalAccessException, PulsarServerException { var tmpChannel = new ServiceUnitStateChannelImpl(pulsar); FieldUtils.writeDeclaredField(tmpChannel, "ownershipMonitorDelayTimeInSecs", 5, true); var channel = spy(tmpChannel); @@ -2261,6 +2296,7 @@ ServiceUnitStateChannelImpl createChannel(PulsarService pulsar) doReturn(loadManagerContext).when(channel).getContext(); doReturn(registry).when(channel).getBrokerRegistry(); doReturn(loadManager).when(channel).getLoadManager(); + doReturn(pulsarAdmin).when(channel).getPulsarAdmin(); var leaderElectionService = new LeaderElectionService( diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Brokers.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Brokers.java index dc0b7c9885a9a..eed73f38282ac 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Brokers.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Brokers.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.pulsar.client.admin.PulsarAdminException.NotAuthorizedException; import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException; @@ -320,10 +321,19 @@ Map getOwnedNamespaces(String cluster, String */ void healthcheck(TopicVersion topicVersion) throws PulsarAdminException; + /** + * Run a healthcheck on the target broker or on the broker. + * @param brokerId target broker id to check the health. If empty, it checks the health on the connected broker. + * + * @throws PulsarAdminException if the healthcheck fails. + */ + void healthcheck(TopicVersion topicVersion, Optional brokerId) throws PulsarAdminException; + /** * Run a healthcheck on the broker asynchronously. */ - CompletableFuture healthcheckAsync(TopicVersion topicVersion); + CompletableFuture healthcheckAsync(TopicVersion topicVersion, Optional brokerId); + /** * Trigger the current broker to graceful-shutdown asynchronously. diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java index b82c3fd0f414b..35b261b196eee 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import javax.ws.rs.client.Entity; import javax.ws.rs.client.InvocationCallback; @@ -168,26 +169,35 @@ public CompletableFuture backlogQuotaCheckAsync() { @Override @Deprecated public void healthcheck() throws PulsarAdminException { - healthcheck(TopicVersion.V1); + healthcheck(TopicVersion.V1, Optional.empty()); } @Override @Deprecated public CompletableFuture healthcheckAsync() { - return healthcheckAsync(TopicVersion.V1); + return healthcheckAsync(TopicVersion.V1, Optional.empty()); } + @Override public void healthcheck(TopicVersion topicVersion) throws PulsarAdminException { - sync(() -> healthcheckAsync(topicVersion)); + sync(() -> healthcheckAsync(topicVersion, Optional.empty())); } @Override - public CompletableFuture healthcheckAsync(TopicVersion topicVersion) { + public void healthcheck(TopicVersion topicVersion, Optional brokerId) throws PulsarAdminException { + sync(() -> healthcheckAsync(topicVersion, brokerId)); + } + + @Override + public CompletableFuture healthcheckAsync(TopicVersion topicVersion, Optional brokerId) { WebTarget path = adminBrokers.path("health"); if (topicVersion != null) { path = path.queryParam("topicVersion", topicVersion); } + if (brokerId.isPresent()) { + path = path.queryParam("brokerId", brokerId.get()); + } final CompletableFuture future = new CompletableFuture<>(); asyncGetRequest(path, new InvocationCallback() { From c2128dc4a1286d5cea8e6a1f9b8ccb49acb9684a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 3 Oct 2024 16:12:06 -0700 Subject: [PATCH 550/580] [fix] Bump commons-io:commons-io from 2.8.0 to 2.14.0 (#23393) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Matteo Merli --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- distribution/shell/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 61d4c2231adad..8bcb7d7346b59 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -284,7 +284,7 @@ The Apache Software License, Version 2.0 - commons-cli-commons-cli-1.5.0.jar - commons-codec-commons-codec-1.15.jar - commons-configuration-commons-configuration-1.10.jar - - commons-io-commons-io-2.8.0.jar + - commons-io-commons-io-2.14.0.jar - commons-lang-commons-lang-2.6.jar - commons-logging-commons-logging-1.1.1.jar - org.apache.commons-commons-collections4-4.4.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index aa3853c6dd926..faad519df2ef5 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -340,7 +340,7 @@ The Apache Software License, Version 2.0 * Apache Commons - commons-codec-1.15.jar - commons-configuration-1.10.jar - - commons-io-2.8.0.jar + - commons-io-2.14.0.jar - commons-lang-2.6.jar - commons-logging-1.2.jar - commons-lang3-3.11.jar diff --git a/pom.xml b/pom.xml index 66009003aa110..70956b4d104eb 100644 --- a/pom.xml +++ b/pom.xml @@ -217,7 +217,7 @@ flexible messaging model and an intuitive client API. 2.12.1 3.11 1.10 - 2.8.0 + 2.14.0 1.15 2.1 2.1.9 From fad67613a4bbf5fa670bc18d7013eff3f44769a6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 4 Oct 2024 02:15:47 +0300 Subject: [PATCH 551/580] [fix][sec] Upgrade Avro to 1.11.4 to address CVE-2024-47561 (#23394) --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- distribution/shell/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 8bcb7d7346b59..8c6e2cfa7159a 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -460,8 +460,8 @@ The Apache Software License, Version 2.0 * zt-zip - org.zeroturnaround-zt-zip-1.17.jar * Apache Avro - - org.apache.avro-avro-1.11.3.jar - - org.apache.avro-avro-protobuf-1.11.3.jar + - org.apache.avro-avro-1.11.4.jar + - org.apache.avro-avro-protobuf-1.11.4.jar * Apache Curator - org.apache.curator-curator-client-5.1.0.jar - org.apache.curator-curator-framework-5.1.0.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index faad519df2ef5..6e0bacb2e8845 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -414,8 +414,8 @@ The Apache Software License, Version 2.0 * Google Error Prone Annotations - error_prone_annotations-2.24.0.jar * Javassist -- javassist-3.25.0-GA.jar * Apache Avro - - avro-1.11.3.jar - - avro-protobuf-1.11.3.jar + - avro-1.11.4.jar + - avro-protobuf-1.11.4.jar * RE2j -- re2j-1.7.jar * Spotify completable-futures -- completable-futures-0.3.6.jar diff --git a/pom.xml b/pom.xml index 70956b4d104eb..c50357b840616 100644 --- a/pom.xml +++ b/pom.xml @@ -185,7 +185,7 @@ flexible messaging model and an intuitive client API. 3.4.0 5.18.0 1.12.638 - 1.11.3 + 1.11.4 2.10.10 2.6.0 5.1.0 From 38322a689b205fa8e4233146a2f9136081f92f26 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 3 Oct 2024 21:58:43 -0700 Subject: [PATCH 552/580] [improve][broker] PIP-327: Support force topic loading for unrecoverable errors (#21759) --- .../mledger/ManagedLedgerConfig.java | 12 ++++ .../mledger/impl/ManagedCursorImpl.java | 6 +- .../mledger/impl/ManagedCursorTest.java | 58 ++++++++++++++++++- .../pulsar/broker/ServiceConfiguration.java | 12 ++++ .../pulsar/broker/service/BrokerService.java | 1 + .../schema/BookkeeperSchemaStorage.java | 30 ++++++---- .../schema/BookkeeperSchemaStorageTest.java | 15 +++-- 7 files changed, 117 insertions(+), 17 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index a24251450b4f4..7b28990f35574 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -64,6 +64,7 @@ public class ManagedLedgerConfig { private long retentionTimeMs = 0; private long retentionSizeInMB = 0; private boolean autoSkipNonRecoverableData; + private boolean ledgerForceRecovery; private boolean lazyCursorRecovery = false; private long metadataOperationsTimeoutSeconds = 60; private long readEntryTimeoutSeconds = 120; @@ -465,6 +466,17 @@ public void setAutoSkipNonRecoverableData(boolean skipNonRecoverableData) { this.autoSkipNonRecoverableData = skipNonRecoverableData; } + /** + * Skip managed ledger failure to recover managed ledger forcefully. + */ + public boolean isLedgerForceRecovery() { + return ledgerForceRecovery; + } + + public void setLedgerForceRecovery(boolean ledgerForceRecovery) { + this.ledgerForceRecovery = ledgerForceRecovery; + } + /** * @return max unacked message ranges that will be persisted and recovered. * diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index b39fd231cdc06..f469b88cae8e6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -182,6 +182,7 @@ public class ManagedCursorImpl implements ManagedCursor { // Wether the current cursorLedger is read-only or writable private boolean isCursorLedgerReadOnly = true; + private boolean ledgerForceRecovery; // Stat of the cursor z-node // NOTE: Don't update cursorLedgerStat alone, @@ -332,6 +333,7 @@ public interface VoidCallback { markDeleteLimiter = null; } this.mbean = new ManagedCursorMXBeanImpl(this); + this.ledgerForceRecovery = getConfig().isLedgerForceRecovery(); } private void updateCursorLedgerStat(ManagedCursorInfo cursorInfo, Stat stat) { @@ -547,7 +549,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (log.isInfoEnabled()) { log.info("[{}] Opened ledger {} for cursor {}. rc={}", ledger.getName(), ledgerId, name, rc); } - if (isBkErrorNotRecoverable(rc)) { + if (isBkErrorNotRecoverable(rc) || ledgerForceRecovery) { log.error("[{}] Error opening metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc)); // Rewind to oldest entry available @@ -575,7 +577,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (log.isDebugEnabled()) { log.debug("[{}} readComplete rc={} entryId={}", ledger.getName(), rc1, lh1.getLastAddConfirmed()); } - if (isBkErrorNotRecoverable(rc1)) { + if (isBkErrorNotRecoverable(rc1) || ledgerForceRecovery) { log.error("[{}] Error reading from metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc1)); // Rewind to oldest entry available diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 1067cda441f6a..8ae5a04a507b1 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -70,11 +70,14 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import lombok.Cleanup; +import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.client.PulsarMockBookKeeper; +import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback; @@ -98,6 +101,7 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; +import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.apache.pulsar.common.api.proto.IntRange; import org.apache.pulsar.common.util.FutureUtil; @@ -4538,7 +4542,6 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { ledger.close(); } - @Test public void testReadEntriesWithSkipDeletedEntries() throws Exception { @Cleanup @@ -4795,5 +4798,58 @@ public void operationFailed(ManagedLedgerException exception) { assertEquals(cursor.getReadPosition(), markDeletedPosition.getNext()); } + @Test + void testForceCursorRecovery() throws Exception { + ManagedLedgerFactoryConfig managedLedgerFactoryConfig = new ManagedLedgerFactoryConfig(); + TestPulsarMockBookKeeper bk = new TestPulsarMockBookKeeper(executor); + factory = new ManagedLedgerFactoryImpl(metadataStore, bk); + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setLedgerForceRecovery(true); + ManagedLedger ledger = factory.open("my_test_ledger", config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); + ledger.addEntry("entry-1".getBytes(Encoding)); + long invalidLedger = -1L; + bk.setErrorCodeMap(invalidLedger, BKException.Code.BookieHandleNotAvailableException); + ManagedCursorInfo info = ManagedCursorInfo.newBuilder().setCursorsLedgerId(invalidLedger).build(); + CountDownLatch latch = new CountDownLatch(1); + MutableBoolean recovered = new MutableBoolean(false); + VoidCallback callback = new VoidCallback() { + @Override + public void operationComplete() { + recovered.setValue(true); + latch.countDown(); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + recovered.setValue(false); + latch.countDown(); + } + }; + c1.recoverFromLedger(info, callback); + latch.await(); + assertTrue(recovered.booleanValue()); + } + + class TestPulsarMockBookKeeper extends PulsarMockBookKeeper { + Map ledgerErrors = new HashMap<>(); + + public TestPulsarMockBookKeeper(OrderedExecutor orderedExecutor) throws Exception { + super(orderedExecutor); + } + + public void setErrorCodeMap(long ledgerId, int rc) { + ledgerErrors.put(ledgerId, rc); + } + + public void asyncOpenLedger(final long lId, final DigestType digestType, final byte[] passwd, + final OpenCallback cb, final Object ctx) { + if (ledgerErrors.containsKey(lId)) { + cb.openComplete(ledgerErrors.get(lId), null, ctx); + } + super.asyncOpenLedger(lId, digestType, passwd, cb, ctx); + } + } + private static final Logger log = LoggerFactory.getLogger(ManagedCursorTest.class); } 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 33b4fbff5f5bb..58d6444e7196a 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 @@ -2249,6 +2249,18 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece + " It helps when data-ledgers gets corrupted at bookkeeper and managed-cursor is stuck at that ledger." ) private boolean autoSkipNonRecoverableData = false; + @FieldContext( + dynamic = true, + category = CATEGORY_STORAGE_ML, + doc = "Skip managed ledger failure to forcefully recover managed ledger." + ) + private boolean managedLedgerForceRecovery = false; + @FieldContext( + dynamic = true, + category = CATEGORY_STORAGE_ML, + doc = "Skip schema ledger failure to forcefully recover topic successfully." + ) + private boolean schemaLedgerForceRecovery = false; @FieldContext( category = CATEGORY_STORAGE_ML, doc = "operation timeout while updating managed-ledger metadata." 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 bfa99eedcadce..dd722dffcfbfc 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 @@ -1970,6 +1970,7 @@ private CompletableFuture getManagedLedgerConfig(@Nonnull T .setRetentionTime(retentionPolicies.getRetentionTimeInMinutes(), TimeUnit.MINUTES); managedLedgerConfig.setRetentionSizeInMB(retentionPolicies.getRetentionSizeInMB()); managedLedgerConfig.setAutoSkipNonRecoverableData(serviceConfig.isAutoSkipNonRecoverableData()); + managedLedgerConfig.setLedgerForceRecovery(serviceConfig.isManagedLedgerForceRecovery()); managedLedgerConfig.setLazyCursorRecovery(serviceConfig.isLazyCursorRecovery()); managedLedgerConfig.setInactiveLedgerRollOverTime( serviceConfig.getManagedLedgerInactiveLedgerRolloverTimeSeconds(), TimeUnit.SECONDS); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java index 99f0249b304b3..85c8aa064581f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java @@ -528,7 +528,7 @@ private CompletableFuture readSchemaEntry( return openLedger(position.getLedgerId()) .thenCompose((ledger) -> - Functions.getLedgerEntry(ledger, position.getEntryId()) + Functions.getLedgerEntry(ledger, position.getEntryId(), config.isSchemaLedgerForceRecovery()) .thenCompose(entry -> closeLedger(ledger) .thenApply(ignore -> entry) ) @@ -560,7 +560,8 @@ private CompletableFuture addEntry(LedgerHandle ledgerHandle, SchemaStorag ledgerHandle.asyncAddEntry(entry.toByteArray(), (rc, handle, entryId, ctx) -> { if (rc != BKException.Code.OK) { - future.completeExceptionally(bkException("Failed to add entry", rc, ledgerHandle.getId(), -1)); + future.completeExceptionally(bkException("Failed to add entry", rc, ledgerHandle.getId(), -1, + config.isSchemaLedgerForceRecovery())); } else { future.complete(entryId); } @@ -582,7 +583,8 @@ private CompletableFuture createLedger(String schemaId) { LedgerPassword, (rc, handle, ctx) -> { if (rc != BKException.Code.OK) { - future.completeExceptionally(bkException("Failed to create ledger", rc, -1, -1)); + future.completeExceptionally(bkException("Failed to create ledger", rc, -1, -1, + config.isSchemaLedgerForceRecovery())); } else { future.complete(handle); } @@ -603,7 +605,8 @@ private CompletableFuture openLedger(Long ledgerId) { LedgerPassword, (rc, handle, ctx) -> { if (rc != BKException.Code.OK) { - future.completeExceptionally(bkException("Failed to open ledger", rc, ledgerId, -1)); + future.completeExceptionally(bkException("Failed to open ledger", rc, ledgerId, -1, + config.isSchemaLedgerForceRecovery())); } else { future.complete(handle); } @@ -617,7 +620,8 @@ private CompletableFuture closeLedger(LedgerHandle ledgerHandle) { CompletableFuture future = new CompletableFuture<>(); ledgerHandle.asyncClose((rc, handle, ctx) -> { if (rc != BKException.Code.OK) { - future.completeExceptionally(bkException("Failed to close ledger", rc, ledgerHandle.getId(), -1)); + future.completeExceptionally(bkException("Failed to close ledger", rc, ledgerHandle.getId(), -1, + config.isSchemaLedgerForceRecovery())); } else { future.complete(null); } @@ -648,12 +652,14 @@ public CompletableFuture> getStoreLedgerIdsBySchemaId(String schemaId } interface Functions { - static CompletableFuture getLedgerEntry(LedgerHandle ledger, long entry) { + static CompletableFuture getLedgerEntry(LedgerHandle ledger, long entry, + boolean forceRecovery) { final CompletableFuture future = new CompletableFuture<>(); ledger.asyncReadEntries(entry, entry, (rc, handle, entries, ctx) -> { if (rc != BKException.Code.OK) { - future.completeExceptionally(bkException("Failed to read entry", rc, ledger.getId(), entry)); + future.completeExceptionally(bkException("Failed to read entry", rc, ledger.getId(), entry, + forceRecovery)); } else { future.complete(entries.nextElement()); } @@ -700,7 +706,8 @@ static class LocatorEntry { } } - public static Exception bkException(String operation, int rc, long ledgerId, long entryId) { + public static Exception bkException(String operation, int rc, long ledgerId, long entryId, + boolean forceRecovery) { String message = org.apache.bookkeeper.client.api.BKException.getMessage(rc) + " - ledger=" + ledgerId + " - operation=" + operation; @@ -709,7 +716,10 @@ public static Exception bkException(String operation, int rc, long ledgerId, lon } boolean recoverable = rc != BKException.Code.NoSuchLedgerExistsException && rc != BKException.Code.NoSuchEntryException - && rc != BKException.Code.NoSuchLedgerExistsOnMetadataServerException; + && rc != BKException.Code.NoSuchLedgerExistsOnMetadataServerException + // if force-recovery is enabled then made it non-recoverable exception + // and force schema to skip this exception and recover immediately + && !forceRecovery; return new SchemaException(recoverable, message); } @@ -732,4 +742,4 @@ public static CompletableFuture ignoreUnrecoverableBKException(Completabl throw t instanceof CompletionException ? (CompletionException) t : new CompletionException(t); }); } -} +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageTest.java index d0c2e149bf438..3653c01daec37 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageTest.java @@ -21,6 +21,7 @@ import java.nio.ByteBuffer; import org.apache.bookkeeper.client.api.BKException; import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.schema.exceptions.SchemaException; import org.apache.pulsar.common.schema.LongSchemaVersion; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.testng.annotations.Test; @@ -29,23 +30,29 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; @Test(groups = "broker") public class BookkeeperSchemaStorageTest { @Test public void testBkException() { - Exception ex = bkException("test", BKException.Code.ReadException, 1, -1); + Exception ex = bkException("test", BKException.Code.ReadException, 1, -1, false); assertEquals("Error while reading ledger - ledger=1 - operation=test", ex.getMessage()); - ex = bkException("test", BKException.Code.ReadException, 1, 0); + ex = bkException("test", BKException.Code.ReadException, 1, 0, false); assertEquals("Error while reading ledger - ledger=1 - operation=test - entry=0", ex.getMessage()); - ex = bkException("test", BKException.Code.QuorumException, 1, -1); + ex = bkException("test", BKException.Code.QuorumException, 1, -1, false); assertEquals("Invalid quorum size on ensemble size - ledger=1 - operation=test", ex.getMessage()); - ex = bkException("test", BKException.Code.QuorumException, 1, 0); + ex = bkException("test", BKException.Code.QuorumException, 1, 0, false); assertEquals("Invalid quorum size on ensemble size - ledger=1 - operation=test - entry=0", ex.getMessage()); + SchemaException sc = (SchemaException) bkException("test", BKException.Code.BookieHandleNotAvailableException, 1, 0, false); + assertTrue(sc.isRecoverable()); + sc = (SchemaException) bkException("test", BKException.Code.BookieHandleNotAvailableException, 1, 0, true); + assertFalse(sc.isRecoverable()); } @Test From 1e936778691b0cee54b6fe34b53ebc1593f5ae92 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 3 Oct 2024 23:11:36 -0700 Subject: [PATCH 553/580] [improve][pip] PIP-360 Add admin API to display Schema metadata (#22938) --- .../admin/impl/SchemasResourceBase.java | 9 ++++ .../broker/admin/v1/SchemasResource.java | 32 +++++++++++ .../broker/admin/v2/SchemasResource.java | 30 +++++++++++ .../schema/BookkeeperSchemaStorage.java | 20 +++++++ .../org/apache/pulsar/schema/SchemaTest.java | 26 +++++++++ .../apache/pulsar/client/admin/Schemas.java | 16 ++++++ .../common/policies/data/SchemaMetadata.java | 48 +++++++++++++++++ .../client/admin/internal/SchemasImpl.java | 18 +++++++ .../apache/pulsar/admin/cli/CmdSchemas.java | 13 +++++ .../pulsar/admin/cli/TestCmdSchema.java | 54 +++++++++++++++++++ 10 files changed, 266 insertions(+) create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SchemaMetadata.java create mode 100644 pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSchema.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SchemasResourceBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SchemasResourceBase.java index 286366c8b5834..886db9c7abb37 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SchemasResourceBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SchemasResourceBase.java @@ -31,6 +31,7 @@ import javax.ws.rs.core.Response; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.admin.AdminResource; +import org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage; import org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.web.RestException; @@ -38,6 +39,7 @@ import org.apache.pulsar.client.internal.DefaultImplementation; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; +import org.apache.pulsar.common.policies.data.SchemaMetadata; import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.protocol.schema.GetAllVersionsSchemaResponse; import org.apache.pulsar.common.protocol.schema.GetSchemaResponse; @@ -105,6 +107,13 @@ public CompletableFuture> getAllSchemasAsync(boolean aut }); } + public CompletableFuture getSchemaMetadataAsync(boolean authoritative) { + String schemaId = getSchemaId(); + BookkeeperSchemaStorage storage = (BookkeeperSchemaStorage) pulsar().getSchemaStorage(); + return validateOwnershipAndOperationAsync(authoritative, TopicOperation.GET_METADATA) + .thenCompose(__ -> storage.getSchemaMetadata(schemaId)); + } + public CompletableFuture deleteSchemaAsync(boolean authoritative, boolean force) { return validateDestinationAndAdminOperationAsync(authoritative) .thenCompose(__ -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/SchemasResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/SchemasResource.java index edc600707a120..0d6c3814bf863 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/SchemasResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/SchemasResource.java @@ -43,6 +43,7 @@ import org.apache.pulsar.broker.admin.impl.SchemasResourceBase; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; import org.apache.pulsar.broker.service.schema.exceptions.InvalidSchemaDataException; +import org.apache.pulsar.common.policies.data.SchemaMetadata; import org.apache.pulsar.common.protocol.schema.DeleteSchemaResponse; import org.apache.pulsar.common.protocol.schema.GetAllVersionsSchemaResponse; import org.apache.pulsar.common.protocol.schema.GetSchemaResponse; @@ -170,6 +171,37 @@ public void getAllSchemas( }); } + @GET + @Path("/{tenant}/{cluster}/{namespace}/{topic}/metadata") + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation(value = "Get the schema metadata of a topic", response = SchemaMetadata.class) + @ApiResponses(value = { + @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), + @ApiResponse(code = 401, message = "Client is not authorized or Don't have admin permission"), + @ApiResponse(code = 403, message = "Client is not authenticated"), + @ApiResponse(code = 404, + message = "Tenant or Namespace or Topic doesn't exist; or Schema is not found for this topic"), + @ApiResponse(code = 412, message = "Failed to find the ownership for the topic"), + @ApiResponse(code = 500, message = "Internal Server Error"), + }) + public void getSchemaMetadata( + @PathParam("tenant") String tenant, + @PathParam("cluster") String cluster, + @PathParam("namespace") String namespace, + @PathParam("topic") String topic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @Suspended final AsyncResponse response + ) { + validateTopicName(tenant, cluster, namespace, topic); + getSchemaMetadataAsync(authoritative) + .thenAccept(response::resume) + .exceptionally(ex -> { + log.error("[{}] Failed to get schema metadata for topic {}", clientAppId(), topicName, ex); + resumeAsyncResponseExceptionally(response, ex); + return null; + }); + } + @DELETE @Path("/{tenant}/{cluster}/{namespace}/{topic}/schema") @Produces(MediaType.APPLICATION_JSON) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/SchemasResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/SchemasResource.java index dd8ed58c853fa..07758436f6ca7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/SchemasResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/SchemasResource.java @@ -166,6 +166,36 @@ public void getAllSchemas( }); } + @GET + @Path("/{tenant}/{namespace}/{topic}/metadata") + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation(value = "Get the schema metadata of a topic", response = GetAllVersionsSchemaResponse.class) + @ApiResponses(value = { + @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), + @ApiResponse(code = 401, message = "Client is not authorized or Don't have admin permission"), + @ApiResponse(code = 403, message = "Client is not authenticated"), + @ApiResponse(code = 404, + message = "Tenant or Namespace or Topic doesn't exist; or Schema is not found for this topic"), + @ApiResponse(code = 412, message = "Failed to find the ownership for the topic"), + @ApiResponse(code = 500, message = "Internal Server Error"), + }) + public void getSchemaMetadata( + @PathParam("tenant") String tenant, + @PathParam("namespace") String namespace, + @PathParam("topic") String topic, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @Suspended final AsyncResponse response + ) { + validateTopicName(tenant, namespace, topic); + getSchemaMetadataAsync(authoritative) + .thenAccept(response::resume) + .exceptionally(ex -> { + log.error("[{}] Failed to get schema metadata for topic {}", clientAppId(), topicName, ex); + resumeAsyncResponseExceptionally(response, ex); + return null; + }); + } + @DELETE @Path("/{tenant}/{namespace}/{topic}/schema") @Produces(MediaType.APPLICATION_JSON) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java index 85c8aa064581f..f68cdd6473e48 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java @@ -52,8 +52,11 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.schema.SchemaStorageFormat.IndexEntry; +import org.apache.pulsar.broker.service.schema.SchemaStorageFormat.SchemaLocator; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; import org.apache.pulsar.broker.service.schema.exceptions.SchemaException; +import org.apache.pulsar.common.policies.data.SchemaMetadata; import org.apache.pulsar.common.protocol.schema.SchemaStorage; import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.protocol.schema.StoredSchema; @@ -554,6 +557,23 @@ private CompletableFuture> getSchemaLocator(String schema o.map(r -> new LocatorEntry(r.getValue(), r.getStat().getVersion()))); } + public CompletableFuture getSchemaMetadata(String schema) { + return getLocator(schema).thenApply(locator -> { + if (!locator.isPresent()) { + return null; + } + SchemaLocator sl = locator.get().locator; + SchemaMetadata metadata = new SchemaMetadata(); + IndexEntry info = sl.getInfo(); + metadata.info = new SchemaMetadata.Entry(info.getPosition().getLedgerId(), info.getPosition().getEntryId(), + info.getVersion()); + metadata.index = sl.getIndexList() == null ? null + : sl.getIndexList().stream().map(i -> new SchemaMetadata.Entry(i.getPosition().getLedgerId(), + i.getPosition().getEntryId(), i.getVersion())).collect(Collectors.toList()); + return metadata; + }); + } + @NotNull private CompletableFuture addEntry(LedgerHandle ledgerHandle, SchemaStorageFormat.SchemaEntry entry) { final CompletableFuture future = new CompletableFuture<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java index ae9ea6d5ae6f4..ab82f981b5df3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java @@ -23,6 +23,7 @@ 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.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertThrows; @@ -84,6 +85,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; +import org.apache.pulsar.common.policies.data.SchemaMetadata; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.schema.KeyValueEncodingType; @@ -1492,4 +1494,28 @@ public SchemaStorageFormat.SchemaLocator deserialize(String path, byte[] content consumer.close(); producer.close(); } + + @Test + public void testTopicSchemaMetadata() throws Exception { + final String tenant = PUBLIC_TENANT; + final String namespace = "test-namespace-" + randomName(16); + final String topicOne = "metadata-topic"; + final String topicName = TopicName.get(TopicDomain.persistent.value(), tenant, namespace, topicOne).toString(); + + admin.namespaces().createNamespace(tenant + "/" + namespace, Sets.newHashSet(CLUSTER_NAME)); + + @Cleanup + Producer producer = pulsarClient + .newProducer(Schema.AVRO(SchemaDefinition. builder().withAlwaysAllowNull(false) + .withSupportSchemaVersioning(true).withPojo(Schemas.PersonTwo.class).build())) + .topic(topicName).create(); + + SchemaMetadata metadata = admin.schemas().getSchemaMetadata(topicName); + + assertNotNull(metadata); + assertNotNull(metadata.info); + assertNotEquals(metadata.info.getLedgerId(), 0); + assertEquals(metadata.info.getEntryId(), 0); + assertEquals(metadata.index.size(), 1); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Schemas.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Schemas.java index 9a1eb67d2e53a..ca8bed253702f 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Schemas.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Schemas.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; +import org.apache.pulsar.common.policies.data.SchemaMetadata; import org.apache.pulsar.common.protocol.schema.IsCompatibilityResponse; import org.apache.pulsar.common.protocol.schema.PostSchemaPayload; import org.apache.pulsar.common.schema.SchemaInfo; @@ -233,4 +234,19 @@ IsCompatibilityResponse testCompatibility(String topic, PostSchemaPayload schema * @param topic topic name, in fully qualified format */ CompletableFuture> getAllSchemasAsync(String topic); + + /** + * Get schema metadata of the topic. + * + * @param topic topic name, in fully qualified format + * @throws PulsarAdminException + */ + SchemaMetadata getSchemaMetadata(String topic) throws PulsarAdminException; + + /** + * Get schema metadata of the topic asynchronously. + * + * @param topic topic name, in fully qualified format + */ + CompletableFuture getSchemaMetadataAsync(String topic); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SchemaMetadata.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SchemaMetadata.java new file mode 100644 index 0000000000000..ff6ba6e86499e --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SchemaMetadata.java @@ -0,0 +1,48 @@ +/* + * 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.policies.data; + +import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +/** + * Schema metadata info. + */ +@Data +public class SchemaMetadata { + + public Entry info; + public List index; + + @Data + @AllArgsConstructor + @NoArgsConstructor + public static class Entry { + private long ledgerId; + private long entryId; + private long version; + + @Override + public String toString() { + return String.format("ledgerId=[%d], entryId=[%d], version=[%d]", ledgerId, entryId, version); + } + } +} \ No newline at end of file diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SchemasImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SchemasImpl.java index 28b435ab5676b..7f2383e1e52ef 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SchemasImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SchemasImpl.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.internal.DefaultImplementation; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.SchemaMetadata; import org.apache.pulsar.common.protocol.schema.DeleteSchemaResponse; import org.apache.pulsar.common.protocol.schema.GetAllVersionsSchemaResponse; import org.apache.pulsar.common.protocol.schema.GetSchemaResponse; @@ -276,6 +277,19 @@ public CompletableFuture> getAllSchemasAsync(String topic) { .collect(Collectors.toList())); } + @Override + public SchemaMetadata getSchemaMetadata(String topic) throws PulsarAdminException { + return sync(() -> getSchemaMetadataAsync(topic)); + } + + @Override + public CompletableFuture getSchemaMetadataAsync(String topic) { + TopicName tn = TopicName.get(topic); + WebTarget path = metadata(tn); + return asyncGetRequest(path, new FutureCallback(){}); + } + + private WebTarget schemaPath(TopicName topicName) { return topicPath(topicName, "schema"); } @@ -292,6 +306,10 @@ private WebTarget compatibilityPath(TopicName topicName) { return topicPath(topicName, "compatibility"); } + private WebTarget metadata(TopicName topicName) { + return topicPath(topicName, "metadata"); + } + private WebTarget topicPath(TopicName topic, String... parts) { final WebTarget base = topic.isV2() ? adminV2 : adminV1; WebTarget topicPath = base.path(topic.getRestPath(false)); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSchemas.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSchemas.java index ab8fdc1f01359..9131f11f3d33d 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSchemas.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdSchemas.java @@ -44,6 +44,7 @@ public CmdSchemas(Supplier admin) { addCommand("delete", new DeleteSchema()); addCommand("upload", new UploadSchema()); addCommand("extract", new ExtractSchema()); + addCommand("metadata", new GetSchemaMetadata()); addCommand("compatibility", new TestCompatibility()); } @@ -77,6 +78,18 @@ void run() throws Exception { } } + @Command(description = "Get the schema for a topic") + private class GetSchemaMetadata extends CliCommand { + @Parameters(description = "persistent://tenant/namespace/topic", arity = "1") + private String topicName; + + @Override + void run() throws Exception { + String topic = validateTopicName(topicName); + print(getAdmin().schemas().getSchemaMetadata(topic)); + } + } + @Command(description = "Delete all versions schema of a topic") private class DeleteSchema extends CliCommand { @Parameters(description = "persistent://tenant/namespace/topic", arity = "1") diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSchema.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSchema.java new file mode 100644 index 0000000000000..b61ac3b8ef3d5 --- /dev/null +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdSchema.java @@ -0,0 +1,54 @@ +/* + * 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.admin.cli; + +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.Schemas; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class TestCmdSchema { + + private PulsarAdmin pulsarAdmin; + + private CmdSchemas cmdSchemas; + + private Schemas schemas; + + @BeforeMethod + public void setup() throws Exception { + pulsarAdmin = mock(PulsarAdmin.class); + schemas = mock(Schemas.class); + when(pulsarAdmin.schemas()).thenReturn(schemas); + cmdSchemas = spy(new CmdSchemas(() -> pulsarAdmin)); + } + + @Test + public void testCmdClusterConfigFile() throws Exception { + String topic = "persistent://tenant/ns1/t1"; + cmdSchemas.run(new String[]{"metadata", topic}); + verify(schemas).getSchemaMetadata(eq(topic)); + } +} From 56200aabc56e75ca9ea5be1edb52d6c9d3f07fe5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 4 Oct 2024 18:14:42 +0300 Subject: [PATCH 554/580] [improve][ci] Continue Pulsar CI build even when Trivy scanner fails (#23397) --- .github/workflows/pulsar-ci.yaml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index ad017674ac6ee..091dab25ec696 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -890,8 +890,10 @@ jobs: run: src/check-binary-license.sh ./distribution/server/target/apache-pulsar-*-bin.tar.gz && src/check-binary-license.sh ./distribution/shell/target/apache-pulsar-shell-*-bin.tar.gz - name: Run Trivy container scan + id: trivy_scan uses: aquasecurity/trivy-action@master if: ${{ github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} + continue-on-error: true with: image-ref: "apachepulsar/pulsar:latest" scanners: vuln @@ -902,7 +904,8 @@ jobs: - name: Upload Trivy scan results to GitHub Security tab uses: github/codeql-action/upload-sarif@v3 - if: ${{ github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} + if: ${{ steps.trivy_scan.outcome == 'success' && github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} + continue-on-error: true with: sarif_file: 'trivy-results.sarif' From aa125616aeb3d71ad23bd6bfeb651ae7f4fe6f55 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 4 Oct 2024 10:51:45 -0700 Subject: [PATCH 555/580] [fix][broker] Fix Broker was failing to create producer with broken schema ledger (#23395) --- .../pulsar/broker/service/AbstractTopic.java | 13 +++++-- .../service/schema/ClientGetSchemaTest.java | 35 ++++++++++++++++++- 2 files changed, 44 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index dce50a54db1f6..76dd277159cf4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -37,6 +37,7 @@ import java.util.Set; import java.util.TreeSet; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; @@ -64,6 +65,7 @@ import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; +import org.apache.pulsar.broker.service.schema.exceptions.SchemaException; import org.apache.pulsar.broker.stats.prometheus.metrics.Summary; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.naming.NamespaceName; @@ -666,9 +668,14 @@ protected String getSchemaId() { } @Override public CompletableFuture hasSchema() { - return brokerService.pulsar() - .getSchemaRegistryService() - .getSchema(getSchemaId()).thenApply(Objects::nonNull); + return brokerService.pulsar().getSchemaRegistryService().getSchema(getSchemaId()).thenApply(Objects::nonNull) + .exceptionally(e -> { + Throwable ex = e.getCause(); + if (ex instanceof SchemaException || !((SchemaException) ex).isRecoverable()) { + return false; + } + throw ex instanceof CompletionException ? (CompletionException) ex : new CompletionException(ex); + }); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/ClientGetSchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/ClientGetSchemaTest.java index 970e6b2712981..ec81f39fef92c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/ClientGetSchemaTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/ClientGetSchemaTest.java @@ -20,8 +20,9 @@ import static org.apache.pulsar.common.naming.TopicName.PUBLIC_TENANT; import static org.apache.pulsar.schema.compatibility.SchemaCompatibilityCheckTest.randomName; -import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertNotNull; import java.util.ArrayList; import java.util.List; @@ -177,4 +178,36 @@ public void testSchemaFailure() throws Exception { producer.close(); consumer.close(); } + + @Test + public void testAddProducerOnDeletedSchemaLedgerTopic() throws Exception { + final String tenant = PUBLIC_TENANT; + final String namespace = "test-namespace-" + randomName(16); + final String topicOne = "test-deleted-schema-ledger"; + final String fqtnOne = TopicName.get(TopicDomain.persistent.value(), tenant, namespace, topicOne).toString(); + + //pulsar.getConfig().setManagedLedgerForceRecovery(true); + admin.namespaces().createNamespace(tenant + "/" + namespace, Sets.newHashSet("test")); + + // (1) create topic with schema + Producer producer = pulsarClient + .newProducer(Schema.AVRO(SchemaDefinition. builder().withAlwaysAllowNull(false) + .withSupportSchemaVersioning(true).withPojo(Schemas.PersonTwo.class).build())) + .topic(fqtnOne).create(); + + producer.close(); + + String key = TopicName.get(fqtnOne).getSchemaName(); + BookkeeperSchemaStorage schemaStrogate = (BookkeeperSchemaStorage) pulsar.getSchemaStorage(); + long schemaLedgerId = schemaStrogate.getSchemaLedgerList(key).get(0); + + // (2) break schema locator by deleting schema-ledger + schemaStrogate.getBookKeeper().deleteLedger(schemaLedgerId); + + admin.topics().unload(fqtnOne); + + Producer producerWihtoutSchema = pulsarClient.newProducer().topic(fqtnOne).create(); + + assertNotNull(producerWihtoutSchema); + } } From 64e9687bb55c42a28f77dca73fdbd68f7f390ca4 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 5 Oct 2024 00:26:30 +0300 Subject: [PATCH 556/580] [improve][pip] PIP-379: Key_Shared Draining Hashes for Improved Message Ordering (#23309) --- pip/pip-379.md | 407 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 407 insertions(+) create mode 100644 pip/pip-379.md diff --git a/pip/pip-379.md b/pip/pip-379.md new file mode 100644 index 0000000000000..3215bb541f11e --- /dev/null +++ b/pip/pip-379.md @@ -0,0 +1,407 @@ +# PIP-379: Key_Shared Draining Hashes for Improved Message Ordering + +## Background Knowledge + +Apache Pulsar's Key_Shared subscription mode is designed to provide ordered message delivery on a per-key basis while allowing multiple consumers to process messages concurrently. This mode is particularly useful in scenarios where maintaining message order for specific keys is crucial, but overall throughput can be improved by parallelizing message consumption across multiple consumers. + +Key concepts: + +- **Key_Shared subscription**: A subscription mode that maintains message ordering per key while allowing multiple consumers. +- **Hash ranges**: In AUTO_SPLIT mode, the hash space is divided among active consumers to distribute message processing. +- **Pending messages**: Messages that have been sent to a consumer but not yet acknowledged (also called "pending acks" or "unacknowledged messages"). + +### Current contract of preserving ordering + +The Key_Shared subscription is described in the [Pulsar documentation](https://pulsar.apache.org/docs/concepts-messaging/#key_shared). + +For this PIP, the most important detail is the "Preserving order of processing" section. +There are recent changes in this section that apply to the master branch of Pulsar and, therefore, to the upcoming Pulsar 4.0. The changes were made as part of ["PIP-282: Change definition of the recently joined consumers position"](https://github.com/apache/pulsar/blob/master/pip/pip-282.md). + +[PIP-282 (master branch / Pulsar 4.0) version of the "Preserving order of processing" section](https://pulsar.apache.org/docs/next/concepts-messaging/#preserving-order-of-processing): + +> Key_Shared Subscription type guarantees a key will be processed by a *single* consumer at any given time. When a new consumer is connected, some keys will change their mapping from existing consumers to the new consumer. Once the connection has been established, the broker will record the current `lastSentPosition` and associate it with the new consumer. The `lastSentPosition` is a marker indicating that messages have been dispatched to the consumers up to this point. The broker will start delivering messages to the new consumer *only* when all messages up to the `lastSentPosition` have been acknowledged. This will guarantee that a certain key is processed by a single consumer at any given time. The trade-off is that if one of the existing consumers is stuck and no time-out was defined (acknowledging for you), the new consumer won't receive any messages until the stuck consumer resumes or gets disconnected. + +[Previous version (applies to Pulsar 3.x) of the "Preserving order of processing" section](https://pulsar.apache.org/docs/3.3.x/concepts-messaging/#preserving-order-of-processing): + +> Key Shared Subscription type guarantees a key will be processed by a *single* consumer at any given time. When a new consumer is connected, some keys will change their mapping from existing consumers to the new consumer. Once the connection has been established, the broker will record the current read position and associate it with the new consumer. The read position is a marker indicating that messages have been dispatched to the consumers up to this point, and after it, no messages have been dispatched yet. The broker will start delivering messages to the new consumer *only* when all messages up to the read position have been acknowledged. This will guarantee that a certain key is processed by a single consumer at any given time. The trade-off is that if one of the existing consumers is stuck and no time-out was defined (acknowledging for you), the new consumer won't receive any messages until the stuck consumer resumes or gets disconnected. + +## Motivation + +The current implementation of Key_Shared subscriptions faces several challenges: + +1. **Complex Contract of Preserving Ordering**: The current contract of preserving ordering is hard to understand and contains a fundamental problem. It explains a solution and then ties the guarantee to the provided solution. It could be interpreted that there's a guarantee as long as this solution is able to handle the case. +2. **Incomplete Ordering Contract Fulfillment**: The current contract seems to make a conditional guarantee that a certain key is processed by a single consumer at any given time. Outside of the described solution in the contract, the current implementation struggles to consistently prevent messages from being sent to another consumer while pending on the original consumer. While Key_Shared subscriptions aim to preserve message ordering per key, the current implementation may not always achieve this, especially during consumer changes. There's a potential corner case reported in [issue #23307](https://github.com/apache/pulsar/issues/23307). +3. **Usability Issues**: Understanding the current system and detecting the reason why messages get blocked is time-consuming and difficult. +4. **Unnecessary Message Blocking**: The current implementation blocks delivery for all messages when any hash range is blocked, even if other keys could be processed independently. This leads to suboptimal utilization of consumers and increased latency for messages that could otherwise be processed. +5. **Observability Challenges**: The current implementation lacks clear visibility into the consuming state when processing gets stuck, making it harder to build automation for detecting and mitigating issues. +6. **Complexity**: The existing solution for managing "recently joined consumers" is overly complex, making the system harder to maintain and debug. + +## Goals + +### In Scope + +- Clarify and fulfill the key-ordered message delivery contract for Key_Shared AUTO_SPLIT mode. +- Fix current issues where messages are sent out-of-order or when a single key is outstanding in multiple consumers at a time. +- Improve the handling of unacknowledged messages to prevent indefinite blocking and consumers getting stuck. +- Minimize memory usage for pending message tracking, eliminating PIP-282's "sent positions" tracking. +- Implement a new "draining hashes" concept to efficiently manage message ordering in Key_Shared subscriptions. +- Enhance the reliability, usability, and scalability of Key_Shared subscriptions. +- Improve observability of Key_Shared subscriptions to aid in troubleshooting and automation. +- Ensure strict ordering guarantees for messages with the same key, even during consumer changes. + +### Out of Scope + +- Changes to other subscription types (Exclusive, Failover, Shared). +- Adding support key based ordering guarantees when negative acknowledgements are used + +## High-Level Design + +### Updated contract of preserving ordering + +The "Preserving order of processing" section of the Key_Shared documentation would be updated to contain this contract: + +_In Key_Shared subscriptions, messages with the same key are delivered and allowed to be in unacknowledged state to only one consumer at a time._ + +When new consumers join or leave, the consumer handling a message key can change when the default AUTO_SPLIT mode is used, but only after all pending messages for a particular key are acknowledged or the original consumer disconnects. + +The Key_Shared subscription doesn't prevent using any methods in the consumer API. For example, the application might call `negativeAcknowledge` or the `redeliverUnacknowledgedMessages` method. When messages are scheduled for delivery due to these methods, they will get redelivered as soon as possible. There's no ordering guarantee in these cases, however the guarantee of delivering a message key to a single consumer at a time will continue to be preserved. + +### Computer Science Perspective: Invariants + +Wikipedia tells us about [invariants](https://en.wikipedia.org/wiki/Invariant_(mathematics)#Invariants_in_computer_science): "In computer science, an invariant is a logical assertion that is always held to be true during a certain phase of execution of a computer program." + +The contract _"In Key_Shared subscriptions, messages with the same key are delivered and allowed to be in an unacknowledged state to only one consumer at a time."_ can be seen as an invariant for Key_Shared subscriptions. It is something that must always be held true for Key_Shared subscriptions. The design and implementation in PIP-379 focuses on ensuring this. + +### Future work in needed for supporting key-based ordering with negative acknowledgements + +The updated contract explicitly states that it is not possible to retain key-based ordering of messages when negative acknowledgements are used. Changing this is out of scope for PIP-379. A potential future solution for handling this would be to modify the client so that when a message is negatively acknowledged, it would also reject all further messages with the same key until the original message gets redelivered. It's already possible to attempt to implement this in client-side code. However, a proper solution would require support on the broker side to block further delivery of the specific key when there are pending negatively acknowledged messages until all negatively acknowledged messages for that particular key have been acknowledged by the consumer. This solution is out of scope for PIP-379. A future implementation to address these problems could build upon PIP-379 concepts such as "draining hashes" and extend that to cover the negative acknowledgement scenarios. + +### High-Level implementation plan + +The proposed solution introduces a "draining hashes" concept to efficiently manage message ordering in Key_Shared subscriptions: + +**1. When consumer hash ranges change (e.g., a consumer joins or leaves), affected hashes of pending messages are added to a "draining hashes" set.** + +Pending messages of the consumer are iterated, and if the hash of a pending message belongs to one of the impacted ranges, the hash gets added to the "draining hashes" tracker. + +Code example to illustrate the implementation: + +```java + private synchronized void registerDrainingHashes(Consumer skipConsumer, + Map> impactedRangesByConsumer) { + for (Map.Entry> entry : impactedRangesByConsumer.entrySet()) { + Consumer c = entry.getKey(); + if (c != skipConsumer) { + // perf optimization: convert the set to an array to avoid iterator allocation in the pending acks loop + Range[] ranges = entry.getValue().toArray(new Range[0]); + // add all pending acks in the impacted hash ranges to the draining hashes tracker + c.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { + for (Range range : ranges) { + if (range.contains(stickyKeyHash)) { + // add the pending ack to the draining hashes tracker if the hash is in the range + drainingHashesTracker.addEntry(c, stickyKeyHash); + break; + } + // Since ranges are sorted, stop checking further ranges if the start of the current range is + // greater than the stickyKeyHash. + if (range.getStart() > stickyKeyHash) { + break; + } + } + }); + } + } + } +``` + +**2. Following messages with hashes in the "draining hashes" set are blocked from further delivery until pending messages are processed.** + +Code example to illustrate the implementation: + +```java + // If the hash is draining, do not send the message + if (drainingHashesTracker.shouldBlockStickyKeyHash(consumer, stickyKeyHash)) { + return false; + } +``` + +**3. A reference counter tracks pending messages for each hash in the "draining hashes" set.** + +Code example to illustrate the implementation: + +```java + // optimize the memory consumption of the map by using primitive int keys + private final Int2ObjectOpenHashMap drainingHashes = new Int2ObjectOpenHashMap<>(); + + public static class DrainingHashEntry { + private final Consumer consumer; + private int refCount; + private int blockedCount; + + DrainingHashEntry(Consumer consumer) { + this.consumer = consumer; + } + + public Consumer getConsumer() { + return consumer; + } + + void incrementRefCount() { + refCount++; + } + + boolean decrementRefCount() { + return --refCount == 0; + } + + void incrementBlockedCount() { + blockedCount++; + } + + boolean isBlocking() { + return blockedCount > 0; + } + } +``` + +The memory consumption estimate for tracking a hash is 52 bytes: +key: 16 bytes (object header) + 4 bytes (int) = 20 bytes +entry: 16 bytes (object header) + 8 bytes (long) + 4 bytes (int) + 4 bytes (int) = 32 bytes + +Although the estimate is 52 bytes per entry, calculations have been made with 80 bytes per entry to account for possible additional overheads such as memory alignment and the overhead of the Int2ObjectOpenHashMap. + +Memory usage estimate for each subscription after there have been consumer changes: + +- Worst case (all 64k hashes draining for a subscription): about 5MB +- Practical case (less than 1000 hashes draining): less than 80 kilobytes +- For 10,000 draining hashes: about 800 kB + +The memory usage of draining hashes tracking will go down to 0 after all hashes have "drained" and are no longer blocked. This memory usage isn't an overhead that applies at all times. + +The hash range size is reduced to 65535 (2^16-1) from the current 2^31-1 (Integer.MAX_VALUE) in ConsistentHashingStickyKeyConsumerSelector to reduce the worst-case memory consumption. Reducing the hash range size won't significantly impact the accuracy of distributing messages across connected consumers. The proof-of-concept implementation of PIP-379 includes the changes to reduce the hash range size. + +**4. As messages are acknowledged or consumers disconnect and therefore get removed from pending messages, the reference counter is decremented.** + +Individual acks are removed in Consumer's `removePendingAcks` method: + +```java + private boolean removePendingAcks(Consumer ackOwnedConsumer, Position position) { + PendingAcksMap ownedConsumerPendingAcks = ackOwnedConsumer.getPendingAcks(); + if (!ownedConsumerPendingAcks.remove(position.getLedgerId(), position.getEntryId())) { + // Message was already removed by the other consumer + return false; + } +``` + +When the `remove` method in `PendingAcksMap` is called, it will use the `PendingAcksMap.PendingAcksRemoveHandler` callback method `handleRemoving` provided by the dispatcher to trigger the removal also from the `DrainingHashesTracker`: + +```java + consumer.setPendingAcksRemoveHandler(new PendingAcksMap.PendingAcksRemoveHandler() { + @Override + public void handleRemoving(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash, + boolean closing) { + drainingHashesTracker.reduceRefCount(consumer, stickyKeyHash, closing); + } + +``` + +Also when a consumer disconnects, hashes of pending acks are removed. This happens in the `PersistentDispatcherMultipleConsumers`'s `removeConsumer` consumer method: + +```java + consumer.getPendingAcks().forEachAndClose((ledgerId, entryId, batchSize, stickyKeyHash) -> { + addMessageToReplay(ledgerId, entryId, stickyKeyHash); + }); +``` + +`PendingAcksMap`'s `forEachAndClose` method will trigger removals from `DrainingHashesTracker` using the `PendingAcksMap.PendingAcksRemoveHandler` callback method `handleRemoving` after processing each entry. This is how the `DrainingHashesTracker` stays in sync with the `PendingAcksMap` state without having the need to add all logic to `PendingAcksMap`. This is about following the "separation of concerns" design principle where each class handles a specific concern. + +**5. When the reference counter reaches zero, the hash is removed from the set, allowing new message delivery. The dispatcher is notified about this so that the delivery of the blocked messages can occur. Unblocked hashes are batched together to prevent a new notification for each call. This is handled with the `keySharedUnblockingIntervalMs` configuration setting.** + +In the implementation, this is handled in the DrainingHashesTracker's reduceRefCount method: + +```java + // code example is simplified for focus on the essential details + + public synchronized void reduceRefCount(Consumer consumer, int stickyHash) { + DrainingHashEntry entry = drainingHashes.get(stickyHash); + if (entry == null) { + return; + } + if (entry.decrementRefCount()) { + DrainingHashEntry removed = drainingHashes.remove(stickyHash); + if (removed.isBlocking()) { + unblockingHandler.stickyKeyHashUnblocked(stickyHash); + } + } + } +``` + +The `isBlocking()` method of `DrainingHashEntry` returns true when delivery was attempted for that hash, indicating a need to unblock it when it's removed. +The dispatcher is notified via the `unblockingHandler.stickyKeyHashUnblocked(stickyHash)` callback. The implementation simply schedules a read, batching all calls together, and then calls `readMoreEntries` in the dispatcher. + +```java + // code example is simplified for focus on the essential details + + private void stickyKeyHashUnblocked(int stickyKeyHash) { + reScheduleReadInMs(keySharedUnblockingIntervalMsSupplier.getAsLong()); + } + + protected void reScheduleReadInMs(long readAfterMs) { + if (isRescheduleReadInProgress.compareAndSet(false, true)) { + Runnable runnable = () -> { + isRescheduleReadInProgress.set(false); + readMoreEntries(); + }; + topic.getBrokerService().executor().schedule(runnable, readAfterMs, TimeUnit.MILLISECONDS); + } + } +``` + +**6. Consumer hash assignments may change multiple times, and a draining hash might be reassigned to the original consumer.** + +The draining hash data structure contains information about the draining consumer. When a message is attempted for delivery, the system can check if the target consumer is the same as the draining consumer. If they match, there's no need to block the hash. The implementation should also remove such hashes from the draining hashes set. This "lazy" approach reduces the need for actively scanning all draining hashes whenever hash assignments change. + +This is handled in the `DrainingHashesTracker` + +```java + public synchronized boolean shouldBlockStickyKeyHash(Consumer consumer, int stickyKeyHash) { + DrainingHashEntry entry = drainingHashes.get(stickyKeyHash); + // if the entry is not found, the hash is not draining. Don't block the hash. + if (entry == null) { + return false; + } + // hash has been reassigned to the original consumer, remove the entry + // and don't block the hash + if (entry.getConsumer() == consumer) { + drainingHashes.remove(stickyKeyHash, entry); + return false; + } + // increment the blocked count which is used to determine if the hash is blocking + // dispatching to other consumers + entry.incrementBlockedCount(); + // block the hash + return true; + } +``` + +**7. When sending out messages, there are potential race conditions that could allow the delivery of a message that should be blocked.** + +This could happen when a consumer is added while reading and sending messages are already in progress. In PIP-379, the sending process has been modified to perform a check when adding the message to the pending acknowledgments map. There are also additional locks in the pending acks handling which prevent race conditions. + +`addPendingAckIfAllowed` method in `PendingAcksMap` class: + +```java + public boolean addPendingAckIfAllowed(long ledgerId, long entryId, int batchSize, int stickyKeyHash) { + try { + writeLock.lock(); + // prevent adding sticky hash to pending acks if the PendingAcksMap has already been closed + // and there's a race condition between closing the consumer and sending new messages + if (closed) { + return false; + } + // prevent adding sticky hash to pending acks if it's already in draining hashes + // to avoid any race conditions that would break consistency + PendingAcksAddHandler pendingAcksAddHandler = pendingAcksAddHandlerSupplier.get(); + if (pendingAcksAddHandler != null + && !pendingAcksAddHandler.handleAdding(consumer, ledgerId, entryId, stickyKeyHash)) { + return false; + } + Long2ObjectSortedMap ledgerPendingAcks = + pendingAcks.computeIfAbsent(ledgerId, k -> new Long2ObjectRBTreeMap<>()); + ledgerPendingAcks.put(entryId, IntIntPair.of(batchSize, stickyKeyHash)); + return true; + } finally { + writeLock.unlock(); + } + } +``` + +This `addPendingAckIfAllowed` method is called from Consumer's `sendMessages` method: + +```java + boolean sendingAllowed = + pendingAcks.addPendingAckIfAllowed(entry.getLedgerId(), entry.getEntryId(), batchSize, stickyKeyHash); + if (!sendingAllowed) { + // sending isn't allowed when pending acks doesn't accept adding the entry + // this happens when Key_Shared draining hashes contains the stickyKeyHash + // because of race conditions, it might be resolved at the time of sending + totalEntries--; + entries.set(i, null); + entry.release(); + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Skipping sending of {}:{} ledger entry with batchSize of {} since adding" + + " to pending acks failed in broker.service.Consumer for consumerId: {}", + topicName, subscription, entry.getLedgerId(), entry.getEntryId(), batchSize, + consumerId); + } +``` + +If sending isn't allowed, the entry will be skipped from delivery. The `PendingAcksAddHandler` callback will add the message to redelivery if this is the case. +The callback maps to `handleAddingPendingAck` in the dispatcher (`PersistentStickyKeyDispatcherMultipleConsumers`). + +```java + private boolean handleAddingPendingAck(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash) { + DrainingHashesTracker.DrainingHashEntry drainingHashEntry = drainingHashesTracker.getEntry(stickyKeyHash); + if (drainingHashEntry != null && drainingHashEntry.getConsumer() != consumer) { + log.warn("[{}] Another consumer id {} is already draining hash {}. Skipping adding {}:{} to pending acks " + + "for consumer {}. Adding the message to replay.", + getName(), drainingHashEntry.getConsumer(), stickyKeyHash, ledgerId, entryId, consumer); + addMessageToReplay(ledgerId, entryId, stickyKeyHash); + // block message from sending + return false; + } + if (recentReadTypeInSending == ReadType.Normal && redeliveryMessages.containsStickyKeyHash(stickyKeyHash)) { + log.warn("[{}] Sticky hash {} is already in the replay queue. " + + "Skipping adding {}:{} to pending acks. Adding the message to replay.", + getName(), stickyKeyHash, ledgerId, entryId); + addMessageToReplay(ledgerId, entryId, stickyKeyHash); + // block message from sending + return false; + } + // allow adding the message to pending acks and sending the message to the consumer + return true; + } +``` + +This logic will prevent any inconsistency when consumers get added or removed and hash ranges change while the sending of messages is already in progress. It will ensure that the view on pending acknowledgments is consistent so that the tracking of draining hashes will also be consistent in all cases. In addition, this logic will block hashes of messages that have recently been added to the redelivery queue and therefore, for message ordering reasons, should get delivered before any further message delivery happens. + +**Summary** + +This high-level design approach will meet the updated contract of preserving ordering: _"In Key_Shared subscriptions, messages with the same key are delivered and allowed to be in an unacknowledged state to only one consumer at a time."_ + +It also minimizes the impact on performance and memory usage. **The tracking only comes into play during transition states.** When consumers have been connected for a longer duration and all draining hashes have been removed, there won't be a need to check any special rules or maintain any extra state. **When the draining hashes are empty, lookups will essentially be no-ops and won't consume CPU or memory resources.** + +## Public-facing Changes + +### Topic Stats Changes & Observability + +Topic stats for the removed PIP-282 "recently joined consumers"/"last sent position" solution are removed: +- `lastSentPositionWhenJoining` field for each consumer +- `consumersAfterMarkDeletePosition` field for each Key_Shared subscription +- `individuallySentPositions` field for each Key_Shared subscription + +New topic stats will be added to monitor the "draining hashes" feature at the subscription level and consumer level: +1. `draining_hashes_count`: The current number of hashes in the draining state. +2. `draining_hashes_pending_messages`: The total number of pending messages for all draining hashes. +3. `draining_hashes_cleared_total`: The total number of hashes cleared from the draining state. +4. `draining_hashes`: Details at the hash level (available at the consumer level to reduce redundancy of information) + - hash + - number of pending messages + +For improved observability, a separate REST API for listing all pending messages ("pending acks") for a consumer will be considered. This API would allow querying which messages are currently part of a draining hash, providing a way to identify specific message IDs of messages that are holding onto a specific hash and blocking delivery to another consumer. + +## Backward & Forward Compatibility + +The "draining hashes" feature doesn't introduce backward or forward compatibility issues. The state is handled at runtime, and the changes are on the broker side without changes to the client protocol. + +Slightly unrelated to PIP-379 changes, there's a need to ensure that users upgrading from Pulsar 3.x can revert to the "recently joined consumers" logic (before PIP-282) in case of possible regressions caused by PIP-379. Since PIP-282 is also new in Pulsar 4.0.0, there needs to be a feature flag that toggles between the PIP-379 implementation for Key_Shared and the "recently joined consumers" logic before PIP-282. Implemention details for this feature toggle can be handled in the pull request for implementing this. + +## Links + +- Mailing List discussion thread: https://lists.apache.org/thread/l5zjq0fb2dscys3rsn6kfl7505tbndlx +- Mailing List voting thread: https://lists.apache.org/thread/z1kgo34qfkkvdnn3l007bdvjr3qqf4rw +- PIP-379 implementation PR: https://github.com/apache/pulsar/pull/23352 + +- [PIP-282: Change definition of the recently joined consumers position](https://github.com/apache/pulsar/blob/master/pip/pip-282.md) +- [Pulsar issue #23307: Message ordering isn't retained in Key_Shared AUTO_SPLIT mode in a rolling restart type of test scenario](https://github.com/apache/pulsar/issues/23307) +- [Pulsar issue #21199: Key_Shared subscription gets stuck after consumer reconnects](https://github.com/apache/pulsar/issues/21199) \ No newline at end of file From 6bca70b1380fd97a0a1ea23bdc0bf918bd6defee Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 4 Oct 2024 16:09:48 -0700 Subject: [PATCH 557/580] [fix][client] Add more info while throwing reader creation exception (#23401) --- .../java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java index ef230475be53b..d0ab90068ed31 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java @@ -92,7 +92,8 @@ public CompletableFuture> createAsync() { return FutureUtil .failedFuture(new IllegalArgumentException( "Start message id or start message from roll back must be specified but they cannot be" - + " specified at the same time")); + + " specified at the same time. MessageId =" + conf.getStartMessageId() + + ", rollback seconds =" + conf.getStartMessageFromRollbackDurationInSec())); } if (conf.getStartMessageFromRollbackDurationInSec() > 0) { From b9ededc30e8f7d73e1226d22d327e10968df42d4 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 4 Oct 2024 16:10:00 -0700 Subject: [PATCH 558/580] [fix][broker] Fix delivery-test with unnecessary invocation-count (#23399) --- .../org/apache/pulsar/client/impl/MessageRedeliveryTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java index e2895b1d01e9f..7269df3b6b8b2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageRedeliveryTest.java @@ -483,7 +483,7 @@ public void testMultiConsumerRedeliveryAddEpoch(boolean enableBatch) throws Exce assertNull(message); } - @Test(dataProvider = "enableBatch", invocationCount = 10) + @Test(dataProvider = "enableBatch") public void testMultiConsumerBatchRedeliveryAddEpoch(boolean enableBatch) throws Exception{ final String topic = "testMultiConsumerBatchRedeliveryAddEpoch"; From 6c7ec4c38c8dce8351096a494b412952be5cc77b Mon Sep 17 00:00:00 2001 From: vineeth1995 Date: Sun, 6 Oct 2024 22:58:25 -0700 Subject: [PATCH 559/580] [feat] [broker] Add broker health check status into prometheus metrics (#20147) --- conf/broker.conf | 2 + .../pulsar/broker/ServiceConfiguration.java | 6 +++ .../pulsar/broker/admin/impl/BrokersBase.java | 50 ++++++++++++------- .../pulsar/broker/service/BrokerService.java | 23 +++++++++ .../stats/BrokerOperabilityMetrics.java | 18 ++++++- .../service/PersistentTopicE2ETest.java | 32 ++++++++++++ .../broker/stats/PrometheusMetricsTest.java | 17 ++++++- 7 files changed, 128 insertions(+), 20 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 617e202e5ec65..e745fcb2b0a8f 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1689,6 +1689,8 @@ exposePublisherStats=true statsUpdateFrequencyInSecs=60 statsUpdateInitialDelayInSecs=60 +healthCheckMetricsUpdateTimeInSeconds=-1 + # Enable expose the precise backlog stats. # Set false to use published counter and consumed counter to calculate, this would be more efficient but may be inaccurate. # Default is false. 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 58d6444e7196a..81073b1731b24 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 @@ -3280,6 +3280,12 @@ public double getLoadBalancerBandwidthOutResourceWeight() { doc = "Stats update initial delay in seconds" ) private int statsUpdateInitialDelayInSecs = 60; + @FieldContext( + category = CATEGORY_METRICS, + minValue = -1, + doc = "HealthCheck update frequency in seconds. Disable health check with value -1 (Default value -1)" + ) + private int healthCheckMetricsUpdateTimeInSeconds = -1; @FieldContext( category = CATEGORY_METRICS, doc = "If true, aggregate publisher stats of PartitionedTopicStats by producerName" diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index e13cb1858f79d..da4cee7b4651c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -51,6 +51,7 @@ import org.apache.commons.lang.StringUtils; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.PulsarServerException; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.PulsarService.State; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.admin.AdminResource; @@ -422,26 +423,35 @@ public static String getHeartbeatTopicName(String brokerId, ServiceConfiguration } private CompletableFuture internalRunHealthCheck(TopicVersion topicVersion) { - String brokerId = pulsar().getBrokerId(); + return internalRunHealthCheck(topicVersion, pulsar(), clientAppId()); + } + + + public static CompletableFuture internalRunHealthCheck(TopicVersion topicVersion, PulsarService pulsar, + String clientAppId) { + NamespaceName namespaceName = (topicVersion == TopicVersion.V2) + ? NamespaceService.getHeartbeatNamespaceV2(pulsar.getAdvertisedAddress(), pulsar.getConfiguration()) + : NamespaceService.getHeartbeatNamespace(pulsar.getAdvertisedAddress(), pulsar.getConfiguration()); + String brokerId = pulsar.getBrokerId(); final String topicName = - getHeartbeatTopicName(brokerId, pulsar().getConfiguration(), (topicVersion == TopicVersion.V2)); - LOG.info("[{}] Running healthCheck with topic={}", clientAppId(), topicName); + getHeartbeatTopicName(brokerId, pulsar.getConfiguration(), (topicVersion == TopicVersion.V2)); + LOG.info("[{}] Running healthCheck with topic={}", clientAppId, topicName); final String messageStr = UUID.randomUUID().toString(); final String subscriptionName = "healthCheck-" + messageStr; // create non-partitioned topic manually and close the previous reader if present. - return pulsar().getBrokerService().getTopic(topicName, true) + return pulsar.getBrokerService().getTopic(topicName, true) .thenCompose(topicOptional -> { if (!topicOptional.isPresent()) { LOG.error("[{}] Fail to run health check while get topic {}. because get null value.", - clientAppId(), topicName); + clientAppId, topicName); throw new RestException(Status.NOT_FOUND, String.format("Topic [%s] not found after create.", topicName)); } PulsarClient client; try { - client = pulsar().getClient(); + client = pulsar.getClient(); } catch (PulsarServerException e) { - LOG.error("[{}] Fail to run health check while get client.", clientAppId()); + LOG.error("[{}] Fail to run health check while get client.", clientAppId); throw new RestException(e); } CompletableFuture resultFuture = new CompletableFuture<>(); @@ -451,17 +461,18 @@ private CompletableFuture internalRunHealthCheck(TopicVersion topicVersion .startMessageId(MessageId.latest) .createAsync().exceptionally(createException -> { producer.closeAsync().exceptionally(ex -> { - LOG.error("[{}] Close producer fail while heath check.", clientAppId()); + LOG.error("[{}] Close producer fail while heath check.", clientAppId); return null; }); throw FutureUtil.wrapToCompletionException(createException); }).thenCompose(reader -> producer.sendAsync(messageStr) .thenCompose(__ -> FutureUtil.addTimeoutHandling( healthCheckRecursiveReadNext(reader, messageStr), - HEALTH_CHECK_READ_TIMEOUT, pulsar().getBrokerService().executor(), + HEALTH_CHECK_READ_TIMEOUT, pulsar.getBrokerService().executor(), () -> HEALTH_CHECK_TIMEOUT_EXCEPTION)) .whenComplete((__, ex) -> { - closeAndReCheck(producer, reader, topicOptional.get(), subscriptionName) + closeAndReCheck(producer, reader, topicOptional.get(), subscriptionName, + clientAppId) .whenComplete((unused, innerEx) -> { if (ex != null) { resultFuture.completeExceptionally(ex); @@ -479,6 +490,11 @@ HEALTH_CHECK_READ_TIMEOUT, pulsar().getBrokerService().executor(), }); } + private CompletableFuture closeAndReCheck(Producer producer, Reader reader, + Topic topic, String subscriptionName) { + return closeAndReCheck(producer, reader, topic, subscriptionName, clientAppId()); + } + /** * Close producer and reader and then to re-check if this operation is success. * @@ -491,8 +507,8 @@ HEALTH_CHECK_READ_TIMEOUT, pulsar().getBrokerService().executor(), * @param topic Topic * @param subscriptionName Subscription name */ - private CompletableFuture closeAndReCheck(Producer producer, Reader reader, - Topic topic, String subscriptionName) { + private static CompletableFuture closeAndReCheck(Producer producer, Reader reader, + Topic topic, String subscriptionName, String clientAppId) { // no matter exception or success, we still need to // close producer/reader CompletableFuture producerFuture = producer.closeAsync(); @@ -503,7 +519,7 @@ private CompletableFuture closeAndReCheck(Producer producer, Reade return FutureUtil.waitForAll(Collections.unmodifiableList(futures)) .exceptionally(closeException -> { if (readerFuture.isCompletedExceptionally()) { - LOG.error("[{}] Close reader fail while heath check.", clientAppId()); + LOG.error("[{}] Close reader fail while heath check.", clientAppId); Subscription subscription = topic.getSubscription(subscriptionName); // re-check subscription after reader close @@ -511,24 +527,24 @@ private CompletableFuture closeAndReCheck(Producer producer, Reade LOG.warn("[{}] Force delete subscription {} " + "when it still exists after the" + " reader is closed.", - clientAppId(), subscription); + clientAppId, subscription); subscription.deleteForcefully() .exceptionally(ex -> { LOG.error("[{}] Force delete subscription fail" + " while health check", - clientAppId(), ex); + clientAppId, ex); return null; }); } } else { // producer future fail. - LOG.error("[{}] Close producer fail while heath check.", clientAppId()); + LOG.error("[{}] Close producer fail while heath check.", clientAppId); } return null; }); } - private CompletableFuture healthCheckRecursiveReadNext(Reader reader, String content) { + private static CompletableFuture healthCheckRecursiveReadNext(Reader reader, String content) { return reader.readNextAsync() .thenCompose(msg -> { if (!Objects.equals(content, msg.getValue())) { 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 dd722dffcfbfc..c240c758dcda6 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 @@ -24,6 +24,7 @@ import static org.apache.bookkeeper.mledger.ManagedLedgerConfig.PROPERTY_SOURCE_TOPIC_KEY; import static org.apache.commons.collections4.CollectionUtils.isEmpty; import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.pulsar.broker.admin.impl.BrokersBase.internalRunHealthCheck; import static org.apache.pulsar.client.util.RetryMessageUtil.DLQ_GROUP_TOPIC_SUFFIX; import static org.apache.pulsar.client.util.RetryMessageUtil.RETRY_GROUP_TOPIC_SUFFIX; import static org.apache.pulsar.common.naming.SystemTopicNames.isTransactionInternalName; @@ -157,6 +158,7 @@ import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.naming.TopicVersion; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AutoSubscriptionCreationOverride; import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; @@ -241,6 +243,7 @@ public class BrokerService implements Closeable { private AuthorizationService authorizationService; private final ScheduledExecutorService statsUpdater; + @Getter private final ScheduledExecutorService backlogQuotaChecker; @@ -346,6 +349,7 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws this.acceptorGroup = EventLoopUtil.newEventLoopGroup( pulsar.getConfiguration().getNumAcceptorThreads(), false, acceptorThreadFactory); this.workerGroup = eventLoopGroup; + this.statsUpdater = OrderedScheduler.newSchedulerBuilder() .name("pulsar-stats-updater") .numThreads(1) @@ -611,6 +615,7 @@ public void start() throws Exception { this.startStatsUpdater( serviceConfig.getStatsUpdateInitialDelayInSecs(), serviceConfig.getStatsUpdateFrequencyInSecs()); + this.initializeHealthChecker(); this.startInactivityMonitor(); this.startMessageExpiryMonitor(); this.startCompactionMonitor(); @@ -640,6 +645,24 @@ protected void startStatsUpdater(int statsUpdateInitialDelayInSecs, int statsUpd updateRates(); } + protected void initializeHealthChecker() { + ServiceConfiguration config = pulsar().getConfiguration(); + if (config.getHealthCheckMetricsUpdateTimeInSeconds() > 0) { + int interval = config.getHealthCheckMetricsUpdateTimeInSeconds(); + statsUpdater.scheduleAtFixedRate(this::checkHealth, + interval, interval, TimeUnit.SECONDS); + } + } + + public CompletableFuture checkHealth() { + return internalRunHealthCheck(TopicVersion.V2, pulsar(), null).thenAccept(__ -> { + this.pulsarStats.getBrokerOperabilityMetrics().recordHealthCheckStatusSuccess(); + }).exceptionally(ex -> { + this.pulsarStats.getBrokerOperabilityMetrics().recordHealthCheckStatusFail(); + return null; + }); + } + protected void startDeduplicationSnapshotMonitor() { // We do not know whether users will enable deduplication on namespace level/topic level or not, so keep this // scheduled task runs. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BrokerOperabilityMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BrokerOperabilityMetrics.java index 3f991be8184ab..1855e1798b465 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BrokerOperabilityMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/BrokerOperabilityMetrics.java @@ -42,6 +42,7 @@ public class BrokerOperabilityMetrics implements AutoCloseable { private final LongAdder connectionTotalCreatedCount; private final LongAdder connectionTotalClosedCount; private final LongAdder connectionActive; + private volatile int healthCheckStatus; // 1=success, 0=failure, -1=unknown private final LongAdder connectionCreateSuccessCount; private final LongAdder connectionCreateFailCount; @@ -61,7 +62,7 @@ public BrokerOperabilityMetrics(PulsarService pulsar) { this.connectionTotalCreatedCount = new LongAdder(); this.connectionTotalClosedCount = new LongAdder(); this.connectionActive = new LongAdder(); - + this.healthCheckStatus = -1; this.connectionCreateSuccessCount = new LongAdder(); this.connectionCreateFailCount = new LongAdder(); @@ -103,6 +104,7 @@ private void generate() { reset(); metricsList.add(getTopicLoadMetrics()); metricsList.add(getConnectionMetrics()); + metricsList.add(getHealthMetrics()); } public Metrics generateConnectionMetrics() { @@ -119,6 +121,12 @@ Metrics getConnectionMetrics() { return rMetrics; } + Metrics getHealthMetrics() { + Metrics rMetrics = Metrics.create(getDimensionMap("broker_health")); + rMetrics.put("brk_health", healthCheckStatus); + return rMetrics; + } + Map getDimensionMap(String metricsName) { Map dimensionMap = new HashMap<>(); dimensionMap.put("broker", brokerName); @@ -179,4 +187,12 @@ public void recordConnectionCreateSuccess() { public void recordConnectionCreateFail() { this.connectionCreateFailCount.increment(); } + + public void recordHealthCheckStatusSuccess() { + this.healthCheckStatus = 1; + } + + public void recordHealthCheckStatusFail() { + this.healthCheckStatus = 0; + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java index 640cd2d37e399..36e741f8fa9cd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java @@ -1607,6 +1607,38 @@ public void testBrokerConnectionStats() throws Exception { assertEquals((long) map.get("brk_connection_create_fail_count"), 1); } + /** + * There is detailed info about this test. + * see: https://github.com/apache/pulsar/issues/10150#issuecomment-1112380074 + */ + @Test + public void testBrokerHealthCheckStatus() throws Exception { + + cleanup(); + conf.setSystemTopicEnabled(false); + conf.setTopicLevelPoliciesEnabled(false); + conf.setHealthCheckMetricsUpdateTimeInSeconds(60); + setup(); + BrokerService brokerService = this.pulsar.getBrokerService(); + + Map map = null; + + brokerService.checkHealth().get(); + brokerService.updateRates(); + Awaitility.await().until(() -> this.activeCount.get() == 1); + List metrics = brokerService.getTopicMetrics(); + System.out.println(metrics); + + for (int i = 0; i < metrics.size(); i++) { + if (metrics.get(i).getDimensions().containsValue("broker_health")) { + map = metrics.get(i).getMetrics(); + break; + } + } + assertNotNull(map); + assertEquals(map.get("brk_health"), 1); + } + @Test public void testPayloadCorruptionDetection() throws Exception { final String topicName = "persistent://prop/ns-abc/topic1"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index a92f5a4acc208..fa073d3694b26 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -79,6 +79,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.manager.UnloadManager; import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; import org.apache.pulsar.broker.service.AbstractTopic; +import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentMessageExpiryMonitor; @@ -1789,6 +1790,20 @@ public void testBrokerConnection() throws Exception { compareBrokerConnectionStateCount(cm, 2.0); } + @Test + public void testBrokerHealthCheckMetric() throws Exception { + conf.setHealthCheckMetricsUpdateTimeInSeconds(60); + BrokerService brokerService = pulsar.getBrokerService(); + brokerService.checkHealth().get(); + brokerService.updateRates(); + ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); + String metricsStr = statsOut.toString(); + Multimap metrics = parseMetrics(metricsStr); + List cm = (List) metrics.get("pulsar_health"); + compareBrokerConnectionStateCount(cm, 1); + } + private void compareBrokerConnectionStateCount(List cm, double count) { assertEquals(cm.size(), 1); assertEquals(cm.get(0).tags.get("cluster"), "test"); @@ -1894,7 +1909,6 @@ public void testMetricsWithCache() throws Throwable { PrometheusMetricsGenerator prometheusMetricsGenerator = new PrometheusMetricsGenerator(pulsar, true, false, false, false, clock); - String previousMetrics = null; for (int a = 0; a < 4; a++) { ByteArrayOutputStream statsOut1 = new ByteArrayOutputStream(); @@ -1908,7 +1922,6 @@ public void testMetricsWithCache() throws Throwable { assertEquals(metricsStr1, metricsStr2); assertNotEquals(metricsStr1, previousMetrics); previousMetrics = metricsStr1; - // move time forward currentTimeMillis.addAndGet(TimeUnit.SECONDS.toMillis(2)); } From 806fdf86866813edbc8f6dea688df823c2889cc5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 7 Oct 2024 15:44:47 +0300 Subject: [PATCH 560/580] [improve][misc] Upgrade Jetty to 9.4.56.v20240826 (#23405) --- .../server/src/assemble/LICENSE.bin.txt | 38 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 16 ++++---- pom.xml | 2 +- 3 files changed, 28 insertions(+), 28 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 8c6e2cfa7159a..24eb6b8066df1 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -392,25 +392,25 @@ The Apache Software License, Version 2.0 - org.asynchttpclient-async-http-client-2.12.1.jar - org.asynchttpclient-async-http-client-netty-utils-2.12.1.jar * Jetty - - org.eclipse.jetty-jetty-client-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-continuation-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-http-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-io-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-proxy-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-security-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-server-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-servlet-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-servlets-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-util-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-util-ajax-9.4.54.v20240208.jar - - org.eclipse.jetty.websocket-javax-websocket-client-impl-9.4.54.v20240208.jar - - org.eclipse.jetty.websocket-websocket-api-9.4.54.v20240208.jar - - org.eclipse.jetty.websocket-websocket-client-9.4.54.v20240208.jar - - org.eclipse.jetty.websocket-websocket-common-9.4.54.v20240208.jar - - org.eclipse.jetty.websocket-websocket-server-9.4.54.v20240208.jar - - org.eclipse.jetty.websocket-websocket-servlet-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-alpn-conscrypt-server-9.4.54.v20240208.jar - - org.eclipse.jetty-jetty-alpn-server-9.4.54.v20240208.jar + - org.eclipse.jetty-jetty-client-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-continuation-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-http-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-io-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-proxy-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-security-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-server-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-servlet-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-servlets-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-util-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-util-ajax-9.4.56.v20240826.jar + - org.eclipse.jetty.websocket-javax-websocket-client-impl-9.4.56.v20240826.jar + - org.eclipse.jetty.websocket-websocket-api-9.4.56.v20240826.jar + - org.eclipse.jetty.websocket-websocket-client-9.4.56.v20240826.jar + - org.eclipse.jetty.websocket-websocket-common-9.4.56.v20240826.jar + - org.eclipse.jetty.websocket-websocket-server-9.4.56.v20240826.jar + - org.eclipse.jetty.websocket-websocket-servlet-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-alpn-conscrypt-server-9.4.56.v20240826.jar + - org.eclipse.jetty-jetty-alpn-server-9.4.56.v20240826.jar * SnakeYaml -- org.yaml-snakeyaml-2.0.jar * RocksDB - org.rocksdb-rocksdbjni-7.9.2.jar * Google Error Prone Annotations - com.google.errorprone-error_prone_annotations-2.24.0.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 6e0bacb2e8845..15b2a918b9ebc 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -402,14 +402,14 @@ The Apache Software License, Version 2.0 - async-http-client-2.12.1.jar - async-http-client-netty-utils-2.12.1.jar * Jetty - - jetty-client-9.4.54.v20240208.jar - - jetty-http-9.4.54.v20240208.jar - - jetty-io-9.4.54.v20240208.jar - - jetty-util-9.4.54.v20240208.jar - - javax-websocket-client-impl-9.4.54.v20240208.jar - - websocket-api-9.4.54.v20240208.jar - - websocket-client-9.4.54.v20240208.jar - - websocket-common-9.4.54.v20240208.jar + - jetty-client-9.4.56.v20240826.jar + - jetty-http-9.4.56.v20240826.jar + - jetty-io-9.4.56.v20240826.jar + - jetty-util-9.4.56.v20240826.jar + - javax-websocket-client-impl-9.4.56.v20240826.jar + - websocket-api-9.4.56.v20240826.jar + - websocket-client-9.4.56.v20240826.jar + - websocket-common-9.4.56.v20240826.jar * SnakeYaml -- snakeyaml-2.0.jar * Google Error Prone Annotations - error_prone_annotations-2.24.0.jar * Javassist -- javassist-3.25.0-GA.jar diff --git a/pom.xml b/pom.xml index c50357b840616..e0bce0442e158 100644 --- a/pom.xml +++ b/pom.xml @@ -148,7 +148,7 @@ flexible messaging model and an intuitive client API. 5.1.0 4.1.113.Final 0.0.24.Final - 9.4.54.v20240208 + 9.4.56.v20240826 2.5.2 2.42 1.10.50 From 5451921cd49dca03c541617c92ee8a3c83af9e50 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 7 Oct 2024 18:37:55 +0300 Subject: [PATCH 561/580] [improve] PIP-384: ManagedLedger interface decoupling (#23363) --- pip/pip-384.md | 158 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 158 insertions(+) create mode 100644 pip/pip-384.md diff --git a/pip/pip-384.md b/pip/pip-384.md new file mode 100644 index 0000000000000..ba02a147d857c --- /dev/null +++ b/pip/pip-384.md @@ -0,0 +1,158 @@ +# PIP-384: ManagedLedger interface decoupling + +## Background knowledge + +Apache Pulsar uses a component called ManagedLedger to handle persistent storage of messages. + +The ManagedLedger interfaces and implementation were initially tightly coupled, making it difficult to introduce alternative implementations or improve the architecture. +This PIP documents changes that have been made in the master branch for Pulsar 4.0. Pull Requests [#22891](https://github.com/apache/pulsar/pull/22891) and [#23311](https://github.com/apache/pulsar/pull/23311) have already been merged. +This work happened after lazy consensus on the dev mailing list based on the discussion thread ["Preparing for Pulsar 4.0: cleaning up the Managed Ledger interfaces"](https://lists.apache.org/thread/l5zjq0fb2dscys3rsn6kfl7505tbndlx). +There is one remaining PR [#23313](https://github.com/apache/pulsar/pull/23313) at the time of writing this document. +The goal of this PIP is to document the changes in this area for later reference. + +Key concepts: + +- **ManagedLedger**: A component that handles the persistent storage of messages in Pulsar. +- **BookKeeper**: The default storage system used by ManagedLedger. +- **ManagedLedgerStorage interface**: A factory for configuring and creating the `ManagedLedgerFactory` instance. [ManagedLedgerStorage.java source code](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java) +- **ManagedLedgerFactory interface**: Creates and manages ManagedLedger instances. [ManagedLedgerFactory.java source code](https://github.com/apache/pulsar/blob/master/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java) +- **ManagedLedger interface**: Handles the persistent storage of messages in Pulsar. [ManagedLedger.java source code](https://github.com/apache/pulsar/blob/master/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java) +- **ManagedCursor interface**: Handles the persistent storage of Pulsar subscriptions and related message acknowledgements. [ManagedCursor.java source code](https://github.com/apache/pulsar/blob/master/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java) + +## Motivation + +The current ManagedLedger implementation faces several challenges: + +1. **Tight coupling**: The interfaces are tightly coupled with their implementation, making it difficult to introduce alternative implementations. + +2. **Limited flexibility**: The current architecture doesn't allow for easy integration of different storage systems or optimizations. + +3. **Dependency on BookKeeper**: The ManagedLedger implementation is closely tied to BookKeeper, limiting options for alternative storage solutions. + +4. **Complexity**: The tight coupling increases the overall complexity of the system, making it harder to maintain, test and evolve. + +5. **Limited extensibility**: Introducing new features or optimizations often requires changes to both interfaces and implementations. + +## Goals + +### In Scope + +- Decouple ManagedLedger interfaces from their current implementation. +- Introduce a ReadOnlyManagedLedger interface. +- Decouple OpAddEntry and LedgerHandle from ManagedLedgerInterceptor. +- Enable support for multiple ManagedLedgerFactory instances. +- Decouple BookKeeper client from ManagedLedgerStorage. +- Improve overall architecture by reducing coupling between core Pulsar components and specific ManagedLedger implementations. +- Prepare the groundwork for alternative ManagedLedger implementations in Pulsar 4.0. + +### Out of Scope + +- Implementing alternative ManagedLedger storage backends. +- Changes to external APIs or behaviors. +- Comprehensive JavaDocs for the interfaces. + +## High Level Design + +1. **Decouple interfaces from implementations**: + - Move required methods from implementation classes to their respective interfaces. + - Update code to use interfaces instead of concrete implementations. + +2. **Introduce ReadOnlyManagedLedger interface**: + - Extract this interface to decouple from ReadOnlyManagedLedgerImpl. + - Adjust code to use the new interface where appropriate. + +3. **Decouple ManagedLedgerInterceptor**: + - Introduce AddEntryOperation and LastEntryHandle interfaces. + - Adjust ManagedLedgerInterceptor to use these new interfaces. + +4. **Enable multiple ManagedLedgerFactory instances**: + - Modify ManagedLedgerStorage interface to support multiple "storage classes". + - Implement BookkeeperManagedLedgerStorageClass for BookKeeper support. + - Update PulsarService and related classes to support multiple ManagedLedgerFactory instances. + - Add "storage class" to persistence policy part of the namespace level or topic level policies. + +5. **Decouple BookKeeper client**: + - Move BookKeeper client creation and management to BookkeeperManagedLedgerStorageClass. + - Update ManagedLedgerStorage interface to remove direct BookKeeper dependencies. + +## Detailed Design + +### Interface Decoupling + +1. Update ManagedLedger interface: + - Add methods from ManagedLedgerImpl to the interface. + - Remove dependencies on implementation-specific classes. + +2. Update ManagedLedgerFactory interface: + - Add necessary methods from ManagedLedgerFactoryImpl. + - Remove dependencies on implementation-specific classes. + +3. Update ManagedCursor interface: + - Add required methods from ManagedCursorImpl. + - Remove dependencies on implementation-specific classes. + +4. Introduce ReadOnlyManagedLedger interface: + - Extract methods specific to read-only operations. + - Update relevant code to use this interface where appropriate. + +5. Decouple ManagedLedgerInterceptor: + - Introduce AddEntryOperation interface for beforeAddEntry method. + - Introduce LastEntryHandle interface for onManagedLedgerLastLedgerInitialize method. + - Update ManagedLedgerInterceptor to use these new interfaces. + +### Multiple ManagedLedgerFactory Instances + +1. Update ManagedLedgerStorage interface: + - Add methods to support multiple storage classes. + - Introduce getManagedLedgerStorageClass method to retrieve specific storage implementations. + +2. Implement BookkeeperManagedLedgerStorageClass: + - Create a new class implementing ManagedLedgerStorageClass for BookKeeper. + - Move BookKeeper client creation and management to this class. + +3. Update PulsarService and related classes: + - Modify to support creation and management of multiple ManagedLedgerFactory instances. + - Update configuration to allow specifying different storage classes for different namespaces or topics. + +### BookKeeper Client Decoupling + +1. Update ManagedLedgerStorage interface: + - Remove direct dependencies on BookKeeper client. + - Introduce methods to interact with storage without exposing BookKeeper specifics. + +2. Implement BookkeeperManagedLedgerStorageClass: + - Encapsulate BookKeeper client creation and management. + - Implement storage operations using BookKeeper client. + +3. Update relevant code: + - Replace direct BookKeeper client usage with calls to ManagedLedgerStorage methods. + - Update configuration handling to support BookKeeper-specific settings through the new storage class. + +## Public-facing Changes + +### Configuration + +- Add new configuration option to specify default ManagedLedger "storage class" at broker level. + +### API Changes + +- No major changes to external APIs are planned. +- The only API change is to add `managedLedgerStorageClassName` to `PersistencePolicies` which can be used by a custom `ManagedLedgerStorage` to control the ManagedLedgerFactory instance that is used for a particular namespace or topic. + +## Backward & Forward Compatibility + +The changes are internal and don't affect external APIs or behaviors. +Backward compatibility is fully preserved in Apache Pulsar. + +## Security Considerations + +The decoupling of interfaces and implementation doesn't introduce new security concerns. + +## Links + +- Initial mailing List discussion thread: [Preparing for Pulsar 4.0: cleaning up the Managed Ledger interfaces](https://lists.apache.org/thread/l5zjq0fb2dscys3rsn6kfl7505tbndlx) + - Merged Pull Request #22891: [Replace dependencies on PositionImpl with Position interface](https://github.com/apache/pulsar/pull/22891) + - Merged Pull Request #23311: [Decouple ManagedLedger interfaces from the current implementation](https://github.com/apache/pulsar/pull/23311) + - Implementation Pull Request #23313: [Decouple Bookkeeper client from ManagedLedgerStorage and enable multiple ManagedLedgerFactory instances](https://github.com/apache/pulsar/pull/23313) +- Mailing List PIP discussion thread: https://lists.apache.org/thread/rtnktrj7tp5ppog0235t2mf9sxrdpfr8 +- Mailing List PIP voting thread: https://lists.apache.org/thread/4jj5dmk6jtpq05lcd6dxlkqpn7hov5gv \ No newline at end of file From 731ec8364f050e3db1532ec8316cf76109865e3d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 8 Oct 2024 04:48:09 +0300 Subject: [PATCH 562/580] [improve][broker][PIP-384] Decouple Bookkeeper client from ManagedLedgerStorage and enable multiple ManagedLedgerFactory instances (#23313) --- .../mledger/ManagedLedgerConfig.java | 4 +- .../broker/ManagedLedgerClientFactory.java | 50 ++++- .../apache/pulsar/broker/PulsarService.java | 28 ++- .../TransactionMetadataStoreService.java | 4 +- .../broker/admin/impl/NamespacesBase.java | 2 +- .../admin/impl/PersistentTopicsBase.java | 36 ++-- .../pulsar/broker/service/BrokerService.java | 87 ++++++--- .../pulsar/broker/service/ServerCnx.java | 95 ++++++---- .../service/persistent/PersistentTopic.java | 44 +++-- .../broker/stats/metrics/AbstractMetrics.java | 4 +- .../stats/metrics/ManagedLedgerMetrics.java | 3 +- .../PrometheusMetricsGenerator.java | 21 ++- .../BookkeeperManagedLedgerStorageClass.java | 42 +++++ .../broker/storage/ManagedLedgerStorage.java | 36 ++-- .../storage/ManagedLedgerStorageClass.java | 45 +++++ ...napshotSegmentAbortedTxnProcessorImpl.java | 9 +- .../impl/MLPendingAckStoreProvider.java | 175 +++++++++++------- .../broker/admin/AdminApiOffloadTest.java | 2 +- .../pulsar/broker/admin/AdminApiTest.java | 18 +- .../broker/admin/PersistentTopicsTest.java | 2 +- .../service/BrokerBkEnsemblesTests.java | 6 +- .../service/BrokerBookieIsolationTest.java | 6 +- .../broker/service/BrokerServiceTest.java | 7 +- ...sistentDispatcherFailoverConsumerTest.java | 6 +- .../PersistentTopicConcurrentTest.java | 2 +- .../service/PersistentTopicE2ETest.java | 4 +- .../broker/service/PersistentTopicTest.java | 15 +- .../broker/service/ReplicationTxnTest.java | 4 +- .../pulsar/broker/service/ReplicatorTest.java | 3 +- .../pulsar/broker/service/ServerCnxTest.java | 22 ++- .../service/TransactionMarkerDeleteTest.java | 2 +- .../stats/ManagedLedgerMetricsTest.java | 4 +- .../NonStartableTestPulsarService.java | 2 +- .../broker/testcontext/PulsarTestContext.java | 87 ++++++--- .../pulsar/broker/testcontext/SpyConfig.java | 3 + .../StartableTestPulsarService.java | 21 +++ .../TopicTransactionBufferRecoverTest.java | 2 +- .../transaction/TransactionProduceTest.java | 2 +- .../broker/transaction/TransactionTest.java | 2 +- .../pendingack/PendingAckPersistentTest.java | 17 +- .../client/api/OrphanPersistentTopicTest.java | 2 +- .../api/SimpleProducerConsumerTest.java | 39 +++- ...SubscriptionPauseOnAckStatPersistTest.java | 2 +- .../api/v1/V1_ProducerConsumerTest.java | 2 +- .../client/impl/SequenceIdWithErrorTest.java | 2 +- .../pulsar/compaction/CompactionTest.java | 4 +- .../policies/data/PersistencePolicies.java | 20 +- .../pulsar/admin/cli/CmdNamespaces.java | 8 +- .../pulsar/admin/cli/CmdTopicPolicies.java | 8 +- .../apache/pulsar/admin/cli/CmdTopics.java | 8 +- 50 files changed, 701 insertions(+), 318 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/BookkeeperManagedLedgerStorageClass.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorageClass.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 7b28990f35574..a1e1deb503e20 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -87,7 +87,9 @@ public class ManagedLedgerConfig { private int minimumBacklogEntriesForCaching = 1000; private int maxBacklogBetweenCursorsForCaching = 1000; private boolean triggerOffloadOnTopicLoad = false; - + @Getter + @Setter + private String storageClassName; @Getter @Setter private String shadowSourceName; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java index 9bbc2857863ff..737bc69bf24df 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java @@ -24,6 +24,8 @@ import io.netty.channel.EventLoopGroup; import io.opentelemetry.api.OpenTelemetry; import java.io.IOException; +import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -39,16 +41,18 @@ import org.apache.bookkeeper.stats.StatsProvider; import org.apache.commons.configuration.Configuration; import org.apache.pulsar.broker.stats.prometheus.metrics.PrometheusMetricsProvider; +import org.apache.pulsar.broker.storage.BookkeeperManagedLedgerStorageClass; import org.apache.pulsar.broker.storage.ManagedLedgerStorage; +import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; import org.apache.pulsar.common.policies.data.EnsemblePlacementPolicyConfig; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ManagedLedgerClientFactory implements ManagedLedgerStorage { - private static final Logger log = LoggerFactory.getLogger(ManagedLedgerClientFactory.class); - + private static final String DEFAULT_STORAGE_CLASS_NAME = "bookkeeper"; + private BookkeeperManagedLedgerStorageClass defaultStorageClass; private ManagedLedgerFactory managedLedgerFactory; private BookKeeper defaultBkClient; private final AsyncCache @@ -119,20 +123,50 @@ public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadata defaultBkClient.close(); throw e; } + + defaultStorageClass = new BookkeeperManagedLedgerStorageClass() { + @Override + public String getName() { + return DEFAULT_STORAGE_CLASS_NAME; + } + + @Override + public ManagedLedgerFactory getManagedLedgerFactory() { + return managedLedgerFactory; + } + + @Override + public StatsProvider getStatsProvider() { + return statsProvider; + } + + @Override + public BookKeeper getBookKeeperClient() { + return defaultBkClient; + } + }; } - public ManagedLedgerFactory getManagedLedgerFactory() { - return managedLedgerFactory; + @Override + public Collection getStorageClasses() { + return List.of(getDefaultStorageClass()); } - public BookKeeper getBookKeeperClient() { - return defaultBkClient; + @Override + public Optional getManagedLedgerStorageClass(String name) { + if (name == null || DEFAULT_STORAGE_CLASS_NAME.equals(name)) { + return Optional.of(getDefaultStorageClass()); + } else { + return Optional.empty(); + } } - public StatsProvider getStatsProvider() { - return statsProvider; + @Override + public ManagedLedgerStorageClass getDefaultStorageClass() { + return defaultStorageClass; } + @VisibleForTesting public Map getBkEnsemblePolicyToBookKeeperMap() { return bkEnsemblePolicyToBkClientMap.synchronous().asMap(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 6c768a078974f..dcc0e961275bd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -124,7 +124,9 @@ import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; import org.apache.pulsar.broker.stats.prometheus.PulsarPrometheusMetricsServlet; +import org.apache.pulsar.broker.storage.BookkeeperManagedLedgerStorageClass; import org.apache.pulsar.broker.storage.ManagedLedgerStorage; +import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl; import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider; @@ -210,7 +212,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private static final int DEFAULT_MONOTONIC_CLOCK_GRANULARITY_MILLIS = 8; private final ServiceConfiguration config; private NamespaceService nsService = null; - private ManagedLedgerStorage managedLedgerClientFactory = null; + private ManagedLedgerStorage managedLedgerStorage = null; private LeaderElectionService leaderElectionService = null; private BrokerService brokerService = null; private WebService webService = null; @@ -606,13 +608,13 @@ public CompletableFuture closeAsync() { this.brokerService = null; } - if (this.managedLedgerClientFactory != null) { + if (this.managedLedgerStorage != null) { try { - this.managedLedgerClientFactory.close(); + this.managedLedgerStorage.close(); } catch (Exception e) { LOG.warn("ManagedLedgerClientFactory closing failed {}", e.getMessage()); } - this.managedLedgerClientFactory = null; + this.managedLedgerStorage = null; } if (bkClientFactory != null) { @@ -899,7 +901,7 @@ public void start() throws PulsarServerException { // Now we are ready to start services this.bkClientFactory = newBookKeeperClientFactory(); - managedLedgerClientFactory = newManagedLedgerClientFactory(); + managedLedgerStorage = newManagedLedgerStorage(); this.brokerService = newBrokerService(this); @@ -1122,7 +1124,7 @@ protected OrderedExecutor newOrderedExecutor() { } @VisibleForTesting - protected ManagedLedgerStorage newManagedLedgerClientFactory() throws Exception { + protected ManagedLedgerStorage newManagedLedgerStorage() throws Exception { return ManagedLedgerStorage.create( config, localMetadataStore, bkClientFactory, ioEventLoopGroup, openTelemetry.getOpenTelemetryService().getOpenTelemetry() @@ -1348,7 +1350,7 @@ private synchronized void startLoadBalancerTasks() { long resourceQuotaUpdateInterval = TimeUnit.MINUTES .toMillis(getConfiguration().getLoadBalancerResourceQuotaUpdateIntervalMinutes()); loadSheddingTask = new LoadSheddingTask(loadManager, loadManagerExecutor, - config, getManagedLedgerFactory()); + config, getDefaultManagedLedgerFactory()); loadSheddingTask.start(); loadResourceQuotaTask = loadManagerExecutor.scheduleAtFixedRate( new LoadResourceQuotaUpdaterTask(loadManager), resourceQuotaUpdateInterval, @@ -1535,11 +1537,17 @@ public WorkerService getWorkerService() throws UnsupportedOperationException { } public BookKeeper getBookKeeperClient() { - return getManagedLedgerClientFactory().getBookKeeperClient(); + ManagedLedgerStorageClass defaultStorageClass = getManagedLedgerStorage().getDefaultStorageClass(); + if (defaultStorageClass instanceof BookkeeperManagedLedgerStorageClass bkStorageClass) { + return bkStorageClass.getBookKeeperClient(); + } else { + // TODO: Refactor code to support other than default bookkeeper based storage class + throw new UnsupportedOperationException("BookKeeper client is not available"); + } } - public ManagedLedgerFactory getManagedLedgerFactory() { - return getManagedLedgerClientFactory().getManagedLedgerFactory(); + public ManagedLedgerFactory getDefaultManagedLedgerFactory() { + return getManagedLedgerStorage().getDefaultStorageClass().getManagedLedgerFactory(); } /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/TransactionMetadataStoreService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/TransactionMetadataStoreService.java index c80580b02f19a..bd19a8e860255 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/TransactionMetadataStoreService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/TransactionMetadataStoreService.java @@ -227,7 +227,9 @@ public CompletableFuture handleTcClientConnect(TransactionCoordinatorID tc .setBatchedWriteMaxDelayInMillis(serviceConfiguration.getTransactionLogBatchedWriteMaxDelayInMillis()); return pulsarService.getBrokerService().getManagedLedgerConfig(getMLTransactionLogName(tcId)).thenCompose( - v -> transactionMetadataStoreProvider.openStore(tcId, pulsarService.getManagedLedgerFactory(), v, + v -> transactionMetadataStoreProvider.openStore(tcId, + pulsarService.getManagedLedgerStorage().getManagedLedgerStorageClass(v.getStorageClassName()) + .get().getManagedLedgerFactory(), v, timeoutTracker, recoverTracker, pulsarService.getConfig().getMaxActiveTransactionsPerCoordinator(), txnLogBufferedWriterConfig, brokerClientSharedTimer)); 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 4d26fe2a4c35b..18c80d6bef4bf 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 @@ -2540,7 +2540,7 @@ protected void internalScanOffloadedLedgers(OffloaderObjectsScannerUtils.Scanner String localClusterName = pulsar().getConfiguration().getClusterName(); OffloaderObjectsScannerUtils.scanOffloadedLedgers(managedLedgerOffloader, - localClusterName, pulsar().getManagedLedgerFactory(), sink); + localClusterName, pulsar().getDefaultManagedLedgerFactory(), sink); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 8860c9bb06d4d..6070093cc3585 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -1405,19 +1405,27 @@ protected void internalGetManagedLedgerInfoForNonPartitionedTopic(AsyncResponse validateTopicOperationAsync(topicName, TopicOperation.GET_STATS) .thenAccept(__ -> { String managedLedger = topicName.getPersistenceNamingEncoding(); - pulsar().getManagedLedgerFactory() - .asyncGetManagedLedgerInfo(managedLedger, new ManagedLedgerInfoCallback() { - @Override - public void getInfoComplete(ManagedLedgerInfo info, Object ctx) { - asyncResponse.resume((StreamingOutput) output -> { - objectWriter().writeValue(output, info); + pulsar().getBrokerService().getManagedLedgerFactoryForTopic(topicName) + .thenAccept(managedLedgerFactory -> { + managedLedgerFactory.asyncGetManagedLedgerInfo(managedLedger, + new ManagedLedgerInfoCallback() { + @Override + public void getInfoComplete(ManagedLedgerInfo info, Object ctx) { + asyncResponse.resume((StreamingOutput) output -> { + objectWriter().writeValue(output, info); + }); + } + + @Override + public void getInfoFailed(ManagedLedgerException exception, Object ctx) { + asyncResponse.resume(exception); + } + }, null); + }) + .exceptionally(ex -> { + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; }); - } - @Override - public void getInfoFailed(ManagedLedgerException exception, Object ctx) { - asyncResponse.resume(exception); - } - }, null); }).exceptionally(ex -> { log.error("[{}] Failed to get managed info for {}", clientAppId(), topicName, ex); resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3174,7 +3182,9 @@ protected CompletableFuture internalGetBacklogAsync try { PersistentOfflineTopicStats estimateOfflineTopicStats = offlineTopicBacklog.estimateUnloadedTopicBacklog( - pulsar().getManagedLedgerFactory(), + pulsar().getBrokerService() + .getManagedLedgerFactoryForTopic(topicName, + config.getStorageClassName()), topicName); pulsar().getBrokerService() .cacheOfflineTopicStats(topicName, estimateOfflineTopicStats); 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 c240c758dcda6..ed0cdf18b47ca 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 @@ -64,6 +64,7 @@ import java.util.Set; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; @@ -136,6 +137,8 @@ import org.apache.pulsar.broker.stats.ClusterReplicationMetrics; import org.apache.pulsar.broker.stats.prometheus.metrics.ObserverGauge; 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.validator.BindAddressValidator; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; @@ -216,7 +219,7 @@ public class BrokerService implements Closeable { .register(); private final PulsarService pulsar; - private final ManagedLedgerFactory managedLedgerFactory; + private final ManagedLedgerStorage managedLedgerStorage; private final Map>> topics = new ConcurrentHashMap<>(); @@ -335,7 +338,7 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws this.brokerPublishRateLimiter = new PublishRateLimiterImpl(pulsar.getMonotonicSnapshotClock()); this.preciseTopicPublishRateLimitingEnable = pulsar.getConfiguration().isPreciseTopicPublishRateLimiterEnable(); - this.managedLedgerFactory = pulsar.getManagedLedgerFactory(); + this.managedLedgerStorage = pulsar.getManagedLedgerStorage(); this.keepAliveIntervalSeconds = pulsar.getConfiguration().getKeepAliveIntervalSeconds(); this.pendingTopicLoadingQueue = Queues.newConcurrentLinkedQueue(); this.pulsarStats = new PulsarStats(pulsar); @@ -1241,23 +1244,51 @@ private CompletableFuture deleteTopicInternal(String topic, boolean forceD return; } CompletableFuture mlConfigFuture = getManagedLedgerConfig(topicName); - managedLedgerFactory.asyncDelete(tn.getPersistenceNamingEncoding(), - mlConfigFuture, new DeleteLedgerCallback() { - @Override - public void deleteLedgerComplete(Object ctx) { - future.complete(null); - } + mlConfigFuture.thenAccept(config -> { + getManagedLedgerFactoryForTopic(topicName, config.getStorageClassName()) + .asyncDelete(tn.getPersistenceNamingEncoding(), + mlConfigFuture, new DeleteLedgerCallback() { + @Override + public void deleteLedgerComplete(Object ctx) { + future.complete(null); + } - @Override - public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { - future.completeExceptionally(exception); - } - }, null); + @Override + public void deleteLedgerFailed(ManagedLedgerException exception, + Object ctx) { + future.completeExceptionally(exception); + } + }, null); + }).exceptionally(ex1 -> { + log.error("Failed to get managed ledger config for topic {}", topic, ex1); + future.completeExceptionally(ex1); + return null; + }); }); return future; } + public CompletableFuture getManagedLedgerFactoryForTopic(TopicName topicName) { + return getManagedLedgerConfig(topicName) + .thenApply(config -> { + String storageClassName = config.getStorageClassName(); + return getManagedLedgerFactoryForTopic(topicName, storageClassName); + }); + } + + public ManagedLedgerFactory getManagedLedgerFactoryForTopic(TopicName topicName, String storageClassName) { + Optional managedLedgerStorageClass = + managedLedgerStorage.getManagedLedgerStorageClass(storageClassName); + if (!managedLedgerStorageClass.isPresent()) { + throw new CompletionException(new ManagedLedgerException( + "ManagedLedgerStorageClass " + storageClassName + " not found for topic " + topicName)); + } + return managedLedgerStorageClass + .get() + .getManagedLedgerFactory(); + } + public void deleteTopicAuthenticationWithRetry(String topic, CompletableFuture future, int count) { if (count == 0) { log.error("The number of retries has exhausted for topic {}", topic); @@ -1624,14 +1655,17 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S @VisibleForTesting protected CompletableFuture> fetchTopicPropertiesAsync(TopicName topicName) { if (!topicName.isPartitioned()) { - return managedLedgerFactory.getManagedLedgerPropertiesAsync(topicName.getPersistenceNamingEncoding()); + return getManagedLedgerFactoryForTopic(topicName).thenCompose( + managedLedgerFactory -> managedLedgerFactory.getManagedLedgerPropertiesAsync( + topicName.getPersistenceNamingEncoding())); } else { TopicName partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName()); return fetchPartitionedTopicMetadataAsync(partitionedTopicName) .thenCompose(metadata -> { if (metadata.partitions == PartitionedTopicMetadata.NON_PARTITIONED) { - return managedLedgerFactory.getManagedLedgerPropertiesAsync( - topicName.getPersistenceNamingEncoding()); + return getManagedLedgerFactoryForTopic(topicName).thenCompose( + managedLedgerFactory -> managedLedgerFactory.getManagedLedgerPropertiesAsync( + topicName.getPersistenceNamingEncoding())); } else { // Check if the partitioned topic is a ShadowTopic if (MapUtils.getString(metadata.properties, PROPERTY_SOURCE_TOPIC_KEY) != null) { @@ -1756,6 +1790,8 @@ private void createPersistentTopic(final String topic, boolean createIfMissing, topicEventsDispatcher.notifyOnCompletion(loadFuture, topic, TopicEvent.LOAD); // Once we have the configuration, we can proceed with the async open operation + ManagedLedgerFactory managedLedgerFactory = + getManagedLedgerFactoryForTopic(topicName, managedLedgerConfig.getStorageClassName()); managedLedgerFactory.asyncOpen(topicName.getPersistenceNamingEncoding(), managedLedgerConfig, new OpenLedgerCallback() { @Override @@ -1918,6 +1954,7 @@ private CompletableFuture getManagedLedgerConfig(@Nonnull T managedLedgerConfig.setEnsembleSize(persistencePolicies.getBookkeeperEnsemble()); managedLedgerConfig.setWriteQuorumSize(persistencePolicies.getBookkeeperWriteQuorum()); managedLedgerConfig.setAckQuorumSize(persistencePolicies.getBookkeeperAckQuorum()); + managedLedgerConfig.setStorageClassName(persistencePolicies.getManagedLedgerStorageClassName()); if (serviceConfig.isStrictBookieAffinityEnabled()) { managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyClassName( @@ -2745,25 +2782,29 @@ private void updateConfigurationAndRegisterListeners() { }); }); + + ManagedLedgerFactory defaultManagedLedgerFactory = + managedLedgerStorage.getDefaultStorageClass().getManagedLedgerFactory(); + // add listener to notify broker managedLedgerCacheSizeMB dynamic config registerConfigurationListener("managedLedgerCacheSizeMB", (managedLedgerCacheSizeMB) -> { - managedLedgerFactory.getEntryCacheManager() + defaultManagedLedgerFactory.getEntryCacheManager() .updateCacheSizeAndThreshold(((int) managedLedgerCacheSizeMB) * 1024L * 1024L); }); // add listener to notify broker managedLedgerCacheEvictionWatermark dynamic config registerConfigurationListener( "managedLedgerCacheEvictionWatermark", (cacheEvictionWatermark) -> { - managedLedgerFactory.getEntryCacheManager() - .updateCacheEvictionWatermark((double) cacheEvictionWatermark); - }); + defaultManagedLedgerFactory.getEntryCacheManager() + .updateCacheEvictionWatermark((double) cacheEvictionWatermark); + }); // add listener to notify broker managedLedgerCacheEvictionTimeThresholdMillis dynamic config registerConfigurationListener( "managedLedgerCacheEvictionTimeThresholdMillis", (cacheEvictionTimeThresholdMills) -> { - managedLedgerFactory.updateCacheEvictionTimeThreshold(MILLISECONDS - .toNanos((long) cacheEvictionTimeThresholdMills)); - }); + defaultManagedLedgerFactory.updateCacheEvictionTimeThreshold(MILLISECONDS + .toNanos((long) cacheEvictionTimeThresholdMills)); + }); // add listener to update message-dispatch-rate in msg for topic 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 aedd68d416fe7..37b431e833983 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 @@ -2902,7 +2902,8 @@ protected void handleEndTxnOnPartition(CommandEndTxnOnPartition command) { log.debug("[{}] handleEndTxnOnPartition txnId: [{}], txnAction: [{}]", topic, txnID, txnAction); } - CompletableFuture> topicFuture = service.getTopicIfExists(TopicName.get(topic).toString()); + TopicName topicName = TopicName.get(topic); + CompletableFuture> topicFuture = service.getTopicIfExists(topicName.toString()); topicFuture.thenAcceptAsync(optionalTopic -> { if (optionalTopic.isPresent()) { // we only accept superuser because this endpoint is reserved for tc to broker communication @@ -2928,24 +2929,29 @@ protected void handleEndTxnOnPartition(CommandEndTxnOnPartition command) { txnID.getLeastSigBits(), txnID.getMostSigBits())); }); } else { - getBrokerService().getManagedLedgerFactory() - .asyncExists(TopicName.get(topic).getPersistenceNamingEncoding()) - .thenAccept((b) -> { - if (b) { - log.error("handleEndTxnOnPartition fail ! The topic {} does not exist in broker, " - + "txnId: [{}], txnAction: [{}]", topic, - txnID, TxnAction.valueOf(txnAction)); - writeAndFlush(Commands.newEndTxnOnPartitionResponse(requestId, - ServerError.ServiceNotReady, - "The topic " + topic + " does not exist in broker.", - txnID.getLeastSigBits(), txnID.getMostSigBits())); - } else { - log.warn("handleEndTxnOnPartition fail ! The topic {} has not been created, " - + "txnId: [{}], txnAction: [{}]", - topic, txnID, TxnAction.valueOf(txnAction)); - writeAndFlush(Commands.newEndTxnOnPartitionResponse(requestId, - txnID.getLeastSigBits(), txnID.getMostSigBits())); - } + getBrokerService().getManagedLedgerFactoryForTopic(topicName) + .thenCompose(managedLedgerFactory -> { + return managedLedgerFactory.asyncExists(topicName.getPersistenceNamingEncoding()) + .thenAccept((b) -> { + if (b) { + log.error( + "handleEndTxnOnPartition fail ! The topic {} does not exist in " + + "broker, " + + "txnId: [{}], txnAction: [{}]", topic, + txnID, TxnAction.valueOf(txnAction)); + writeAndFlush(Commands.newEndTxnOnPartitionResponse(requestId, + ServerError.ServiceNotReady, + "The topic " + topic + " does not exist in broker.", + txnID.getLeastSigBits(), txnID.getMostSigBits())); + } else { + log.warn( + "handleEndTxnOnPartition fail ! The topic {} has not been created, " + + "txnId: [{}], txnAction: [{}]", + topic, txnID, TxnAction.valueOf(txnAction)); + writeAndFlush(Commands.newEndTxnOnPartitionResponse(requestId, + txnID.getLeastSigBits(), txnID.getMostSigBits())); + } + }); }).exceptionally(e -> { log.error("handleEndTxnOnPartition fail ! topic {}, " + "txnId: [{}], txnAction: [{}]", topic, txnID, @@ -2954,7 +2960,8 @@ protected void handleEndTxnOnPartition(CommandEndTxnOnPartition command) { requestId, ServerError.ServiceNotReady, e.getMessage(), txnID.getLeastSigBits(), txnID.getMostSigBits())); return null; - }); + + }); } }, ctx.executor()).exceptionally(e -> { log.error("handleEndTxnOnPartition fail ! topic {}, " @@ -2984,7 +2991,8 @@ protected void handleEndTxnOnSubscription(CommandEndTxnOnSubscription command) { new TxnID(txnidMostBits, txnidLeastBits), txnAction); } - CompletableFuture> topicFuture = service.getTopicIfExists(TopicName.get(topic).toString()); + TopicName topicName = TopicName.get(topic); + CompletableFuture> topicFuture = service.getTopicIfExists(topicName.toString()); topicFuture.thenAcceptAsync(optionalTopic -> { if (optionalTopic.isPresent()) { Subscription subscription = optionalTopic.get().getSubscription(subName); @@ -3019,24 +3027,31 @@ protected void handleEndTxnOnSubscription(CommandEndTxnOnSubscription command) { Commands.newEndTxnOnSubscriptionResponse(requestId, txnidLeastBits, txnidMostBits)); }); } else { - getBrokerService().getManagedLedgerFactory() - .asyncExists(TopicName.get(topic).getPersistenceNamingEncoding()) - .thenAccept((b) -> { - if (b) { - log.error("handleEndTxnOnSubscription fail! The topic {} does not exist in broker, " - + "subscription: {}, txnId: [{}], txnAction: [{}]", topic, subName, - txnID, TxnAction.valueOf(txnAction)); - writeAndFlush(Commands.newEndTxnOnSubscriptionResponse( - requestId, txnID.getLeastSigBits(), txnID.getMostSigBits(), - ServerError.ServiceNotReady, - "The topic " + topic + " does not exist in broker.")); - } else { - log.warn("handleEndTxnOnSubscription fail ! The topic {} has not been created, " - + "subscription: {} txnId: [{}], txnAction: [{}]", - topic, subName, txnID, TxnAction.valueOf(txnAction)); - writeAndFlush(Commands.newEndTxnOnSubscriptionResponse(requestId, - txnID.getLeastSigBits(), txnID.getMostSigBits())); - } + getBrokerService().getManagedLedgerFactoryForTopic(topicName) + .thenCompose(managedLedgerFactory -> { + return managedLedgerFactory.asyncExists(topicName.getPersistenceNamingEncoding()) + .thenAccept((b) -> { + if (b) { + log.error( + "handleEndTxnOnSubscription fail! The topic {} does not exist in " + + "broker, " + + "subscription: {}, txnId: [{}], txnAction: [{}]", topic, + subName, + txnID, TxnAction.valueOf(txnAction)); + writeAndFlush(Commands.newEndTxnOnSubscriptionResponse( + requestId, txnID.getLeastSigBits(), txnID.getMostSigBits(), + ServerError.ServiceNotReady, + "The topic " + topic + " does not exist in broker.")); + } else { + log.warn( + "handleEndTxnOnSubscription fail ! The topic {} has not been " + + "created, " + + "subscription: {} txnId: [{}], txnAction: [{}]", + topic, subName, txnID, TxnAction.valueOf(txnAction)); + writeAndFlush(Commands.newEndTxnOnSubscriptionResponse(requestId, + txnID.getLeastSigBits(), txnID.getMostSigBits())); + } + }); }).exceptionally(e -> { log.error("handleEndTxnOnSubscription fail ! topic {}, subscription: {}" + "txnId: [{}], txnAction: [{}]", topic, subName, @@ -3045,7 +3060,7 @@ protected void handleEndTxnOnSubscription(CommandEndTxnOnSubscription command) { requestId, txnID.getLeastSigBits(), txnID.getMostSigBits(), ServerError.ServiceNotReady, e.getMessage())); return null; - }); + }); } }, ctx.executor()).exceptionally(e -> { log.error("handleEndTxnOnSubscription fail ! topic: {}, subscription: {}" diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index f8581cfc79985..3cce175660e70 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -74,12 +74,14 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursor.IndividualDeletedEntries; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerAlreadyClosedException; import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerFencedException; import org.apache.bookkeeper.mledger.ManagedLedgerException.ManagedLedgerTerminatedException; import org.apache.bookkeeper.mledger.ManagedLedgerException.MetadataNotFoundException; import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionBound; import org.apache.bookkeeper.mledger.PositionFactory; @@ -1232,26 +1234,34 @@ public CompletableFuture unsubscribe(String subscriptionName) { .getTransactionPendingAckStoreSuffix(topic, Codec.encode(subscriptionName))); if (brokerService.pulsar().getConfiguration().isTransactionCoordinatorEnabled()) { - getBrokerService().getManagedLedgerFactory().asyncDelete(tn.getPersistenceNamingEncoding(), - getBrokerService().getManagedLedgerConfig(tn), - new AsyncCallbacks.DeleteLedgerCallback() { - @Override - public void deleteLedgerComplete(Object ctx) { - asyncDeleteCursorWithClearDelayedMessage(subscriptionName, unsubscribeFuture); - } - - @Override - public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { - if (exception instanceof MetadataNotFoundException) { + CompletableFuture managedLedgerConfig = getBrokerService().getManagedLedgerConfig(tn); + managedLedgerConfig.thenAccept(config -> { + ManagedLedgerFactory managedLedgerFactory = + getBrokerService().getManagedLedgerFactoryForTopic(tn, config.getStorageClassName()); + managedLedgerFactory.asyncDelete(tn.getPersistenceNamingEncoding(), + managedLedgerConfig, + new AsyncCallbacks.DeleteLedgerCallback() { + @Override + public void deleteLedgerComplete(Object ctx) { asyncDeleteCursorWithClearDelayedMessage(subscriptionName, unsubscribeFuture); - return; } - unsubscribeFuture.completeExceptionally(exception); - log.error("[{}][{}] Error deleting subscription pending ack store", - topic, subscriptionName, exception); - } - }, null); + @Override + public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { + if (exception instanceof MetadataNotFoundException) { + asyncDeleteCursorWithClearDelayedMessage(subscriptionName, unsubscribeFuture); + return; + } + + unsubscribeFuture.completeExceptionally(exception); + log.error("[{}][{}] Error deleting subscription pending ack store", + topic, subscriptionName, exception); + } + }, null); + }).exceptionally(ex -> { + unsubscribeFuture.completeExceptionally(ex); + return null; + }); } else { asyncDeleteCursorWithClearDelayedMessage(subscriptionName, unsubscribeFuture); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java index 489d37dd0a307..114f962cb81d9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/AbstractMetrics.java @@ -131,7 +131,7 @@ protected Metrics createMetrics(Map dimensionMap) { * @return */ protected ManagedLedgerFactoryMXBean getManagedLedgerCacheStats() { - return pulsar.getManagedLedgerFactory().getCacheStats(); + return pulsar.getDefaultManagedLedgerFactory().getCacheStats(); } /** @@ -140,7 +140,7 @@ protected ManagedLedgerFactoryMXBean getManagedLedgerCacheStats() { * @return */ protected Map getManagedLedgers() { - return pulsar.getManagedLedgerFactory().getManagedLedgers(); + return pulsar.getDefaultManagedLedgerFactory().getManagedLedgers(); } protected String getLocalClusterName() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java index 52c69265c2f1f..925fcb28b7a03 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/metrics/ManagedLedgerMetrics.java @@ -52,8 +52,7 @@ public ManagedLedgerMetrics(PulsarService pulsar) { this.metricsCollection = new ArrayList<>(); this.ledgersByDimensionMap = new HashMap<>(); this.tempAggregatedMetricsMap = new HashMap<>(); - this.statsPeriodSeconds = pulsar.getManagedLedgerFactory() - .getConfig().getStatsPeriodSeconds(); + this.statsPeriodSeconds = pulsar.getDefaultManagedLedgerFactory().getConfig().getStatsPeriodSeconds(); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java index 6b4d08c359d42..8c3cb39c925d7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java @@ -55,6 +55,8 @@ import org.apache.pulsar.broker.stats.metrics.ManagedCursorMetrics; import org.apache.pulsar.broker.stats.metrics.ManagedLedgerCacheMetrics; import org.apache.pulsar.broker.stats.metrics.ManagedLedgerMetrics; +import org.apache.pulsar.broker.storage.BookkeeperManagedLedgerStorageClass; +import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.SimpleTextOutputStream; @@ -485,12 +487,14 @@ private static SimpleTextOutputStream writeNameReplacingBrkPrefix(SimpleTextOutp } private static void generateManagedLedgerBookieClientMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { - StatsProvider statsProvider = pulsar.getManagedLedgerClientFactory().getStatsProvider(); - if (statsProvider instanceof NullStatsProvider) { - return; - } + ManagedLedgerStorageClass defaultStorageClass = pulsar.getManagedLedgerStorage().getDefaultStorageClass(); + if (defaultStorageClass instanceof BookkeeperManagedLedgerStorageClass bkStorageClass) { + StatsProvider statsProvider = bkStorageClass.getStatsProvider(); + if (statsProvider instanceof NullStatsProvider) { + return; + } - try (Writer writer = new OutputStreamWriter(new BufferedOutputStream(new OutputStream() { + try (Writer writer = new OutputStreamWriter(new BufferedOutputStream(new OutputStream() { @Override public void write(int b) throws IOException { stream.writeByte(b); @@ -501,9 +505,10 @@ public void write(byte b[], int off, int len) throws IOException { stream.write(b, off, len); } }), StandardCharsets.UTF_8)) { - statsProvider.writeAllMetrics(writer); - } catch (IOException e) { - log.error("Failed to write managed ledger bookie client metrics", e); + statsProvider.writeAllMetrics(writer); + } catch (IOException e) { + log.error("Failed to write managed ledger bookie client metrics", e); + } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/BookkeeperManagedLedgerStorageClass.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/BookkeeperManagedLedgerStorageClass.java new file mode 100644 index 0000000000000..1f05cde72a5b5 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/BookkeeperManagedLedgerStorageClass.java @@ -0,0 +1,42 @@ +/* + * 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.storage; + +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.stats.StatsProvider; + +/** + * ManagedLedgerStorageClass represents a configured instance of ManagedLedgerFactory for managed ledgers. + * This instance is backed by a bookkeeper storage. + */ +public interface BookkeeperManagedLedgerStorageClass extends ManagedLedgerStorageClass { + /** + * Return the bookkeeper client instance used by this instance. + * + * @return the bookkeeper client. + */ + BookKeeper getBookKeeperClient(); + + /** + * Return the stats provider to expose the stats of the storage implementation. + * + * @return the stats provider. + */ + StatsProvider getStatsProvider(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java index 944d2badf75f2..720798123e7b4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorage.java @@ -21,9 +21,8 @@ import io.netty.channel.EventLoopGroup; import io.opentelemetry.api.OpenTelemetry; import java.io.IOException; -import org.apache.bookkeeper.client.BookKeeper; -import org.apache.bookkeeper.mledger.ManagedLedgerFactory; -import org.apache.bookkeeper.stats.StatsProvider; +import java.util.Collection; +import java.util.Optional; import org.apache.pulsar.broker.BookKeeperClientFactory; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.common.classification.InterfaceAudience.Private; @@ -33,6 +32,12 @@ /** * Storage to access {@link org.apache.bookkeeper.mledger.ManagedLedger}s. + *

+ * The interface provides the abstraction to access the storage layer for managed ledgers. + * The interface supports multiple storage classes, each with its own configuration. The default + * implementation supports a single instance of {@link BookkeeperManagedLedgerStorageClass}. + * Implementations can provide multiple storage classes. The default storage class is used + * for topics unless it is overridden by the persistency policy at topic or namespace level. */ @Private @Unstable @@ -52,25 +57,25 @@ void initialize(ServiceConfiguration conf, OpenTelemetry openTelemetry) throws Exception; /** - * Return the factory to create {@link ManagedLedgerFactory}. - * - * @return the factory to create {@link ManagedLedgerFactory}. + * Get all configured storage class instances. + * @return all configured storage class instances */ - ManagedLedgerFactory getManagedLedgerFactory(); + Collection getStorageClasses(); /** - * Return the stats provider to expose the stats of the storage implementation. - * - * @return the stats provider. + * Get the default storage class. + * @return default storage class */ - StatsProvider getStatsProvider(); + default ManagedLedgerStorageClass getDefaultStorageClass() { + return getStorageClasses().stream().findFirst().get(); + } /** - * Return the default bookkeeper client. - * - * @return the default bookkeeper client. + * Lookup a storage class by name. + * @param name storage class name + * @return storage class instance, or empty if not found */ - BookKeeper getBookKeeperClient(); + Optional getManagedLedgerStorageClass(String name); /** * Close the storage. @@ -97,5 +102,4 @@ static ManagedLedgerStorage create(ServiceConfiguration conf, storage.initialize(conf, metadataStore, bkProvider, eventLoopGroup, openTelemetry); return storage; } - } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorageClass.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorageClass.java new file mode 100644 index 0000000000000..8cbe5c3b411e5 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/storage/ManagedLedgerStorageClass.java @@ -0,0 +1,45 @@ +/* + * 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.storage; + +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; +import org.apache.pulsar.common.classification.InterfaceAudience; +import org.apache.pulsar.common.classification.InterfaceStability; + +/** + * ManagedLedgerStorageClass represents a configured instance of ManagedLedgerFactory for managed ledgers. + * The {@link ManagedLedgerStorage} can hold multiple storage classes, and each storage class can have its own + * configuration. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public interface ManagedLedgerStorageClass { + /** + * Return the name of the storage class. + * + * @return the name of the storage class. + */ + String getName(); + /** + * Return the factory to create {@link ManagedLedgerFactory}. + * + * @return the factory to create {@link ManagedLedgerFactory}. + */ + ManagedLedgerFactory getManagedLedgerFactory(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java index 88a3968b7b430..f2ff5d519d8c0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java @@ -311,8 +311,13 @@ public String toString() { SnapshotSegmentAbortedTxnProcessorImpl.this.topic.getName()); } }; - topic.getBrokerService().getPulsar().getManagedLedgerFactory().asyncOpenReadOnlyManagedLedger( - topicName.getPersistenceNamingEncoding(), callback, topic.getManagedLedger().getConfig(), null); + topic.getBrokerService().getManagedLedgerFactoryForTopic(topicName).thenAccept(managedLedgerFactory -> + managedLedgerFactory.asyncOpenReadOnlyManagedLedger(topicName.getPersistenceNamingEncoding(), + callback, topic.getManagedLedger().getConfig(), null)) + .exceptionally(e -> { + future.completeExceptionally(e); + return null; + }); return wait(future, "open read only ml for " + topicName); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreProvider.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreProvider.java index 6fc61d423ce85..12f761bb4df5e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreProvider.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreProvider.java @@ -25,9 +25,11 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.transaction.exception.pendingack.TransactionPendingAckException; @@ -84,7 +86,8 @@ public CompletableFuture newPendingAckStore(PersistentSubscript } PersistentTopic originPersistentTopic = (PersistentTopic) subscription.getTopic(); - PulsarService pulsarService = originPersistentTopic.getBrokerService().getPulsar(); + BrokerService brokerService = originPersistentTopic.getBrokerService(); + PulsarService pulsarService = brokerService.getPulsar(); final Timer brokerClientSharedTimer = pulsarService.getBrokerClientSharedTimer(); @@ -103,93 +106,127 @@ public CompletableFuture newPendingAckStore(PersistentSubscript String pendingAckTopicName = MLPendingAckStore .getTransactionPendingAckStoreSuffix(originPersistentTopic.getName(), subscription.getName()); - originPersistentTopic.getBrokerService().getManagedLedgerFactory() - .asyncExists(TopicName.get(pendingAckTopicName) - .getPersistenceNamingEncoding()).thenAccept(exist -> { - TopicName topicName; - if (exist) { - topicName = TopicName.get(pendingAckTopicName); - } else { - topicName = TopicName.get(originPersistentTopic.getName()); - } - originPersistentTopic.getBrokerService() - .getManagedLedgerConfig(topicName).thenAccept(config -> { - config.setCreateIfMissing(true); - originPersistentTopic.getBrokerService().getManagedLedgerFactory() - .asyncOpen(TopicName.get(pendingAckTopicName).getPersistenceNamingEncoding(), - config, new AsyncCallbacks.OpenLedgerCallback() { - @Override - public void openLedgerComplete(ManagedLedger ledger, Object ctx) { - ledger.asyncOpenCursor( - MLPendingAckStore.getTransactionPendingAckStoreCursorName(), - InitialPosition.Earliest, new AsyncCallbacks.OpenCursorCallback() { - @Override - public void openCursorComplete(ManagedCursor cursor, Object ctx) { - pendingAckStoreFuture.complete(new MLPendingAckStore(ledger, - cursor, - subscription.getCursor(), - originPersistentTopic - .getBrokerService() - .getPulsar() - .getConfiguration() - .getTransactionPendingAckLogIndexMinLag(), - txnLogBufferedWriterConfig, - brokerClientSharedTimer, bufferedWriterMetrics, - originPersistentTopic - .getBrokerService() - .getPulsar() - .getOrderedExecutor() - .chooseThread())); - if (log.isDebugEnabled()) { - log.debug("{},{} open MLPendingAckStore cursor success", - originPersistentTopic.getName(), - subscription.getName()); - } - } - - @Override - public void openCursorFailed(ManagedLedgerException exception, - Object ctx) { - log.error("{},{} open MLPendingAckStore cursor failed." - , originPersistentTopic.getName(), - subscription.getName(), exception); - pendingAckStoreFuture.completeExceptionally(exception); - } - }, null); - } - - @Override - public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { - log.error("{}, {} open MLPendingAckStore managedLedger failed." - , originPersistentTopic.getName(), subscription.getName(), exception); - pendingAckStoreFuture.completeExceptionally(exception); - } - }, () -> CompletableFuture.completedFuture(true), null); + TopicName pendingAckTopicNameObject = TopicName.get(pendingAckTopicName); + brokerService.getManagedLedgerFactoryForTopic(pendingAckTopicNameObject) + .thenAccept(managedLedgerFactory -> { + managedLedgerFactory.asyncExists(pendingAckTopicNameObject + .getPersistenceNamingEncoding()).thenAccept(exist -> { + TopicName topicName; + if (exist) { + topicName = pendingAckTopicNameObject; + } else { + topicName = TopicName.get(originPersistentTopic.getName()); + } + brokerService.getManagedLedgerConfig(topicName).thenAccept(config -> { + internalNewPendingAckStore(subscription, config, brokerService, topicName, + pendingAckTopicNameObject, pendingAckStoreFuture, txnLogBufferedWriterConfig, + brokerClientSharedTimer, originPersistentTopic); + }).exceptionally(e -> { + Throwable t = FutureUtil.unwrapCompletionException(e); + log.error("[{}] [{}] Failed to get managedLedger config when init pending ack " + + "store!", + originPersistentTopic, subscription, t); + pendingAckStoreFuture.completeExceptionally(t); + return null; + + }); }).exceptionally(e -> { Throwable t = FutureUtil.unwrapCompletionException(e); - log.error("[{}] [{}] Failed to get managedLedger config when init pending ack store!", + log.error("[{}] [{}] Failed to check the pending ack topic exist when init pending ack store!", originPersistentTopic, subscription, t); pendingAckStoreFuture.completeExceptionally(t); return null; - }); }).exceptionally(e -> { Throwable t = FutureUtil.unwrapCompletionException(e); - log.error("[{}] [{}] Failed to check the pending ack topic exist when init pending ack store!", - originPersistentTopic, subscription, t); + log.error("[{}] [{}] Failed to get managedLedger config when init pending ack store!", + pendingAckTopicNameObject, subscription, t); pendingAckStoreFuture.completeExceptionally(t); return null; }); return pendingAckStoreFuture; } + private static void internalNewPendingAckStore(PersistentSubscription subscription, ManagedLedgerConfig config, + BrokerService brokerService, TopicName topicName, + TopicName pendingAckTopicNameObject, + CompletableFuture pendingAckStoreFuture, + TxnLogBufferedWriterConfig txnLogBufferedWriterConfig, + Timer brokerClientSharedTimer, + PersistentTopic originPersistentTopic) { + config.setCreateIfMissing(true); + brokerService + .getManagedLedgerFactoryForTopic(topicName, config.getStorageClassName()) + .asyncOpen(pendingAckTopicNameObject.getPersistenceNamingEncoding(), + config, new AsyncCallbacks.OpenLedgerCallback() { + @Override + public void openLedgerComplete(ManagedLedger ledger, Object ctx) { + ledger.asyncOpenCursor( + MLPendingAckStore.getTransactionPendingAckStoreCursorName(), + InitialPosition.Earliest, + new AsyncCallbacks.OpenCursorCallback() { + @Override + public void openCursorComplete(ManagedCursor cursor, + Object ctx) { + pendingAckStoreFuture.complete( + new MLPendingAckStore(ledger, + cursor, + subscription.getCursor(), + brokerService + .getPulsar() + .getConfiguration() + .getTransactionPendingAckLogIndexMinLag(), + txnLogBufferedWriterConfig, + brokerClientSharedTimer, + bufferedWriterMetrics, + brokerService + .getPulsar() + .getOrderedExecutor() + .chooseThread())); + if (log.isDebugEnabled()) { + log.debug( + "{},{} open MLPendingAckStore cursor " + + "success", + originPersistentTopic.getName(), + subscription.getName()); + } + } + + @Override + public void openCursorFailed( + ManagedLedgerException exception, + Object ctx) { + log.error( + "{},{} open MLPendingAckStore cursor " + + "failed." + , originPersistentTopic.getName(), + subscription.getName(), exception); + pendingAckStoreFuture.completeExceptionally( + exception); + } + }, null); + } + + @Override + public void openLedgerFailed(ManagedLedgerException exception, + Object ctx) { + log.error("{}, {} open MLPendingAckStore managedLedger failed." + , originPersistentTopic.getName(), subscription.getName(), + exception); + pendingAckStoreFuture.completeExceptionally(exception); + } + }, () -> CompletableFuture.completedFuture(true), null); + } + @Override public CompletableFuture checkInitializedBefore(PersistentSubscription subscription) { PersistentTopic originPersistentTopic = (PersistentTopic) subscription.getTopic(); String pendingAckTopicName = MLPendingAckStore .getTransactionPendingAckStoreSuffix(originPersistentTopic.getName(), subscription.getName()); - return originPersistentTopic.getBrokerService().getManagedLedgerFactory() - .asyncExists(TopicName.get(pendingAckTopicName).getPersistenceNamingEncoding()); + TopicName topicName = TopicName.get(pendingAckTopicName); + return originPersistentTopic.getBrokerService().getManagedLedgerFactoryForTopic(topicName) + .thenCompose(managedLedgerFactory -> managedLedgerFactory.asyncExists( + topicName.getPersistenceNamingEncoding())); } private static class MLTxnPendingAckLogBufferedWriterMetrics extends TxnLogBufferedWriterMetricsStats{ diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java index 1ea29c9d431bd..9aa2dcc700c9b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiOffloadTest.java @@ -135,7 +135,7 @@ private void testOffload(String topicName, String mlName) throws Exception { } } - ManagedLedgerInfo info = pulsar.getManagedLedgerFactory().getManagedLedgerInfo(mlName); + ManagedLedgerInfo info = pulsar.getDefaultManagedLedgerFactory().getManagedLedgerInfo(mlName); assertEquals(info.ledgers.size(), 2); assertEquals(admin.topics().offloadStatus(topicName).getStatus(), LongRunningProcessStatus.Status.NOT_RUN); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index 4a1dbface2c63..26da4116d09cb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -588,24 +588,24 @@ public void testUpdateDynamicCacheConfigurationWithZkWatch() throws Exception { // wait config to be updated Awaitility.await().until(() -> { - return pulsar.getManagedLedgerFactory().getEntryCacheManager().getMaxSize() == 1 * 1024L * 1024L - && pulsar.getManagedLedgerFactory().getEntryCacheManager().getCacheEvictionWatermark() == 0.8 - && pulsar.getManagedLedgerFactory().getCacheEvictionTimeThreshold() == TimeUnit.MILLISECONDS + return pulsar.getDefaultManagedLedgerFactory().getEntryCacheManager().getMaxSize() == 1 * 1024L * 1024L + && pulsar.getDefaultManagedLedgerFactory().getEntryCacheManager().getCacheEvictionWatermark() == 0.8 + && pulsar.getDefaultManagedLedgerFactory().getCacheEvictionTimeThreshold() == TimeUnit.MILLISECONDS .toNanos(2000); }); // verify value is updated - assertEquals(pulsar.getManagedLedgerFactory().getEntryCacheManager().getMaxSize(), 1 * 1024L * 1024L); - assertEquals(pulsar.getManagedLedgerFactory().getEntryCacheManager().getCacheEvictionWatermark(), 0.8); - assertEquals(pulsar.getManagedLedgerFactory().getCacheEvictionTimeThreshold(), TimeUnit.MILLISECONDS + assertEquals(pulsar.getDefaultManagedLedgerFactory().getEntryCacheManager().getMaxSize(), 1 * 1024L * 1024L); + assertEquals(pulsar.getDefaultManagedLedgerFactory().getEntryCacheManager().getCacheEvictionWatermark(), 0.8); + assertEquals(pulsar.getDefaultManagedLedgerFactory().getCacheEvictionTimeThreshold(), TimeUnit.MILLISECONDS .toNanos(2000)); restartBroker(); // verify value again - assertEquals(pulsar.getManagedLedgerFactory().getEntryCacheManager().getMaxSize(), 1 * 1024L * 1024L); - assertEquals(pulsar.getManagedLedgerFactory().getEntryCacheManager().getCacheEvictionWatermark(), 0.8); - assertEquals(pulsar.getManagedLedgerFactory().getCacheEvictionTimeThreshold(), TimeUnit.MILLISECONDS + assertEquals(pulsar.getDefaultManagedLedgerFactory().getEntryCacheManager().getMaxSize(), 1 * 1024L * 1024L); + assertEquals(pulsar.getDefaultManagedLedgerFactory().getEntryCacheManager().getCacheEvictionWatermark(), 0.8); + assertEquals(pulsar.getDefaultManagedLedgerFactory().getCacheEvictionTimeThreshold(), TimeUnit.MILLISECONDS .toNanos(2000)); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index 18fd3dd1c8bb3..aae2f7b8830e8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -708,7 +708,7 @@ public void testUpdatePartitionedTopicHavingNonPartitionTopicWithPartitionSuffix // partitioned topic to more than 10. final String nonPartitionTopicName2 = "special-topic-partition-10"; final String partitionedTopicName = "special-topic"; - pulsar.getBrokerService().getManagedLedgerFactory() + pulsar.getDefaultManagedLedgerFactory() .open(TopicName.get(nonPartitionTopicName2).getPersistenceNamingEncoding()); doAnswer(invocation -> { persistentTopics.namespaceName = NamespaceName.get("tenant", "namespace"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java index 82892ad353aa1..68a52c4b4c381 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java @@ -127,7 +127,7 @@ public void testCrashBrokerWithoutCursorLedgerLeak() throws Exception { consumer.close(); producer.close(); pulsar.getBrokerService().removeTopicFromCache(topic); - ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory(); + ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory(); Field field = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers"); field.setAccessible(true); @SuppressWarnings("unchecked") @@ -250,7 +250,7 @@ public void testSkipCorruptDataLedger() throws Exception { // clean managed-ledger and recreate topic to clean any data from the cache producer.close(); pulsar.getBrokerService().removeTopicFromCache(topic); - ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory(); + ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory(); Field field = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers"); field.setAccessible(true); @SuppressWarnings("unchecked") @@ -399,7 +399,7 @@ public void testTruncateCorruptDataLedger() throws Exception { @Test public void testDeleteLedgerFactoryCorruptLedger() throws Exception { - ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory(); + ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory(); ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open("test"); // bookkeeper client diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java index d7272fcffa964..be6f7c9143742 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java @@ -274,7 +274,7 @@ public void testBookieIsolation() throws Exception { assertAffinityBookies(ledgerManager, ml4.getLedgersInfoAsList(), isolatedBookies); ManagedLedgerClientFactory mlFactory = - (ManagedLedgerClientFactory) pulsarService.getManagedLedgerClientFactory(); + (ManagedLedgerClientFactory) pulsarService.getManagedLedgerStorage(); Map bkPlacementPolicyToBkClientMap = mlFactory .getBkEnsemblePolicyToBookKeeperMap(); @@ -588,7 +588,7 @@ public void testStrictBookieIsolation() throws Exception { assertAffinityBookies(ledgerManager, ml4.getLedgersInfoAsList(), isolatedBookies); ManagedLedgerClientFactory mlFactory = - (ManagedLedgerClientFactory) pulsarService.getManagedLedgerClientFactory(); + (ManagedLedgerClientFactory) pulsarService.getManagedLedgerStorage(); Map bkPlacementPolicyToBkClientMap = mlFactory .getBkEnsemblePolicyToBookKeeperMap(); @@ -751,7 +751,7 @@ public void testBookieIsolationWithSecondaryGroup() throws Exception { assertAffinityBookies(ledgerManager, ml3.getLedgersInfoAsList(), isolatedBookies); ManagedLedgerClientFactory mlFactory = - (ManagedLedgerClientFactory) pulsarService.getManagedLedgerClientFactory(); + (ManagedLedgerClientFactory) pulsarService.getManagedLedgerStorage(); Map bkPlacementPolicyToBkClientMap = mlFactory .getBkEnsemblePolicyToBookKeeperMap(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 17209c83c13ea..e05bb836a3ce6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -76,8 +76,8 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; import org.apache.bookkeeper.mledger.impl.NonAppendableLedgerOffloader; +import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; import org.apache.http.HttpResponse; import org.apache.http.client.methods.HttpGet; import org.apache.http.impl.client.CloseableHttpClient; @@ -1451,7 +1451,7 @@ public void testLedgerOpenFailureShouldNotHaveDeadLock() throws Exception { ledgerField.setAccessible(true); @SuppressWarnings("unchecked") ConcurrentHashMap> ledgers = (ConcurrentHashMap>) ledgerField - .get(pulsar.getManagedLedgerFactory()); + .get(pulsar.getDefaultManagedLedgerFactory()); CompletableFuture future = new CompletableFuture<>(); future.completeExceptionally(new ManagedLedgerException("ledger opening failed")); ledgers.put(namespace + "/persistent/deadLockTestTopic", future); @@ -1517,8 +1517,7 @@ public void testStuckTopicUnloading() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); - ManagedLedgerFactoryImpl mlFactory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerClientFactory() - .getManagedLedgerFactory(); + ManagedLedgerFactoryImpl mlFactory = (ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory(); Field ledgersField = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers"); ledgersField.setAccessible(true); ConcurrentHashMap> ledgers = (ConcurrentHashMap>) ledgersField 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 000ea7af91525..69f3e2e4d3917 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 @@ -61,6 +61,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.pulsar.broker.ServiceConfiguration; @@ -184,10 +185,11 @@ void setupMLAsyncCallbackMocks() { doReturn("mockCursor").when(cursorMock).getName(); // call openLedgerComplete with ledgerMock on ML factory asyncOpen + ManagedLedgerFactory managedLedgerFactory = pulsarTestContext.getDefaultManagedLedgerFactory(); doAnswer(invocationOnMock -> { ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -196,7 +198,7 @@ void setupMLAsyncCallbackMocks() { ((OpenLedgerCallback) invocationOnMock.getArguments()[2]) .openLedgerFailed(new ManagedLedgerException("Managed ledger failure"), null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java index 85e0887465db2..f75a32567473d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicConcurrentTest.java @@ -92,7 +92,7 @@ public void setup(Method m) throws Exception { cursorMock = ledger.openCursor("c1"); ledgerMock = ledger; mlFactoryMock = factory; - doReturn(mlFactoryMock).when(pulsar).getManagedLedgerFactory(); + doReturn(mlFactoryMock).when(pulsar).getDefaultManagedLedgerFactory(); brokerService = spyWithClassAndConstructorArgs(BrokerService.class, pulsar, eventLoopGroup); doReturn(brokerService).when(pulsar).getBrokerService(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java index 36e741f8fa9cd..2896c13af0093 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java @@ -596,7 +596,7 @@ public void testUnloadNamespace() throws Exception { pulsarClient.close(); assertNotNull(pulsar.getBrokerService().getTopicReference(topic)); - assertTrue(((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getManagedLedgers() + assertTrue(((ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory()).getManagedLedgers() .containsKey(topicName.getPersistenceNamingEncoding())); admin.namespaces().unload("prop/ns-abc"); @@ -613,7 +613,7 @@ public void testUnloadNamespace() throws Exception { } // ML should have been closed as well - assertFalse(((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getManagedLedgers() + assertFalse(((ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory()).getManagedLedgers() .containsKey(topicName.getPersistenceNamingEncoding())); } 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 81c12df4f3918..1e96da737dd51 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 @@ -90,6 +90,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; @@ -169,6 +170,7 @@ public class PersistentTopicTest extends MockedBookKeeperTestCase { private BrokerService brokerService; private EventLoopGroup eventLoopGroup; + private ManagedLedgerFactory managedLedgerFactory; @BeforeMethod(alwaysRun = true) public void setup() throws Exception { @@ -190,13 +192,14 @@ public void setup() throws Exception { .build(); brokerService = pulsarTestContext.getBrokerService(); + managedLedgerFactory = pulsarTestContext.getDefaultManagedLedgerFactory(); doAnswer(invocationOnMock -> CompletableFuture.completedFuture(null)) - .when(pulsarTestContext.getManagedLedgerFactory()).getManagedLedgerPropertiesAsync(any()); + .when(managedLedgerFactory).getManagedLedgerPropertiesAsync(any()); doAnswer(invocation -> { DeleteLedgerCallback deleteLedgerCallback = invocation.getArgument(1); deleteLedgerCallback.deleteLedgerComplete(null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()).asyncDelete(any(), any(), any()); + }).when(managedLedgerFactory).asyncDelete(any(), any(), any()); // Mock serviceCnx. serverCnx = spyWithClassAndConstructorArgsRecordingInvocations(ServerCnx.class, pulsarTestContext.getPulsarService()); @@ -247,7 +250,7 @@ public void testCreateTopic() { doAnswer(invocationOnMock -> { ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(anyString(), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -273,7 +276,7 @@ public void testCreateTopicMLFailure() { .openLedgerFailed(new ManagedLedgerException("Managed ledger failure"), null)).start(); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(anyString(), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -1395,7 +1398,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { doAnswer(invocationOnMock -> { ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -1404,7 +1407,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { ((OpenLedgerCallback) invocationOnMock.getArguments()[2]) .openLedgerFailed(new ManagedLedgerException("Managed ledger failure"), null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java index 3caf4a1f2398c..bd4a0889c730f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java @@ -187,14 +187,14 @@ public void testTxnLogNotBeReplicated() throws Exception { for (int i = 0; i < txnLogPartitions; i++) { TopicName txnLog = TopicName.get(TopicDomain.persistent.value(), NamespaceName.SYSTEM_NAMESPACE, TRANSACTION_LOG_PREFIX + i); - assertNotNull(pulsar1.getManagedLedgerFactory() + assertNotNull(pulsar1.getDefaultManagedLedgerFactory() .getManagedLedgerInfo(txnLog.getPersistenceNamingEncoding())); assertFalse(broker1.getTopics().containsKey(txnLog.toString())); } // __transaction_pending_ack: it only uses ML, will not create topic. TopicName pendingAck = TopicName.get( MLPendingAckStore.getTransactionPendingAckStoreSuffix(topic, subscription)); - assertNotNull(pulsar1.getManagedLedgerFactory() + assertNotNull(pulsar1.getDefaultManagedLedgerFactory() .getManagedLedgerInfo(pendingAck.getPersistenceNamingEncoding())); assertFalse(broker1.getTopics().containsKey(pendingAck.toString())); // __transaction_buffer_snapshot. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java index aac7a85f477c5..2420ed58bed27 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java @@ -1419,8 +1419,7 @@ public void testCleanupTopic() throws Exception { config1.setTopicLoadTimeoutSeconds(topicLoadTimeoutSeconds); config2.setTopicLoadTimeoutSeconds(topicLoadTimeoutSeconds); - ManagedLedgerFactoryImpl mlFactory = (ManagedLedgerFactoryImpl) pulsar1.getManagedLedgerClientFactory() - .getManagedLedgerFactory(); + ManagedLedgerFactoryImpl mlFactory = (ManagedLedgerFactoryImpl) pulsar1.getDefaultManagedLedgerFactory(); Field ledgersField = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers"); ledgersField.setAccessible(true); ConcurrentHashMap> ledgers = (ConcurrentHashMap>) ledgersField diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index 42b52d901e32f..9a85995ab771f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -83,6 +83,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.commons.lang3.mutable.MutableInt; @@ -202,6 +203,7 @@ public class ServerCnxTest { private ManagedLedger ledgerMock; private ManagedCursor cursorMock; private ConcurrentHashSet channelsStoppedAnswerHealthCheck = new ConcurrentHashSet<>(); + private ManagedLedgerFactory managedLedgerFactory; @BeforeMethod(alwaysRun = true) @@ -218,7 +220,7 @@ public void setup() throws Exception { .spyByDefault() .build(); pulsar = pulsarTestContext.getPulsarService(); - + managedLedgerFactory = pulsarTestContext.getDefaultManagedLedgerFactory(); brokerService = pulsarTestContext.getBrokerService(); namespaceService = pulsar.getNamespaceService(); @@ -2043,7 +2045,7 @@ public void testCreateProducerTimeout() throws Exception { () -> ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null)); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2098,7 +2100,7 @@ public void testCreateProducerTimeoutThenCreateSameNamedProducerShouldFail() thr () -> ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null)); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2165,7 +2167,7 @@ public void testCreateProducerMultipleTimeouts() throws Exception { ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2244,7 +2246,7 @@ public void testCreateProducerBookieTimeout() throws Exception { () -> ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null)); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2316,7 +2318,7 @@ public void testSubscribeTimeout() throws Exception { null)); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2391,7 +2393,7 @@ public void testSubscribeBookieTimeout() throws Exception { () -> ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null)); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2400,7 +2402,7 @@ public void testSubscribeBookieTimeout() throws Exception { openTopicFail.complete(() -> ((OpenLedgerCallback) invocationOnMock.getArguments()[2]) .openLedgerFailed(new ManagedLedgerException("Managed ledger failure"), null)); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2926,7 +2928,7 @@ private void setupMLAsyncCallbackMocks() { Thread.sleep(300); ((OpenLedgerCallback) invocationOnMock.getArguments()[2]).openLedgerComplete(ledgerMock, null); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); @@ -2937,7 +2939,7 @@ private void setupMLAsyncCallbackMocks() { .openLedgerFailed(new ManagedLedgerException("Managed ledger failure"), null)).start(); return null; - }).when(pulsarTestContext.getManagedLedgerFactory()) + }).when(managedLedgerFactory) .asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any(Supplier.class), any()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMarkerDeleteTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMarkerDeleteTest.java index 7e8454f6c7eef..fc10d315cb14a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMarkerDeleteTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionMarkerDeleteTest.java @@ -70,7 +70,7 @@ protected void cleanup() throws Exception { @Test public void testMarkerDeleteTimes() throws Exception { ManagedLedgerImpl managedLedger = - spy((ManagedLedgerImpl) getPulsarServiceList().get(0).getManagedLedgerFactory().open("test")); + spy((ManagedLedgerImpl) getPulsarServiceList().get(0).getDefaultManagedLedgerFactory().open("test")); PersistentTopic topic = mock(PersistentTopic.class); BrokerService brokerService = mock(BrokerService.class); PulsarService pulsarService = mock(PulsarService.class); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java index d0fd384ba78fb..d72e8f75427bd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ManagedLedgerMetricsTest.java @@ -105,7 +105,7 @@ public void testManagedLedgerMetrics() throws Exception { producer.send(message.getBytes()); } - var managedLedgerFactory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory(); + var managedLedgerFactory = (ManagedLedgerFactoryImpl) pulsar.getDefaultManagedLedgerFactory(); for (Entry ledger : managedLedgerFactory.getManagedLedgers().entrySet()) { ManagedLedgerMBeanImpl stats = (ManagedLedgerMBeanImpl) ledger.getValue().getStats(); stats.refreshStats(1, TimeUnit.SECONDS); @@ -205,7 +205,7 @@ public void testTransactionTopic() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); managedLedgerConfig.setMaxEntriesPerLedger(2); MLTransactionLogImpl mlTransactionLog = new MLTransactionLogImpl(TransactionCoordinatorID.get(0), - pulsar.getManagedLedgerFactory(), managedLedgerConfig, txnLogBufferedWriterConfig, + pulsar.getDefaultManagedLedgerFactory(), managedLedgerConfig, txnLogBufferedWriterConfig, transactionTimer, DISABLED_BUFFERED_WRITER_METRICS); mlTransactionLog.initialize().get(2, TimeUnit.SECONDS); ManagedLedgerMetrics metrics = new ManagedLedgerMetrics(pulsar); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/NonStartableTestPulsarService.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/NonStartableTestPulsarService.java index 7860b0708e35e..70e386c68aa26 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/NonStartableTestPulsarService.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/NonStartableTestPulsarService.java @@ -71,7 +71,7 @@ public NonStartableTestPulsarService(SpyConfig spyConfig, ServiceConfiguration c super(spyConfig, config, localMetadataStore, configurationMetadataStore, compactionServiceFactory, brokerInterceptor, bookKeeperClientFactory, null); setPulsarResources(pulsarResources); - setManagedLedgerClientFactory(managedLedgerClientFactory); + setManagedLedgerStorage(managedLedgerClientFactory); try { setBrokerService(brokerServiceCustomizer.apply( spyConfig.getBrokerService().spy(TestBrokerService.class, this, getIoEventLoopGroup()))); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java index 3d79a17a90f50..cdb047079bfcd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/PulsarTestContext.java @@ -28,6 +28,7 @@ import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Optional; import java.util.function.Consumer; @@ -56,7 +57,9 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.ServerCnx; import org.apache.pulsar.broker.stats.BrokerOpenTelemetryTestUtil; +import org.apache.pulsar.broker.storage.BookkeeperManagedLedgerStorageClass; import org.apache.pulsar.broker.storage.ManagedLedgerStorage; +import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; import org.apache.pulsar.common.util.GracefulExecutorServicesShutdown; import org.apache.pulsar.common.util.PortManager; import org.apache.pulsar.compaction.CompactionServiceFactory; @@ -136,7 +139,7 @@ public class PulsarTestContext implements AutoCloseable { private final OrderedExecutor executor; - private final ManagedLedgerStorage managedLedgerClientFactory; + private final ManagedLedgerStorage managedLedgerStorage; private final PulsarService pulsarService; @@ -167,8 +170,12 @@ public class PulsarTestContext implements AutoCloseable { private final boolean enableOpenTelemetry; private final InMemoryMetricReader openTelemetryMetricReader; - public ManagedLedgerFactory getManagedLedgerFactory() { - return managedLedgerClientFactory.getManagedLedgerFactory(); + public ManagedLedgerStorage getManagedLedgerStorage() { + return managedLedgerStorage; + } + + public ManagedLedgerFactory getDefaultManagedLedgerFactory() { + return getManagedLedgerStorage().getDefaultStorageClass().getManagedLedgerFactory(); } public PulsarMockBookKeeper getMockBookKeeper() { @@ -524,8 +531,8 @@ public Builder useTestPulsarResources(MetadataStore metadataStore) { */ public Builder managedLedgerClients(BookKeeper bookKeeperClient, ManagedLedgerFactory managedLedgerFactory) { - return managedLedgerClientFactory( - PulsarTestContext.createManagedLedgerClientFactory(bookKeeperClient, managedLedgerFactory)); + return managedLedgerStorage( + PulsarTestContext.createManagedLedgerStorage(bookKeeperClient, managedLedgerFactory)); } /** @@ -569,6 +576,9 @@ public final PulsarTestContext build() { if (configOverrideCustomizer != null) { configOverrideCustomizer.accept(super.config); } + if (super.managedLedgerStorage != null && !MockUtil.isMock(super.managedLedgerStorage)) { + super.managedLedgerStorage = spyConfig.getManagedLedgerStorage().spy(super.managedLedgerStorage); + } initializeCommonPulsarServices(spyConfig); initializePulsarServices(spyConfig, this); if (pulsarServiceCustomizer != null) { @@ -622,7 +632,7 @@ protected void handlePreallocatePorts(ServiceConfiguration config) { } private void initializeCommonPulsarServices(SpyConfig spyConfig) { - if (super.bookKeeperClient == null && super.managedLedgerClientFactory == null) { + if (super.bookKeeperClient == null && super.managedLedgerStorage == null) { if (super.executor == null) { OrderedExecutor createdExecutor = OrderedExecutor.newBuilder().numThreads(1) .name(PulsarTestContext.class.getSimpleName() + "-executor").build(); @@ -645,8 +655,11 @@ private void initializeCommonPulsarServices(SpyConfig spyConfig) { }); bookKeeperClient(mockBookKeeper); } - if (super.bookKeeperClient == null && super.managedLedgerClientFactory != null) { - bookKeeperClient(super.managedLedgerClientFactory.getBookKeeperClient()); + if (super.bookKeeperClient == null && super.managedLedgerStorage != null) { + bookKeeperClient(super.managedLedgerStorage.getStorageClasses().stream() + .filter(BookkeeperManagedLedgerStorageClass.class::isInstance) + .map(BookkeeperManagedLedgerStorageClass.class::cast) + .map(BookkeeperManagedLedgerStorageClass::getBookKeeperClient).findFirst().get()); } if (super.localMetadataStore == null || super.configurationMetadataStore == null) { if (super.mockZooKeeper != null) { @@ -725,8 +738,8 @@ static class StartableCustomBuilder extends AbstractCustomBuilder { } @Override - public Builder managedLedgerClientFactory(ManagedLedgerStorage managedLedgerClientFactory) { - throw new IllegalStateException("Cannot set managedLedgerClientFactory when startable."); + public Builder managedLedgerStorage(ManagedLedgerStorage managedLedgerStorage) { + throw new IllegalStateException("Cannot set managedLedgerStorage when startable."); } @Override @@ -788,10 +801,12 @@ static class NonStartableCustomBuilder extends AbstractCustomBuilder { @Override protected void initializePulsarServices(SpyConfig spyConfig, Builder builder) { - if (builder.managedLedgerClientFactory == null) { + if (builder.managedLedgerStorage == null) { ManagedLedgerFactory mlFactoryMock = Mockito.mock(ManagedLedgerFactory.class); - managedLedgerClientFactory( - PulsarTestContext.createManagedLedgerClientFactory(builder.bookKeeperClient, mlFactoryMock)); + managedLedgerStorage( + spyConfig.getManagedLedgerStorage() + .spy(PulsarTestContext.createManagedLedgerStorage(builder.bookKeeperClient, + mlFactoryMock))); } if (builder.pulsarResources == null) { SpyConfig.SpyType spyConfigPulsarResources = spyConfig.getPulsarResources(); @@ -825,7 +840,7 @@ protected void initializePulsarServices(SpyConfig spyConfig, Builder builder) { builder.configurationMetadataStore, compactionServiceFactory, builder.brokerInterceptor, bookKeeperClientFactory, builder.pulsarResources, - builder.managedLedgerClientFactory, builder.brokerServiceCustomizer); + builder.managedLedgerStorage, builder.brokerServiceCustomizer); if (compactionServiceFactory != null) { compactionServiceFactory.initialize(pulsarService); } @@ -838,10 +853,31 @@ protected void initializePulsarServices(SpyConfig spyConfig, Builder builder) { } @NotNull - private static ManagedLedgerStorage createManagedLedgerClientFactory(BookKeeper bookKeeperClient, - ManagedLedgerFactory managedLedgerFactory) { - return new ManagedLedgerStorage() { + private static ManagedLedgerStorage createManagedLedgerStorage(BookKeeper bookKeeperClient, + ManagedLedgerFactory managedLedgerFactory) { + BookkeeperManagedLedgerStorageClass managedLedgerStorageClass = + new BookkeeperManagedLedgerStorageClass() { + @Override + public String getName() { + return "bookkeeper"; + } + + @Override + public ManagedLedgerFactory getManagedLedgerFactory() { + return managedLedgerFactory; + } + + @Override + public StatsProvider getStatsProvider() { + return new NullStatsProvider(); + } + @Override + public BookKeeper getBookKeeperClient() { + return bookKeeperClient; + } + }; + return new ManagedLedgerStorage() { @Override public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadataStore, BookKeeperClientFactory bookkeeperProvider, EventLoopGroup eventLoopGroup, @@ -849,18 +885,17 @@ public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadata } @Override - public ManagedLedgerFactory getManagedLedgerFactory() { - return managedLedgerFactory; - } - - @Override - public StatsProvider getStatsProvider() { - return new NullStatsProvider(); + public Collection getStorageClasses() { + return List.of(managedLedgerStorageClass); } @Override - public BookKeeper getBookKeeperClient() { - return bookKeeperClient; + public Optional getManagedLedgerStorageClass(String name) { + if (name == null || name.equals("bookkeeper")) { + return Optional.of(managedLedgerStorageClass); + } else { + return Optional.empty(); + } } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/SpyConfig.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/SpyConfig.java index 64789d1f0d487..285eb1bba6d6e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/SpyConfig.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/SpyConfig.java @@ -112,6 +112,8 @@ public T spy(Class clazz, Object... args) { */ private final SpyType namespaceService; + private final SpyType managedLedgerStorage; + /** * Create a builder for SpyConfig with no spies by default. * @@ -141,5 +143,6 @@ public static void configureDefaults(Builder spyConfigBuilder, SpyType defaultSp spyConfigBuilder.compactor(defaultSpyType); spyConfigBuilder.compactedServiceFactory(defaultSpyType); spyConfigBuilder.namespaceService(defaultSpyType); + spyConfigBuilder.managedLedgerStorage(defaultSpyType); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/StartableTestPulsarService.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/StartableTestPulsarService.java index a0774414492dc..d82cd69c83dd0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/StartableTestPulsarService.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/testcontext/StartableTestPulsarService.java @@ -23,6 +23,7 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; +import lombok.SneakyThrows; import org.apache.pulsar.broker.BookKeeperClientFactory; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -30,6 +31,7 @@ import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.storage.ManagedLedgerStorage; import org.apache.pulsar.compaction.CompactionServiceFactory; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @@ -62,4 +64,23 @@ protected BrokerService newBrokerService(PulsarService pulsar) throws Exception public Supplier getNamespaceServiceProvider() throws PulsarServerException { return () -> spyConfig.getNamespaceService().spy(NamespaceService.class, this); } + + @SneakyThrows + @Override + public ManagedLedgerStorage getManagedLedgerStorage() { + // support adding spy to managedLedgerStorage in beforePulsarStart method + if (super.getManagedLedgerStorage() == null) { + setManagedLedgerStorage(createManagedLedgerStorageSpy()); + } + return super.getManagedLedgerStorage(); + } + + @Override + protected ManagedLedgerStorage newManagedLedgerStorage() throws Exception { + return getManagedLedgerStorage(); + } + + private ManagedLedgerStorage createManagedLedgerStorageSpy() throws Exception { + return spyConfig.getManagedLedgerStorage().spy(super.newManagedLedgerStorage()); + } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index f21e11b980209..14cc813a17ddd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -806,7 +806,7 @@ public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Ob // } }; - pulsarService.getManagedLedgerFactory() + pulsarService.getDefaultManagedLedgerFactory() .asyncOpenReadOnlyManagedLedger(snapshotSegmentTopicName.getPersistenceNamingEncoding(), callback, brokerService.getManagedLedgerConfig(snapshotSegmentTopicName).get(),null); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java index 14b1d563c11ec..3d7ab902bf494 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionProduceTest.java @@ -247,7 +247,7 @@ private ReadOnlyCursor getOriginTopicCursor(String topic, int partition) { if (partition >= 0) { topic = TopicName.get(topic).toString() + TopicName.PARTITIONED_TOPIC_SUFFIX + partition; } - return getPulsarServiceList().get(0).getManagedLedgerFactory().openReadOnlyCursor( + return getPulsarServiceList().get(0).getDefaultManagedLedgerFactory().openReadOnlyCursor( TopicName.get(topic).getPersistenceNamingEncoding(), PositionFactory.EARLIEST, new ManagedLedgerConfig()); } catch (Exception e) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index 5480b1a21d5a0..35c9048ebb554 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -510,7 +510,7 @@ public void testSubscriptionRecreateTopic() admin.topics().createNonPartitionedTopic(topic); PulsarService pulsarService = super.getPulsarServiceList().get(0); pulsarService.getBrokerService().getTopics().clear(); - ManagedLedgerFactory managedLedgerFactory = pulsarService.getBrokerService().getManagedLedgerFactory(); + ManagedLedgerFactory managedLedgerFactory = pulsarService.getDefaultManagedLedgerFactory(); Field field = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers"); field.setAccessible(true); ConcurrentHashMap> ledgers = diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java index 72aa078d5da1e..fc6a10e385a54 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/PendingAckPersistentTest.java @@ -23,7 +23,7 @@ import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -45,6 +45,7 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.api.BKException; @@ -952,8 +953,14 @@ public void testGetManagedLegerConfigFailThenUnload() throws Exception { assertNotNull(persistentTopic); BrokerService brokerService = spy(persistentTopic.getBrokerService()); - doReturn(FutureUtil.failedFuture(new BrokerServiceException.ServiceUnitNotReadyException("test"))) - .when(brokerService).getManagedLedgerConfig(any()); + AtomicBoolean isGetManagedLedgerConfigFail = new AtomicBoolean(false); + doAnswer(invocation -> { + if (isGetManagedLedgerConfigFail.get()) { + return FutureUtil.failedFuture(new BrokerServiceException.ServiceUnitNotReadyException("test")); + } else { + return invocation.callRealMethod(); + } + }).when(brokerService).getManagedLedgerConfig(any()); Field field = AbstractTopic.class.getDeclaredField("brokerService"); field.setAccessible(true); field.set(persistentTopic, brokerService); @@ -968,11 +975,13 @@ public void testGetManagedLegerConfigFailThenUnload() throws Exception { producer.send("test"); Transaction transaction = pulsarClient.newTransaction() - .withTransactionTimeout(30, TimeUnit.SECONDS).build().get(); + .withTransactionTimeout(10, TimeUnit.SECONDS).build().get(); + isGetManagedLedgerConfigFail.set(true); // pending ack init fail, so the ack will throw exception try { consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); + fail("ack should fail"); } catch (Exception e) { assertTrue(e.getCause() instanceof PulsarClientException.LookupException); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java index 9396a80cf2557..6f79c573ed3d0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java @@ -291,7 +291,7 @@ public void testTopicLoadAndDeleteAtTheSameTime(boolean injectTimeout) throws Ex Thread.sleep(10 * 1000); } log.info("Race condition occurs {} times", mockRaceConditionCounter.get()); - pulsar.getManagedLedgerFactory().delete(TopicName.get(tpName).getPersistenceNamingEncoding()); + pulsar.getDefaultManagedLedgerFactory().delete(TopicName.get(tpName).getPersistenceNamingEncoding()); } return invocation.callRealMethod(); }).when(namespaceService).isServiceUnitActiveAsync(any(TopicName.class)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index 2e71e8cc28c3e..e76c3d8fb845d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -91,8 +91,13 @@ import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.ServerCnx; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.storage.ManagedLedgerStorage; +import org.apache.pulsar.broker.storage.ManagedLedgerStorageClass; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.broker.testcontext.SpyConfig; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.client.impl.BatchMessageIdImpl; @@ -150,6 +155,14 @@ protected void setup() throws Exception { super.producerBaseSetup(); } + @Override + protected PulsarTestContext.Builder createPulsarTestContextBuilder(ServiceConfiguration conf) { + return super.createPulsarTestContextBuilder(conf) + .spyConfig(SpyConfig.builder() + .managedLedgerStorage(SpyConfig.SpyType.SPY_ALSO_INVOCATIONS) + .build()); + } + @AfterMethod(alwaysRun = true) public void cleanupAfterMethod() throws Exception { try { @@ -1097,18 +1110,25 @@ public void testSendBigMessageSizeButCompressed() throws Exception { } + @Override protected void beforePulsarStart(PulsarService pulsar) throws Exception { super.beforePulsarStart(pulsar); - doAnswer(i0 -> { - ManagedLedgerFactory factory = (ManagedLedgerFactory) spy(i0.callRealMethod()); - doAnswer(i1 -> { - EntryCacheManager manager = (EntryCacheManager) spy(i1.callRealMethod()); - doAnswer(i2 -> spy(i2.callRealMethod())).when(manager).getEntryCache(any()); - return manager; - }).when(factory).getEntryCacheManager(); - return factory; - }).when(pulsar).getManagedLedgerFactory(); + ManagedLedgerStorage managedLedgerStorage = pulsar.getManagedLedgerStorage(); + doAnswer(invocation -> { + ManagedLedgerStorageClass managedLedgerStorageClass = + (ManagedLedgerStorageClass) spy(invocation.callRealMethod()); + doAnswer(i0 -> { + ManagedLedgerFactory factory = (ManagedLedgerFactory) spy(i0.callRealMethod()); + doAnswer(i1 -> { + EntryCacheManager manager = (EntryCacheManager) spy(i1.callRealMethod()); + doAnswer(i2 -> spy(i2.callRealMethod())).when(manager).getEntryCache(any()); + return manager; + }).when(factory).getEntryCacheManager(); + return factory; + }).when(managedLedgerStorageClass).getManagedLedgerFactory(); + return managedLedgerStorageClass; + }).when(managedLedgerStorage).getDefaultStorageClass(); } /** @@ -1126,6 +1146,7 @@ protected void beforePulsarStart(PulsarService pulsar) throws Exception { public void testActiveAndInActiveConsumerEntryCacheBehavior() throws Exception { log.info("-- Starting {} test --", methodName); + final long batchMessageDelayMs = 100; final int receiverSize = 10; final String topicName = "cache-topic"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java index 36c36735c067e..390e81ad664f9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java @@ -563,7 +563,7 @@ public void testNeverCallCursorIsCursorDataFullyPersistableIfDisabledTheFeature( final String subscription = "s1"; final int msgSendCount = 100; // Inject a injection to record the counter of calling "cursor.isCursorDataFullyPersistable". - final ManagedLedgerImpl ml = (ManagedLedgerImpl) pulsar.getBrokerService().getManagedLedgerFactory().open(mlName); + final ManagedLedgerImpl ml = (ManagedLedgerImpl) pulsar.getDefaultManagedLedgerFactory().open(mlName); final ManagedCursorImpl cursor = (ManagedCursorImpl) ml.openCursor(subscription); final ManagedCursorImpl spyCursor = Mockito.spy(cursor); AtomicInteger callingIsCursorDataFullyPersistableCounter = new AtomicInteger(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/v1/V1_ProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/v1/V1_ProducerConsumerTest.java index d3cb1d60d37ed..0b3ff345acfc3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/v1/V1_ProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/v1/V1_ProducerConsumerTest.java @@ -624,7 +624,7 @@ protected void beforePulsarStart(PulsarService pulsar) throws Exception { return manager; }).when(factory).getEntryCacheManager(); return factory; - }).when(pulsar).getManagedLedgerFactory(); + }).when(pulsar).getDefaultManagedLedgerFactory(); } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SequenceIdWithErrorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SequenceIdWithErrorTest.java index 1395424b14123..2b1b409b71ce8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SequenceIdWithErrorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/SequenceIdWithErrorTest.java @@ -62,7 +62,7 @@ public void testCheckSequenceId() throws Exception { ManagedLedgerClientFactory clientFactory = new ManagedLedgerClientFactory(); clientFactory.initialize(pulsar.getConfiguration(), pulsar.getLocalMetadataStore(), pulsar.getBookKeeperClientFactory(), eventLoopGroup, OpenTelemetry.noop()); - ManagedLedgerFactory mlFactory = clientFactory.getManagedLedgerFactory(); + ManagedLedgerFactory mlFactory = clientFactory.getDefaultStorageClass().getManagedLedgerFactory(); ManagedLedger ml = mlFactory.open(TopicName.get(topicName).getPersistenceNamingEncoding()); ml.close(); clientFactory.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java index 19f42a7e0570f..d75ccce7ff39c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java @@ -927,7 +927,7 @@ public void testCompactorReadsCompacted() throws Exception { // verify second ledger created String managedLedgerName = ((PersistentTopic)pulsar.getBrokerService().getTopicReference(topic).get()) .getManagedLedger().getName(); - ManagedLedgerInfo info = pulsar.getManagedLedgerFactory().getManagedLedgerInfo(managedLedgerName); + ManagedLedgerInfo info = pulsar.getDefaultManagedLedgerFactory().getManagedLedgerInfo(managedLedgerName); Assert.assertEquals(info.ledgers.size(), 2); Assert.assertTrue(ledgersOpened.isEmpty()); // no ledgers should have been opened @@ -950,7 +950,7 @@ public void testCompactorReadsCompacted() throws Exception { .send(); } - info = pulsar.getManagedLedgerFactory().getManagedLedgerInfo(managedLedgerName); + info = pulsar.getDefaultManagedLedgerFactory().getManagedLedgerInfo(managedLedgerName); Assert.assertEquals(info.ledgers.size(), 3); // should only have opened the penultimate ledger to get stat diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PersistencePolicies.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PersistencePolicies.java index df4e086748f30..3fbc91e7d2eaa 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PersistencePolicies.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/PersistencePolicies.java @@ -30,17 +30,24 @@ public class PersistencePolicies { private int bookkeeperWriteQuorum; private int bookkeeperAckQuorum; private double managedLedgerMaxMarkDeleteRate; + private String managedLedgerStorageClassName; public PersistencePolicies() { - this(2, 2, 2, 0.0); + this(2, 2, 2, 0.0, null); } public PersistencePolicies(int bookkeeperEnsemble, int bookkeeperWriteQuorum, int bookkeeperAckQuorum, - double managedLedgerMaxMarkDeleteRate) { + double managedLedgerMaxMarkDeleteRate) { + this(bookkeeperEnsemble, bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate, null); + } + + public PersistencePolicies(int bookkeeperEnsemble, int bookkeeperWriteQuorum, int bookkeeperAckQuorum, + double managedLedgerMaxMarkDeleteRate, String managedLedgerStorageClassName) { this.bookkeeperEnsemble = bookkeeperEnsemble; this.bookkeeperWriteQuorum = bookkeeperWriteQuorum; this.bookkeeperAckQuorum = bookkeeperAckQuorum; this.managedLedgerMaxMarkDeleteRate = managedLedgerMaxMarkDeleteRate; + this.managedLedgerStorageClassName = managedLedgerStorageClassName; } public int getBookkeeperEnsemble() { @@ -59,10 +66,14 @@ public double getManagedLedgerMaxMarkDeleteRate() { return managedLedgerMaxMarkDeleteRate; } + public String getManagedLedgerStorageClassName() { + return managedLedgerStorageClassName; + } + @Override public int hashCode() { return Objects.hash(bookkeeperEnsemble, bookkeeperWriteQuorum, - bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate); + bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate, managedLedgerStorageClassName); } @Override public boolean equals(Object obj) { @@ -71,7 +82,8 @@ public boolean equals(Object obj) { return bookkeeperEnsemble == other.bookkeeperEnsemble && bookkeeperWriteQuorum == other.bookkeeperWriteQuorum && bookkeeperAckQuorum == other.bookkeeperAckQuorum - && managedLedgerMaxMarkDeleteRate == other.managedLedgerMaxMarkDeleteRate; + && managedLedgerMaxMarkDeleteRate == other.managedLedgerMaxMarkDeleteRate + && managedLedgerStorageClassName == other.managedLedgerStorageClassName; } return false; diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java index e8e644b688029..8adedcd14ac40 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java @@ -1379,6 +1379,11 @@ private class SetPersistence extends CliCommand { description = "Throttling rate of mark-delete operation (0 means no throttle)") private double managedLedgerMaxMarkDeleteRate = 0; + @Option(names = { "-c", + "--ml-storage-class" }, + description = "Managed ledger storage class name") + private String managedLedgerStorageClassName; + @Override void run() throws PulsarAdminException { String namespace = validateNamespace(namespaceName); @@ -1390,7 +1395,8 @@ void run() throws PulsarAdminException { throw new ParameterException("[--ml-mark-delete-max-rate] cannot less than 0."); } getAdmin().namespaces().setPersistence(namespace, new PersistencePolicies(bookkeeperEnsemble, - bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate)); + bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate, + managedLedgerStorageClassName)); } } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopicPolicies.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopicPolicies.java index 3cc72db2e95f1..10850d107edf5 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopicPolicies.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopicPolicies.java @@ -1197,6 +1197,11 @@ private class SetPersistence extends CliCommand { + "If set to true, the policy will be replicate to other clusters asynchronously", arity = "0") private boolean isGlobal = false; + @Option(names = { "-c", + "--ml-storage-class" }, + description = "Managed ledger storage class name") + private String managedLedgerStorageClassName; + @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(topicName); @@ -1208,7 +1213,8 @@ void run() throws PulsarAdminException { throw new ParameterException("[--ml-mark-delete-max-rate] cannot less than 0."); } getTopicPolicies(isGlobal).setPersistence(persistentTopic, new PersistencePolicies(bookkeeperEnsemble, - bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate)); + bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate, + managedLedgerStorageClassName)); } } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index 261bd81a5b7bd..955d6e13e1d04 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -2148,6 +2148,11 @@ private class SetPersistence extends CliCommand { + "(0 means no throttle)") private double managedLedgerMaxMarkDeleteRate = 0; + @Option(names = { "-c", + "--ml-storage-class" }, + description = "Managed ledger storage class name") + private String managedLedgerStorageClassName; + @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(topicName); @@ -2159,7 +2164,8 @@ void run() throws PulsarAdminException { throw new ParameterException("[--ml-mark-delete-max-rate] cannot less than 0."); } getTopics().setPersistence(persistentTopic, new PersistencePolicies(bookkeeperEnsemble, - bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate)); + bookkeeperWriteQuorum, bookkeeperAckQuorum, managedLedgerMaxMarkDeleteRate, + managedLedgerStorageClassName)); } } From 06fc259ce568ba6f1dfdee82b2981952357f6e72 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 8 Oct 2024 13:28:16 +0300 Subject: [PATCH 563/580] [improve][build] Update maven-wrapper (mvnw) to recent stable version 3.3.2 (#23410) --- .mvn/wrapper/maven-wrapper.properties | 9 +- mvnw | 457 +++++++++++--------------- mvnw.cmd | 287 +++++++--------- 3 files changed, 329 insertions(+), 424 deletions(-) diff --git a/.mvn/wrapper/maven-wrapper.properties b/.mvn/wrapper/maven-wrapper.properties index db95c131dde6f..d58dfb70bab56 100644 --- a/.mvn/wrapper/maven-wrapper.properties +++ b/.mvn/wrapper/maven-wrapper.properties @@ -5,14 +5,15 @@ # 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. -distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.8.5/apache-maven-3.8.5-bin.zip -wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar +wrapperVersion=3.3.2 +distributionType=only-script +distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.9.9/apache-maven-3.9.9-bin.zip diff --git a/mvnw b/mvnw index 5643201c7d822..19529ddf8c6ea 100755 --- a/mvnw +++ b/mvnw @@ -19,298 +19,241 @@ # ---------------------------------------------------------------------------- # ---------------------------------------------------------------------------- -# Maven Start Up Batch script -# -# Required ENV vars: -# ------------------ -# JAVA_HOME - location of a JDK home dir +# Apache Maven Wrapper startup batch script, version 3.3.2 # # Optional ENV vars # ----------------- -# M2_HOME - location of maven2's installed home dir -# MAVEN_OPTS - parameters passed to the Java VM when running Maven -# e.g. to debug Maven itself, use -# set MAVEN_OPTS=-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=8000 -# MAVEN_SKIP_RC - flag to disable loading of mavenrc files +# JAVA_HOME - location of a JDK home dir, required when download maven via java source +# MVNW_REPOURL - repo url base for downloading maven distribution +# MVNW_USERNAME/MVNW_PASSWORD - user and password for downloading maven +# MVNW_VERBOSE - true: enable verbose log; debug: trace the mvnw script; others: silence the output # ---------------------------------------------------------------------------- -if [ -z "$MAVEN_SKIP_RC" ] ; then - - if [ -f /usr/local/etc/mavenrc ] ; then - . /usr/local/etc/mavenrc - fi - - if [ -f /etc/mavenrc ] ; then - . /etc/mavenrc - fi +set -euf +[ "${MVNW_VERBOSE-}" != debug ] || set -x - if [ -f "$HOME/.mavenrc" ] ; then - . "$HOME/.mavenrc" - fi +# OS specific support. +native_path() { printf %s\\n "$1"; } +case "$(uname)" in +CYGWIN* | MINGW*) + [ -z "${JAVA_HOME-}" ] || JAVA_HOME="$(cygpath --unix "$JAVA_HOME")" + native_path() { cygpath --path --windows "$1"; } + ;; +esac -fi +# set JAVACMD and JAVACCMD +set_java_home() { + # For Cygwin and MinGW, ensure paths are in Unix format before anything is touched + if [ -n "${JAVA_HOME-}" ]; then + if [ -x "$JAVA_HOME/jre/sh/java" ]; then + # IBM's JDK on AIX uses strange locations for the executables + JAVACMD="$JAVA_HOME/jre/sh/java" + JAVACCMD="$JAVA_HOME/jre/sh/javac" + else + JAVACMD="$JAVA_HOME/bin/java" + JAVACCMD="$JAVA_HOME/bin/javac" -# OS specific support. $var _must_ be set to either true or false. -cygwin=false; -darwin=false; -mingw=false -case "`uname`" in - CYGWIN*) cygwin=true ;; - MINGW*) mingw=true;; - Darwin*) darwin=true - # Use /usr/libexec/java_home if available, otherwise fall back to /Library/Java/Home - # See https://developer.apple.com/library/mac/qa/qa1170/_index.html - if [ -z "$JAVA_HOME" ]; then - if [ -x "/usr/libexec/java_home" ]; then - export JAVA_HOME="`/usr/libexec/java_home`" - else - export JAVA_HOME="/Library/Java/Home" + if [ ! -x "$JAVACMD" ] || [ ! -x "$JAVACCMD" ]; then + echo "The JAVA_HOME environment variable is not defined correctly, so mvnw cannot run." >&2 + echo "JAVA_HOME is set to \"$JAVA_HOME\", but \"\$JAVA_HOME/bin/java\" or \"\$JAVA_HOME/bin/javac\" does not exist." >&2 + return 1 fi fi - ;; -esac - -if [ -z "$JAVA_HOME" ] ; then - if [ -r /etc/gentoo-release ] ; then - JAVA_HOME=`java-config --jre-home` + else + JAVACMD="$( + 'set' +e + 'unset' -f command 2>/dev/null + 'command' -v java + )" || : + JAVACCMD="$( + 'set' +e + 'unset' -f command 2>/dev/null + 'command' -v javac + )" || : + + if [ ! -x "${JAVACMD-}" ] || [ ! -x "${JAVACCMD-}" ]; then + echo "The java/javac command does not exist in PATH nor is JAVA_HOME set, so mvnw cannot run." >&2 + return 1 + fi fi -fi - -if [ -z "$M2_HOME" ] ; then - ## resolve links - $0 may be a link to maven's home - PRG="$0" +} - # need this for relative symlinks - while [ -h "$PRG" ] ; do - ls=`ls -ld "$PRG"` - link=`expr "$ls" : '.*-> \(.*\)$'` - if expr "$link" : '/.*' > /dev/null; then - PRG="$link" - else - PRG="`dirname "$PRG"`/$link" - fi +# hash string like Java String::hashCode +hash_string() { + str="${1:-}" h=0 + while [ -n "$str" ]; do + char="${str%"${str#?}"}" + h=$(((h * 31 + $(LC_CTYPE=C printf %d "'$char")) % 4294967296)) + str="${str#?}" done + printf %x\\n $h +} - saveddir=`pwd` +verbose() { :; } +[ "${MVNW_VERBOSE-}" != true ] || verbose() { printf %s\\n "${1-}"; } - M2_HOME=`dirname "$PRG"`/.. +die() { + printf %s\\n "$1" >&2 + exit 1 +} - # make it fully qualified - M2_HOME=`cd "$M2_HOME" && pwd` +trim() { + # MWRAPPER-139: + # Trims trailing and leading whitespace, carriage returns, tabs, and linefeeds. + # Needed for removing poorly interpreted newline sequences when running in more + # exotic environments such as mingw bash on Windows. + printf "%s" "${1}" | tr -d '[:space:]' +} - cd "$saveddir" - # echo Using m2 at $M2_HOME -fi +# parse distributionUrl and optional distributionSha256Sum, requires .mvn/wrapper/maven-wrapper.properties +while IFS="=" read -r key value; do + case "${key-}" in + distributionUrl) distributionUrl=$(trim "${value-}") ;; + distributionSha256Sum) distributionSha256Sum=$(trim "${value-}") ;; + esac +done <"${0%/*}/.mvn/wrapper/maven-wrapper.properties" +[ -n "${distributionUrl-}" ] || die "cannot read distributionUrl property in ${0%/*}/.mvn/wrapper/maven-wrapper.properties" + +case "${distributionUrl##*/}" in +maven-mvnd-*bin.*) + MVN_CMD=mvnd.sh _MVNW_REPO_PATTERN=/maven/mvnd/ + case "${PROCESSOR_ARCHITECTURE-}${PROCESSOR_ARCHITEW6432-}:$(uname -a)" in + *AMD64:CYGWIN* | *AMD64:MINGW*) distributionPlatform=windows-amd64 ;; + :Darwin*x86_64) distributionPlatform=darwin-amd64 ;; + :Darwin*arm64) distributionPlatform=darwin-aarch64 ;; + :Linux*x86_64*) distributionPlatform=linux-amd64 ;; + *) + echo "Cannot detect native platform for mvnd on $(uname)-$(uname -m), use pure java version" >&2 + distributionPlatform=linux-amd64 + ;; + esac + distributionUrl="${distributionUrl%-bin.*}-$distributionPlatform.zip" + ;; +maven-mvnd-*) MVN_CMD=mvnd.sh _MVNW_REPO_PATTERN=/maven/mvnd/ ;; +*) MVN_CMD="mvn${0##*/mvnw}" _MVNW_REPO_PATTERN=/org/apache/maven/ ;; +esac -# For Cygwin, ensure paths are in UNIX format before anything is touched -if $cygwin ; then - [ -n "$M2_HOME" ] && - M2_HOME=`cygpath --unix "$M2_HOME"` - [ -n "$JAVA_HOME" ] && - JAVA_HOME=`cygpath --unix "$JAVA_HOME"` - [ -n "$CLASSPATH" ] && - CLASSPATH=`cygpath --path --unix "$CLASSPATH"` -fi +# apply MVNW_REPOURL and calculate MAVEN_HOME +# maven home pattern: ~/.m2/wrapper/dists/{apache-maven-,maven-mvnd--}/ +[ -z "${MVNW_REPOURL-}" ] || distributionUrl="$MVNW_REPOURL$_MVNW_REPO_PATTERN${distributionUrl#*"$_MVNW_REPO_PATTERN"}" +distributionUrlName="${distributionUrl##*/}" +distributionUrlNameMain="${distributionUrlName%.*}" +distributionUrlNameMain="${distributionUrlNameMain%-bin}" +MAVEN_USER_HOME="${MAVEN_USER_HOME:-${HOME}/.m2}" +MAVEN_HOME="${MAVEN_USER_HOME}/wrapper/dists/${distributionUrlNameMain-}/$(hash_string "$distributionUrl")" + +exec_maven() { + unset MVNW_VERBOSE MVNW_USERNAME MVNW_PASSWORD MVNW_REPOURL || : + exec "$MAVEN_HOME/bin/$MVN_CMD" "$@" || die "cannot exec $MAVEN_HOME/bin/$MVN_CMD" +} -# For Mingw, ensure paths are in UNIX format before anything is touched -if $mingw ; then - [ -n "$M2_HOME" ] && - M2_HOME="`(cd "$M2_HOME"; pwd)`" - [ -n "$JAVA_HOME" ] && - JAVA_HOME="`(cd "$JAVA_HOME"; pwd)`" +if [ -d "$MAVEN_HOME" ]; then + verbose "found existing MAVEN_HOME at $MAVEN_HOME" + exec_maven "$@" fi -if [ -z "$JAVA_HOME" ]; then - javaExecutable="`which javac`" - if [ -n "$javaExecutable" ] && ! [ "`expr \"$javaExecutable\" : '\([^ ]*\)'`" = "no" ]; then - # readlink(1) is not available as standard on Solaris 10. - readLink=`which readlink` - if [ ! `expr "$readLink" : '\([^ ]*\)'` = "no" ]; then - if $darwin ; then - javaHome="`dirname \"$javaExecutable\"`" - javaExecutable="`cd \"$javaHome\" && pwd -P`/javac" - else - javaExecutable="`readlink -f \"$javaExecutable\"`" - fi - javaHome="`dirname \"$javaExecutable\"`" - javaHome=`expr "$javaHome" : '\(.*\)/bin'` - JAVA_HOME="$javaHome" - export JAVA_HOME - fi - fi -fi +case "${distributionUrl-}" in +*?-bin.zip | *?maven-mvnd-?*-?*.zip) ;; +*) die "distributionUrl is not valid, must match *-bin.zip or maven-mvnd-*.zip, but found '${distributionUrl-}'" ;; +esac -if [ -z "$JAVACMD" ] ; then - if [ -n "$JAVA_HOME" ] ; then - if [ -x "$JAVA_HOME/jre/sh/java" ] ; then - # IBM's JDK on AIX uses strange locations for the executables - JAVACMD="$JAVA_HOME/jre/sh/java" - else - JAVACMD="$JAVA_HOME/bin/java" - fi - else - JAVACMD="`\\unset -f command; \\command -v java`" - fi +# prepare tmp dir +if TMP_DOWNLOAD_DIR="$(mktemp -d)" && [ -d "$TMP_DOWNLOAD_DIR" ]; then + clean() { rm -rf -- "$TMP_DOWNLOAD_DIR"; } + trap clean HUP INT TERM EXIT +else + die "cannot create temp dir" fi -if [ ! -x "$JAVACMD" ] ; then - echo "Error: JAVA_HOME is not defined correctly." >&2 - echo " We cannot execute $JAVACMD" >&2 - exit 1 -fi +mkdir -p -- "${MAVEN_HOME%/*}" -if [ -z "$JAVA_HOME" ] ; then - echo "Warning: JAVA_HOME environment variable is not set." +# Download and Install Apache Maven +verbose "Couldn't find MAVEN_HOME, downloading and installing it ..." +verbose "Downloading from: $distributionUrl" +verbose "Downloading to: $TMP_DOWNLOAD_DIR/$distributionUrlName" + +# select .zip or .tar.gz +if ! command -v unzip >/dev/null; then + distributionUrl="${distributionUrl%.zip}.tar.gz" + distributionUrlName="${distributionUrl##*/}" fi -CLASSWORLDS_LAUNCHER=org.codehaus.plexus.classworlds.launcher.Launcher +# verbose opt +__MVNW_QUIET_WGET=--quiet __MVNW_QUIET_CURL=--silent __MVNW_QUIET_UNZIP=-q __MVNW_QUIET_TAR='' +[ "${MVNW_VERBOSE-}" != true ] || __MVNW_QUIET_WGET='' __MVNW_QUIET_CURL='' __MVNW_QUIET_UNZIP='' __MVNW_QUIET_TAR=v -# traverses directory structure from process work directory to filesystem root -# first directory with .mvn subdirectory is considered project base directory -find_maven_basedir() { +# normalize http auth +case "${MVNW_PASSWORD:+has-password}" in +'') MVNW_USERNAME='' MVNW_PASSWORD='' ;; +has-password) [ -n "${MVNW_USERNAME-}" ] || MVNW_USERNAME='' MVNW_PASSWORD='' ;; +esac - if [ -z "$1" ] - then - echo "Path not specified to find_maven_basedir" - return 1 - fi +if [ -z "${MVNW_USERNAME-}" ] && command -v wget >/dev/null; then + verbose "Found wget ... using wget" + wget ${__MVNW_QUIET_WGET:+"$__MVNW_QUIET_WGET"} "$distributionUrl" -O "$TMP_DOWNLOAD_DIR/$distributionUrlName" || die "wget: Failed to fetch $distributionUrl" +elif [ -z "${MVNW_USERNAME-}" ] && command -v curl >/dev/null; then + verbose "Found curl ... using curl" + curl ${__MVNW_QUIET_CURL:+"$__MVNW_QUIET_CURL"} -f -L -o "$TMP_DOWNLOAD_DIR/$distributionUrlName" "$distributionUrl" || die "curl: Failed to fetch $distributionUrl" +elif set_java_home; then + verbose "Falling back to use Java to download" + javaSource="$TMP_DOWNLOAD_DIR/Downloader.java" + targetZip="$TMP_DOWNLOAD_DIR/$distributionUrlName" + cat >"$javaSource" <<-END + public class Downloader extends java.net.Authenticator + { + protected java.net.PasswordAuthentication getPasswordAuthentication() + { + return new java.net.PasswordAuthentication( System.getenv( "MVNW_USERNAME" ), System.getenv( "MVNW_PASSWORD" ).toCharArray() ); + } + public static void main( String[] args ) throws Exception + { + setDefault( new Downloader() ); + java.nio.file.Files.copy( java.net.URI.create( args[0] ).toURL().openStream(), java.nio.file.Paths.get( args[1] ).toAbsolutePath().normalize() ); + } + } + END + # For Cygwin/MinGW, switch paths to Windows format before running javac and java + verbose " - Compiling Downloader.java ..." + "$(native_path "$JAVACCMD")" "$(native_path "$javaSource")" || die "Failed to compile Downloader.java" + verbose " - Running Downloader.java ..." + "$(native_path "$JAVACMD")" -cp "$(native_path "$TMP_DOWNLOAD_DIR")" Downloader "$distributionUrl" "$(native_path "$targetZip")" +fi - basedir="$1" - wdir="$1" - while [ "$wdir" != '/' ] ; do - if [ -d "$wdir"/.mvn ] ; then - basedir=$wdir - break +# If specified, validate the SHA-256 sum of the Maven distribution zip file +if [ -n "${distributionSha256Sum-}" ]; then + distributionSha256Result=false + if [ "$MVN_CMD" = mvnd.sh ]; then + echo "Checksum validation is not supported for maven-mvnd." >&2 + echo "Please disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." >&2 + exit 1 + elif command -v sha256sum >/dev/null; then + if echo "$distributionSha256Sum $TMP_DOWNLOAD_DIR/$distributionUrlName" | sha256sum -c >/dev/null 2>&1; then + distributionSha256Result=true fi - # workaround for JBEAP-8937 (on Solaris 10/Sparc) - if [ -d "${wdir}" ]; then - wdir=`cd "$wdir/.."; pwd` + elif command -v shasum >/dev/null; then + if echo "$distributionSha256Sum $TMP_DOWNLOAD_DIR/$distributionUrlName" | shasum -a 256 -c >/dev/null 2>&1; then + distributionSha256Result=true fi - # end of workaround - done - echo "${basedir}" -} - -# concatenates all lines of a file -concat_lines() { - if [ -f "$1" ]; then - echo "$(tr -s '\n' ' ' < "$1")" + else + echo "Checksum validation was requested but neither 'sha256sum' or 'shasum' are available." >&2 + echo "Please install either command, or disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." >&2 + exit 1 + fi + if [ $distributionSha256Result = false ]; then + echo "Error: Failed to validate Maven distribution SHA-256, your Maven distribution might be compromised." >&2 + echo "If you updated your Maven version, you need to update the specified distributionSha256Sum property." >&2 + exit 1 fi -} - -BASE_DIR=`find_maven_basedir "$(pwd)"` -if [ -z "$BASE_DIR" ]; then - exit 1; fi -########################################################################################## -# Extension to allow automatically downloading the maven-wrapper.jar from Maven-central -# This allows using the maven wrapper in projects that prohibit checking in binary data. -########################################################################################## -if [ -r "$BASE_DIR/.mvn/wrapper/maven-wrapper.jar" ]; then - if [ "$MVNW_VERBOSE" = true ]; then - echo "Found .mvn/wrapper/maven-wrapper.jar" - fi +# unzip and move +if command -v unzip >/dev/null; then + unzip ${__MVNW_QUIET_UNZIP:+"$__MVNW_QUIET_UNZIP"} "$TMP_DOWNLOAD_DIR/$distributionUrlName" -d "$TMP_DOWNLOAD_DIR" || die "failed to unzip" else - if [ "$MVNW_VERBOSE" = true ]; then - echo "Couldn't find .mvn/wrapper/maven-wrapper.jar, downloading it ..." - fi - if [ -n "$MVNW_REPOURL" ]; then - jarUrl="$MVNW_REPOURL/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar" - else - jarUrl="https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar" - fi - while IFS="=" read key value; do - case "$key" in (wrapperUrl) jarUrl="$value"; break ;; - esac - done < "$BASE_DIR/.mvn/wrapper/maven-wrapper.properties" - if [ "$MVNW_VERBOSE" = true ]; then - echo "Downloading from: $jarUrl" - fi - wrapperJarPath="$BASE_DIR/.mvn/wrapper/maven-wrapper.jar" - if $cygwin; then - wrapperJarPath=`cygpath --path --windows "$wrapperJarPath"` - fi - - if command -v wget > /dev/null; then - if [ "$MVNW_VERBOSE" = true ]; then - echo "Found wget ... using wget" - fi - if [ -z "$MVNW_USERNAME" ] || [ -z "$MVNW_PASSWORD" ]; then - wget "$jarUrl" -O "$wrapperJarPath" || rm -f "$wrapperJarPath" - else - wget --http-user=$MVNW_USERNAME --http-password=$MVNW_PASSWORD "$jarUrl" -O "$wrapperJarPath" || rm -f "$wrapperJarPath" - fi - elif command -v curl > /dev/null; then - if [ "$MVNW_VERBOSE" = true ]; then - echo "Found curl ... using curl" - fi - if [ -z "$MVNW_USERNAME" ] || [ -z "$MVNW_PASSWORD" ]; then - curl -o "$wrapperJarPath" "$jarUrl" -f - else - curl --user $MVNW_USERNAME:$MVNW_PASSWORD -o "$wrapperJarPath" "$jarUrl" -f - fi - - else - if [ "$MVNW_VERBOSE" = true ]; then - echo "Falling back to using Java to download" - fi - javaClass="$BASE_DIR/.mvn/wrapper/MavenWrapperDownloader.java" - # For Cygwin, switch paths to Windows format before running javac - if $cygwin; then - javaClass=`cygpath --path --windows "$javaClass"` - fi - if [ -e "$javaClass" ]; then - if [ ! -e "$BASE_DIR/.mvn/wrapper/MavenWrapperDownloader.class" ]; then - if [ "$MVNW_VERBOSE" = true ]; then - echo " - Compiling MavenWrapperDownloader.java ..." - fi - # Compiling the Java class - ("$JAVA_HOME/bin/javac" "$javaClass") - fi - if [ -e "$BASE_DIR/.mvn/wrapper/MavenWrapperDownloader.class" ]; then - # Running the downloader - if [ "$MVNW_VERBOSE" = true ]; then - echo " - Running MavenWrapperDownloader.java ..." - fi - ("$JAVA_HOME/bin/java" -cp .mvn/wrapper MavenWrapperDownloader "$MAVEN_PROJECTBASEDIR") - fi - fi - fi -fi -########################################################################################## -# End of extension -########################################################################################## - -export MAVEN_PROJECTBASEDIR=${MAVEN_BASEDIR:-"$BASE_DIR"} -if [ "$MVNW_VERBOSE" = true ]; then - echo $MAVEN_PROJECTBASEDIR + tar xzf${__MVNW_QUIET_TAR:+"$__MVNW_QUIET_TAR"} "$TMP_DOWNLOAD_DIR/$distributionUrlName" -C "$TMP_DOWNLOAD_DIR" || die "failed to untar" fi -MAVEN_OPTS="$(concat_lines "$MAVEN_PROJECTBASEDIR/.mvn/jvm.config") $MAVEN_OPTS" - -# For Cygwin, switch paths to Windows format before running java -if $cygwin; then - [ -n "$M2_HOME" ] && - M2_HOME=`cygpath --path --windows "$M2_HOME"` - [ -n "$JAVA_HOME" ] && - JAVA_HOME=`cygpath --path --windows "$JAVA_HOME"` - [ -n "$CLASSPATH" ] && - CLASSPATH=`cygpath --path --windows "$CLASSPATH"` - [ -n "$MAVEN_PROJECTBASEDIR" ] && - MAVEN_PROJECTBASEDIR=`cygpath --path --windows "$MAVEN_PROJECTBASEDIR"` -fi - -# Provide a "standardized" way to retrieve the CLI args that will -# work with both Windows and non-Windows executions. -MAVEN_CMD_LINE_ARGS="$MAVEN_CONFIG $@" -export MAVEN_CMD_LINE_ARGS - -WRAPPER_LAUNCHER=org.apache.maven.wrapper.MavenWrapperMain +printf %s\\n "$distributionUrl" >"$TMP_DOWNLOAD_DIR/$distributionUrlNameMain/mvnw.url" +mv -- "$TMP_DOWNLOAD_DIR/$distributionUrlNameMain" "$MAVEN_HOME" || [ -d "$MAVEN_HOME" ] || die "fail to move MAVEN_HOME" -exec "$JAVACMD" \ - $MAVEN_OPTS \ - $MAVEN_DEBUG_OPTS \ - -classpath "$MAVEN_PROJECTBASEDIR/.mvn/wrapper/maven-wrapper.jar" \ - "-Dmaven.home=${M2_HOME}" \ - "-Dmaven.multiModuleProjectDirectory=${MAVEN_PROJECTBASEDIR}" \ - ${WRAPPER_LAUNCHER} $MAVEN_CONFIG "$@" +clean || : +exec_maven "$@" diff --git a/mvnw.cmd b/mvnw.cmd index 23b7079a3d4c7..b150b91ed5005 100644 --- a/mvnw.cmd +++ b/mvnw.cmd @@ -1,3 +1,4 @@ +<# : batch portion @REM ---------------------------------------------------------------------------- @REM Licensed to the Apache Software Foundation (ASF) under one @REM or more contributor license agreements. See the NOTICE file @@ -18,171 +19,131 @@ @REM ---------------------------------------------------------------------------- @REM ---------------------------------------------------------------------------- -@REM Maven Start Up Batch script -@REM -@REM Required ENV vars: -@REM JAVA_HOME - location of a JDK home dir +@REM Apache Maven Wrapper startup batch script, version 3.3.2 @REM @REM Optional ENV vars -@REM M2_HOME - location of maven2's installed home dir -@REM MAVEN_BATCH_ECHO - set to 'on' to enable the echoing of the batch commands -@REM MAVEN_BATCH_PAUSE - set to 'on' to wait for a keystroke before ending -@REM MAVEN_OPTS - parameters passed to the Java VM when running Maven -@REM e.g. to debug Maven itself, use -@REM set MAVEN_OPTS=-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=8000 -@REM MAVEN_SKIP_RC - flag to disable loading of mavenrc files +@REM MVNW_REPOURL - repo url base for downloading maven distribution +@REM MVNW_USERNAME/MVNW_PASSWORD - user and password for downloading maven +@REM MVNW_VERBOSE - true: enable verbose log; others: silence the output @REM ---------------------------------------------------------------------------- -@REM Begin all REM lines with '@' in case MAVEN_BATCH_ECHO is 'on' -@echo off -@REM set title of command window -title %0 -@REM enable echoing by setting MAVEN_BATCH_ECHO to 'on' -@if "%MAVEN_BATCH_ECHO%" == "on" echo %MAVEN_BATCH_ECHO% - -@REM set %HOME% to equivalent of $HOME -if "%HOME%" == "" (set "HOME=%HOMEDRIVE%%HOMEPATH%") - -@REM Execute a user defined script before this one -if not "%MAVEN_SKIP_RC%" == "" goto skipRcPre -@REM check for pre script, once with legacy .bat ending and once with .cmd ending -if exist "%USERPROFILE%\mavenrc_pre.bat" call "%USERPROFILE%\mavenrc_pre.bat" %* -if exist "%USERPROFILE%\mavenrc_pre.cmd" call "%USERPROFILE%\mavenrc_pre.cmd" %* -:skipRcPre - -@setlocal - -set ERROR_CODE=0 - -@REM To isolate internal variables from possible post scripts, we use another setlocal -@setlocal - -@REM ==== START VALIDATION ==== -if not "%JAVA_HOME%" == "" goto OkJHome - -echo. -echo Error: JAVA_HOME not found in your environment. >&2 -echo Please set the JAVA_HOME variable in your environment to match the >&2 -echo location of your Java installation. >&2 -echo. -goto error - -:OkJHome -if exist "%JAVA_HOME%\bin\java.exe" goto init - -echo. -echo Error: JAVA_HOME is set to an invalid directory. >&2 -echo JAVA_HOME = "%JAVA_HOME%" >&2 -echo Please set the JAVA_HOME variable in your environment to match the >&2 -echo location of your Java installation. >&2 -echo. -goto error - -@REM ==== END VALIDATION ==== - -:init - -@REM Find the project base dir, i.e. the directory that contains the folder ".mvn". -@REM Fallback to current working directory if not found. - -set MAVEN_PROJECTBASEDIR=%MAVEN_BASEDIR% -IF NOT "%MAVEN_PROJECTBASEDIR%"=="" goto endDetectBaseDir - -set EXEC_DIR=%CD% -set WDIR=%EXEC_DIR% -:findBaseDir -IF EXIST "%WDIR%"\.mvn goto baseDirFound -cd .. -IF "%WDIR%"=="%CD%" goto baseDirNotFound -set WDIR=%CD% -goto findBaseDir - -:baseDirFound -set MAVEN_PROJECTBASEDIR=%WDIR% -cd "%EXEC_DIR%" -goto endDetectBaseDir - -:baseDirNotFound -set MAVEN_PROJECTBASEDIR=%EXEC_DIR% -cd "%EXEC_DIR%" - -:endDetectBaseDir - -IF NOT EXIST "%MAVEN_PROJECTBASEDIR%\.mvn\jvm.config" goto endReadAdditionalConfig - -@setlocal EnableExtensions EnableDelayedExpansion -for /F "usebackq delims=" %%a in ("%MAVEN_PROJECTBASEDIR%\.mvn\jvm.config") do set JVM_CONFIG_MAVEN_PROPS=!JVM_CONFIG_MAVEN_PROPS! %%a -@endlocal & set JVM_CONFIG_MAVEN_PROPS=%JVM_CONFIG_MAVEN_PROPS% - -:endReadAdditionalConfig - -SET MAVEN_JAVA_EXE="%JAVA_HOME%\bin\java.exe" -set WRAPPER_JAR="%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.jar" -set WRAPPER_LAUNCHER=org.apache.maven.wrapper.MavenWrapperMain - -set DOWNLOAD_URL="https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar" - -FOR /F "usebackq tokens=1,2 delims==" %%A IN ("%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.properties") DO ( - IF "%%A"=="wrapperUrl" SET DOWNLOAD_URL=%%B -) - -@REM Extension to allow automatically downloading the maven-wrapper.jar from Maven-central -@REM This allows using the maven wrapper in projects that prohibit checking in binary data. -if exist %WRAPPER_JAR% ( - if "%MVNW_VERBOSE%" == "true" ( - echo Found %WRAPPER_JAR% - ) -) else ( - if not "%MVNW_REPOURL%" == "" ( - SET DOWNLOAD_URL="%MVNW_REPOURL%/org/apache/maven/wrapper/maven-wrapper/3.1.0/maven-wrapper-3.1.0.jar" - ) - if "%MVNW_VERBOSE%" == "true" ( - echo Couldn't find %WRAPPER_JAR%, downloading it ... - echo Downloading from: %DOWNLOAD_URL% - ) - - powershell -Command "&{"^ - "$webclient = new-object System.Net.WebClient;"^ - "if (-not ([string]::IsNullOrEmpty('%MVNW_USERNAME%') -and [string]::IsNullOrEmpty('%MVNW_PASSWORD%'))) {"^ - "$webclient.Credentials = new-object System.Net.NetworkCredential('%MVNW_USERNAME%', '%MVNW_PASSWORD%');"^ - "}"^ - "[Net.ServicePointManager]::SecurityProtocol = [Net.SecurityProtocolType]::Tls12; $webclient.DownloadFile('%DOWNLOAD_URL%', '%WRAPPER_JAR%')"^ - "}" - if "%MVNW_VERBOSE%" == "true" ( - echo Finished downloading %WRAPPER_JAR% - ) +@IF "%__MVNW_ARG0_NAME__%"=="" (SET __MVNW_ARG0_NAME__=%~nx0) +@SET __MVNW_CMD__= +@SET __MVNW_ERROR__= +@SET __MVNW_PSMODULEP_SAVE=%PSModulePath% +@SET PSModulePath= +@FOR /F "usebackq tokens=1* delims==" %%A IN (`powershell -noprofile "& {$scriptDir='%~dp0'; $script='%__MVNW_ARG0_NAME__%'; icm -ScriptBlock ([Scriptblock]::Create((Get-Content -Raw '%~f0'))) -NoNewScope}"`) DO @( + IF "%%A"=="MVN_CMD" (set __MVNW_CMD__=%%B) ELSE IF "%%B"=="" (echo %%A) ELSE (echo %%A=%%B) ) -@REM End of extension - -@REM Provide a "standardized" way to retrieve the CLI args that will -@REM work with both Windows and non-Windows executions. -set MAVEN_CMD_LINE_ARGS=%* - -%MAVEN_JAVA_EXE% ^ - %JVM_CONFIG_MAVEN_PROPS% ^ - %MAVEN_OPTS% ^ - %MAVEN_DEBUG_OPTS% ^ - -classpath %WRAPPER_JAR% ^ - "-Dmaven.multiModuleProjectDirectory=%MAVEN_PROJECTBASEDIR%" ^ - %WRAPPER_LAUNCHER% %MAVEN_CONFIG% %* -if ERRORLEVEL 1 goto error -goto end - -:error -set ERROR_CODE=1 - -:end -@endlocal & set ERROR_CODE=%ERROR_CODE% - -if not "%MAVEN_SKIP_RC%"=="" goto skipRcPost -@REM check for post script, once with legacy .bat ending and once with .cmd ending -if exist "%USERPROFILE%\mavenrc_post.bat" call "%USERPROFILE%\mavenrc_post.bat" -if exist "%USERPROFILE%\mavenrc_post.cmd" call "%USERPROFILE%\mavenrc_post.cmd" -:skipRcPost - -@REM pause the script if MAVEN_BATCH_PAUSE is set to 'on' -if "%MAVEN_BATCH_PAUSE%"=="on" pause - -if "%MAVEN_TERMINATE_CMD%"=="on" exit %ERROR_CODE% - -cmd /C exit /B %ERROR_CODE% +@SET PSModulePath=%__MVNW_PSMODULEP_SAVE% +@SET __MVNW_PSMODULEP_SAVE= +@SET __MVNW_ARG0_NAME__= +@SET MVNW_USERNAME= +@SET MVNW_PASSWORD= +@IF NOT "%__MVNW_CMD__%"=="" (%__MVNW_CMD__% %*) +@echo Cannot start maven from wrapper >&2 && exit /b 1 +@GOTO :EOF +: end batch / begin powershell #> + +$ErrorActionPreference = "Stop" +if ($env:MVNW_VERBOSE -eq "true") { + $VerbosePreference = "Continue" +} + +# calculate distributionUrl, requires .mvn/wrapper/maven-wrapper.properties +$distributionUrl = (Get-Content -Raw "$scriptDir/.mvn/wrapper/maven-wrapper.properties" | ConvertFrom-StringData).distributionUrl +if (!$distributionUrl) { + Write-Error "cannot read distributionUrl property in $scriptDir/.mvn/wrapper/maven-wrapper.properties" +} + +switch -wildcard -casesensitive ( $($distributionUrl -replace '^.*/','') ) { + "maven-mvnd-*" { + $USE_MVND = $true + $distributionUrl = $distributionUrl -replace '-bin\.[^.]*$',"-windows-amd64.zip" + $MVN_CMD = "mvnd.cmd" + break + } + default { + $USE_MVND = $false + $MVN_CMD = $script -replace '^mvnw','mvn' + break + } +} + +# apply MVNW_REPOURL and calculate MAVEN_HOME +# maven home pattern: ~/.m2/wrapper/dists/{apache-maven-,maven-mvnd--}/ +if ($env:MVNW_REPOURL) { + $MVNW_REPO_PATTERN = if ($USE_MVND) { "/org/apache/maven/" } else { "/maven/mvnd/" } + $distributionUrl = "$env:MVNW_REPOURL$MVNW_REPO_PATTERN$($distributionUrl -replace '^.*'+$MVNW_REPO_PATTERN,'')" +} +$distributionUrlName = $distributionUrl -replace '^.*/','' +$distributionUrlNameMain = $distributionUrlName -replace '\.[^.]*$','' -replace '-bin$','' +$MAVEN_HOME_PARENT = "$HOME/.m2/wrapper/dists/$distributionUrlNameMain" +if ($env:MAVEN_USER_HOME) { + $MAVEN_HOME_PARENT = "$env:MAVEN_USER_HOME/wrapper/dists/$distributionUrlNameMain" +} +$MAVEN_HOME_NAME = ([System.Security.Cryptography.MD5]::Create().ComputeHash([byte[]][char[]]$distributionUrl) | ForEach-Object {$_.ToString("x2")}) -join '' +$MAVEN_HOME = "$MAVEN_HOME_PARENT/$MAVEN_HOME_NAME" + +if (Test-Path -Path "$MAVEN_HOME" -PathType Container) { + Write-Verbose "found existing MAVEN_HOME at $MAVEN_HOME" + Write-Output "MVN_CMD=$MAVEN_HOME/bin/$MVN_CMD" + exit $? +} + +if (! $distributionUrlNameMain -or ($distributionUrlName -eq $distributionUrlNameMain)) { + Write-Error "distributionUrl is not valid, must end with *-bin.zip, but found $distributionUrl" +} + +# prepare tmp dir +$TMP_DOWNLOAD_DIR_HOLDER = New-TemporaryFile +$TMP_DOWNLOAD_DIR = New-Item -Itemtype Directory -Path "$TMP_DOWNLOAD_DIR_HOLDER.dir" +$TMP_DOWNLOAD_DIR_HOLDER.Delete() | Out-Null +trap { + if ($TMP_DOWNLOAD_DIR.Exists) { + try { Remove-Item $TMP_DOWNLOAD_DIR -Recurse -Force | Out-Null } + catch { Write-Warning "Cannot remove $TMP_DOWNLOAD_DIR" } + } +} + +New-Item -Itemtype Directory -Path "$MAVEN_HOME_PARENT" -Force | Out-Null + +# Download and Install Apache Maven +Write-Verbose "Couldn't find MAVEN_HOME, downloading and installing it ..." +Write-Verbose "Downloading from: $distributionUrl" +Write-Verbose "Downloading to: $TMP_DOWNLOAD_DIR/$distributionUrlName" + +$webclient = New-Object System.Net.WebClient +if ($env:MVNW_USERNAME -and $env:MVNW_PASSWORD) { + $webclient.Credentials = New-Object System.Net.NetworkCredential($env:MVNW_USERNAME, $env:MVNW_PASSWORD) +} +[Net.ServicePointManager]::SecurityProtocol = [Net.SecurityProtocolType]::Tls12 +$webclient.DownloadFile($distributionUrl, "$TMP_DOWNLOAD_DIR/$distributionUrlName") | Out-Null + +# If specified, validate the SHA-256 sum of the Maven distribution zip file +$distributionSha256Sum = (Get-Content -Raw "$scriptDir/.mvn/wrapper/maven-wrapper.properties" | ConvertFrom-StringData).distributionSha256Sum +if ($distributionSha256Sum) { + if ($USE_MVND) { + Write-Error "Checksum validation is not supported for maven-mvnd. `nPlease disable validation by removing 'distributionSha256Sum' from your maven-wrapper.properties." + } + Import-Module $PSHOME\Modules\Microsoft.PowerShell.Utility -Function Get-FileHash + if ((Get-FileHash "$TMP_DOWNLOAD_DIR/$distributionUrlName" -Algorithm SHA256).Hash.ToLower() -ne $distributionSha256Sum) { + Write-Error "Error: Failed to validate Maven distribution SHA-256, your Maven distribution might be compromised. If you updated your Maven version, you need to update the specified distributionSha256Sum property." + } +} + +# unzip and move +Expand-Archive "$TMP_DOWNLOAD_DIR/$distributionUrlName" -DestinationPath "$TMP_DOWNLOAD_DIR" | Out-Null +Rename-Item -Path "$TMP_DOWNLOAD_DIR/$distributionUrlNameMain" -NewName $MAVEN_HOME_NAME | Out-Null +try { + Move-Item -Path "$TMP_DOWNLOAD_DIR/$MAVEN_HOME_NAME" -Destination $MAVEN_HOME_PARENT | Out-Null +} catch { + if (! (Test-Path -Path "$MAVEN_HOME" -PathType Container)) { + Write-Error "fail to move MAVEN_HOME" + } +} finally { + try { Remove-Item $TMP_DOWNLOAD_DIR -Recurse -Force | Out-Null } + catch { Write-Warning "Cannot remove $TMP_DOWNLOAD_DIR" } +} + +Write-Output "MVN_CMD=$MAVEN_HOME/bin/$MVN_CMD" From c598974346a5df2bb328e679f49b6cb0b56ab84b Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Tue, 8 Oct 2024 19:22:31 +0800 Subject: [PATCH 564/580] [improve][PIP] PIP-383: Support granting/revoking permissions for multiple topics (#23355) --- pip/pip-383.md | 144 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 144 insertions(+) create mode 100644 pip/pip-383.md diff --git a/pip/pip-383.md b/pip/pip-383.md new file mode 100644 index 0000000000000..72f4e182ea7ea --- /dev/null +++ b/pip/pip-383.md @@ -0,0 +1,144 @@ +# PIP-383: Support granting/revoking permissions for multiple topics + +## Background + +In AuthorizationProvider, the authorization interface `grantPermissionAsync(TopicName topicName, Set actions, String role, String authDataJson)` currently only supports granting permissions to a single topic at a time. +When multiple topics need to be authorized under a namespace, the client makes the calls to the authorization interface concurrently. +Since the permissions information is stored in the namespace-level policies, and multiple topics may be on different brokers, concurrent authorization modification will cause concurrent modification exceptions. +Therefore, supporting granting permissions for multiple topics is very beneficial. + + +## Motivation + +Supporting granting/revoking permissions for multiple topics, +add `grantPermissionAsync(List options)` and `revokePermissionAsync(List options)` in AuthorizationProvider. + +## Goals + +### In Scope + +- Add `grantPermissionAsync(List options)` in AuthorizationProvider. +- Add `revokePermissionAsync(List options)` in AuthorizationProvider. + +## High-Level Design + +### Design & Implementation Details + +Add default method implementation in AuthorizationProvider +```java + +public interface AuthorizationProvider extends Closeable { + + default CompletableFuture grantPermissionAsync(List options) { + return FutureUtil.failedFuture(new IllegalStateException( + String.format("grantPermissionAsync is not supported by the Authorization"))); + } + + default CompletableFuture revokePermissionAsync(List options) { + return FutureUtil.failedFuture(new IllegalStateException( + String.format("revokePermissionAsync is not supported by the Authorization"))); + } +} +``` + +``` +@Data +@Builder +public class GrantTopicPermissionOptions { + + private final String topic; + + private final String role; + + private final Set actions; +} + +@Data +@Builder +public class RevokeTopicPermissionOptions { + + private final String topic; + + private final String role; +} +``` + +Add namespace admin API. + +```java +public interface Namespaces { + + CompletableFuture grantPermissionOnTopicsAsync(List options); + + void grantPermissionOnTopics(List options) throws PulsarAdminException; + + CompletableFuture revokePermissionOnTopicsAsync(List options); + + void revokePermissionOnTopics(List options) throws PulsarAdminException; +} +``` + +Add namespace rest implementation in broker side. +```java +@POST +@Path("/grantPermissions") +public void grantPermissionOnTopics(@Suspended final AsyncResponse asyncResponse, + List options) { + internalGrantPermissionsAsync(options) + .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) + .exceptionally(ex -> { + log.error("[{}] Failed to grant permissions {}", + clientAppId(), options, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); +} + +@POST +@Path("/revokePermissions") +public void revokePermissionOnTopics(@Suspended final AsyncResponse asyncResponse, + List options) { + internalRevokePermissionsAsync(options) + .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) + .exceptionally(ex -> { + log.error("[{}] Failed to revoke permissions {}", + clientAppId(), options, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); +} +``` + +so user can grant/revoke permissions to multi-topics like : +```java +public class TestAuthorization { + + @Test + public void testGrantPermission() { + // grant permission for multi-topics + List grantPermissions = new ArrayList<>(); + grantPermissions.add(GrantPermissionOptions.builder().topic("topic1").role("role1").actions(Set.of(AuthAction.produce)).build()); + grantPermissions.add(GrantPermissionOptions.builder().topic("topic2").role("role2").actions(Set.of(AuthAction.consume)).build()); + admin.namespaces().grantPermissionOnTopics(grantPermissions); + // revoke permission topics + List revokePermissions = new ArrayList<>(); + revokePermissions.add(RevokePermissionOptions.builder().topic("topic1").role("role1").build()); + revokePermissions.add(RevokePermissionOptions.builder().topic("topic2").role("role2").build()); + admin.namespaces().revokePermissionOnTopics(revokePermissions); + } +} + +``` + +## Backward & Forward Compatibility + + + +## Alternatives + +## General Notes + +## Links + +* Mailing List discussion thread: https://lists.apache.org/thread/6n2jdl9bsf1f6xz2orygz3kvxmy11ykh +* Mailing List voting thread: https://lists.apache.org/thread/qbyvs75r0d64h6jk8w1swr782l85b77h From 4efcc1502f5d56047094113a7f14468c1ef90a05 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 8 Oct 2024 15:40:38 +0300 Subject: [PATCH 565/580] [improve][ci] Move some flaky ExtensibleLoadManager tests to flaky group until they are fixed (#23414) --- .../loadbalance/extensions/BrokerRegistryIntegrationTest.java | 2 +- .../extensions/BrokerRegistryMetadataStoreIntegrationTest.java | 2 +- .../loadbalance/extensions/ExtensibleLoadManagerCloseTest.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java index e975671fa12e8..189c29400c4f9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryIntegrationTest.java @@ -37,7 +37,7 @@ import org.testng.annotations.Test; @Slf4j -@Test(groups = "broker") +@Test(groups = "flaky") public class BrokerRegistryIntegrationTest { private static final String clusterName = "test"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java index 3e01b1fad0f21..15097b565db6c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryMetadataStoreIntegrationTest.java @@ -22,7 +22,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateMetadataStoreTableViewImpl; import org.testng.annotations.Test; -@Test(groups = "broker") +@Test(groups = "flaky") public class BrokerRegistryMetadataStoreIntegrationTest extends BrokerRegistryIntegrationTest { @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java index fa63ce566c603..ca44f6bc4d6d9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerCloseTest.java @@ -38,7 +38,7 @@ import org.testng.annotations.Test; @Slf4j -@Test(groups = "broker") +@Test(groups = "flaky") public class ExtensibleLoadManagerCloseTest { private static final String clusterName = "test"; From 3d0625ba64294fb0fe7dafc27c7a34883b4be51b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 8 Oct 2024 17:03:13 +0300 Subject: [PATCH 566/580] [improve][broker] PIP-379: Key_Shared Draining Hashes for Improved Message Ordering (#23352) --- .../server/src/assemble/LICENSE.bin.txt | 1 + pom.xml | 7 + .../pulsar/broker/ServiceConfiguration.java | 10 + pulsar-broker/pom.xml | 5 + .../AbstractDelayedDeliveryTracker.java | 2 +- .../pulsar/broker/service/BrokerService.java | 2 +- ...stentHashingStickyKeyConsumerSelector.java | 105 ++- .../pulsar/broker/service/Consumer.java | 134 +-- .../ConsumerHashAssignmentsSnapshot.java | 224 +++++ .../broker/service/DrainingHashesTracker.java | 241 ++++++ .../broker/service/EntryAndMetadata.java | 46 +- .../broker/service/HashRangeAssignment.java | 26 + ...ngeAutoSplitStickyKeyConsumerSelector.java | 47 +- ...ngeExclusiveStickyKeyConsumerSelector.java | 48 +- .../service/ImpactedConsumersResult.java | 59 ++ .../pulsar/broker/service/PendingAcksMap.java | 424 ++++++++++ .../pulsar/broker/service/Producer.java | 20 +- .../broker/service/RemovedHashRanges.java | 74 ++ .../service/StickyKeyConsumerSelector.java | 74 +- .../StickyKeyConsumerSelectorUtils.java | 51 ++ ...tStickyKeyDispatcherMultipleConsumers.java | 8 +- .../MessageRedeliveryController.java | 24 +- ...PersistentDispatcherMultipleConsumers.java | 157 ++-- ...tStickyKeyDispatcherMultipleConsumers.java | 480 ++++------- .../persistent/PersistentSubscription.java | 26 - .../persistent/RescheduleReadHandler.java | 102 +++ .../ConcurrentBitmapSortedLongPairSet.java | 14 +- .../apache/pulsar/broker/BrokerTestUtil.java | 19 + .../pulsar/broker/admin/AdminApiTest.java | 206 ----- .../delayed/AbstractDeliveryTrackerTest.java | 14 +- ...tHashingStickyKeyConsumerSelectorTest.java | 85 +- .../ConsumerHashAssignmentsSnapshotTest.java | 204 +++++ .../service/DrainingHashesTrackerTest.java | 213 +++++ .../broker/service/PendingAcksMapTest.java | 196 +++++ .../service/PersistentTopicE2ETest.java | 87 -- ...istentDispatcherMultipleConsumersTest.java | 4 +- ...ckyKeyDispatcherMultipleConsumersTest.java | 469 ++--------- .../persistent/RescheduleReadHandlerTest.java | 161 ++++ .../broker/stats/SubscriptionStatsTest.java | 44 - .../client/api/KeySharedSubscriptionTest.java | 778 +++++++----------- .../impl/KeySharedSubscriptionTest.java | 161 +++- .../policies/data/SubscriptionStats.java | 9 - .../org/apache/pulsar/client/api/Range.java | 8 + .../apache/pulsar/client/api/RangeTest.java | 40 + .../data/stats/SubscriptionStatsImpl.java | 10 - .../pulsar/common/protocol/Commands.java | 38 +- .../src/main/resources/findbugsExclude.xml | 5 + 47 files changed, 3324 insertions(+), 1838 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshot.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAssignment.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ImpactedConsumersResult.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PendingAcksMap.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelectorUtils.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshotTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DrainingHashesTrackerTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PendingAcksMapTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandlerTest.java diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 24eb6b8066df1..271f6dc6cebf7 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -262,6 +262,7 @@ The Apache Software License, Version 2.0 - com.fasterxml.jackson.module-jackson-module-parameter-names-2.17.2.jar * Caffeine -- com.github.ben-manes.caffeine-caffeine-2.9.1.jar * Conscrypt -- org.conscrypt-conscrypt-openjdk-uber-2.5.2.jar + * Fastutil -- it.unimi.dsi-fastutil-8.5.14.jar * Proto Google Common Protos -- com.google.api.grpc-proto-google-common-protos-2.17.0.jar * Bitbucket -- org.bitbucket.b_c-jose4j-0.9.4.jar * Gson diff --git a/pom.xml b/pom.xml index e0bce0442e158..f99eb3066d5e6 100644 --- a/pom.xml +++ b/pom.xml @@ -162,6 +162,7 @@ flexible messaging model and an intuitive client API. 1.0.7 1.0.2.5 2.17.2 + 8.5.14 0.10.2 1.6.2 10.14.2 @@ -911,6 +912,12 @@ flexible messaging model and an intuitive client API. import + + it.unimi.dsi + fastutil + ${fastutil.version} + + org.codehaus.jettison jettison 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 81073b1731b24..1b021bd569969 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 @@ -979,6 +979,16 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece ) private int keySharedLookAheadMsgInReplayThresholdPerSubscription = 20000; + @FieldContext( + category = CATEGORY_POLICIES, + doc = "For Key_Shared subscriptions, when a blocked key hash gets unblocked," + + " a redelivery will be attempted after a delay. This setting controls the delay." + + " The reason to have the delay is to batch multiple unblocking events instead of triggering" + + " redelivery for each unblocking event.", + dynamic = true + ) + private long keySharedUnblockingIntervalMs = 10L; + @FieldContext( category = CATEGORY_POLICIES, doc = "Once broker reaches maxUnackedMessagesPerBroker limit, it blocks subscriptions which has higher " diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index ee22762719175..a9521e76296de 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -63,6 +63,11 @@ protobuf-java + + it.unimi.dsi + fastutil + + ${project.groupId} pulsar-client-original diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java index 5c99e4c307d7c..f93a627bca7b8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java @@ -146,7 +146,7 @@ public void run(Timeout timeout) throws Exception { lastTickRun = clock.millis(); currentTimeoutTarget = -1; this.timeout = null; - dispatcher.readMoreEntries(); + dispatcher.readMoreEntriesAsync(); } } 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 ed0cdf18b47ca..8d0b9a4a84e6a 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 @@ -3416,7 +3416,7 @@ public void unblockDispatchersOnUnAckMessages(List { dispatcher.unBlockDispatcherOnUnackedMsgs(); - executor().execute(() -> dispatcher.readMoreEntries()); + dispatcher.readMoreEntriesAsync(); log.info("[{}] Dispatcher is unblocked", dispatcher.getName()); blockedDispatchers.remove(dispatcher); }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java index 1ae9a6ff96b7d..8381f9543bdc2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.service; import java.util.ArrayList; -import java.util.IdentityHashMap; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -28,13 +27,10 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.pulsar.client.api.Range; -import org.apache.pulsar.common.util.Murmur3_32Hash; /** - * This is a consumer selector based fixed hash range. - * - * The implementation uses consistent hashing to evenly split, the - * number of keys assigned to each consumer. + * This is a consumer selector using consistent hashing to evenly split + * the number of keys assigned to each consumer. */ public class ConsistentHashingStickyKeyConsumerSelector implements StickyKeyConsumerSelector { // use NUL character as field separator for hash key calculation @@ -47,14 +43,22 @@ public class ConsistentHashingStickyKeyConsumerSelector implements StickyKeyCons private final ConsumerNameIndexTracker consumerNameIndexTracker = new ConsumerNameIndexTracker(); private final int numberOfPoints; + private final Range keyHashRange; + private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot; public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints) { + this(numberOfPoints, DEFAULT_RANGE_SIZE - 1); + } + + public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints, int rangeMaxValue) { this.hashRing = new TreeMap<>(); this.numberOfPoints = numberOfPoints; + this.keyHashRange = Range.of(STICKY_KEY_HASH_NOT_SET + 1, rangeMaxValue); + this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty(); } @Override - public CompletableFuture addConsumer(Consumer consumer) { + public CompletableFuture addConsumer(Consumer consumer) { rwLock.writeLock().lock(); try { ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer); @@ -72,7 +76,11 @@ public CompletableFuture addConsumer(Consumer consumer) { consumerNameIndexTracker.decreaseConsumerRefCount(removed); } } - return CompletableFuture.completedFuture(null); + ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); + ImpactedConsumersResult impactedConsumers = + consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); + consumerHashAssignmentsSnapshot = assignmentsAfter; + return CompletableFuture.completedFuture(impactedConsumers); } finally { rwLock.writeLock().unlock(); } @@ -88,14 +96,14 @@ public CompletableFuture addConsumer(Consumer consumer) { * @param hashRingPointIndex the index of the hash ring point * @return the hash value */ - private static int calculateHashForConsumerAndIndex(Consumer consumer, int consumerNameIndex, + private int calculateHashForConsumerAndIndex(Consumer consumer, int consumerNameIndex, int hashRingPointIndex) { String key = consumer.consumerName() + KEY_SEPARATOR + consumerNameIndex + KEY_SEPARATOR + hashRingPointIndex; - return Murmur3_32Hash.getInstance().makeHash(key.getBytes()); + return makeStickyKeyHash(key.getBytes()); } @Override - public void removeConsumer(Consumer consumer) { + public ImpactedConsumersResult removeConsumer(Consumer consumer) { rwLock.writeLock().lock(); try { ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer); @@ -109,6 +117,11 @@ public void removeConsumer(Consumer consumer) { } } } + ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); + ImpactedConsumersResult impactedConsumers = + consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); + consumerHashAssignmentsSnapshot = assignmentsAfter; + return impactedConsumers; } finally { rwLock.writeLock().unlock(); } @@ -134,32 +147,58 @@ public Consumer select(int hash) { } @Override - public Map> getConsumerKeyHashRanges() { - Map> result = new IdentityHashMap<>(); + public Range getKeyHashRange() { + return keyHashRange; + } + + @Override + public ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() { rwLock.readLock().lock(); try { - if (hashRing.isEmpty()) { - return result; - } - int start = 0; - int lastKey = 0; - for (Map.Entry entry: hashRing.entrySet()) { - Consumer consumer = entry.getValue().consumer; - result.computeIfAbsent(consumer, key -> new ArrayList<>()) - .add(Range.of(start, entry.getKey())); - lastKey = entry.getKey(); - start = lastKey + 1; - } - // Handle wrap-around in the hash ring, the first consumer will also contain the range from the last key - // to the maximum value of the hash range - Consumer firstConsumer = hashRing.firstEntry().getValue().consumer; - List ranges = result.get(firstConsumer); - if (lastKey != Integer.MAX_VALUE - 1) { - ranges.add(Range.of(lastKey + 1, Integer.MAX_VALUE - 1)); - } + return consumerHashAssignmentsSnapshot; } finally { rwLock.readLock().unlock(); } - return result; + } + + private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() { + if (hashRing.isEmpty()) { + return ConsumerHashAssignmentsSnapshot.empty(); + } + List result = new ArrayList<>(); + int start = getKeyHashRange().getStart(); + int lastKey = -1; + Consumer previousConsumer = null; + Range previousRange = null; + for (Map.Entry entry: hashRing.entrySet()) { + Consumer consumer = entry.getValue().consumer; + Range range; + if (consumer == previousConsumer) { + // join ranges + result.remove(result.size() - 1); + range = Range.of(previousRange.getStart(), entry.getKey()); + } else { + range = Range.of(start, entry.getKey()); + } + result.add(new HashRangeAssignment(range, consumer)); + lastKey = entry.getKey(); + start = lastKey + 1; + previousConsumer = consumer; + previousRange = range; + } + // Handle wrap-around + Consumer firstConsumer = hashRing.firstEntry().getValue().consumer; + if (lastKey != getKeyHashRange().getEnd()) { + Range range; + if (firstConsumer == previousConsumer && previousRange.getEnd() == lastKey) { + // join ranges + result.remove(result.size() - 1); + range = Range.of(previousRange.getStart(), getKeyHashRange().getEnd()); + } else { + range = Range.of(lastKey + 1, getKeyHashRange().getEnd()); + } + result.add(new HashRangeAssignment(range, firstConsumer)); + } + return ConsumerHashAssignmentsSnapshot.of(result); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index 7f46e8969eb53..c9584f2c1790f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; @@ -26,6 +27,8 @@ import io.netty.util.concurrent.Future; import io.netty.util.concurrent.Promise; import io.opentelemetry.api.common.Attributes; +import it.unimi.dsi.fastutil.ints.IntIntPair; +import it.unimi.dsi.fastutil.objects.ObjectIntPair; import java.time.Instant; import java.util.ArrayList; import java.util.BitSet; @@ -65,14 +68,11 @@ import org.apache.pulsar.common.policies.data.ClusterPolicies.ClusterUrl; import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; -import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.stats.Rate; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.BitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap; -import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap.LongPair; import org.apache.pulsar.opentelemetry.OpenTelemetryAttributes; import org.apache.pulsar.transaction.common.exception.TransactionConflictException; import org.slf4j.Logger; @@ -119,7 +119,7 @@ public class Consumer { AtomicIntegerFieldUpdater.newUpdater(Consumer.class, "permitsReceivedWhileConsumerBlocked"); private volatile int permitsReceivedWhileConsumerBlocked = 0; - private final ConcurrentLongLongPairHashMap pendingAcks; + private final PendingAcksMap pendingAcks; private final ConsumerStatsImpl stats; @@ -167,6 +167,13 @@ public class Consumer { private static final AtomicReferenceFieldUpdater OPEN_TELEMETRY_ATTRIBUTES_FIELD_UPDATER = AtomicReferenceFieldUpdater.newUpdater(Consumer.class, Attributes.class, "openTelemetryAttributes"); + @Getter + @Setter + private volatile PendingAcksMap.PendingAcksAddHandler pendingAcksAddHandler; + @Getter + @Setter + private volatile PendingAcksMap.PendingAcksRemoveHandler pendingAcksRemoveHandler; + public Consumer(Subscription subscription, SubType subType, String topicName, long consumerId, int priorityLevel, String consumerName, boolean isDurable, TransportCnx cnx, String appId, @@ -223,12 +230,8 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo stats.metadata = this.metadata; if (Subscription.isIndividualAckMode(subType)) { - this.pendingAcks = ConcurrentLongLongPairHashMap.newBuilder() - .autoShrink(subscription.getTopic().getBrokerService() - .getPulsar().getConfiguration().isAutoShrinkForConsumerPendingAcksMap()) - .expectedItems(256) - .concurrencyLevel(1) - .build(); + this.pendingAcks = new PendingAcksMap(this, this::getPendingAcksAddHandler, + this::getPendingAcksRemoveHandler); } else { // We don't need to keep track of pending acks if the subscription is not shared this.pendingAcks = null; @@ -359,17 +362,43 @@ public Future sendMessages(final List entries, // because this consumer is possible to disconnect at this time. if (pendingAcks != null) { int batchSize = batchSizes.getBatchSize(i); - int stickyKeyHash = stickyKeyHashes == null ? getStickyKeyHash(entry) : stickyKeyHashes.get(i); - long[] ackSet = batchIndexesAcks == null ? null : batchIndexesAcks.getAckSet(i); - if (ackSet != null) { - unackedMessages -= (batchSize - BitSet.valueOf(ackSet).cardinality()); + int stickyKeyHash; + if (stickyKeyHashes == null) { + if (entry instanceof EntryAndMetadata entryAndMetadata) { + stickyKeyHash = entryAndMetadata.getCachedStickyKeyHash(); + } else { + stickyKeyHash = STICKY_KEY_HASH_NOT_SET; + } + } else { + stickyKeyHash = stickyKeyHashes.get(i); } - pendingAcks.put(entry.getLedgerId(), entry.getEntryId(), batchSize, stickyKeyHash); - if (log.isDebugEnabled()) { - log.debug("[{}-{}] Added {}:{} ledger entry with batchSize of {} to pendingAcks in" - + " broker.service.Consumer for consumerId: {}", - topicName, subscription, entry.getLedgerId(), entry.getEntryId(), batchSize, - consumerId); + boolean sendingAllowed = + pendingAcks.addPendingAckIfAllowed(entry.getLedgerId(), entry.getEntryId(), batchSize, + stickyKeyHash); + if (!sendingAllowed) { + // sending isn't allowed when pending acks doesn't accept adding the entry + // this happens when Key_Shared draining hashes contains the stickyKeyHash + // because of race conditions, it might be resolved at the time of sending + totalEntries--; + entries.set(i, null); + entry.release(); + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Skipping sending of {}:{} ledger entry with batchSize of {} since adding" + + " to pending acks failed in broker.service.Consumer for consumerId: {}", + topicName, subscription, entry.getLedgerId(), entry.getEntryId(), batchSize, + consumerId); + } + } else { + long[] ackSet = batchIndexesAcks == null ? null : batchIndexesAcks.getAckSet(i); + if (ackSet != null) { + unackedMessages -= (batchSize - BitSet.valueOf(ackSet).cardinality()); + } + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Added {}:{} ledger entry with batchSize of {} to pendingAcks in" + + " broker.service.Consumer for consumerId: {}", + topicName, subscription, entry.getLedgerId(), entry.getEntryId(), batchSize, + consumerId); + } } } } @@ -537,11 +566,11 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map ackOwnerConsumerAndBatchSize = + ObjectIntPair ackOwnerConsumerAndBatchSize = getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(), msgId.getEntryId()); - Consumer ackOwnerConsumer = ackOwnerConsumerAndBatchSize.getLeft(); + Consumer ackOwnerConsumer = ackOwnerConsumerAndBatchSize.left(); long ackedCount; - long batchSize = ackOwnerConsumerAndBatchSize.getRight(); + int batchSize = ackOwnerConsumerAndBatchSize.rightInt(); if (msgId.getAckSetsCount() > 0) { long[] ackSets = new long[msgId.getAckSetsCount()]; for (int j = 0; j < msgId.getAckSetsCount(); j++) { @@ -607,11 +636,17 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { for (int i = 0; i < ack.getMessageIdsCount(); i++) { MessageIdData msgId = ack.getMessageIdAt(i); Position position = AckSetStateUtil.createPositionWithAckSet(msgId.getLedgerId(), msgId.getEntryId(), null); - Consumer ackOwnerConsumer = getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(), - msgId.getEntryId()).getLeft(); + ObjectIntPair ackOwnerConsumerAndBatchSize = getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(), + msgId.getEntryId()); + if (ackOwnerConsumerAndBatchSize == null) { + log.warn("[{}] [{}] Acknowledging message at {} that was already deleted", subscription, + consumerId, position); + continue; + } + Consumer ackOwnerConsumer = ackOwnerConsumerAndBatchSize.left(); // acked count at least one long ackedCount; - long batchSize; + int batchSize; if (msgId.hasBatchSize()) { batchSize = msgId.getBatchSize(); // ack batch messages set ackeCount = batchSize @@ -660,7 +695,7 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { return completableFuture.thenApply(__ -> totalAckCount.sum()); } - private long getAckedCountForMsgIdNoAckSets(long batchSize, Position position, Consumer consumer) { + private long getAckedCountForMsgIdNoAckSets(int batchSize, Position position, Consumer consumer) { if (isAcknowledgmentAtBatchIndexLevelEnabled && Subscription.isIndividualAckMode(subType)) { long[] cursorAckSet = getCursorAckSet(position); if (cursorAckSet != null) { @@ -670,11 +705,11 @@ private long getAckedCountForMsgIdNoAckSets(long batchSize, Position position, C return batchSize; } - private long getAckedCountForBatchIndexLevelEnabled(Position position, long batchSize, long[] ackSets, + private long getAckedCountForBatchIndexLevelEnabled(Position position, int batchSize, long[] ackSets, Consumer consumer) { long ackedCount = 0; if (isAcknowledgmentAtBatchIndexLevelEnabled && Subscription.isIndividualAckMode(subType) - && consumer.getPendingAcks().get(position.getLedgerId(), position.getEntryId()) != null) { + && consumer.getPendingAcks().contains(position.getLedgerId(), position.getEntryId())) { long[] cursorAckSet = getCursorAckSet(position); if (cursorAckSet != null) { BitSetRecyclable cursorBitSet = BitSetRecyclable.create().resetWords(cursorAckSet); @@ -692,14 +727,14 @@ private long getAckedCountForBatchIndexLevelEnabled(Position position, long batc return ackedCount; } - private long getAckedCountForTransactionAck(long batchSize, long[] ackSets) { + private long getAckedCountForTransactionAck(int batchSize, long[] ackSets) { BitSetRecyclable bitset = BitSetRecyclable.create().resetWords(ackSets); long ackedCount = batchSize - bitset.cardinality(); bitset.recycle(); return ackedCount; } - private long getUnAckedCountForBatchIndexLevelEnabled(Position position, long batchSize) { + private long getUnAckedCountForBatchIndexLevelEnabled(Position position, int batchSize) { long unAckedCount = batchSize; if (isAcknowledgmentAtBatchIndexLevelEnabled) { long[] cursorAckSet = getCursorAckSet(position); @@ -734,24 +769,24 @@ private boolean checkCanRemovePendingAcksAndHandle(Consumer ackOwnedConsumer, * @param entryId The ID of the entry. * @return Pair */ - private Pair getAckOwnerConsumerAndBatchSize(long ledgerId, long entryId) { + private ObjectIntPair getAckOwnerConsumerAndBatchSize(long ledgerId, long entryId) { if (Subscription.isIndividualAckMode(subType)) { - LongPair longPair = getPendingAcks().get(ledgerId, entryId); - if (longPair != null) { - return Pair.of(this, longPair.first); + IntIntPair pendingAck = getPendingAcks().get(ledgerId, entryId); + if (pendingAck != null) { + return ObjectIntPair.of(this, pendingAck.leftInt()); } else { // If there are more consumers, this step will consume more CPU, and it should be optimized later. for (Consumer consumer : subscription.getConsumers()) { if (consumer != this) { - longPair = consumer.getPendingAcks().get(ledgerId, entryId); - if (longPair != null) { - return Pair.of(consumer, longPair.first); + pendingAck = consumer.getPendingAcks().get(ledgerId, entryId); + if (pendingAck != null) { + return ObjectIntPair.of(consumer, pendingAck.leftInt()); } } } } } - return Pair.of(this, 1L); + return ObjectIntPair.of(this, 1); } private long[] getCursorAckSet(Position position) { @@ -1027,7 +1062,8 @@ public int hashCode() { * @param position */ private boolean removePendingAcks(Consumer ackOwnedConsumer, Position position) { - if (!ackOwnedConsumer.getPendingAcks().remove(position.getLedgerId(), position.getEntryId())) { + PendingAcksMap ownedConsumerPendingAcks = ackOwnedConsumer.getPendingAcks(); + if (!ownedConsumerPendingAcks.remove(position.getLedgerId(), position.getEntryId())) { // Message was already removed by the other consumer return false; } @@ -1046,7 +1082,7 @@ private boolean removePendingAcks(Consumer ackOwnedConsumer, Position position) return true; } - public ConcurrentLongLongPairHashMap getPendingAcks() { + public PendingAcksMap getPendingAcks() { return pendingAcks; } @@ -1093,9 +1129,9 @@ public void redeliverUnacknowledgedMessages(List messageIds) { List pendingPositions = new ArrayList<>(); for (MessageIdData msg : messageIds) { Position position = PositionFactory.create(msg.getLedgerId(), msg.getEntryId()); - LongPair longPair = pendingAcks.get(position.getLedgerId(), position.getEntryId()); - if (longPair != null) { - int unAckedCount = (int) getUnAckedCountForBatchIndexLevelEnabled(position, longPair.first); + IntIntPair pendingAck = pendingAcks.get(position.getLedgerId(), position.getEntryId()); + if (pendingAck != null) { + int unAckedCount = (int) getUnAckedCountForBatchIndexLevelEnabled(position, pendingAck.leftInt()); pendingAcks.remove(position.getLedgerId(), position.getEntryId()); totalRedeliveryMessages += unAckedCount; pendingPositions.add(position); @@ -1191,16 +1227,6 @@ public Map getMetadata() { return metadata; } - private int getStickyKeyHash(Entry entry) { - final byte[] stickyKey; - if (entry instanceof EntryAndMetadata) { - stickyKey = ((EntryAndMetadata) entry).getStickyKey(); - } else { - stickyKey = Commands.peekStickyKey(entry.getDataBuffer(), topicName, subscription.getName()); - } - return StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey); - } - private static final Logger log = LoggerFactory.getLogger(Consumer.class); public Attributes getOpenTelemetryAttributes() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshot.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshot.java new file mode 100644 index 0000000000000..d2bd113e69d1e --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshot.java @@ -0,0 +1,224 @@ +/* + * 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.service; + +import com.google.common.annotations.VisibleForTesting; +import java.util.ArrayList; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; +import lombok.EqualsAndHashCode; +import lombok.ToString; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.client.api.Range; +import org.jetbrains.annotations.NotNull; + +/** + * Represents the hash ranges assigned to each consumer in a {@link StickyKeyConsumerSelector} at a point in time. + */ +@EqualsAndHashCode(exclude = "cachedRangesByConsumer") +@ToString(exclude = "cachedRangesByConsumer") +public class ConsumerHashAssignmentsSnapshot { + private final List hashRangeAssignments; + private Map> cachedRangesByConsumer; + + private ConsumerHashAssignmentsSnapshot(List hashRangeAssignments) { + validate(hashRangeAssignments); + this.hashRangeAssignments = hashRangeAssignments; + } + + private void validate(List hashRangeAssignments) { + Range previousRange = null; + for (HashRangeAssignment hashRangeAssignment : hashRangeAssignments) { + Range range = hashRangeAssignment.range(); + Consumer consumer = hashRangeAssignment.consumer(); + if (range == null || consumer == null) { + throw new IllegalArgumentException("Range and consumer must not be null"); + } + if (previousRange != null && previousRange.compareTo(range) >= 0) { + throw new IllegalArgumentException("Ranges must be non-overlapping and sorted"); + } + previousRange = range; + } + } + + public static ConsumerHashAssignmentsSnapshot of(List hashRangeAssignments) { + return new ConsumerHashAssignmentsSnapshot(hashRangeAssignments); + } + + public static ConsumerHashAssignmentsSnapshot empty() { + return new ConsumerHashAssignmentsSnapshot(Collections.emptyList()); + } + + public ImpactedConsumersResult resolveImpactedConsumers(ConsumerHashAssignmentsSnapshot other) { + return resolveConsumerRemovedHashRanges(this.hashRangeAssignments, other.hashRangeAssignments); + } + + /** + * Get the ranges assigned to each consumer. The ranges are merged if they are overlapping. + * @return the ranges assigned to each consumer + */ + public synchronized Map> getRangesByConsumer() { + if (cachedRangesByConsumer == null) { + cachedRangesByConsumer = internalGetRangesByConsumer(); + } + return cachedRangesByConsumer; + } + + private @NotNull Map> internalGetRangesByConsumer() { + Map> rangesByConsumer = new IdentityHashMap<>(); + hashRangeAssignments.forEach(entry -> { + Range range = entry.range(); + Consumer consumer = entry.consumer(); + rangesByConsumer.computeIfAbsent(consumer, k -> new TreeSet<>()).add(range); + }); + Map> mergedOverlappingRangesByConsumer = new IdentityHashMap<>(); + rangesByConsumer.forEach((consumer, ranges) -> { + mergedOverlappingRangesByConsumer.put(consumer, mergeOverlappingRanges(ranges)); + }); + return mergedOverlappingRangesByConsumer; + } + + @VisibleForTesting + Map> diffRanges(ConsumerHashAssignmentsSnapshot other) { + return diffRanges(this.hashRangeAssignments, other.hashRangeAssignments); + } + + /** + * Resolve the consumers where the existing range was removed by a change. + * @param mappingBefore the range mapping before the change + * @param mappingAfter the range mapping after the change + * @return consumers and ranges where the existing range changed + */ + static ImpactedConsumersResult resolveConsumerRemovedHashRanges(List mappingBefore, + List mappingAfter) { + Map> impactedRanges = diffRanges(mappingBefore, mappingAfter); + Map> removedRangesByConsumer = impactedRanges.entrySet().stream() + .collect(IdentityHashMap::new, (resultMap, entry) -> { + Range range = entry.getKey(); + // filter out only where the range was removed + Consumer consumerBefore = entry.getValue().getLeft(); + if (consumerBefore != null) { + resultMap.computeIfAbsent(consumerBefore, k -> new TreeSet<>()).add(range); + } + }, IdentityHashMap::putAll); + return mergedOverlappingRangesAndConvertToImpactedConsumersResult(removedRangesByConsumer); + } + + static ImpactedConsumersResult mergedOverlappingRangesAndConvertToImpactedConsumersResult( + Map> removedRangesByConsumer) { + Map mergedRangesByConsumer = new IdentityHashMap<>(); + removedRangesByConsumer.forEach((consumer, ranges) -> { + mergedRangesByConsumer.put(consumer, RemovedHashRanges.of(mergeOverlappingRanges(ranges))); + }); + return ImpactedConsumersResult.of(mergedRangesByConsumer); + } + + /** + * Merge overlapping ranges. + * @param ranges the ranges to merge + * @return the merged ranges + */ + static List mergeOverlappingRanges(SortedSet ranges) { + List mergedRanges = new ArrayList<>(); + Iterator rangeIterator = ranges.iterator(); + Range currentRange = rangeIterator.hasNext() ? rangeIterator.next() : null; + while (rangeIterator.hasNext()) { + Range nextRange = rangeIterator.next(); + if (currentRange.getEnd() >= nextRange.getStart() - 1) { + currentRange = Range.of(currentRange.getStart(), Math.max(currentRange.getEnd(), nextRange.getEnd())); + } else { + mergedRanges.add(currentRange); + currentRange = nextRange; + } + } + if (currentRange != null) { + mergedRanges.add(currentRange); + } + return mergedRanges; + } + + /** + * Calculate the diff of two range mappings. + * @param mappingBefore the range mapping before + * @param mappingAfter the range mapping after + * @return the impacted ranges where the consumer is changed from the before to the after + */ + static Map> diffRanges(List mappingBefore, + List mappingAfter) { + Map> impactedRanges = new LinkedHashMap<>(); + Iterator beforeIterator = mappingBefore.iterator(); + Iterator afterIterator = mappingAfter.iterator(); + + HashRangeAssignment beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null; + HashRangeAssignment afterEntry = afterIterator.hasNext() ? afterIterator.next() : null; + + while (beforeEntry != null && afterEntry != null) { + Range beforeRange = beforeEntry.range(); + Range afterRange = afterEntry.range(); + Consumer beforeConsumer = beforeEntry.consumer(); + Consumer afterConsumer = afterEntry.consumer(); + + if (beforeRange.equals(afterRange)) { + if (!beforeConsumer.equals(afterConsumer)) { + impactedRanges.put(afterRange, Pair.of(beforeConsumer, afterConsumer)); + } + beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null; + afterEntry = afterIterator.hasNext() ? afterIterator.next() : null; + } else if (beforeRange.getEnd() < afterRange.getStart()) { + impactedRanges.put(beforeRange, Pair.of(beforeConsumer, afterConsumer)); + beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null; + } else if (afterRange.getEnd() < beforeRange.getStart()) { + impactedRanges.put(afterRange, Pair.of(beforeConsumer, afterConsumer)); + afterEntry = afterIterator.hasNext() ? afterIterator.next() : null; + } else { + Range overlapRange = Range.of( + Math.max(beforeRange.getStart(), afterRange.getStart()), + Math.min(beforeRange.getEnd(), afterRange.getEnd()) + ); + if (!beforeConsumer.equals(afterConsumer)) { + impactedRanges.put(overlapRange, Pair.of(beforeConsumer, afterConsumer)); + } + if (beforeRange.getEnd() <= overlapRange.getEnd()) { + beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null; + } + if (afterRange.getEnd() <= overlapRange.getEnd()) { + afterEntry = afterIterator.hasNext() ? afterIterator.next() : null; + } + } + } + + while (beforeEntry != null) { + impactedRanges.put(beforeEntry.range(), Pair.of(beforeEntry.consumer(), null)); + beforeEntry = beforeIterator.hasNext() ? beforeIterator.next() : null; + } + + while (afterEntry != null) { + impactedRanges.put(afterEntry.range(), Pair.of(null, afterEntry.consumer())); + afterEntry = afterIterator.hasNext() ? afterIterator.next() : null; + } + + return impactedRanges; + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java new file mode 100644 index 0000000000000..3521fa197a13d --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java @@ -0,0 +1,241 @@ +/* + * 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.service; + +import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import lombok.ToString; +import lombok.extern.slf4j.Slf4j; + +/** + * A thread-safe map to store draining hashes in the consumer. + */ +@Slf4j +public class DrainingHashesTracker { + private final String dispatcherName; + private final UnblockingHandler unblockingHandler; + // optimize the memory consumption of the map by using primitive int keys + private final Int2ObjectOpenHashMap drainingHashes = new Int2ObjectOpenHashMap<>(); + int batchLevel; + boolean unblockedWhileBatching; + + /** + * Represents an entry in the draining hashes tracker. + */ + @ToString + public static class DrainingHashEntry { + private final Consumer consumer; + private int refCount; + private int blockedCount; + + /** + * Constructs a new DrainingHashEntry with the specified Consumer. + * + * @param consumer the Consumer instance + */ + DrainingHashEntry(Consumer consumer) { + this.consumer = consumer; + } + + /** + * Gets the consumer that contained the hash in pending acks at the time of creating this + * entry. Since a particular hash can be assigned to only one consumer at a time, this consumer + * cannot change. No new pending acks can be added in the {@link PendingAcksMap} when there's + * a draining hash entry for a hash in {@link DrainingHashesTracker}. + * + * @return the consumer instance that contained the hash in pending acks at the time of creating this entry + */ + public Consumer getConsumer() { + return consumer; + } + + /** + * Increments the reference count. + */ + void incrementRefCount() { + refCount++; + } + + /** + * Decrements the reference count. + * + * @return true if the reference count is zero, false otherwise + */ + boolean decrementRefCount() { + return --refCount == 0; + } + + /** + * Increments the blocked count. + */ + void incrementBlockedCount() { + blockedCount++; + } + + /** + * Checks if the entry is blocking. + * + * @return true if the blocked count is greater than zero, false otherwise + */ + boolean isBlocking() { + return blockedCount > 0; + } + } + + /** + * Interface for handling the unblocking of sticky key hashes. + */ + public interface UnblockingHandler { + /** + * Handle the unblocking of a sticky key hash. + * + * @param stickyKeyHash the sticky key hash that has been unblocked, or -1 if hash unblocking is done in batch + */ + void stickyKeyHashUnblocked(int stickyKeyHash); + } + + public DrainingHashesTracker(String dispatcherName, UnblockingHandler unblockingHandler) { + this.dispatcherName = dispatcherName; + this.unblockingHandler = unblockingHandler; + } + + /** + * Add an entry to the draining hashes tracker. + * + * @param consumer the consumer + * @param stickyHash the sticky hash + */ + public synchronized void addEntry(Consumer consumer, int stickyHash) { + if (stickyHash == 0) { + throw new IllegalArgumentException("Sticky hash cannot be 0"); + } + DrainingHashEntry entry = drainingHashes.get(stickyHash); + if (entry == null) { + entry = new DrainingHashEntry(consumer); + drainingHashes.put(stickyHash, entry); + } else if (entry.getConsumer() != consumer) { + throw new IllegalStateException( + "Consumer " + entry.getConsumer() + " is already draining hash " + stickyHash + + " in dispatcher " + dispatcherName + ". Same hash being used for consumer " + consumer + + "."); + } + entry.incrementRefCount(); + } + + /** + * Start a batch operation. There could be multiple nested batch operations. + * The unblocking of sticky key hashes will be done only when the last batch operation ends. + */ + public synchronized void startBatch() { + batchLevel++; + } + + /** + * End a batch operation. + */ + public synchronized void endBatch() { + if (--batchLevel == 0 && unblockedWhileBatching) { + unblockedWhileBatching = false; + unblockingHandler.stickyKeyHashUnblocked(-1); + } + } + + /** + * Reduce the reference count for a given sticky hash. + * + * @param consumer the consumer + * @param stickyHash the sticky hash + * @param closing + */ + public synchronized void reduceRefCount(Consumer consumer, int stickyHash, boolean closing) { + if (stickyHash == 0) { + return; + } + DrainingHashEntry entry = drainingHashes.get(stickyHash); + if (entry == null) { + return; + } + if (entry.getConsumer() != consumer) { + throw new IllegalStateException( + "Consumer " + entry.getConsumer() + " is already draining hash " + stickyHash + + " in dispatcher " + dispatcherName + ". Same hash being used for consumer " + consumer + + "."); + } + if (entry.decrementRefCount()) { + DrainingHashEntry removed = drainingHashes.remove(stickyHash); + if (!closing && removed.isBlocking()) { + if (batchLevel > 0) { + unblockedWhileBatching = true; + } else { + unblockingHandler.stickyKeyHashUnblocked(stickyHash); + } + } + } + } + + /** + * Check if a sticky key hash should be blocked. + * + * @param consumer the consumer + * @param stickyKeyHash the sticky key hash + * @return true if the sticky key hash should be blocked, false otherwise + */ + public synchronized boolean shouldBlockStickyKeyHash(Consumer consumer, int stickyKeyHash) { + if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { + log.warn("[{}] Sticky key hash is not set. Allowing dispatching", dispatcherName); + return false; + } + DrainingHashEntry entry = drainingHashes.get(stickyKeyHash); + // if the entry is not found, the hash is not draining. Don't block the hash. + if (entry == null) { + return false; + } + // hash has been reassigned to the original consumer, remove the entry + // and don't block the hash + if (entry.getConsumer() == consumer) { + log.info("[{}] Hash {} has been reassigned consumer {}. " + + "The draining hash entry with refCount={} will be removed.", + dispatcherName, stickyKeyHash, entry.getConsumer(), entry.refCount); + drainingHashes.remove(stickyKeyHash, entry); + return false; + } + // increment the blocked count which is used to determine if the hash is blocking + // dispatching to other consumers + entry.incrementBlockedCount(); + // block the hash + return true; + } + + /** + * Get the entry for a given sticky key hash. + * + * @param stickyKeyHash the sticky key hash + * @return the draining hash entry, or null if not found + */ + public synchronized DrainingHashEntry getEntry(int stickyKeyHash) { + return stickyKeyHash != 0 ? drainingHashes.get(stickyKeyHash) : null; + } + + /** + * Clear all entries in the draining hashes tracker. + */ + public synchronized void clear() { + drainingHashes.clear(); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java index efa89a8ff16f6..e4870bf251ecb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryAndMetadata.java @@ -20,7 +20,7 @@ import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; -import java.nio.charset.StandardCharsets; +import java.util.function.ToIntFunction; import javax.annotation.Nullable; import lombok.Getter; import org.apache.bookkeeper.mledger.Entry; @@ -29,11 +29,12 @@ import org.apache.pulsar.common.protocol.Commands; public class EntryAndMetadata implements Entry { - + private static final int STICKY_KEY_HASH_NOT_INITIALIZED = -1; private final Entry entry; @Getter @Nullable private final MessageMetadata metadata; + int stickyKeyHash = STICKY_KEY_HASH_NOT_INITIALIZED; private EntryAndMetadata(final Entry entry, @Nullable final MessageMetadata metadata) { this.entry = entry; @@ -45,22 +46,15 @@ public static EntryAndMetadata create(final Entry entry, final MessageMetadata m } @VisibleForTesting - static EntryAndMetadata create(final Entry entry) { + public static EntryAndMetadata create(final Entry entry) { return create(entry, Commands.peekAndCopyMessageMetadata(entry.getDataBuffer(), "", -1)); } public byte[] getStickyKey() { if (metadata != null) { - if (metadata.hasOrderingKey()) { - return metadata.getOrderingKey(); - } else if (metadata.hasPartitionKey()) { - return metadata.getPartitionKey().getBytes(StandardCharsets.UTF_8); - } else if (metadata.hasProducerName() && metadata.hasSequenceId()) { - String fallbackKey = metadata.getProducerName() + "-" + metadata.getSequenceId(); - return fallbackKey.getBytes(StandardCharsets.UTF_8); - } + return Commands.resolveStickyKey(metadata); } - return "NONE_KEY".getBytes(StandardCharsets.UTF_8); + return Commands.NONE_KEY; } @Override @@ -114,4 +108,32 @@ public long getEntryId() { public boolean release() { return entry.release(); } + + /** + * Get cached sticky key hash or calculate it based on the sticky key if it's not cached. + * + * @param makeStickyKeyHash function to calculate the sticky key hash + * @return the sticky key hash + */ + public int getOrUpdateCachedStickyKeyHash(ToIntFunction makeStickyKeyHash) { + if (stickyKeyHash == STICKY_KEY_HASH_NOT_INITIALIZED) { + stickyKeyHash = makeStickyKeyHash.applyAsInt(getStickyKey()); + } + return stickyKeyHash; + } + + /** + * Get cached sticky key hash or return STICKY_KEY_HASH_NOT_SET if it's not cached. + * + * @return the cached sticky key hash or STICKY_KEY_HASH_NOT_SET if it's not cached + */ + public int getCachedStickyKeyHash() { + return stickyKeyHash != STICKY_KEY_HASH_NOT_INITIALIZED ? stickyKeyHash + : StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; + } + + @VisibleForTesting + public Entry unwrap() { + return entry; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAssignment.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAssignment.java new file mode 100644 index 0000000000000..80996c395e352 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAssignment.java @@ -0,0 +1,26 @@ +/* + * 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.service; + +import org.apache.pulsar.client.api.Range; + +/** + * Hash range assignment in {@link StickyKeyConsumerSelector} implementations. + */ +public record HashRangeAssignment(Range range, Consumer consumer) {} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java index a9fea5b39bf82..b90aef739f2b1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java @@ -27,7 +27,6 @@ import java.util.concurrent.ConcurrentSkipListMap; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException; import org.apache.pulsar.client.api.Range; -import org.apache.pulsar.common.util.FutureUtil; /** * This is a consumer selector based fixed hash range. @@ -56,11 +55,11 @@ * */ public class HashRangeAutoSplitStickyKeyConsumerSelector implements StickyKeyConsumerSelector { - private final int rangeSize; - + private final Range keyHashRange; private final ConcurrentSkipListMap rangeMap; private final Map consumerRange; + private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot; public HashRangeAutoSplitStickyKeyConsumerSelector() { this(DEFAULT_RANGE_SIZE); @@ -76,10 +75,12 @@ public HashRangeAutoSplitStickyKeyConsumerSelector(int rangeSize) { this.rangeMap = new ConcurrentSkipListMap<>(); this.consumerRange = new HashMap<>(); this.rangeSize = rangeSize; + this.keyHashRange = Range.of(0, rangeSize - 1); + this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty(); } @Override - public synchronized CompletableFuture addConsumer(Consumer consumer) { + public synchronized CompletableFuture addConsumer(Consumer consumer) { if (rangeMap.isEmpty()) { rangeMap.put(rangeSize, consumer); consumerRange.put(consumer, rangeSize); @@ -87,14 +88,18 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { try { splitRange(findBiggestRange(), consumer); } catch (ConsumerAssignException e) { - return FutureUtil.failedFuture(e); + return CompletableFuture.failedFuture(e); } } - return CompletableFuture.completedFuture(null); + ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); + ImpactedConsumersResult impactedConsumers = + consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); + consumerHashAssignmentsSnapshot = assignmentsAfter; + return CompletableFuture.completedFuture(impactedConsumers); } @Override - public synchronized void removeConsumer(Consumer consumer) { + public synchronized ImpactedConsumersResult removeConsumer(Consumer consumer) { Integer removeRange = consumerRange.remove(consumer); if (removeRange != null) { if (removeRange == rangeSize && rangeMap.size() > 1) { @@ -106,28 +111,40 @@ public synchronized void removeConsumer(Consumer consumer) { rangeMap.remove(removeRange); } } + ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); + ImpactedConsumersResult impactedConsumers = + consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); + consumerHashAssignmentsSnapshot = assignmentsAfter; + return impactedConsumers; } @Override public Consumer select(int hash) { if (!rangeMap.isEmpty()) { - int slot = hash % rangeSize; - return rangeMap.ceilingEntry(slot).getValue(); + return rangeMap.ceilingEntry(hash).getValue(); } else { return null; } } @Override - public Map> getConsumerKeyHashRanges() { - Map> result = new HashMap<>(); + public Range getKeyHashRange() { + return keyHashRange; + } + + @Override + public synchronized ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() { + return consumerHashAssignmentsSnapshot; + } + + private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() { + List result = new ArrayList<>(); int start = 0; - for (Map.Entry entry: rangeMap.entrySet()) { - result.computeIfAbsent(entry.getValue(), key -> new ArrayList<>()) - .add(Range.of(start, entry.getKey())); + for (Entry entry: rangeMap.entrySet()) { + result.add(new HashRangeAssignment(Range.of(start, entry.getKey()), entry.getValue())); start = entry.getKey() + 1; } - return result; + return ConsumerHashAssignmentsSnapshot.of(result); } private int findBiggestRange() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java index 78bad1b2c400e..7c76d9dca7456 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java @@ -20,7 +20,6 @@ import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -36,9 +35,10 @@ * else there'll be chance that a key fall in a `whole` that not handled by any consumer. */ public class HashRangeExclusiveStickyKeyConsumerSelector implements StickyKeyConsumerSelector { - private final int rangeSize; + private final Range keyHashRange; private final ConcurrentSkipListMap rangeMap; + private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot; public HashRangeExclusiveStickyKeyConsumerSelector() { this(DEFAULT_RANGE_SIZE); @@ -50,21 +50,23 @@ public HashRangeExclusiveStickyKeyConsumerSelector(int rangeSize) { throw new IllegalArgumentException("range size must greater than 0"); } this.rangeSize = rangeSize; + this.keyHashRange = Range.of(0, rangeSize - 1); this.rangeMap = new ConcurrentSkipListMap<>(); + this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty(); } @Override - public synchronized CompletableFuture addConsumer(Consumer consumer) { - return validateKeySharedMeta(consumer).thenRun(() -> { + public synchronized CompletableFuture addConsumer(Consumer consumer) { + return validateKeySharedMeta(consumer).thenApply(__ -> { try { - internalAddConsumer(consumer); + return internalAddConsumer(consumer); } catch (BrokerServiceException.ConsumerAssignException e) { throw FutureUtil.wrapToCompletionException(e); } }); } - private synchronized void internalAddConsumer(Consumer consumer) + private synchronized ImpactedConsumersResult internalAddConsumer(Consumer consumer) throws BrokerServiceException.ConsumerAssignException { Consumer conflictingConsumer = findConflictingConsumer(consumer.getKeySharedMeta().getHashRangesList()); if (conflictingConsumer != null) { @@ -75,37 +77,49 @@ private synchronized void internalAddConsumer(Consumer consumer) rangeMap.put(intRange.getStart(), consumer); rangeMap.put(intRange.getEnd(), consumer); } + ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); + ImpactedConsumersResult impactedConsumers = + consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); + consumerHashAssignmentsSnapshot = assignmentsAfter; + return impactedConsumers; } @Override - public void removeConsumer(Consumer consumer) { + public synchronized ImpactedConsumersResult removeConsumer(Consumer consumer) { rangeMap.entrySet().removeIf(entry -> entry.getValue().equals(consumer)); + ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); + ImpactedConsumersResult impactedConsumers = + consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); + consumerHashAssignmentsSnapshot = assignmentsAfter; + return impactedConsumers; } @Override - public Map> getConsumerKeyHashRanges() { - Map> result = new HashMap<>(); + public synchronized ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() { + return consumerHashAssignmentsSnapshot; + } + + private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() { + List result = new ArrayList<>(); Map.Entry prev = null; for (Map.Entry entry: rangeMap.entrySet()) { if (prev == null) { prev = entry; } else { if (prev.getValue().equals(entry.getValue())) { - result.computeIfAbsent(entry.getValue(), key -> new ArrayList<>()) - .add(Range.of(prev.getKey(), entry.getKey())); + result.add(new HashRangeAssignment(Range.of(prev.getKey(), entry.getKey()), entry.getValue())); } prev = null; } } - return result; + return ConsumerHashAssignmentsSnapshot.of(result); } @Override public Consumer select(int hash) { if (rangeMap.size() > 0) { - int slot = hash % rangeSize; - Map.Entry ceilingEntry = rangeMap.ceilingEntry(slot); - Map.Entry floorEntry = rangeMap.floorEntry(slot); + Map.Entry ceilingEntry = rangeMap.ceilingEntry(hash); + Map.Entry floorEntry = rangeMap.floorEntry(hash); Consumer ceilingConsumer = ceilingEntry != null ? ceilingEntry.getValue() : null; Consumer floorConsumer = floorEntry != null ? floorEntry.getValue() : null; if (floorConsumer != null && floorConsumer.equals(ceilingConsumer)) { @@ -173,4 +187,8 @@ Map getRangeConsumer() { return Collections.unmodifiableMap(rangeMap); } + @Override + public Range getKeyHashRange() { + return keyHashRange; + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ImpactedConsumersResult.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ImpactedConsumersResult.java new file mode 100644 index 0000000000000..a525b0501d767 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ImpactedConsumersResult.java @@ -0,0 +1,59 @@ +/* + * 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.service; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Map; +import lombok.EqualsAndHashCode; +import lombok.ToString; + +/** + * Represents the consumers that were impacted by a hash range change in a {@link StickyKeyConsumerSelector} + * at a point in time. + */ +@EqualsAndHashCode +@ToString +public class ImpactedConsumersResult { + public interface RemovedHashRangesProcessor { + void process(Consumer consumer, RemovedHashRanges removedHashRanges); + } + + private final Map removedHashRanges; + + private ImpactedConsumersResult(Map removedHashRanges) { + this.removedHashRanges = removedHashRanges; + } + + public static ImpactedConsumersResult of(Map removedHashRanges) { + return new ImpactedConsumersResult(removedHashRanges); + } + + public void processRemovedHashRanges(RemovedHashRangesProcessor processor) { + removedHashRanges.forEach((c, r) -> processor.process(c, r)); + } + + public boolean isEmpty() { + return removedHashRanges.isEmpty(); + } + + @VisibleForTesting + Map getRemovedHashRanges() { + return removedHashRanges; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PendingAcksMap.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PendingAcksMap.java new file mode 100644 index 0000000000000..7a728a037dc62 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PendingAcksMap.java @@ -0,0 +1,424 @@ +/* + * 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.service; + +import it.unimi.dsi.fastutil.ints.IntIntPair; +import it.unimi.dsi.fastutil.longs.Long2ObjectMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectRBTreeMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectSortedMap; +import it.unimi.dsi.fastutil.objects.ObjectBidirectionalIterator; +import java.util.Map; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Supplier; + +/** + * A thread-safe map to store pending acks in the consumer. + * + * The locking solution is used for the draining hashes solution + * to ensure that there's a consistent view of the pending acks. This is needed in the DrainingHashesTracker + * to ensure that the reference counts are consistent at all times. + * Calling forEachAndClose will ensure that no more entries can be added, + * therefore no other thread cannot send out entries while the forEachAndClose is being called. + * remove is also locked to ensure that there aren't races in the removal of entries while forEachAndClose is + * running. + */ +public class PendingAcksMap { + /** + * Callback interface for handling the addition of pending acknowledgments. + */ + public interface PendingAcksAddHandler { + /** + * Handle the addition of a pending acknowledgment. + * + * @param consumer the consumer + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @param stickyKeyHash the sticky key hash + * @return true if the addition is allowed, false otherwise + */ + boolean handleAdding(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash); + } + + /** + * Callback interface for handling the removal of pending acknowledgments. + */ + public interface PendingAcksRemoveHandler { + /** + * Handle the removal of a pending acknowledgment. + * + * @param consumer the consumer + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @param stickyKeyHash the sticky key hash + * @param closing true if the pending ack is being removed because the map is being closed, false + * otherwise + */ + void handleRemoving(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash, boolean closing); + /** + * Start a batch of pending acknowledgment removals. + */ + void startBatch(); + /** + * End a batch of pending acknowledgment removals. + */ + void endBatch(); + } + + /** + * Callback interface for processing pending acknowledgments. + */ + public interface PendingAcksConsumer { + /** + * Accept a pending acknowledgment. + * + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @param batchSize the batch size + * @param stickyKeyHash the sticky key hash + */ + void accept(long ledgerId, long entryId, int batchSize, int stickyKeyHash); + } + + private final Consumer consumer; + private final Long2ObjectSortedMap> pendingAcks; + private final Supplier pendingAcksAddHandlerSupplier; + private final Supplier pendingAcksRemoveHandlerSupplier; + private final Lock readLock; + private final Lock writeLock; + private boolean closed = false; + + PendingAcksMap(Consumer consumer, Supplier pendingAcksAddHandlerSupplier, + Supplier pendingAcksRemoveHandlerSupplier) { + this.consumer = consumer; + this.pendingAcks = new Long2ObjectRBTreeMap<>(); + this.pendingAcksAddHandlerSupplier = pendingAcksAddHandlerSupplier; + this.pendingAcksRemoveHandlerSupplier = pendingAcksRemoveHandlerSupplier; + ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); + this.writeLock = readWriteLock.writeLock(); + this.readLock = readWriteLock.readLock(); + } + + /** + * Add a pending ack to the map if it's allowed to send a message with the given sticky key hash. + * If this method returns false, it means that the pending ack was not added, and it's not allowed to send a + * message. In that case, the caller should not send a message and skip the entry. + * The sending could be disallowed if the sticky key hash is blocked in the Key_Shared subscription. + * + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @param batchSize the batch size + * @param stickyKeyHash the sticky key hash + * @return true if the pending ack was added, and it's allowed to send a message, false otherwise + */ + public boolean addPendingAckIfAllowed(long ledgerId, long entryId, int batchSize, int stickyKeyHash) { + try { + writeLock.lock(); + // prevent adding sticky hash to pending acks if the PendingAcksMap has already been closed + // and there's a race condition between closing the consumer and sending new messages + if (closed) { + return false; + } + // prevent adding sticky hash to pending acks if it's already in draining hashes + // to avoid any race conditions that would break consistency + PendingAcksAddHandler pendingAcksAddHandler = pendingAcksAddHandlerSupplier.get(); + if (pendingAcksAddHandler != null + && !pendingAcksAddHandler.handleAdding(consumer, ledgerId, entryId, stickyKeyHash)) { + return false; + } + Long2ObjectSortedMap ledgerPendingAcks = + pendingAcks.computeIfAbsent(ledgerId, k -> new Long2ObjectRBTreeMap<>()); + ledgerPendingAcks.put(entryId, IntIntPair.of(batchSize, stickyKeyHash)); + return true; + } finally { + writeLock.unlock(); + } + } + + /** + * Get the size of the pending acks map. + * + * @return the size of the pending acks map + */ + public long size() { + try { + readLock.lock(); + return pendingAcks.values().stream().mapToInt(Long2ObjectSortedMap::size).sum(); + } finally { + readLock.unlock(); + } + } + + /** + * Iterate over all the pending acks and process them using the given processor. + * + * @param processor the processor to handle each pending ack + */ + public void forEach(PendingAcksConsumer processor) { + try { + readLock.lock(); + processPendingAcks(processor); + } finally { + readLock.unlock(); + } + } + + // iterate all pending acks and process them + private void processPendingAcks(PendingAcksConsumer processor) { + // this code uses for loops intentionally, don't refactor to use forEach + // iterate the outer map + for (Map.Entry> entry : pendingAcks.entrySet()) { + Long ledgerId = entry.getKey(); + Long2ObjectSortedMap ledgerPendingAcks = entry.getValue(); + // iterate the inner map + for (Map.Entry e : ledgerPendingAcks.entrySet()) { + Long entryId = e.getKey(); + IntIntPair batchSizeAndStickyKeyHash = e.getValue(); + processor.accept(ledgerId, entryId, batchSizeAndStickyKeyHash.leftInt(), + batchSizeAndStickyKeyHash.rightInt()); + } + } + } + + /** + * Iterate over all the pending acks and close the map so that no more entries can be added. + * All entries are removed. + * + * @param processor the processor to handle each pending ack + */ + public void forEachAndClose(PendingAcksConsumer processor) { + try { + writeLock.lock(); + closed = true; + PendingAcksRemoveHandler pendingAcksRemoveHandler = pendingAcksRemoveHandlerSupplier.get(); + if (pendingAcksRemoveHandler != null) { + try { + pendingAcksRemoveHandler.startBatch(); + processPendingAcks((ledgerId, entryId, batchSize, stickyKeyHash) -> { + processor.accept(ledgerId, entryId, batchSize, stickyKeyHash); + pendingAcksRemoveHandler.handleRemoving(consumer, ledgerId, entryId, stickyKeyHash, closed); + }); + } finally { + pendingAcksRemoveHandler.endBatch(); + } + } else { + processPendingAcks(processor); + } + pendingAcks.clear(); + } finally { + writeLock.unlock(); + } + } + + /** + * Check if the map contains a pending ack for the given ledger ID and entry ID. + * + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @return true if the map contains the pending ack, false otherwise + */ + public boolean contains(long ledgerId, long entryId) { + try { + readLock.lock(); + Long2ObjectSortedMap ledgerMap = pendingAcks.get(ledgerId); + if (ledgerMap == null) { + return false; + } + return ledgerMap.containsKey(entryId); + } finally { + readLock.unlock(); + } + } + + /** + * Get the pending ack for the given ledger ID and entry ID. + * + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @return the pending ack, or null if not found + */ + public IntIntPair get(long ledgerId, long entryId) { + try { + readLock.lock(); + Long2ObjectSortedMap ledgerMap = pendingAcks.get(ledgerId); + if (ledgerMap == null) { + return null; + } + return ledgerMap.get(entryId); + } finally { + readLock.unlock(); + } + } + + /** + * Remove the pending ack for the given ledger ID, entry ID, batch size, and sticky key hash. + * + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @param batchSize the batch size + * @param stickyKeyHash the sticky key hash + * @return true if the pending ack was removed, false otherwise + */ + public boolean remove(long ledgerId, long entryId, int batchSize, int stickyKeyHash) { + try { + writeLock.lock(); + Long2ObjectSortedMap ledgerMap = pendingAcks.get(ledgerId); + if (ledgerMap == null) { + return false; + } + boolean removed = ledgerMap.remove(entryId, IntIntPair.of(batchSize, stickyKeyHash)); + if (removed) { + handleRemovePendingAck(ledgerId, entryId, stickyKeyHash); + } + if (removed && ledgerMap.isEmpty()) { + pendingAcks.remove(ledgerId); + } + return removed; + } finally { + writeLock.unlock(); + } + } + + /** + * Remove the pending ack for the given ledger ID and entry ID. + * + * @param ledgerId the ledger ID + * @param entryId the entry ID + * @return true if the pending ack was removed, false otherwise + */ + public boolean remove(long ledgerId, long entryId) { + try { + writeLock.lock(); + Long2ObjectSortedMap ledgerMap = pendingAcks.get(ledgerId); + if (ledgerMap == null) { + return false; + } + IntIntPair removedEntry = ledgerMap.remove(entryId); + boolean removed = removedEntry != null; + if (removed) { + int stickyKeyHash = removedEntry.rightInt(); + handleRemovePendingAck(ledgerId, entryId, stickyKeyHash); + } + if (removed && ledgerMap.isEmpty()) { + pendingAcks.remove(ledgerId); + } + return removed; + } finally { + writeLock.unlock(); + } + } + + /** + * Remove all pending acks up to the given ledger ID and entry ID. + * + * @param markDeleteLedgerId the ledger ID up to which to remove pending acks + * @param markDeleteEntryId the entry ID up to which to remove pending acks + */ + public void removeAllUpTo(long markDeleteLedgerId, long markDeleteEntryId) { + internalRemoveAllUpTo(markDeleteLedgerId, markDeleteEntryId, false); + } + + /** + * Removes all pending acknowledgments up to the specified ledger ID and entry ID. + * + * ReadWriteLock doesn't support upgrading from read lock to write lock. + * This method first checks if there's anything to remove using a read lock and if there is, exits + * and retries with a write lock to make the removals. + * + * @param markDeleteLedgerId the ledger ID up to which to remove pending acks + * @param markDeleteEntryId the entry ID up to which to remove pending acks + * @param useWriteLock true if the method should use a write lock, false otherwise + */ + private void internalRemoveAllUpTo(long markDeleteLedgerId, long markDeleteEntryId, boolean useWriteLock) { + PendingAcksRemoveHandler pendingAcksRemoveHandler = pendingAcksRemoveHandlerSupplier.get(); + // track if the write lock was acquired + boolean acquiredWriteLock = false; + // track if a batch was started + boolean batchStarted = false; + // track if the method should retry with a write lock + boolean retryWithWriteLock = false; + try { + if (useWriteLock) { + writeLock.lock(); + acquiredWriteLock = true; + } else { + readLock.lock(); + } + ObjectBidirectionalIterator>> ledgerMapIterator = + pendingAcks.headMap(markDeleteLedgerId + 1).long2ObjectEntrySet().iterator(); + while (ledgerMapIterator.hasNext()) { + Long2ObjectMap.Entry> entry = ledgerMapIterator.next(); + long ledgerId = entry.getLongKey(); + Long2ObjectSortedMap ledgerMap = entry.getValue(); + Long2ObjectSortedMap ledgerMapHead; + if (ledgerId == markDeleteLedgerId) { + ledgerMapHead = ledgerMap.headMap(markDeleteEntryId + 1); + } else { + ledgerMapHead = ledgerMap; + } + ObjectBidirectionalIterator> entryMapIterator = + ledgerMapHead.long2ObjectEntrySet().iterator(); + while (entryMapIterator.hasNext()) { + Long2ObjectMap.Entry intIntPairEntry = entryMapIterator.next(); + long entryId = intIntPairEntry.getLongKey(); + if (!acquiredWriteLock) { + retryWithWriteLock = true; + return; + } + if (pendingAcksRemoveHandler != null) { + if (!batchStarted) { + pendingAcksRemoveHandler.startBatch(); + batchStarted = true; + } + int stickyKeyHash = intIntPairEntry.getValue().rightInt(); + pendingAcksRemoveHandler.handleRemoving(consumer, ledgerId, entryId, stickyKeyHash, closed); + } + entryMapIterator.remove(); + } + if (ledgerMap.isEmpty()) { + if (!acquiredWriteLock) { + retryWithWriteLock = true; + return; + } + ledgerMapIterator.remove(); + } + } + } finally { + if (batchStarted) { + pendingAcksRemoveHandler.endBatch(); + } + if (acquiredWriteLock) { + writeLock.unlock(); + } else { + readLock.unlock(); + if (retryWithWriteLock) { + internalRemoveAllUpTo(markDeleteLedgerId, markDeleteEntryId, true); + } + } + } + } + + private void handleRemovePendingAck(long ledgerId, long entryId, int stickyKeyHash) { + PendingAcksRemoveHandler pendingAcksRemoveHandler = pendingAcksRemoveHandlerSupplier.get(); + if (pendingAcksRemoveHandler != null) { + pendingAcksRemoveHandler.handleRemoving(consumer, ledgerId, entryId, stickyKeyHash, closed); + } + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java index b4578711027ef..c39b722888f71 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java @@ -186,7 +186,7 @@ public boolean isSuccessorTo(Producer other) { && other.getEpoch() < epoch; } - public void publishMessage(long producerId, long sequenceId, ByteBuf headersAndPayload, long batchSize, + public void publishMessage(long producerId, long sequenceId, ByteBuf headersAndPayload, int batchSize, boolean isChunked, boolean isMarker, Position position) { if (checkAndStartPublish(producerId, sequenceId, headersAndPayload, batchSize, position)) { publishMessageToTopic(headersAndPayload, sequenceId, batchSize, isChunked, isMarker, position); @@ -194,7 +194,7 @@ public void publishMessage(long producerId, long sequenceId, ByteBuf headersAndP } public void publishMessage(long producerId, long lowestSequenceId, long highestSequenceId, - ByteBuf headersAndPayload, long batchSize, boolean isChunked, boolean isMarker, Position position) { + ByteBuf headersAndPayload, int batchSize, boolean isChunked, boolean isMarker, Position position) { if (lowestSequenceId > highestSequenceId) { cnx.execute(() -> { cnx.getCommandSender().sendSendError(producerId, highestSequenceId, ServerError.MetadataError, @@ -209,7 +209,7 @@ public void publishMessage(long producerId, long lowestSequenceId, long highestS } } - public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf headersAndPayload, long batchSize, + public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf headersAndPayload, int batchSize, Position position) { if (!isShadowTopic && position != null) { cnx.execute(() -> { @@ -267,7 +267,7 @@ public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf he return true; } - private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, long batchSize, boolean isChunked, + private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, int batchSize, boolean isChunked, boolean isMarker, Position position) { MessagePublishContext messagePublishContext = MessagePublishContext.get(this, sequenceId, headersAndPayload.readableBytes(), @@ -280,7 +280,7 @@ private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, l } private void publishMessageToTopic(ByteBuf headersAndPayload, long lowestSequenceId, long highestSequenceId, - long batchSize, boolean isChunked, boolean isMarker, Position position) { + int batchSize, boolean isChunked, boolean isMarker, Position position) { MessagePublishContext messagePublishContext = MessagePublishContext.get(this, lowestSequenceId, highestSequenceId, headersAndPayload.readableBytes(), batchSize, isChunked, System.nanoTime(), isMarker, position); @@ -375,7 +375,7 @@ private static final class MessagePublishContext implements PublishContext, Runn private long ledgerId; private long entryId; private int msgSize; - private long batchSize; + private int batchSize; private boolean chunked; private boolean isMarker; @@ -551,7 +551,7 @@ public void run() { recycle(); } - static MessagePublishContext get(Producer producer, long sequenceId, int msgSize, long batchSize, + static MessagePublishContext get(Producer producer, long sequenceId, int msgSize, int batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) { MessagePublishContext callback = RECYCLER.get(); callback.producer = producer; @@ -572,7 +572,7 @@ static MessagePublishContext get(Producer producer, long sequenceId, int msgSize } static MessagePublishContext get(Producer producer, long lowestSequenceId, long highestSequenceId, int msgSize, - long batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) { + int batchSize, boolean chunked, long startTimeNs, boolean isMarker, Position position) { MessagePublishContext callback = RECYCLER.get(); callback.producer = producer; callback.sequenceId = lowestSequenceId; @@ -628,7 +628,7 @@ public void recycle() { msgSize = 0; ledgerId = -1L; entryId = -1L; - batchSize = 0L; + batchSize = 0; startTimeNs = -1L; chunked = false; isMarker = false; @@ -795,7 +795,7 @@ public void checkEncryption() { } public void publishTxnMessage(TxnID txnID, long producerId, long sequenceId, long highSequenceId, - ByteBuf headersAndPayload, long batchSize, boolean isChunked, boolean isMarker) { + ByteBuf headersAndPayload, int batchSize, boolean isChunked, boolean isMarker) { if (!checkAndStartPublish(producerId, sequenceId, headersAndPayload, batchSize, null)) { return; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java new file mode 100644 index 0000000000000..1833c243f8955 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/RemovedHashRanges.java @@ -0,0 +1,74 @@ +/* + * 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.service; + +import java.util.List; +import lombok.EqualsAndHashCode; +import lombok.ToString; +import org.apache.pulsar.client.api.Range; + +/** + * Represents the hash ranges which were removed from an existing consumer by a change in the hash range assignments. + */ +@EqualsAndHashCode +@ToString +public class RemovedHashRanges { + private final Range[] sortedRanges; + + private RemovedHashRanges(List ranges) { + // Converts the set of ranges to an array to avoid iterator allocation + // when the ranges are iterator multiple times in the pending acknowledgments loop. + this.sortedRanges = ranges.toArray(new Range[0]); + validateSortedRanges(); + } + + private void validateSortedRanges() { + for (int i = 0; i < sortedRanges.length - 1; i++) { + if (sortedRanges[i].getStart() >= sortedRanges[i + 1].getStart()) { + throw new IllegalArgumentException( + "Ranges must be sorted: " + sortedRanges[i] + " and " + sortedRanges[i + 1]); + } + if (sortedRanges[i].getEnd() >= sortedRanges[i + 1].getStart()) { + throw new IllegalArgumentException( + "Ranges must not overlap: " + sortedRanges[i] + " and " + sortedRanges[i + 1]); + } + } + } + + public static RemovedHashRanges of(List ranges) { + return new RemovedHashRanges(ranges); + } + + /** + * Checks if the sticky key hash is contained in the impacted hash ranges. + */ + public boolean containsStickyKey(int stickyKeyHash) { + for (Range range : sortedRanges) { + if (range.contains(stickyKeyHash)) { + return true; + } + // Since ranges are sorted, stop checking further ranges if the start of the current range is + // greater than the stickyKeyHash. + if (range.getStart() > stickyKeyHash) { + return false; + } + } + return false; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java index e0ed75020bc82..1ead3f946c24d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java @@ -22,50 +22,94 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import org.apache.pulsar.client.api.Range; -import org.apache.pulsar.common.util.Murmur3_32Hash; +/** + * Abstraction for selecting the same consumer based on a key. + * This interface provides methods to add and remove consumers, + * select a consumer based on a sticky key or hash, and retrieve + * the hash range assignments for consumers. This is used by the Key_Shared implementation. + */ public interface StickyKeyConsumerSelector { - + /** + * The default range size used for hashing. + * This should be a power of 2 so that it's compatible with all implementations. + */ int DEFAULT_RANGE_SIZE = 2 << 15; + /** + * The value used to indicate that sticky key hash is not set. + * This value cannot be -1 since some of the data structures require non-negative values. + */ + int STICKY_KEY_HASH_NOT_SET = 0; + /** * Add a new consumer. * - * @param consumer new consumer + * @param consumer the new consumer to be added + * @return a CompletableFuture that completes with the result of impacted consumers. + * The result contains information about the existing consumers whose hash ranges were affected + * by the addition of the new consumer. */ - CompletableFuture addConsumer(Consumer consumer); + CompletableFuture addConsumer(Consumer consumer); /** * Remove the consumer. - * @param consumer consumer to be removed + * + * @param consumer the consumer to be removed + * @return the result of impacted consumers. The result contains information about the existing consumers + * whose hash ranges were affected by the removal of the consumer. */ - void removeConsumer(Consumer consumer); + ImpactedConsumersResult removeConsumer(Consumer consumer); /** * Select a consumer by sticky key. * - * @param stickyKey sticky key - * @return consumer + * @param stickyKey the sticky key to select the consumer + * @return the selected consumer */ default Consumer select(byte[] stickyKey) { return select(makeStickyKeyHash(stickyKey)); } - static int makeStickyKeyHash(byte[] stickyKey) { - return Murmur3_32Hash.getInstance().makeHash(stickyKey); + /** + * Make a hash from the sticky key. The hash value is in the range returned by the {@link #getKeyHashRange()} + * method instead of in the full range of integers. In other words, this returns the "slot". + * + * @param stickyKey the sticky key to hash + * @return the generated hash value + */ + default int makeStickyKeyHash(byte[] stickyKey) { + return StickyKeyConsumerSelectorUtils.makeStickyKeyHash(stickyKey, getKeyHashRange()); } /** * Select a consumer by hash. * - * @param hash hash corresponding to sticky key - * @return consumer + * @param hash the hash corresponding to the sticky key + * @return the selected consumer */ Consumer select(int hash); + /** + * Get the full range of hash values used by this selector. The upper bound is exclusive. + * + * @return the full range of hash values + */ + Range getKeyHashRange(); + /** * Get key hash ranges handled by each consumer. - * @return A map where key is a consumer name and value is list of hash range it receiving message for. + * + * @return a map where the key is a consumer and the value is a list of hash ranges it is receiving messages for + */ + default Map> getConsumerKeyHashRanges() { + return getConsumerHashAssignmentsSnapshot().getRangesByConsumer(); + } + + /** + * Get the current mappings of hash range to consumer. + * + * @return a snapshot of the consumer hash assignments */ - Map> getConsumerKeyHashRanges(); -} + ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot(); +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelectorUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelectorUtils.java new file mode 100644 index 0000000000000..03a107422ddbd --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelectorUtils.java @@ -0,0 +1,51 @@ +/* + * 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.service; + +import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; +import org.apache.pulsar.client.api.Range; +import org.apache.pulsar.common.util.Hash; +import org.apache.pulsar.common.util.Murmur3_32Hash; + +/** + * Internal utility class for {@link StickyKeyConsumerSelector} implementations. + */ +class StickyKeyConsumerSelectorUtils { + private static final Hash HASH_INSTANCE = Murmur3_32Hash.getInstance(); + + /** + * Generates a sticky key hash from the given sticky key within the specified range. + * This method shouldn't be used by other classes than {@link StickyKeyConsumerSelector} implementations. + * To create a sticky key hash, use {@link StickyKeyConsumerSelector#makeStickyKeyHash(byte[])} instead which + * is an instance method of a {@link StickyKeyConsumerSelector}. + * + * @param stickyKey the sticky key to hash + * @param fullHashRange hash range to generate the hash value within + * @return the generated hash value, ensuring it is not zero (since zero is a special value in dispatchers) + */ + static int makeStickyKeyHash(byte[] stickyKey, Range fullHashRange) { + int hashValue = HASH_INSTANCE.makeHash(stickyKey) % fullHashRange.size() + fullHashRange.getStart(); + // Avoid using STICKY_KEY_HASH_NOT_SET as hash value + if (hashValue == STICKY_KEY_HASH_NOT_SET) { + // use next value as hash value + hashValue = STICKY_KEY_HASH_NOT_SET + 1; + } + return hashValue; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java index fb7bd22de94a7..ce674cf471ef0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java @@ -101,10 +101,10 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { consumerList.remove(consumer); } throw FutureUtil.wrapToCompletionException(ex); - } else { - return value; } - })); + return value; + })).thenAccept(__ -> { + }); } @Override @@ -152,7 +152,7 @@ public void sendMessages(List entries) { for (Entry entry : entries) { byte[] stickyKey = peekStickyKey(entry.getDataBuffer()); - int stickyKeyHash = StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey); + int stickyKeyHash = selector.makeStickyKeyHash(stickyKey); Consumer consumer = selector.select(stickyKeyHash); if (consumer != null) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java index fa6e1412151b6..b34a0b454385f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.broker.service.persistent; -import com.google.common.collect.ComparisonChain; +import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; import java.util.ArrayList; import java.util.List; import java.util.NavigableSet; @@ -65,6 +65,9 @@ public void add(long ledgerId, long entryId) { public void add(long ledgerId, long entryId, long stickyKeyHash) { if (!allowOutOfOrderDelivery) { + if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { + throw new IllegalArgumentException("Sticky key hash is not set. It is required."); + } boolean inserted = hashesToBeBlocked.putIfAbsent(ledgerId, entryId, stickyKeyHash, 0); if (!inserted) { hashesToBeBlocked.put(ledgerId, entryId, stickyKeyHash, 0); @@ -108,18 +111,20 @@ public Long getHash(long ledgerId, long entryId) { } public void removeAllUpTo(long markDeleteLedgerId, long markDeleteEntryId) { - if (!allowOutOfOrderDelivery) { + boolean bitsCleared = messagesToRedeliver.removeUpTo(markDeleteLedgerId, markDeleteEntryId + 1); + // only if bits have been clear, and we are not allowing out of order delivery, we need to remove the hashes + // removing hashes is a relatively expensive operation, so we should only do it when necessary + if (bitsCleared && !allowOutOfOrderDelivery) { List keysToRemove = new ArrayList<>(); hashesToBeBlocked.forEach((ledgerId, entryId, stickyKeyHash, none) -> { - if (ComparisonChain.start().compare(ledgerId, markDeleteLedgerId).compare(entryId, markDeleteEntryId) - .result() <= 0) { + if (ledgerId < markDeleteLedgerId || (ledgerId == markDeleteLedgerId && entryId <= markDeleteEntryId)) { keysToRemove.add(new LongPair(ledgerId, entryId)); } }); - keysToRemove.forEach(longPair -> removeFromHashBlocker(longPair.first, longPair.second)); - keysToRemove.clear(); + for (LongPair longPair : keysToRemove) { + removeFromHashBlocker(longPair.first, longPair.second); + } } - messagesToRedeliver.removeUpTo(markDeleteLedgerId, markDeleteEntryId + 1); } public boolean isEmpty() { @@ -141,7 +146,7 @@ public String toString() { public boolean containsStickyKeyHashes(Set stickyKeyHashes) { if (!allowOutOfOrderDelivery) { for (Integer stickyKeyHash : stickyKeyHashes) { - if (hashesRefCount.containsKey(stickyKeyHash)) { + if (stickyKeyHash != STICKY_KEY_HASH_NOT_SET && hashesRefCount.containsKey(stickyKeyHash)) { return true; } } @@ -150,7 +155,8 @@ public boolean containsStickyKeyHashes(Set stickyKeyHashes) { } public boolean containsStickyKeyHash(int stickyKeyHash) { - return !allowOutOfOrderDelivery && hashesRefCount.containsKey(stickyKeyHash); + return !allowOutOfOrderDelivery + && stickyKeyHash != STICKY_KEY_HASH_NOT_SET && hashesRefCount.containsKey(stickyKeyHash); } public Optional getFirstPositionInReplay() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index d479d8f384ee9..73d152bab1a60 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service.persistent; +import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; @@ -46,6 +47,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; @@ -67,7 +69,6 @@ import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; import org.apache.pulsar.broker.service.SendMessageInfo; import org.apache.pulsar.broker.service.SharedConsumerAssignor; -import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; import org.apache.pulsar.broker.transaction.exception.buffer.TransactionBufferException; @@ -132,6 +133,7 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul "blockedDispatcherOnUnackedMsgs"); protected Optional dispatchRateLimiter = Optional.empty(); private AtomicBoolean isRescheduleReadInProgress = new AtomicBoolean(false); + private final AtomicBoolean readMoreEntriesAsyncRequested = new AtomicBoolean(false); protected final ExecutorService dispatchMessagesThread; private final SharedConsumerAssignor assignor; // tracks how many entries were processed by consumers in the last trySendMessagesToConsumers call @@ -144,6 +146,8 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul protected enum ReadType { Normal, Replay } + private Position lastMarkDeletePositionBeforeReadMoreEntries; + private volatile long readMoreEntriesCallCount; public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor, Subscription subscription) { @@ -164,7 +168,7 @@ public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCurso : RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED; this.readBatchSize = serviceConfig.getDispatcherMaxReadBatchSize(); this.initializeDispatchRateLimiterIfNeeded(); - this.assignor = new SharedConsumerAssignor(this::getNextConsumer, this::addMessageToReplay); + this.assignor = new SharedConsumerAssignor(this::getNextConsumer, this::addEntryToReplay); ServiceConfiguration serviceConfiguration = topic.getBrokerService().pulsar().getConfiguration(); this.readFailureBackoff = new Backoff( serviceConfiguration.getDispatcherReadFailureBackoffInitialTimeInMs(), @@ -239,8 +243,12 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE if (log.isDebugEnabled()) { log.debug("[{}] Consumer are left, reading more entries", name); } - consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { - addMessageToReplay(ledgerId, entryId, stickyKeyHash); + MutableBoolean notifyAddedToReplay = new MutableBoolean(false); + consumer.getPendingAcks().forEachAndClose((ledgerId, entryId, batchSize, stickyKeyHash) -> { + boolean addedToReplay = addMessageToReplay(ledgerId, entryId, stickyKeyHash); + if (addedToReplay) { + notifyAddedToReplay.setTrue(); + } }); totalAvailablePermits -= consumer.getAvailablePermits(); if (log.isDebugEnabled()) { @@ -248,7 +256,9 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE + "New dispatcher permit count is {}", name, consumer.getAvailablePermits(), totalAvailablePermits); } - readMoreEntries(); + if (notifyAddedToReplay.booleanValue()) { + notifyRedeliveryMessageAdded(); + } } } else { /** @@ -264,7 +274,12 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE } } - private synchronized void clearComponentsAfterRemovedAllConsumers() { + protected synchronized void internalRemoveConsumer(Consumer consumer) { + consumerSet.removeAll(consumer); + consumerList.remove(consumer); + } + + protected synchronized void clearComponentsAfterRemovedAllConsumers() { cancelPendingRead(); redeliveryMessages.clear(); @@ -298,7 +313,7 @@ private synchronized void internalConsumerFlow(Consumer consumer, int additional + "after adding {} permits", name, consumer, totalAvailablePermits, additionalNumberOfMessages); } - readMoreEntries(); + readMoreEntriesAsync(); } /** @@ -306,7 +321,13 @@ private synchronized void internalConsumerFlow(Consumer consumer, int additional * */ public void readMoreEntriesAsync() { - topic.getBrokerService().executor().execute(this::readMoreEntries); + // deduplication for readMoreEntriesAsync calls + if (readMoreEntriesAsyncRequested.compareAndSet(false, true)) { + topic.getBrokerService().executor().execute(() -> { + readMoreEntriesAsyncRequested.set(false); + readMoreEntries(); + }); + } } public synchronized void readMoreEntries() { @@ -337,6 +358,20 @@ public synchronized void readMoreEntries() { return; } + // increment the counter for readMoreEntries calls, to track the number of times readMoreEntries is called + readMoreEntriesCallCount++; + + // remove possible expired messages from redelivery tracker and pending acks + Position markDeletePosition = cursor.getMarkDeletedPosition(); + if (lastMarkDeletePositionBeforeReadMoreEntries != markDeletePosition) { + redeliveryMessages.removeAllUpTo(markDeletePosition.getLedgerId(), markDeletePosition.getEntryId()); + for (Consumer consumer : consumerList) { + consumer.getPendingAcks() + .removeAllUpTo(markDeletePosition.getLedgerId(), markDeletePosition.getEntryId()); + } + lastMarkDeletePositionBeforeReadMoreEntries = markDeletePosition; + } + // totalAvailablePermits may be updated by other threads int firstAvailableConsumerPermits = getFirstAvailableConsumerPermits(); int currentTotalAvailablePermits = Math.max(totalAvailablePermits, firstAvailableConsumerPermits); @@ -396,21 +431,8 @@ public synchronized void readMoreEntries() { updateMinReplayedPosition(); messagesToRead = Math.min(messagesToRead, getMaxEntriesReadLimit()); - - // Filter out and skip read delayed messages exist in DelayedDeliveryTracker - if (delayedDeliveryTracker.isPresent()) { - Predicate skipCondition = null; - final DelayedDeliveryTracker deliveryTracker = delayedDeliveryTracker.get(); - if (deliveryTracker instanceof BucketDelayedDeliveryTracker) { - skipCondition = position -> ((BucketDelayedDeliveryTracker) deliveryTracker) - .containsMessage(position.getLedgerId(), position.getEntryId()); - } - cursor.asyncReadEntriesWithSkipOrWait(messagesToRead, bytesToRead, this, ReadType.Normal, - topic.getMaxReadPosition(), skipCondition); - } else { - cursor.asyncReadEntriesOrWait(messagesToRead, bytesToRead, this, ReadType.Normal, - topic.getMaxReadPosition()); - } + cursor.asyncReadEntriesWithSkipOrWait(messagesToRead, bytesToRead, this, ReadType.Normal, + topic.getMaxReadPosition(), createReadEntriesSkipConditionForNormalRead()); } else { if (log.isDebugEnabled()) { log.debug("[{}] Cannot schedule next read until previous one is done", name); @@ -423,6 +445,19 @@ public synchronized void readMoreEntries() { } } + protected Predicate createReadEntriesSkipConditionForNormalRead() { + Predicate skipCondition = null; + // Filter out and skip read delayed messages exist in DelayedDeliveryTracker + if (delayedDeliveryTracker.isPresent()) { + final DelayedDeliveryTracker deliveryTracker = delayedDeliveryTracker.get(); + if (deliveryTracker instanceof BucketDelayedDeliveryTracker) { + skipCondition = position -> ((BucketDelayedDeliveryTracker) deliveryTracker) + .containsMessage(position.getLedgerId(), position.getEntryId()); + } + } + return skipCondition; + } + /** * Sets a hard limit on the number of entries to read from the Managed Ledger. * Subclasses can override this method to set a different limit. @@ -447,6 +482,10 @@ protected void handleNormalReadNotAllowed() { // do nothing } + protected long getReadMoreEntriesCallCount() { + return readMoreEntriesCallCount; + } + /** * Controls whether replaying entries is currently enabled. * Subclasses can override this method to temporarily disable replaying entries. @@ -702,7 +741,7 @@ public final synchronized void readEntriesComplete(List entries, Object c entries.forEach(Entry::release); cursor.rewind(); shouldRewindBeforeReadingOrReplaying = false; - readMoreEntries(); + readMoreEntriesAsync(); return; } @@ -799,8 +838,15 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis boolean hasChunk = false; for (int i = 0; i < metadataArray.length; i++) { Entry entry = entries.get(i); - MessageMetadata metadata = entry instanceof EntryAndMetadata ? ((EntryAndMetadata) entry).getMetadata() - : Commands.peekAndCopyMessageMetadata(entry.getDataBuffer(), subscription.toString(), -1); + MessageMetadata metadata; + if (entry instanceof EntryAndMetadata) { + metadata = ((EntryAndMetadata) entry).getMetadata(); + } else { + metadata = Commands.peekAndCopyMessageMetadata(entry.getDataBuffer(), subscription.toString(), -1); + // cache the metadata in the entry with EntryAndMetadata for later use to avoid re-parsing the metadata + // and to carry the metadata and calculated stickyKeyHash with the entry + entries.set(i, EntryAndMetadata.create(entry, metadata)); + } if (metadata != null) { remainingMessages += metadata.getNumMessagesInBatch(); if (!hasChunk && metadata.hasUuid()) { @@ -901,18 +947,17 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis log.debug("[{}] No consumers found with available permits, storing {} positions for later replay", name, entries.size() - start); } - entries.subList(start, entries.size()).forEach(entry -> { - addEntryToReplay(entry); - entry.release(); - }); + entries.subList(start, entries.size()).forEach(this::addEntryToReplay); } return true; } - protected void addEntryToReplay(Entry entry) { + protected boolean addEntryToReplay(Entry entry) { long stickyKeyHash = getStickyKeyHash(entry); - addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + boolean addedToReplay = addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + return addedToReplay; } private boolean sendChunkedMessagesToConsumers(ReadType readType, @@ -930,6 +975,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, long totalEntries = 0; long totalEntriesProcessed = 0; final AtomicInteger numConsumers = new AtomicInteger(assignResult.size()); + boolean notifyAddedToReplay = false; for (Map.Entry> current : assignResult.entrySet()) { final Consumer consumer = current.getKey(); final List entryAndMetadataList = current.getValue(); @@ -941,7 +987,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, if (messagesForC < entryAndMetadataList.size()) { for (int i = messagesForC; i < entryAndMetadataList.size(); i++) { final EntryAndMetadata entry = entryAndMetadataList.get(i); - addMessageToReplay(entry); + notifyAddedToReplay |= addEntryToReplay(entry); entryAndMetadataList.set(i, null); } } @@ -965,7 +1011,7 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, sendMessageInfo.getTotalChunkedMessages(), getRedeliveryTracker() ).addListener(future -> { if (future.isDone() && numConsumers.decrementAndGet() == 0) { - readMoreEntries(); + readMoreEntriesAsync(); } }); @@ -978,7 +1024,8 @@ private boolean sendChunkedMessagesToConsumers(ReadType readType, lastNumberOfEntriesProcessed = (int) totalEntriesProcessed; acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); - return numConsumers.get() == 0; // trigger a new readMoreEntries() call + // trigger a new readMoreEntries() call + return numConsumers.get() == 0 || notifyAddedToReplay; } @Override @@ -1111,31 +1158,39 @@ public boolean isConsumerAvailable(Consumer consumer) { @Override public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpoch) { + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, + redeliveryMessages); + } + MutableBoolean addedToReplay = new MutableBoolean(false); consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { if (addMessageToReplay(ledgerId, entryId, stickyKeyHash)) { redeliveryTracker.incrementAndGetRedeliveryCount((PositionFactory.create(ledgerId, entryId))); + addedToReplay.setTrue(); } }); - if (log.isDebugEnabled()) { - log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, - redeliveryMessages); + if (addedToReplay.booleanValue()) { + notifyRedeliveryMessageAdded(); } - readMoreEntries(); } @Override public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, positions); + } + MutableBoolean addedToReplay = new MutableBoolean(false); positions.forEach(position -> { // TODO: We want to pass a sticky key hash as a third argument to guarantee the order of the messages // on Key_Shared subscription, but it's difficult to get the sticky key here if (addMessageToReplay(position.getLedgerId(), position.getEntryId())) { redeliveryTracker.incrementAndGetRedeliveryCount(position); + addedToReplay.setTrue(); } }); - if (log.isDebugEnabled()) { - log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, positions); + if (addedToReplay.booleanValue()) { + notifyRedeliveryMessageAdded(); } - readMoreEntries(); } @Override @@ -1366,11 +1421,6 @@ public void cursorIsReset() { } } - private void addMessageToReplay(Entry entry) { - addMessageToReplay(entry.getLedgerId(), entry.getEntryId()); - entry.release(); - } - protected boolean addMessageToReplay(long ledgerId, long entryId, long stickyKeyHash) { if (checkIfMessageIsUnacked(ledgerId, entryId)) { redeliveryMessages.add(ledgerId, entryId, stickyKeyHash); @@ -1380,6 +1430,13 @@ protected boolean addMessageToReplay(long ledgerId, long entryId, long stickyKey } } + /** + * Notify the dispatcher that a message has been added to the redelivery list. + */ + private void notifyRedeliveryMessageAdded() { + readMoreEntriesAsync(); + } + protected boolean addMessageToReplay(long ledgerId, long entryId) { if (checkIfMessageIsUnacked(ledgerId, entryId)) { redeliveryMessages.add(ledgerId, entryId); @@ -1404,7 +1461,7 @@ public boolean checkAndUnblockIfStuck() { if (totalAvailablePermits > 0 && !havePendingReplayRead && !havePendingRead && cursor.getNumberOfEntriesInBacklog(false) > 0) { log.warn("{}-{} Dispatcher is stuck and unblocking by issuing reads", topic.getName(), name); - readMoreEntries(); + readMoreEntriesAsync(); return true; } return false; @@ -1436,10 +1493,10 @@ public ManagedCursor getCursor() { } protected int getStickyKeyHash(Entry entry) { - return StickyKeyConsumerSelector.makeStickyKeyHash(peekStickyKey(entry.getDataBuffer())); + // There's no need to calculate the hash for Shared subscription + return STICKY_KEY_HASH_NOT_SET; } - public Subscription getSubscription() { return subscription; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index ecd3f19a14028..a78e4e46c0e5a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -18,12 +18,11 @@ */ package org.apache.pulsar.broker.service.persistent; +import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; import com.google.common.annotations.VisibleForTesting; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -31,24 +30,24 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; -import javax.annotation.Nullable; +import lombok.Getter; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; -import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; -import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.ConsistentHashingStickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.DrainingHashesTracker; +import org.apache.pulsar.broker.service.EntryAndMetadata; import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; import org.apache.pulsar.broker.service.EntryBatchSizes; import org.apache.pulsar.broker.service.HashRangeAutoSplitStickyKeyConsumerSelector; import org.apache.pulsar.broker.service.HashRangeExclusiveStickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.ImpactedConsumersResult; +import org.apache.pulsar.broker.service.PendingAcksMap; import org.apache.pulsar.broker.service.SendMessageInfo; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Subscription; @@ -56,9 +55,8 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.KeySharedMeta; import org.apache.pulsar.common.api.proto.KeySharedMode; +import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; -import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,25 +64,14 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi private final boolean allowOutOfOrderDelivery; private final StickyKeyConsumerSelector selector; - private final boolean recentlyJoinedConsumerTrackingRequired; + private final boolean drainingHashesRequired; private boolean skipNextReplayToTriggerLookAhead = false; private final KeySharedMode keySharedMode; + @Getter + private final DrainingHashesTracker drainingHashesTracker; - /** - * When a consumer joins, it will be added to this map with the current read position. - * This means that, in order to preserve ordering, new consumers can only receive old - * messages, until the mark-delete position will move past this point. - */ - private final LinkedHashMap recentlyJoinedConsumers; - - /** - * The lastSentPosition and the individuallySentPositions are not thread safe. - */ - @Nullable - private Position lastSentPosition; - private final LongPairRangeSet individuallySentPositions; - private static final LongPairRangeSet.LongPairConsumer positionRangeConverter = PositionFactory::create; + private final RescheduleReadHandler rescheduleReadHandler; PersistentStickyKeyDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor, Subscription subscription, ServiceConfiguration conf, KeySharedMeta ksm) { @@ -93,13 +80,13 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi this.allowOutOfOrderDelivery = ksm.isAllowOutOfOrderDelivery(); this.keySharedMode = ksm.getKeySharedMode(); // recent joined consumer tracking is required only for AUTO_SPLIT mode when out-of-order delivery is disabled - this.recentlyJoinedConsumerTrackingRequired = + this.drainingHashesRequired = keySharedMode == KeySharedMode.AUTO_SPLIT && !allowOutOfOrderDelivery; - this.recentlyJoinedConsumers = recentlyJoinedConsumerTrackingRequired ? new LinkedHashMap<>() : null; - this.individuallySentPositions = - recentlyJoinedConsumerTrackingRequired - ? new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter) - : null; + this.drainingHashesTracker = + drainingHashesRequired ? new DrainingHashesTracker(this.getName(), this::stickyKeyHashUnblocked) : null; + this.rescheduleReadHandler = new RescheduleReadHandler(conf::getKeySharedUnblockingIntervalMs, + topic.getBrokerService().executor(), this::cancelPendingRead, () -> reScheduleReadInMs(0), + () -> havePendingRead, this::getReadMoreEntriesCallCount, () -> !redeliveryMessages.isEmpty()); switch (this.keySharedMode) { case AUTO_SPLIT: if (conf.isSubscriptionKeySharedUseConsistentHashing()) { @@ -109,16 +96,29 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi selector = new HashRangeAutoSplitStickyKeyConsumerSelector(); } break; - case STICKY: this.selector = new HashRangeExclusiveStickyKeyConsumerSelector(); break; - default: throw new IllegalArgumentException("Invalid key-shared mode: " + keySharedMode); } } + private void stickyKeyHashUnblocked(int stickyKeyHash) { + if (log.isDebugEnabled()) { + if (stickyKeyHash > -1) { + log.debug("[{}] Sticky key hash {} is unblocked", getName(), stickyKeyHash); + } else { + log.debug("[{}] Some sticky key hashes are unblocked", getName()); + } + } + reScheduleReadWithKeySharedUnblockingInterval(); + } + + private void reScheduleReadWithKeySharedUnblockingInterval() { + rescheduleReadHandler.rescheduleRead(); + } + @VisibleForTesting public StickyKeyConsumerSelector getSelector() { return selector; @@ -131,32 +131,52 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { consumer.disconnect(); return CompletableFuture.completedFuture(null); } - return super.addConsumer(consumer).thenCompose(__ -> - selector.addConsumer(consumer).handle((result, ex) -> { - if (ex != null) { - synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) { - consumerSet.removeAll(consumer); - consumerList.remove(consumer); - } - throw FutureUtil.wrapToCompletionException(ex); + return super.addConsumer(consumer).thenCompose(__ -> selector.addConsumer(consumer)) + .thenAccept(impactedConsumers -> { + // TODO: Add some way to prevent changes in between the time the consumer is added and the + // time the draining hashes are applied. It might be fine for ConsistentHashingStickyKeyConsumerSelector + // since it's not really asynchronous, although it returns a CompletableFuture + if (drainingHashesRequired) { + consumer.setPendingAcksAddHandler(this::handleAddingPendingAck); + consumer.setPendingAcksRemoveHandler(new PendingAcksMap.PendingAcksRemoveHandler() { + @Override + public void handleRemoving(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash, + boolean closing) { + drainingHashesTracker.reduceRefCount(consumer, stickyKeyHash, closing); } - return result; - }) - ).thenRun(() -> { - synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) { - if (recentlyJoinedConsumerTrackingRequired) { - final Position lastSentPositionWhenJoining = updateIfNeededAndGetLastSentPosition(); - if (lastSentPositionWhenJoining != null) { - consumer.setLastSentPositionWhenJoining(lastSentPositionWhenJoining); - // If this was the 1st consumer, or if all the messages are already acked, then we - // don't need to do anything special - if (recentlyJoinedConsumers != null - && consumerList.size() > 1 - && cursor.getNumberOfEntriesSinceFirstNotAckedMessage() > 1) { - recentlyJoinedConsumers.put(consumer, lastSentPositionWhenJoining); - } + + @Override + public void startBatch() { + drainingHashesTracker.startBatch(); } - } + + @Override + public void endBatch() { + drainingHashesTracker.endBatch(); + } + }); + registerDrainingHashes(consumer, impactedConsumers); + } + }).exceptionally(ex -> { + internalRemoveConsumer(consumer); + throw FutureUtil.wrapToCompletionException(ex); + }); + } + + private synchronized void registerDrainingHashes(Consumer skipConsumer, + ImpactedConsumersResult impactedConsumers) { + impactedConsumers.processRemovedHashRanges((c, removedHashRanges) -> { + if (c != skipConsumer) { + c.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { + if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { + log.warn("[{}] Sticky key hash was missing for {}:{}", getName(), ledgerId, entryId); + return; + } + if (removedHashRanges.containsStickyKey(stickyKeyHash)) { + // add the pending ack to the draining hashes tracker if the hash is in the range + drainingHashesTracker.addEntry(c, stickyKeyHash); + } + }); } }); } @@ -164,27 +184,21 @@ public synchronized CompletableFuture addConsumer(Consumer consumer) { @Override public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException { // The consumer must be removed from the selector before calling the superclass removeConsumer method. - // In the superclass removeConsumer method, the pending acks that the consumer has are added to - // redeliveryMessages. If the consumer has not been removed from the selector at this point, - // the broker will try to redeliver the messages to the consumer that has already been closed. - // As a result, the messages are not redelivered to any consumer, and the mark-delete position does not move, - // eventually causing all consumers to get stuck. - selector.removeConsumer(consumer); + ImpactedConsumersResult impactedConsumers = selector.removeConsumer(consumer); super.removeConsumer(consumer); - if (recentlyJoinedConsumerTrackingRequired) { - recentlyJoinedConsumers.remove(consumer); - if (consumerList.size() == 1) { - recentlyJoinedConsumers.clear(); - } else if (consumerList.isEmpty()) { - // The subscription removes consumers if rewind or reset cursor operations are called. - // The dispatcher must clear lastSentPosition and individuallySentPositions because - // these operations trigger re-sending messages. - lastSentPosition = null; - individuallySentPositions.clear(); - } - if (removeConsumersFromRecentJoinedConsumers() || !redeliveryMessages.isEmpty()) { - readMoreEntries(); - } + if (drainingHashesRequired) { + // register draining hashes for the impacted consumers and ranges, in case a hash switched from one + // consumer to another. This will handle the case where a hash gets switched from an existing + // consumer to another existing consumer during removal. + registerDrainingHashes(consumer, impactedConsumers); + } + } + + @Override + protected synchronized void clearComponentsAfterRemovedAllConsumers() { + super.clearComponentsAfterRemovedAllConsumers(); + if (drainingHashesRequired) { + drainingHashesTracker.clear(); } } @@ -226,11 +240,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis name, replayPosition, minReplayedPosition, readType); } if (readType == ReadType.Normal) { - entries.forEach(entry -> { - long stickyKeyHash = getStickyKeyHash(entry); - addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); - entry.release(); - }); + entries.forEach(this::addEntryToReplay); } else if (readType == ReadType.Replay) { entries.forEach(Entry::release); } @@ -241,26 +251,6 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis } } - if (recentlyJoinedConsumerTrackingRequired) { - // Update if the markDeletePosition move forward - updateIfNeededAndGetLastSentPosition(); - - // Should not access to individualDeletedMessages from outside managed cursor - // because it doesn't guarantee thread safety. - if (lastSentPosition == null) { - if (cursor.getMarkDeletedPosition() != null) { - lastSentPosition = ((ManagedCursorImpl) cursor) - .processIndividuallyDeletedMessagesAndGetMarkDeletedPosition(range -> { - final Position lower = range.lowerEndpoint(); - final Position upper = range.upperEndpoint(); - individuallySentPositions.addOpenClosed(lower.getLedgerId(), lower.getEntryId(), - upper.getLedgerId(), upper.getEntryId()); - return true; - }); - } - } - } - // returns a boolean indicating whether look-ahead could be useful, when there's a consumer // with available permits, and it's not able to make progress because of blocked hashes. MutableBoolean triggerLookAhead = new MutableBoolean(); @@ -276,23 +266,11 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis log.debug("[{}] select consumer {} with messages num {}, read type is {}", name, consumer.consumerName(), entriesForConsumer.size(), readType); } - final ManagedLedger managedLedger = cursor.getManagedLedger(); - for (Entry entry : entriesForConsumer) { - // remove positions first from replay list first : sendMessages recycles entries - if (readType == ReadType.Replay) { + // remove positions first from replay list first : sendMessages recycles entries + if (readType == ReadType.Replay) { + for (Entry entry : entriesForConsumer) { redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId()); } - // Add positions to individuallySentPositions if necessary - if (recentlyJoinedConsumerTrackingRequired) { - final Position position = entry.getPosition(); - // Store to individuallySentPositions even if lastSentPosition is null - if ((lastSentPosition == null || position.compareTo(lastSentPosition) > 0) - && !individuallySentPositions.contains(position.getLedgerId(), position.getEntryId())) { - final Position previousPosition = managedLedger.getPreviousPosition(position); - individuallySentPositions.addOpenClosed(previousPosition.getLedgerId(), - previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); - } - } } SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); @@ -306,7 +284,7 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), getRedeliveryTracker()).addListener(future -> { if (future.isDone() && remainingConsumersToFinishSending.decrementAndGet() == 0) { - readMoreEntries(); + readMoreEntriesAsync(); } }); @@ -316,60 +294,6 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis totalBytesSent += sendMessageInfo.getTotalBytes(); } - // Update the last sent position and remove ranges from individuallySentPositions if necessary - if (recentlyJoinedConsumerTrackingRequired && lastSentPosition != null) { - final ManagedLedger managedLedger = cursor.getManagedLedger(); - com.google.common.collect.Range range = individuallySentPositions.firstRange(); - - // If the upper bound is before the last sent position, we need to move ahead as these - // individuallySentPositions are now irrelevant. - if (range != null && range.upperEndpoint().compareTo(lastSentPosition) <= 0) { - individuallySentPositions.removeAtMost(lastSentPosition.getLedgerId(), - lastSentPosition.getEntryId()); - range = individuallySentPositions.firstRange(); - } - - if (range != null) { - // If the lowerBound is ahead of the last sent position, - // verify if there are any entries in-between. - if (range.lowerEndpoint().compareTo(lastSentPosition) <= 0 || managedLedger - .getNumberOfEntries(com.google.common.collect.Range.openClosed(lastSentPosition, - range.lowerEndpoint())) <= 0) { - if (log.isDebugEnabled()) { - log.debug("[{}] Found a position range to last sent: {}", name, range); - } - Position newLastSentPosition = range.upperEndpoint(); - Position positionAfterNewLastSent = managedLedger - .getNextValidPosition(newLastSentPosition); - // sometime ranges are connected but belongs to different ledgers - // so, they are placed sequentially - // eg: (2:10..3:15] can be returned as (2:10..2:15],[3:0..3:15]. - // So, try to iterate over connected range and found the last non-connected range - // which gives new last sent position. - final Position lastConfirmedEntrySnapshot = managedLedger.getLastConfirmedEntry(); - if (lastConfirmedEntrySnapshot != null) { - while (positionAfterNewLastSent.compareTo(lastConfirmedEntrySnapshot) <= 0) { - if (individuallySentPositions.contains(positionAfterNewLastSent.getLedgerId(), - positionAfterNewLastSent.getEntryId())) { - range = individuallySentPositions.rangeContaining( - positionAfterNewLastSent.getLedgerId(), positionAfterNewLastSent.getEntryId()); - newLastSentPosition = range.upperEndpoint(); - positionAfterNewLastSent = managedLedger.getNextValidPosition(newLastSentPosition); - // check if next valid position is also deleted and part of the deleted-range - continue; - } - break; - } - } - - if (lastSentPosition.compareTo(newLastSentPosition) < 0) { - lastSentPosition = newLastSentPosition; - } - individuallySentPositions.removeAtMost(lastSentPosition.getLedgerId(), - lastSentPosition.getEntryId()); - } - } - } lastNumberOfEntriesProcessed = (int) totalEntriesProcessed; @@ -386,7 +310,8 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis // therefore would be most likely only increase the distance between read-position and mark-delete position. skipNextReplayToTriggerLookAhead = true; // skip backoff delay before reading ahead in the "look ahead" mode to prevent any additional latency - skipNextBackoff = true; + // only skip the delay if there are more entries to read + skipNextBackoff = cursor.hasMoreEntries(); return true; } @@ -398,6 +323,37 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis return false; } + /** + * Check if the sticky hash is already draining or blocked in the replay queue. + * If it is, add the message to replay and return false so that the message isn't sent to a consumer. + * + * @param ledgerId the ledger id of the message + * @param entryId the entry id of the message + * @param stickyKeyHash the sticky hash of the message + * @return true if the message should be added to pending acks and allow sending, false otherwise + */ + private boolean handleAddingPendingAck(Consumer consumer, long ledgerId, long entryId, int stickyKeyHash) { + if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { + log.warn("[{}] Sticky key hash is missing for {}:{}", getName(), ledgerId, entryId); + throw new IllegalArgumentException("Sticky key hash is missing for " + ledgerId + ":" + entryId); + } + DrainingHashesTracker.DrainingHashEntry drainingHashEntry = drainingHashesTracker.getEntry(stickyKeyHash); + if (drainingHashEntry != null && drainingHashEntry.getConsumer() != consumer) { + log.warn("[{}] Another consumer id {} is already draining hash {}. Skipping adding {}:{} to pending acks " + + "for consumer {}. Adding the message to replay.", + getName(), drainingHashEntry.getConsumer(), stickyKeyHash, ledgerId, entryId, consumer); + addMessageToReplay(ledgerId, entryId, stickyKeyHash); + // block message from sending + return false; + } + if (log.isDebugEnabled()) { + log.debug("[{}] Adding {}:{} to pending acks for consumer {} with sticky key hash {}", + getName(), ledgerId, entryId, consumer, stickyKeyHash); + } + // allow adding the message to pending acks and sending the message to the consumer + return true; + } + private boolean isReplayQueueSizeBelowLimit() { return redeliveryMessages.size() < getEffectiveLookAheadLimit(); } @@ -442,16 +398,22 @@ private Map> filterAndGroupEntriesForDispatching(List> entriesGroupedByConsumer = new HashMap<>(); // permits for consumer, permits are for entries/batches Map permitsForConsumer = new HashMap<>(); - // maxLastSentPosition cache for consumers, used when recently joined consumers exist - boolean hasRecentlyJoinedConsumers = hasRecentlyJoinedConsumers(); - Map maxLastSentPositionCache = hasRecentlyJoinedConsumers ? new HashMap<>() : null; boolean lookAheadAllowed = isReplayQueueSizeBelowLimit(); // in normal read mode, keep track of consumers that are blocked by hash, to check if look-ahead could be useful Set blockedByHashConsumers = lookAheadAllowed && readType == ReadType.Normal ? new HashSet<>() : null; // in replay read mode, keep track of consumers for entries, used for look-ahead check Set consumersForEntriesForLookaheadCheck = lookAheadAllowed ? new HashSet<>() : null; - for (Entry entry : entries) { + for (Entry inputEntry : entries) { + EntryAndMetadata entry; + if (inputEntry instanceof EntryAndMetadata entryAndMetadataInstance) { + entry = entryAndMetadataInstance; + } else { + // replace the input entry with EntryAndMetadata instance. In addition to the entry and metadata, + // it will also carry the calculated sticky key hash + entry = EntryAndMetadata.create(inputEntry, + Commands.peekAndCopyMessageMetadata(inputEntry.getDataBuffer(), getSubscriptionName(), -1)); + } int stickyKeyHash = getStickyKeyHash(entry); Consumer consumer = selector.select(stickyKeyHash); MutableBoolean blockedByHash = null; @@ -460,15 +422,13 @@ private Map> filterAndGroupEntriesForDispatching(List resolveMaxLastSentPositionForRecentlyJoinedConsumer(consumer, readType)) : null; blockedByHash = lookAheadAllowed && readType == ReadType.Normal ? new MutableBoolean(false) : null; MutableInt permits = permitsForConsumer.computeIfAbsent(consumer, k -> new MutableInt(getAvailablePermits(consumer))); // a consumer was found for the sticky key hash and the entry can be dispatched - if (permits.intValue() > 0 && canDispatchEntry(entry, readType, stickyKeyHash, - maxLastSentPosition, blockedByHash)) { + if (permits.intValue() > 0 + && canDispatchEntry(consumer, entry, readType, stickyKeyHash, blockedByHash)) { // decrement the permits for the consumer permits.decrement(); // allow the entry to be dispatched @@ -491,6 +451,7 @@ private Map> filterAndGroupEntriesForDispatching(List> filterAndGroupEntriesForDispatching(List 0) { - return false; - } - // If redeliveryMessages contains messages that correspond to the same hash as the entry to be dispatched // do not send those messages for order guarantee if (readType == ReadType.Normal && redeliveryMessages.containsStickyKeyHash(stickyKeyHash)) { @@ -545,6 +501,16 @@ private boolean canDispatchEntry(Entry entry, return false; } + if (drainingHashesRequired) { + // If the hash is draining, do not send the message + if (drainingHashesTracker.shouldBlockStickyKeyHash(consumer, stickyKeyHash)) { + if (blockedByHash != null) { + blockedByHash.setTrue(); + } + return false; + } + } + return true; } @@ -566,8 +532,6 @@ private class ReplayPositionFilter implements Predicate { // tracks the available permits for each consumer for the duration of the filter usage // the filter is stateful and shouldn't be shared or reused later private final Map availablePermitsMap = new HashMap<>(); - private final Map maxLastSentPositionCache = - hasRecentlyJoinedConsumers() ? new HashMap<>() : null; @Override public boolean test(Position position) { @@ -585,6 +549,7 @@ public boolean test(Position position) { } return true; } + // find the consumer for the sticky key hash Consumer consumer = selector.select(stickyKeyHash.intValue()); // skip replaying the message position if there's no assigned consumer @@ -599,113 +564,32 @@ public boolean test(Position position) { if (availablePermits.intValue() <= 0) { return false; } - // check if the entry position can be replayed to a recently joined consumer - Position maxLastSentPosition = maxLastSentPositionCache != null - ? maxLastSentPositionCache.computeIfAbsent(consumer, __ -> - resolveMaxLastSentPositionForRecentlyJoinedConsumer(consumer, ReadType.Replay)) - : null; - if (maxLastSentPosition != null && position.compareTo(maxLastSentPosition) > 0) { + + if (drainingHashesRequired + && drainingHashesTracker.shouldBlockStickyKeyHash(consumer, stickyKeyHash.intValue())) { + // the hash is draining and the consumer is not the draining consumer return false; } + availablePermits.decrement(); return true; } } - /** - * Contains the logic to resolve the max last sent position for a consumer - * when the consumer has recently joined. This is only applicable for key shared mode when - * allowOutOfOrderDelivery=false. - */ - private Position resolveMaxLastSentPositionForRecentlyJoinedConsumer(Consumer consumer, ReadType readType) { - if (recentlyJoinedConsumers == null) { - return null; - } - removeConsumersFromRecentJoinedConsumers(); - Position maxLastSentPosition = recentlyJoinedConsumers.get(consumer); - // At this point, all the old messages were already consumed and this consumer - // is now ready to receive any message - if (maxLastSentPosition == null) { - // The consumer has not recently joined, so we can send all messages - return null; - } - - // If the read type is Replay, we should avoid send messages that hold by other consumer to the new consumers, - // For example, we have 10 messages [0,1,2,3,4,5,6,7,8,9] - // If the consumer0 get message 0 and 1, and does not acked message 0, then consumer1 joined, - // when consumer1 get message 2,3, the broker will not dispatch messages to consumer1 - // because of the mark delete position did not move forward. - // So message 2,3 will stored in the redeliver tracker. - // Now, consumer2 joined, it will read new messages from the cursor, - // so the recentJoinedPosition is 4 for consumer2 - // Because of there are messages need to redeliver, so the broker will read the redelivery message first [2,3] - // message [2,3] is lower than the recentJoinedPosition 4, - // so the message [2,3] will dispatched to the consumer2 - // But the message [2,3] should not dispatch to consumer2. - - if (readType == ReadType.Replay) { - Position minLastSentPositionForRecentJoinedConsumer = recentlyJoinedConsumers.values().iterator().next(); - if (minLastSentPositionForRecentJoinedConsumer != null - && minLastSentPositionForRecentJoinedConsumer.compareTo(maxLastSentPosition) < 0) { - maxLastSentPosition = minLastSentPositionForRecentJoinedConsumer; - } + @Override + protected int getStickyKeyHash(Entry entry) { + if (entry instanceof EntryAndMetadata entryAndMetadata) { + // use the cached sticky key hash if available, otherwise calculate the sticky key hash and cache it + return entryAndMetadata.getOrUpdateCachedStickyKeyHash(selector::makeStickyKeyHash); } - - return maxLastSentPosition; + return selector.makeStickyKeyHash(peekStickyKey(entry.getDataBuffer())); } - @Override public void markDeletePositionMoveForward() { - // Execute the notification in different thread to avoid a mutex chain here - // from the delete operation that was completed - topic.getBrokerService().getTopicOrderedExecutor().execute(() -> { - synchronized (PersistentStickyKeyDispatcherMultipleConsumers.this) { - if (hasRecentlyJoinedConsumers() - && removeConsumersFromRecentJoinedConsumers()) { - // After we process acks, we need to check whether the mark-delete position was advanced and we - // can finally read more messages. It's safe to call readMoreEntries() multiple times. - readMoreEntries(); - } - } - }); - } - - private boolean hasRecentlyJoinedConsumers() { - return !MapUtils.isEmpty(recentlyJoinedConsumers); - } - - private boolean removeConsumersFromRecentJoinedConsumers() { - if (MapUtils.isEmpty(recentlyJoinedConsumers)) { - return false; - } - Iterator> itr = recentlyJoinedConsumers.entrySet().iterator(); - boolean hasConsumerRemovedFromTheRecentJoinedConsumers = false; - Position mdp = cursor.getMarkDeletedPosition(); - if (mdp != null) { - while (itr.hasNext()) { - Map.Entry entry = itr.next(); - if (entry.getValue().compareTo(mdp) <= 0) { - itr.remove(); - hasConsumerRemovedFromTheRecentJoinedConsumers = true; - } else { - break; - } - } - } - return hasConsumerRemovedFromTheRecentJoinedConsumers; - } - - @Nullable - private synchronized Position updateIfNeededAndGetLastSentPosition() { - if (lastSentPosition == null) { - return null; - } - final Position mdp = cursor.getMarkDeletedPosition(); - if (mdp != null && mdp.compareTo(lastSentPosition) > 0) { - lastSentPosition = mdp; - } - return lastSentPosition; + // reschedule a read with a backoff after moving the mark-delete position forward since there might have + // been consumers that were blocked by hash and couldn't make progress + reScheduleReadWithKeySharedUnblockingInterval(); } /** @@ -829,34 +713,6 @@ public boolean hasSameKeySharedPolicy(KeySharedMeta ksm) { && ksm.isAllowOutOfOrderDelivery() == this.allowOutOfOrderDelivery); } - public LinkedHashMap getRecentlyJoinedConsumers() { - return recentlyJoinedConsumers; - } - - public synchronized String getLastSentPosition() { - if (lastSentPosition == null) { - return null; - } - return lastSentPosition.toString(); - } - - @VisibleForTesting - public Position getLastSentPositionField() { - return lastSentPosition; - } - - public synchronized String getIndividuallySentPositions() { - if (individuallySentPositions == null) { - return null; - } - return individuallySentPositions.toString(); - } - - @VisibleForTesting - public LongPairRangeSet getIndividuallySentPositionsField() { - return individuallySentPositions; - } - public Map> getConsumerKeyHashRanges() { return selector.getConsumerKeyHashRanges(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 9a0545e6f0ab2..b8d351bddf839 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -25,7 +25,6 @@ import io.netty.buffer.ByteBuf; import java.io.IOException; import java.util.Collections; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -1297,31 +1296,6 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge subStats.allowOutOfOrderDelivery = keySharedDispatcher.isAllowOutOfOrderDelivery(); subStats.keySharedMode = keySharedDispatcher.getKeySharedMode().toString(); - - LinkedHashMap recentlyJoinedConsumers = keySharedDispatcher - .getRecentlyJoinedConsumers(); - if (recentlyJoinedConsumers != null && recentlyJoinedConsumers.size() > 0) { - recentlyJoinedConsumers.forEach((k, v) -> { - // The dispatcher allows same name consumers - final StringBuilder stringBuilder = new StringBuilder(); - stringBuilder.append("consumerName=").append(k.consumerName()) - .append(", consumerId=").append(k.consumerId()); - if (k.cnx() != null) { - stringBuilder.append(", address=").append(k.cnx().clientAddress()); - } - subStats.consumersAfterMarkDeletePosition.put(stringBuilder.toString(), v.toString()); - }); - } - final String lastSentPosition = ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher) - .getLastSentPosition(); - if (lastSentPosition != null) { - subStats.lastSentPosition = lastSentPosition; - } - final String individuallySentPositions = ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher) - .getIndividuallySentPositions(); - if (individuallySentPositions != null) { - subStats.individuallySentPositions = individuallySentPositions; - } } subStats.nonContiguousDeletedMessagesRanges = cursor.getTotalNonContiguousDeletedMessagesRange(); subStats.nonContiguousDeletedMessagesRangesSerializedSize = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java new file mode 100644 index 0000000000000..3554f29255227 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service.persistent; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BooleanSupplier; +import java.util.function.LongSupplier; + +/** + * Reschedules reads so that the possible pending read is cancelled if it's waiting for more entries. + * This will prevent the dispatcher in getting blocked when there are entries in the replay queue + * that should be handled. This will also batch multiple calls together to reduce the number of + * operations. + */ +class RescheduleReadHandler { + private static final int UNSET = -1; + private static final int NO_PENDING_READ = 0; + private final AtomicLong maxReadOpCounter = new AtomicLong(UNSET); + private final LongSupplier readIntervalMsSupplier; + private final ScheduledExecutorService executor; + private final Runnable cancelPendingRead; + private final Runnable rescheduleReadImmediately; + private final BooleanSupplier hasPendingReadRequestThatMightWait; + private final LongSupplier readOpCounterSupplier; + private final BooleanSupplier hasEntriesInReplayQueue; + + RescheduleReadHandler(LongSupplier readIntervalMsSupplier, + ScheduledExecutorService executor, Runnable cancelPendingRead, + Runnable rescheduleReadImmediately, BooleanSupplier hasPendingReadRequestThatMightWait, + LongSupplier readOpCounterSupplier, + BooleanSupplier hasEntriesInReplayQueue) { + this.readIntervalMsSupplier = readIntervalMsSupplier; + this.executor = executor; + this.cancelPendingRead = cancelPendingRead; + this.rescheduleReadImmediately = rescheduleReadImmediately; + this.hasPendingReadRequestThatMightWait = hasPendingReadRequestThatMightWait; + this.readOpCounterSupplier = readOpCounterSupplier; + this.hasEntriesInReplayQueue = hasEntriesInReplayQueue; + } + + public void rescheduleRead() { + long readOpCountWhenPendingRead = + hasPendingReadRequestThatMightWait.getAsBoolean() ? readOpCounterSupplier.getAsLong() : NO_PENDING_READ; + if (maxReadOpCounter.compareAndSet(UNSET, readOpCountWhenPendingRead)) { + Runnable runnable = () -> { + // Read the current value of maxReadOpCounter and set it to UNSET, this will allow scheduling a next + // runnable + long maxReadOpCount = maxReadOpCounter.getAndSet(UNSET); + // Cancel a possible pending read if it's been waiting for more entries since the runnable was + // scheduled. This is detected by checking that the value of the readOpCounter has not changed + // since the runnable was scheduled. Canceling the read request will only be needed if there + // are entries in the replay queue. + if (maxReadOpCount != NO_PENDING_READ && readOpCounterSupplier.getAsLong() == maxReadOpCount + && hasEntriesInReplayQueue.getAsBoolean()) { + cancelPendingRead.run(); + } + // Re-schedule read immediately, or join the next scheduled read + rescheduleReadImmediately.run(); + }; + long rescheduleDelay = readIntervalMsSupplier.getAsLong(); + if (rescheduleDelay > 0) { + executor.schedule(runnable, rescheduleDelay, TimeUnit.MILLISECONDS); + } else { + runnable.run(); + } + } else { + // When there's a scheduled read, update the maxReadOpCounter to carry the state when the later scheduled + // read was done + long updatedValue = maxReadOpCounter.updateAndGet( + // Ignore updating if the value is UNSET + current -> current == UNSET ? UNSET : + // Prefer keeping NO_PENDING_READ if the latest value is NO_PENDING_READ + (readOpCountWhenPendingRead == NO_PENDING_READ ? NO_PENDING_READ : + // Otherwise, keep the maximum value + Math.max(current, readOpCountWhenPendingRead))); + // If the value was unset, it means that the runnable was already run and retrying is needed + // so that we don't miss any entries + if (updatedValue == UNSET) { + // Retry + rescheduleRead(); + } + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java b/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java index cc1eae475fa2d..7a4126fedec64 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java @@ -71,19 +71,30 @@ public boolean contains(long item1, long item2) { } } - public void removeUpTo(long item1, long item2) { + /** + * Remove all items up to (and including) the specified item. + * + * @param item1 the first part of the item key + * @param item2 the second part of the item key + * @return true if any bits were cleared + */ + public boolean removeUpTo(long item1, long item2) { + boolean bitsCleared = false; lock.writeLock().lock(); try { Map.Entry firstEntry = map.firstEntry(); while (firstEntry != null && firstEntry.getKey() <= item1) { if (firstEntry.getKey() < item1) { map.remove(firstEntry.getKey(), firstEntry.getValue()); + bitsCleared = true; } else { RoaringBitmap bitSet = firstEntry.getValue(); if (bitSet != null) { + bitsCleared |= bitSet.contains(0, item2); bitSet.remove(0, item2); if (bitSet.isEmpty()) { map.remove(firstEntry.getKey(), bitSet); + bitsCleared = true; } } break; @@ -93,6 +104,7 @@ public void removeUpTo(long item1, long item2) { } finally { lock.writeLock().unlock(); } + return bitsCleared; } public > Optional first(LongPairSet.LongPairFunction longPairConverter) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java index 5641816ee0b80..ffcc3bf0881db 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.ObjectWriter; import java.io.IOException; @@ -234,4 +236,21 @@ public static void receiveMessagesInThreads(BiFunction, Message< }); }).toList()).join(); } + + private static long mockConsumerIdGenerator = 0; + + public static org.apache.pulsar.broker.service.Consumer createMockConsumer(String consumerName) { + long consumerId = mockConsumerIdGenerator++; + return createMockConsumer(consumerName, consumerName + " consumerId:" + consumerId, consumerId); + } + + public static org.apache.pulsar.broker.service.Consumer createMockConsumer(String consumerName, String toString, long consumerId) { + // without stubOnly, the mock will record method invocations and could run into OOME + org.apache.pulsar.broker.service.Consumer + consumer = mock(org.apache.pulsar.broker.service.Consumer.class, Mockito.withSettings().stubOnly()); + when(consumer.consumerName()).thenReturn(consumerName); + when(consumer.toString()).thenReturn(consumerName + " consumerId:" + consumerId); + when(consumer.consumerId()).thenReturn(consumerId); + return consumer; + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index 26da4116d09cb..70c2b343ec584 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -18,8 +18,6 @@ */ package org.apache.pulsar.broker.admin; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -58,7 +56,6 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; import javax.ws.rs.client.InvocationCallback; import javax.ws.rs.client.WebTarget; import javax.ws.rs.core.Response.Status; @@ -68,8 +65,6 @@ import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedLedgerInfo; -import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -79,8 +74,6 @@ import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.broker.namespace.NamespaceService; -import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; -import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.broker.testcontext.SpyConfig; import org.apache.pulsar.client.admin.GetStatsOptions; @@ -128,7 +121,6 @@ import org.apache.pulsar.common.policies.data.BrokerInfo; import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.ConsumerStats; import org.apache.pulsar.common.policies.data.NamespaceIsolationData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.NamespaceOwnershipStatus; @@ -145,10 +137,7 @@ import org.apache.pulsar.common.policies.data.TopicHashPositions; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.Codec; -import org.apache.pulsar.common.util.Murmur3_32Hash; import org.apache.pulsar.common.util.ObjectMapperFactory; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; -import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.compaction.Compactor; import org.apache.pulsar.compaction.PulsarCompactionServiceFactory; import org.awaitility.Awaitility; @@ -3457,201 +3446,6 @@ public void testGetTtlDurationDefaultInSeconds() throws Exception { assertNull(seconds); } - @Test - public void testGetLastSentPositionWhenJoining() throws Exception { - final String topic = "persistent://prop-xyz/ns1/testGetLastSentPositionWhenJoining-" + UUID.randomUUID().toString(); - final String subName = "my-sub"; - @Cleanup - Producer producer = pulsarClient.newProducer() - .topic(topic) - .enableBatching(false) - .create(); - - @Cleanup - final Consumer consumer1 = pulsarClient.newConsumer() - .topic(topic) - .subscriptionType(SubscriptionType.Key_Shared) - .subscriptionName(subName) - .subscribe(); - - final int messages = 10; - MessageIdImpl messageId = null; - for (int i = 0; i < messages; i++) { - messageId = (MessageIdImpl) producer.send(("Hello Pulsar - " + i).getBytes()); - consumer1.receive(); - } - - @Cleanup - final Consumer consumer2 = pulsarClient.newConsumer() - .topic(topic) - .subscriptionType(SubscriptionType.Key_Shared) - .subscriptionName(subName) - .subscribe(); - - TopicStats stats = admin.topics().getStats(topic); - Assert.assertEquals(stats.getSubscriptions().size(), 1); - SubscriptionStats subStats = stats.getSubscriptions().get(subName); - Assert.assertNotNull(subStats); - Assert.assertEquals(subStats.getConsumers().size(), 2); - ConsumerStats consumerStats = subStats.getConsumers().stream() - .filter(s -> s.getConsumerName().equals(consumer2.getConsumerName())).findFirst().get(); - Assert.assertEquals(consumerStats.getLastSentPositionWhenJoining(), - PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()).toString()); - } - - @Test - public void testGetLastSentPosition() throws Exception { - final String topic = "persistent://prop-xyz/ns1/testGetLastSentPosition-" + UUID.randomUUID().toString(); - final String subName = "my-sub"; - @Cleanup - final Producer producer = pulsarClient.newProducer() - .topic(topic) - .enableBatching(false) - .create(); - final AtomicInteger counter = new AtomicInteger(); - @Cleanup - final Consumer consumer = pulsarClient.newConsumer() - .topic(topic) - .subscriptionType(SubscriptionType.Key_Shared) - .subscriptionName(subName) - .messageListener((c, msg) -> { - try { - c.acknowledge(msg); - counter.getAndIncrement(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }) - .subscribe(); - - TopicStats stats = admin.topics().getStats(topic); - Assert.assertEquals(stats.getSubscriptions().size(), 1); - SubscriptionStats subStats = stats.getSubscriptions().get(subName); - Assert.assertNotNull(subStats); - Assert.assertNull(subStats.getLastSentPosition()); - - final int messages = 10; - MessageIdImpl messageId = null; - for (int i = 0; i < messages; i++) { - messageId = (MessageIdImpl) producer.send(("Hello Pulsar - " + i).getBytes()); - } - - Awaitility.await().untilAsserted(() -> assertEquals(counter.get(), messages)); - - stats = admin.topics().getStats(topic); - Assert.assertEquals(stats.getSubscriptions().size(), 1); - subStats = stats.getSubscriptions().get(subName); - Assert.assertNotNull(subStats); - Assert.assertEquals(subStats.getLastSentPosition(), PositionFactory.create(messageId.getLedgerId(), messageId.getEntryId()).toString()); - } - - @Test - public void testGetIndividuallySentPositions() throws Exception { - // The producer sends messages with two types of keys. - // The dispatcher sends keyA messages to consumer1. - // Consumer1 will not receive any messages. Its receiver queue size is 1. - // Consumer2 will receive and ack any messages immediately. - - final String topic = "persistent://prop-xyz/ns1/testGetIndividuallySentPositions-" + UUID.randomUUID().toString(); - final String subName = "my-sub"; - @Cleanup - final Producer producer = pulsarClient.newProducer() - .topic(topic) - .enableBatching(false) - .create(); - - final String consumer1Name = "c1"; - final String consumer2Name = "c2"; - - @Cleanup - final Consumer consumer1 = pulsarClient.newConsumer() - .topic(topic) - .consumerName(consumer1Name) - .receiverQueueSize(1) - .subscriptionType(SubscriptionType.Key_Shared) - .subscriptionName(subName) - .subscribe(); - - final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topic).get().get().getSubscription(subName).getDispatcher(); - final String keyA = "key-a"; - final String keyB = "key-b"; - final int hashA = Murmur3_32Hash.getInstance().makeHash(keyA.getBytes()); - - final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); - selectorField.setAccessible(true); - final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); - selectorField.set(dispatcher, selector); - - // the selector returns consumer1 if keyA - doAnswer((invocationOnMock -> { - final int hash = invocationOnMock.getArgument(0); - - final String consumerName = hash == hashA ? consumer1Name : consumer2Name; - return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); - })).when(selector).select(anyInt()); - - final AtomicInteger consumer2AckCounter = new AtomicInteger(); - @Cleanup - final Consumer consumer2 = pulsarClient.newConsumer() - .topic(topic) - .consumerName(consumer2Name) - .subscriptionType(SubscriptionType.Key_Shared) - .subscriptionName(subName) - .messageListener((c, msg) -> { - try { - c.acknowledge(msg); - consumer2AckCounter.getAndIncrement(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }) - .subscribe(); - - final LongPairRangeSet.LongPairConsumer positionRangeConverter = PositionFactory::create; - final LongPairRangeSet expectedIndividuallySentPositions = new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter); - - TopicStats stats = admin.topics().getStats(topic); - Assert.assertEquals(stats.getSubscriptions().size(), 1); - SubscriptionStats subStats = stats.getSubscriptions().get(subName); - Assert.assertNotNull(subStats); - Assert.assertEquals(subStats.getIndividuallySentPositions(), expectedIndividuallySentPositions.toString()); - - final Function sendFn = (key) -> { - try { - return (MessageIdImpl) producer.newMessage().key(key).value(("msg").getBytes()).send(); - } catch (PulsarClientException e) { - throw new RuntimeException(e); - } - }; - final List messageIdList = new ArrayList<>(); - - // the dispatcher can send keyA message, but then consumer1's receiver queue will be full - messageIdList.add(sendFn.apply(keyA)); - - // the dispatcher can send messages other than keyA - messageIdList.add(sendFn.apply(keyA)); - messageIdList.add(sendFn.apply(keyB)); - messageIdList.add(sendFn.apply(keyA)); - messageIdList.add(sendFn.apply(keyB)); - messageIdList.add(sendFn.apply(keyB)); - - assertEquals(messageIdList.size(), 6); - Awaitility.await().untilAsserted(() -> assertEquals(consumer2AckCounter.get(), 3)); - - // set expected value - expectedIndividuallySentPositions.addOpenClosed(messageIdList.get(1).getLedgerId(), messageIdList.get(1).getEntryId(), - messageIdList.get(2).getLedgerId(), messageIdList.get(2).getEntryId()); - expectedIndividuallySentPositions.addOpenClosed(messageIdList.get(3).getLedgerId(), messageIdList.get(3).getEntryId(), - messageIdList.get(5).getLedgerId(), messageIdList.get(5).getEntryId()); - - stats = admin.topics().getStats(topic); - Assert.assertEquals(stats.getSubscriptions().size(), 1); - subStats = stats.getSubscriptions().get(subName); - Assert.assertNotNull(subStats); - Assert.assertEquals(subStats.getIndividuallySentPositions(), expectedIndividuallySentPositions.toString()); - } - @Test public void testPartitionedTopicMsgDelayedAggregated() throws Exception { final String topic = "persistent://prop-xyz/ns1/testPartitionedTopicMsgDelayedAggregated-" + UUID.randomUUID().toString(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java index 8b72411329c65..ea6ffa2d70dba 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java @@ -126,10 +126,10 @@ public void testWithTimer(DelayedDeliveryTracker tracker, NavigableMap false); ((AbstractDelayedDeliveryTracker) tracker).run(timeout); - verify(dispatcher, times(1)).readMoreEntries(); + verify(dispatcher, times(1)).readMoreEntriesAsync(); // Add a message that has a delivery time just after the previous run. It will get delivered based on the // tick delay plus the last tick run. @@ -189,11 +189,11 @@ public void testAddMessageWithDeliverAtTimeAfterNowBeforeTickTimeFrequencyWithSt // Wait longer than the tick time plus the HashedWheelTimer's tick time to ensure that enough time has // passed where it would have been triggered if the tick time was doing the triggering. Thread.sleep(600); - verify(dispatcher, times(1)).readMoreEntries(); + verify(dispatcher, times(1)).readMoreEntriesAsync(); // Not wait for the message delivery to get triggered. Awaitility.await().atMost(10, TimeUnit.SECONDS) - .untilAsserted(() -> verify(dispatcher).readMoreEntries()); + .untilAsserted(() -> verify(dispatcher).readMoreEntriesAsync()); tracker.close(); } @@ -212,7 +212,7 @@ public void testAddMessageWithDeliverAtTimeAfterNowAfterTickTimeFrequencyWithStr // Wait long enough for the runnable to run, but not longer than the tick time. The point is that the delivery // should get scheduled early when the tick duration has passed since the last tick. Awaitility.await().atMost(10, TimeUnit.SECONDS) - .untilAsserted(() -> verify(dispatcher).readMoreEntries()); + .untilAsserted(() -> verify(dispatcher).readMoreEntriesAsync()); tracker.close(); } @@ -233,7 +233,7 @@ public void testAddMessageWithDeliverAtTimeAfterFullTickTimeWithStrict(DelayedDe // Not wait for the message delivery to get triggered. Awaitility.await().atMost(10, TimeUnit.SECONDS) - .untilAsserted(() -> verify(dispatcher).readMoreEntries()); + .untilAsserted(() -> verify(dispatcher).readMoreEntriesAsync()); tracker.close(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java index 04aafc49b47e6..e2feb2050652b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java @@ -168,20 +168,20 @@ public void testGetConsumerKeyHashRanges() throws BrokerServiceException.Consume Map> expectedResult = new HashMap<>(); assertThat(consumers.get(0).consumerName()).isEqualTo("consumer1"); expectedResult.put(consumers.get(0), Arrays.asList( - Range.of(95615213, 440020355), - Range.of(440020356, 455987436), - Range.of(1189794593, 1264144431))); + Range.of(14359, 18366), + Range.of(29991, 39817), + Range.of(52980, 60442))); assertThat(consumers.get(1).consumerName()).isEqualTo("consumer2"); expectedResult.put(consumers.get(1), Arrays.asList( - Range.of(939655188, 1189794592), - Range.of(1314727625, 1977451233), - Range.of(1977451234, 2016237253))); + Range.of(1, 6668), + Range.of(39818, 52979), + Range.of(60443, 63679), + Range.of(65184, 65535))); assertThat(consumers.get(2).consumerName()).isEqualTo("consumer3"); expectedResult.put(consumers.get(2), Arrays.asList( - Range.of(0, 95615212), - Range.of(455987437, 939655187), - Range.of(1264144432, 1314727624), - Range.of(2016237254, 2147483646))); + Range.of(6669, 14358), + Range.of(18367, 29990), + Range.of(63680, 65183))); Map> consumerKeyHashRanges = selector.getConsumerKeyHashRanges(); assertThat(consumerKeyHashRanges).containsExactlyInAnyOrderEntriesOf(expectedResult); @@ -195,7 +195,8 @@ public void testGetConsumerKeyHashRanges() throws BrokerServiceException.Consume } previousRange = range; } - assertThat(allRanges.stream().mapToInt(r -> r.getEnd() - r.getStart() + 1).sum()).isEqualTo(Integer.MAX_VALUE); + Range totalRange = selector.getKeyHashRange(); + assertThat(allRanges.stream().mapToInt(Range::size).sum()).isEqualTo(totalRange.size()); } @Test @@ -247,12 +248,12 @@ private static void printSelectionCountStats(Map consumerS private static void printConsumerRangesStats(ConsistentHashingStickyKeyConsumerSelector selector) { selector.getConsumerKeyHashRanges().entrySet().stream() .map(entry -> Map.entry(entry.getKey(), - entry.getValue().stream().mapToInt(r -> r.getEnd() - r.getStart() + 1).sum())) + entry.getValue().stream().mapToInt(Range::size).sum())) .sorted(Map.Entry.comparingByKey(Comparator.comparing(Consumer::toString))) .forEach(entry -> System.out.println( String.format("consumer: %s total ranges size: %d ratio: %.2f%%", entry.getKey(), entry.getValue(), - ((double) entry.getValue() / (Integer.MAX_VALUE - 1)) * 100.0d))); + ((double) entry.getValue() / selector.getKeyHashRange().size()) * 100.0d))); } private static Consumer createMockConsumer(String consumerName, String toString, long id) { @@ -323,7 +324,7 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsRemoved() { selector.addConsumer(consumer); } - int hashRangeSize = Integer.MAX_VALUE; + int hashRangeSize = selector.getKeyHashRange().size(); int validationPointCount = 200; int increment = hashRangeSize / (validationPointCount + 1); List selectedConsumerBeforeRemoval = new ArrayList<>(); @@ -342,13 +343,14 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsRemoved() { for (Consumer removedConsumer : consumers) { selector.removeConsumer(removedConsumer); removedConsumers.add(removedConsumer); + Map> consumerKeyHashRanges = selector.getConsumerKeyHashRanges(); for (int i = 0; i < validationPointCount; i++) { int hash = i * increment; Consumer selected = selector.select(hash); Consumer expected = selectedConsumerBeforeRemoval.get(i); if (!removedConsumers.contains(expected)) { assertThat(selected.consumerId()).as("validationPoint %d, removed %s, hash %d ranges %s", i, - removedConsumer.toString(), hash, selector.getConsumerKeyHashRanges()).isEqualTo(expected.consumerId()); + removedConsumer.toString(), hash, consumerKeyHashRanges).isEqualTo(expected.consumerId()); } } } @@ -441,7 +443,7 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsAdded() { selector.addConsumer(consumer); } - int hashRangeSize = Integer.MAX_VALUE; + int hashRangeSize = selector.getKeyHashRange().size(); int validationPointCount = 200; int increment = hashRangeSize / (validationPointCount + 1); List selectedConsumerBeforeRemoval = new ArrayList<>(); @@ -473,10 +475,10 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsAdded() { } @Test - public void testShouldNotChangeMappingWhenConsumerLeavesAndRejoins() { + public void testShouldContainMinimalMappingChangesWhenConsumerLeavesAndRejoins() { final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); final String consumerName = "consumer"; - final int numOfInitialConsumers = 25; + final int numOfInitialConsumers = 10; List consumers = new ArrayList<>(); for (int i = 0; i < numOfInitialConsumers; i++) { final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); @@ -484,6 +486,8 @@ public void testShouldNotChangeMappingWhenConsumerLeavesAndRejoins() { selector.addConsumer(consumer); } + ConsumerHashAssignmentsSnapshot assignmentsBefore = selector.getConsumerHashAssignmentsSnapshot(); + Map> expected = selector.getConsumerKeyHashRanges(); assertThat(selector.getConsumerKeyHashRanges()).as("sanity check").containsExactlyInAnyOrderEntriesOf(expected); @@ -492,7 +496,15 @@ public void testShouldNotChangeMappingWhenConsumerLeavesAndRejoins() { selector.addConsumer(consumers.get(0)); selector.addConsumer(consumers.get(numOfInitialConsumers / 2)); - assertThat(selector.getConsumerKeyHashRanges()).as("ranges shouldn't change").containsExactlyInAnyOrderEntriesOf(expected); + ConsumerHashAssignmentsSnapshot assignmentsAfter = selector.getConsumerHashAssignmentsSnapshot(); + int removedRangesSize = assignmentsBefore.diffRanges(assignmentsAfter).keySet().stream() + .mapToInt(Range::size) + .sum(); + double allowedremovedRangesPercentage = 1; // 1% + int hashRangeSize = selector.getKeyHashRange().size(); + int allowedremovedRanges = (int) (hashRangeSize * (allowedremovedRangesPercentage / 100.0d)); + assertThat(removedRangesSize).describedAs("Allow up to %d%% of total hash range size to be impacted", + allowedremovedRangesPercentage).isLessThan(allowedremovedRanges); } @Test @@ -501,7 +513,7 @@ public void testConsumersReconnect() { final String consumerName = "consumer"; final int numOfInitialConsumers = 50; final int validationPointCount = 200; - final List pointsToTest = pointsToTest(validationPointCount); + final List pointsToTest = pointsToTest(validationPointCount, selector.getKeyHashRange().size()); List consumers = new ArrayList<>(); for (int i = 0; i < numOfInitialConsumers; i++) { final Consumer consumer = createMockConsumer(consumerName, "index " + i, i); @@ -537,13 +549,38 @@ public void testConsumersReconnect() { } } - private List pointsToTest(int validationPointCount) { + private List pointsToTest(int validationPointCount, int hashRangeSize) { List res = new ArrayList<>(); - int hashRangeSize = Integer.MAX_VALUE; final int increment = hashRangeSize / (validationPointCount + 1); for (int i = 0; i < validationPointCount; i++) { - res.add(i * increment); + res.add(Math.max(i * increment, hashRangeSize - 1)); } return res; } -} + + @Test(enabled = false) + public void testPerformanceOfAdding1000ConsumersWith100Points() { + // test that adding 1000 consumers with 100 points runs in a reasonable time. + // This takes about 1 second on Apple M3 + // this unit test can be used for basic profiling + final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + for (int i = 0; i < 1000; i++) { + // use real class to avoid Mockito over head + final Consumer consumer = new Consumer("consumer" + i, 0) { + @Override + public int hashCode() { + return consumerName().hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof Consumer) { + return consumerName().equals(((Consumer) obj).consumerName()); + } + return false; + } + }; + selector.addConsumer(consumer); + } + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshotTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshotTest.java new file mode 100644 index 0000000000000..5c886b6eec9f3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumerHashAssignmentsSnapshotTest.java @@ -0,0 +1,204 @@ +/* + * 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.service; + +import static org.apache.pulsar.broker.BrokerTestUtil.createMockConsumer; +import static org.assertj.core.api.Assertions.assertThat; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.client.api.Range; +import org.testng.annotations.Test; + +@Test(groups = "broker-impl") +public class ConsumerHashAssignmentsSnapshotTest { + @Test + public void testMergeOverlappingRanges() { + SortedSet ranges = new TreeSet<>(); + ranges.add(Range.of(1, 5)); + ranges.add(Range.of(6, 10)); + ranges.add(Range.of(8, 12)); + ranges.add(Range.of(15, 20)); + ranges.add(Range.of(21, 25)); + + SortedSet expectedMergedRanges = new TreeSet<>(); + expectedMergedRanges.add(Range.of(1, 12)); + expectedMergedRanges.add(Range.of(15, 25)); + + List mergedRanges = ConsumerHashAssignmentsSnapshot.mergeOverlappingRanges(ranges); + + assertThat(mergedRanges).containsExactlyElementsOf(expectedMergedRanges); + } + + @Test + public void testDiffRanges_NoChanges() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + + Map> diff = + ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter); + + assertThat(diff).isEmpty(); + } + + @Test + public void testDiffRanges_ConsumerChanged() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + Consumer consumer2 = createMockConsumer("consumer2"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer2)); + + Map> diff = + ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter); + + assertThat(diff).containsEntry(Range.of(1, 5), Pair.of(consumer1, consumer2)); + } + + @Test + public void testDiffRanges_RangeAdded() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + Consumer consumer1 = createMockConsumer("consumer1"); + + mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + + Map> diff = + ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter); + + assertThat(diff).containsEntry(Range.of(1, 5), Pair.of(null, consumer1)); + } + + @Test + public void testDiffRanges_RangeRemoved() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + + Map> diff = + ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter); + + assertThat(diff).containsEntry(Range.of(1, 5), Pair.of(consumer1, null)); + } + + @Test + public void testDiffRanges_OverlappingRanges() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + Consumer consumer2 = createMockConsumer("consumer2"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + mappingAfter.add(new HashRangeAssignment(Range.of(3, 7), consumer2)); + + Map> diff = + ConsumerHashAssignmentsSnapshot.diffRanges(mappingBefore, mappingAfter); + + assertThat(diff).containsEntry(Range.of(3, 5), Pair.of(consumer1, consumer2)); + } + + @Test + public void testResolveConsumerRemovedHashRanges_NoChanges() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + + ImpactedConsumersResult impactedConsumers = + ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter); + + assertThat(impactedConsumers.getRemovedHashRanges()).isEmpty(); + } + + @Test + public void testResolveConsumerRemovedHashRanges_ConsumerChanged() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + Consumer consumer2 = createMockConsumer("consumer2"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer2)); + + ImpactedConsumersResult impactedConsumers = + ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter); + + assertThat(impactedConsumers.getRemovedHashRanges()).containsExactlyInAnyOrderEntriesOf( + Map.of(consumer1, RemovedHashRanges.of(List.of(Range.of(1, 5))))); + } + + @Test + public void testResolveConsumerRemovedHashRanges_RangeAdded() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + mappingAfter.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + + ImpactedConsumersResult impactedConsumers = + ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter); + + assertThat(impactedConsumers.getRemovedHashRanges()).isEmpty(); + } + + @Test + public void testResolveConsumerRemovedHashRanges_RangeRemoved() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + + ImpactedConsumersResult impactedConsumers = + ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter); + + assertThat(impactedConsumers.getRemovedHashRanges()).containsExactlyInAnyOrderEntriesOf( + Map.of(consumer1, RemovedHashRanges.of(List.of(Range.of(1, 5))))); + } + + @Test + public void testResolveConsumerRemovedHashRanges_OverlappingRanges() { + List mappingBefore = new ArrayList<>(); + List mappingAfter = new ArrayList<>(); + + Consumer consumer1 = createMockConsumer("consumer1"); + Consumer consumer2 = createMockConsumer("consumer2"); + mappingBefore.add(new HashRangeAssignment(Range.of(1, 5), consumer1)); + mappingAfter.add(new HashRangeAssignment(Range.of(3, 7), consumer2)); + + ImpactedConsumersResult impactedConsumers = + ConsumerHashAssignmentsSnapshot.resolveConsumerRemovedHashRanges(mappingBefore, mappingAfter); + + assertThat(impactedConsumers.getRemovedHashRanges()).containsExactlyInAnyOrderEntriesOf( + Map.of(consumer1, RemovedHashRanges.of(List.of(Range.of(3, 5))))); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DrainingHashesTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DrainingHashesTrackerTest.java new file mode 100644 index 0000000000000..ecb20beeb648a --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DrainingHashesTrackerTest.java @@ -0,0 +1,213 @@ +/* + * 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.service; + +import static org.apache.pulsar.broker.BrokerTestUtil.createMockConsumer; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; +import org.apache.pulsar.broker.service.DrainingHashesTracker.UnblockingHandler; +import org.testng.annotations.Test; + +public class DrainingHashesTrackerTest { + @Test + public void addEntry_AddsNewEntry() { + Consumer consumer = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + + tracker.addEntry(consumer, 1); + + assertNotNull(tracker.getEntry(1)); + assertSame(tracker.getEntry(1).getConsumer(), consumer); + } + + @Test + public void addEntry_ThrowsExceptionForZeroStickyHash() { + Consumer consumer = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + + assertThrows(IllegalArgumentException.class, () -> tracker.addEntry(consumer, 0)); + } + + @Test + public void reduceRefCount_ReducesReferenceCount() { + Consumer consumer = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + tracker.addEntry(consumer, 1); + + tracker.reduceRefCount(consumer, 1, false); + + assertNull(tracker.getEntry(1)); + } + + @Test + public void reduceRefCount_DoesNotReduceForDifferentConsumer() { + Consumer consumer1 = createMockConsumer("consumer1"); + Consumer consumer2 = createMockConsumer("consumer2"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + tracker.addEntry(consumer1, 1); + + assertThrows(IllegalStateException.class, () -> tracker.reduceRefCount(consumer2, 1, false)); + + assertNotNull(tracker.getEntry(1)); + assertSame(tracker.getEntry(1).getConsumer(), consumer1); + } + + @Test + public void shouldBlockStickyKeyHash_DoesNotBlockForExistingEntryWhenSameConsumer() { + Consumer consumer = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + tracker.addEntry(consumer, 1); + + boolean result = tracker.shouldBlockStickyKeyHash(consumer, 1); + + assertFalse(result); + } + + @Test + public void shouldBlockStickyKeyHash_BlocksForExistingEntryWhenDifferentConsumer() { + Consumer consumer1 = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + tracker.addEntry(consumer1, 1); + + Consumer consumer2 = createMockConsumer("consumer2"); + boolean result = tracker.shouldBlockStickyKeyHash(consumer2, 1); + + assertTrue(result); + } + + + @Test + public void shouldBlockStickyKeyHash_DoesNotBlockForNewEntry() { + Consumer consumer = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + + boolean result = tracker.shouldBlockStickyKeyHash(consumer, 1); + + assertFalse(result); + } + + @Test + public void startBatch_IncrementsBatchLevel() { + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + + tracker.startBatch(); + assertEquals(tracker.batchLevel, 1); + + tracker.startBatch(); + assertEquals(tracker.batchLevel, 2); + + tracker.startBatch(); + assertEquals(tracker.batchLevel, 3); + } + + @Test + public void endBatch_DecrementsBatchLevel() { + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + tracker.startBatch(); + + tracker.endBatch(); + + assertEquals(tracker.batchLevel, 0); + } + + @Test + public void endBatch_InvokesUnblockingHandlerWhenUnblockedWhileBatching() { + // given a tracker with unblocking handler + UnblockingHandler unblockingHandler = mock(UnblockingHandler.class); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", unblockingHandler); + + // when a hash is draining + Consumer consumer1 = createMockConsumer("consumer1"); + tracker.addEntry(consumer1, 1); + // and batch starts + tracker.startBatch(); + + // when hash gets blocked + Consumer consumer2 = createMockConsumer("consumer2"); + tracker.shouldBlockStickyKeyHash(consumer2, 1); + // and it gets unblocked + tracker.reduceRefCount(consumer1, 1, false); + + // then no unblocking call should be done + verify(unblockingHandler, never()).stickyKeyHashUnblocked(anyInt()); + + // when batch ends + tracker.endBatch(); + // then unblocking call should be done + verify(unblockingHandler).stickyKeyHashUnblocked(-1); + } + + @Test + public void clear_RemovesAllEntries() { + Consumer consumer = createMockConsumer("consumer1"); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", mock(UnblockingHandler.class)); + tracker.addEntry(consumer, 1); + + tracker.clear(); + + assertNull(tracker.getEntry(1)); + } + + @Test + public void unblockingHandler_InvokesStickyKeyHashUnblocked() { + // given a tracker with unblocking handler + UnblockingHandler unblockingHandler = mock(UnblockingHandler.class); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", unblockingHandler); + + // when a hash is draining + Consumer consumer = createMockConsumer("consumer1"); + tracker.addEntry(consumer, 1); + // aand hash gets blocked + Consumer consumer2 = createMockConsumer("consumer2"); + tracker.shouldBlockStickyKeyHash(consumer2, 1); + // and hash gets unblocked + tracker.reduceRefCount(consumer, 1, false); + + // then unblocking call should be done + verify(unblockingHandler).stickyKeyHashUnblocked(1); + } + + @Test + public void unblockingHandler_DoesNotInvokeStickyKeyHashUnblockedWhenClosing() { + // given a tracker with unblocking handler + UnblockingHandler unblockingHandler = mock(UnblockingHandler.class); + DrainingHashesTracker tracker = new DrainingHashesTracker("dispatcher1", unblockingHandler); + + // when a hash is draining + Consumer consumer = createMockConsumer("consumer1"); + tracker.addEntry(consumer, 1); + // aand hash gets blocked + Consumer consumer2 = createMockConsumer("consumer2"); + tracker.shouldBlockStickyKeyHash(consumer2, 1); + // and hash gets unblocked + tracker.reduceRefCount(consumer, 1, true); + + // then unblocking call should be done + verify(unblockingHandler, never()).stickyKeyHashUnblocked(anyInt()); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PendingAcksMapTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PendingAcksMapTest.java new file mode 100644 index 0000000000000..42f5935ca88ff --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PendingAcksMapTest.java @@ -0,0 +1,196 @@ +/* + * 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.service; + +import static org.apache.pulsar.broker.BrokerTestUtil.createMockConsumer; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +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.assertTrue; +import java.util.ArrayList; +import java.util.List; +import org.testng.annotations.Test; + +public class PendingAcksMapTest { + @Test + public void addPendingAckIfAllowed_AddsAckWhenAllowed() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + + boolean result = pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + + assertTrue(result); + assertTrue(pendingAcksMap.contains(1L, 1L)); + } + + @Test + public void addPendingAckIfAllowed_DoesNotAddAckWhenNotAllowed() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap.PendingAcksAddHandler addHandler = mock(PendingAcksMap.PendingAcksAddHandler.class); + when(addHandler.handleAdding(any(), anyLong(), anyLong(), anyInt())).thenReturn(false); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> addHandler, () -> null); + + boolean result = pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + + assertFalse(result); + assertFalse(pendingAcksMap.contains(1L, 1L)); + } + + @Test + public void addPendingAckIfAllowed_DoesNotAddAfterClosed() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.forEachAndClose((ledgerId, entryId, batchSize, stickyKeyHash) -> {}); + + boolean result = pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + + assertFalse(result); + assertFalse(pendingAcksMap.contains(1L, 1L)); + } + + @Test + public void forEach_ProcessesAllPendingAcks() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124); + + List processedEntries = new ArrayList<>(); + pendingAcksMap.forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> processedEntries.add(entryId)); + + assertEquals(processedEntries, List.of(1L, 2L)); + } + + @Test + public void forEachAndClose_ProcessesAndClearsAllPendingAcks() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124); + + List processedEntries = new ArrayList<>(); + pendingAcksMap.forEachAndClose((ledgerId, entryId, batchSize, stickyKeyHash) -> processedEntries.add(entryId)); + + assertEquals(processedEntries, List.of(1L, 2L)); + assertEquals(pendingAcksMap.size(), 0); + } + + @Test + public void remove_RemovesPendingAck() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + + boolean result = pendingAcksMap.remove(1L, 1L); + + assertTrue(result); + assertFalse(pendingAcksMap.contains(1L, 1L)); + } + + @Test + public void removeAllUpTo_RemovesAllPendingAcksUpToSpecifiedEntry() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124); + pendingAcksMap.addPendingAckIfAllowed(2L, 1L, 1, 125); + + pendingAcksMap.removeAllUpTo(1L, 2L); + + assertFalse(pendingAcksMap.contains(1L, 1L)); + assertFalse(pendingAcksMap.contains(1L, 2L)); + assertTrue(pendingAcksMap.contains(2L, 1L)); + } + + @Test + public void removeAllUpTo_RemovesAllPendingAcksUpToSpecifiedEntryAcrossMultipleLedgers() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124); + pendingAcksMap.addPendingAckIfAllowed(2L, 1L, 1, 125); + pendingAcksMap.addPendingAckIfAllowed(2L, 2L, 1, 126); + pendingAcksMap.addPendingAckIfAllowed(3L, 1L, 1, 127); + + pendingAcksMap.removeAllUpTo(2L, 1L); + + assertFalse(pendingAcksMap.contains(1L, 1L)); + assertFalse(pendingAcksMap.contains(1L, 2L)); + assertFalse(pendingAcksMap.contains(2L, 1L)); + assertTrue(pendingAcksMap.contains(2L, 2L)); + assertTrue(pendingAcksMap.contains(3L, 1L)); + } + + @Test + public void addPendingAckIfAllowed_InvokesAddHandler() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap.PendingAcksAddHandler addHandler = mock(PendingAcksMap.PendingAcksAddHandler.class); + when(addHandler.handleAdding(any(), anyLong(), anyLong(), anyInt())).thenReturn(true); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> addHandler, () -> null); + + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + + verify(addHandler).handleAdding(consumer, 1L, 1L, 123); + } + + @Test + public void remove_InvokesRemoveHandler() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap.PendingAcksRemoveHandler removeHandler = mock(PendingAcksMap.PendingAcksRemoveHandler.class); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> removeHandler); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + + pendingAcksMap.remove(1L, 1L); + + verify(removeHandler).handleRemoving(consumer, 1L, 1L, 123, false); + } + + @Test + public void removeAllUpTo_InvokesRemoveHandlerForEachEntry() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap.PendingAcksRemoveHandler removeHandler = mock(PendingAcksMap.PendingAcksRemoveHandler.class); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> removeHandler); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124); + pendingAcksMap.addPendingAckIfAllowed(2L, 1L, 1, 125); + + pendingAcksMap.removeAllUpTo(1L, 2L); + + verify(removeHandler).handleRemoving(consumer, 1L, 1L, 123, false); + verify(removeHandler).handleRemoving(consumer, 1L, 2L, 124, false); + verify(removeHandler, never()).handleRemoving(consumer, 2L, 1L, 125, false); + } + + @Test + public void size_ReturnsCorrectSize() { + Consumer consumer = createMockConsumer("consumer1"); + PendingAcksMap pendingAcksMap = new PendingAcksMap(consumer, () -> null, () -> null); + pendingAcksMap.addPendingAckIfAllowed(1L, 1L, 1, 123); + pendingAcksMap.addPendingAckIfAllowed(1L, 2L, 1, 124); + pendingAcksMap.addPendingAckIfAllowed(2L, 1L, 1, 125); + + assertEquals(pendingAcksMap.size(), 3); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java index 2896c13af0093..4d79e7ccdf0d1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service; -import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -55,8 +54,6 @@ import org.apache.bookkeeper.mledger.impl.cache.EntryCache; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; -import org.apache.pulsar.broker.service.persistent.MessageRedeliveryController; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.schema.SchemaRegistry; @@ -1753,90 +1750,6 @@ public void testMessageRedelivery() throws Exception { producer.close(); } - /** - * Verify: 1. Broker should not replay already acknowledged messages 2. Dispatcher should not stuck while - * dispatching new messages due to previous-replay of invalid/already-acked messages - * - * @throws Exception - */ - @Test - public void testMessageReplay() throws Exception { - - final String topicName = "persistent://prop/ns-abc/topic2"; - final String subName = "sub2"; - - Message msg; - int totalMessages = 10; - int replayIndex = totalMessages / 2; - - Consumer consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subName) - .subscriptionType(SubscriptionType.Shared).receiverQueueSize(1).subscribe(); - Producer producer = pulsarClient.newProducer() - .topic(topicName) - .enableBatching(false) - .messageRoutingMode(MessageRoutingMode.SinglePartition) - .create(); - - PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); - assertNotNull(topicRef); - PersistentSubscription subRef = topicRef.getSubscription(subName); - PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers) subRef - .getDispatcher(); - Field redeliveryMessagesField = PersistentDispatcherMultipleConsumers.class - .getDeclaredField("redeliveryMessages"); - redeliveryMessagesField.setAccessible(true); - MessageRedeliveryController redeliveryMessages = new MessageRedeliveryController(true); - - assertNotNull(subRef); - - // (1) Produce messages - for (int i = 0; i < totalMessages; i++) { - String message = "my-message-" + i; - producer.send(message.getBytes()); - } - - MessageIdImpl firstAckedMsg = null; - // (2) Consume and ack messages except first message - for (int i = 0; i < totalMessages; i++) { - msg = consumer.receive(); - consumer.acknowledge(msg); - MessageIdImpl msgId = (MessageIdImpl) msg.getMessageId(); - if (i == 0) { - firstAckedMsg = msgId; - } - if (i < replayIndex) { - // (3) accumulate acked messages for replay - redeliveryMessages.add(msgId.getLedgerId(), msgId.getEntryId()); - } - } - - // (4) redelivery : should redeliver only unacked messages - Thread.sleep(1000); - - redeliveryMessagesField.set(dispatcher, redeliveryMessages); - // (a) redelivery with all acked-message should clear messageReply bucket - dispatcher.redeliverUnacknowledgedMessages(dispatcher.getConsumers().get(0), DEFAULT_CONSUMER_EPOCH); - Awaitility.await().atMost(10, TimeUnit.SECONDS).until(() -> { - return redeliveryMessages.isEmpty(); - }); - assertTrue(redeliveryMessages.isEmpty()); - - // (b) fill messageReplyBucket with already acked entry again: and try to publish new msg and read it - redeliveryMessages.add(firstAckedMsg.getLedgerId(), firstAckedMsg.getEntryId()); - redeliveryMessagesField.set(dispatcher, redeliveryMessages); - // send new message - final String testMsg = "testMsg"; - producer.send(testMsg.getBytes()); - // consumer should be able to receive only new message and not the - dispatcher.consumerFlow(dispatcher.getConsumers().get(0), 1); - msg = consumer.receive(1, TimeUnit.SECONDS); - assertNotNull(msg); - assertEquals(msg.getData(), testMsg.getBytes()); - - consumer.close(); - producer.close(); - } - @Test public void testCreateProducerWithSameName() throws Exception { String topic = "persistent://prop/ns-abc/testCreateProducerWithSameName"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java index a03ed92b81590..052c5ceb5cdde 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java @@ -158,8 +158,8 @@ public void testSkipReadEntriesFromCloseCursor() throws Exception { dispatcher1.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException("cursor closed"), null); return null; - }).when(cursor).asyncReadEntriesOrWait(Mockito.anyInt(), Mockito.anyLong(), Mockito.eq(dispatcher), - Mockito.any(), Mockito.any()); + }).when(cursor).asyncReadEntriesWithSkipOrWait(Mockito.anyInt(), Mockito.anyLong(), Mockito.eq(dispatcher), + Mockito.any(), Mockito.any(), Mockito.any()); dispatcher.readMoreEntries(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index a0054f7e71425..4b29ead984e7a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -37,8 +37,6 @@ import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNull; -import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.google.common.collect.BoundType; @@ -51,16 +49,15 @@ import io.netty.util.concurrent.GenericFutureListener; import java.lang.reflect.Field; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Queue; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.mledger.Entry; @@ -73,6 +70,7 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.EntryAndMetadata; import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; import org.apache.pulsar.broker.service.EntryBatchSizes; import org.apache.pulsar.broker.service.RedeliveryTracker; @@ -85,8 +83,6 @@ import org.apache.pulsar.common.policies.data.HierarchyTopicPolicies; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.Markers; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; -import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.awaitility.Awaitility; import org.mockito.ArgumentCaptor; import org.testng.Assert; @@ -123,9 +119,10 @@ public void setup() throws Exception { doReturn(true).when(configMock).isSubscriptionRedeliveryTrackerEnabled(); doReturn(100).when(configMock).getDispatcherMaxReadBatchSize(); doReturn(true).when(configMock).isSubscriptionKeySharedUseConsistentHashing(); - doReturn(1).when(configMock).getSubscriptionKeySharedConsistentHashingReplicaPoints(); + doReturn(20).when(configMock).getSubscriptionKeySharedConsistentHashingReplicaPoints(); doReturn(false).when(configMock).isDispatcherDispatchMessagesInSubscriptionThread(); doReturn(false).when(configMock).isAllowOverrideEntryFilters(); + doReturn(false).when(configMock).isDispatchThrottlingOnNonBacklogConsumerEnabled(); doAnswer(invocation -> retryBackoffInitialTimeInMs).when(configMock).getDispatcherRetryBackoffInitialTimeInMs(); doAnswer(invocation -> retryBackoffMaxTimeInMs).when(configMock).getDispatcherRetryBackoffMaxTimeInMs(); pulsarMock = mock(PulsarService.class); @@ -223,6 +220,8 @@ protected static Consumer createMockConsumer() { TransportCnx transportCnx = mock(TransportCnx.class); doReturn(transportCnx).when(consumerMock).cnx(); doReturn(true).when(transportCnx).isActive(); + doReturn(100).when(consumerMock).getMaxUnackedMessages(); + doReturn(1).when(consumerMock).getAvgMessagesPerEntry(); return consumerMock; } @@ -321,13 +320,16 @@ public void testSendMessage() { @Test public void testSkipRedeliverTemporally() { final Consumer slowConsumerMock = createMockConsumer(); + AtomicInteger slowConsumerPermits = new AtomicInteger(0); + doAnswer(invocation -> slowConsumerPermits.get()).when(slowConsumerMock).getAvailablePermits(); + final ChannelPromise slowChannelMock = mock(ChannelPromise.class); // add entries to redeliver and read target final List redeliverEntries = new ArrayList<>(); - redeliverEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key1"))); + redeliverEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key123"))); final List readEntries = new ArrayList<>(); - readEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); - readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key2"))); + readEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key123"))); + readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key222"))); try { Field totalAvailablePermitsField = PersistentDispatcherMultipleConsumers.class.getDeclaredField("totalAvailablePermits"); @@ -348,9 +350,6 @@ public void testSkipRedeliverTemporally() { // Create 2Consumers try { doReturn("consumer2").when(slowConsumerMock).consumerName(); - when(slowConsumerMock.getAvailablePermits()) - .thenReturn(0) - .thenReturn(1); doReturn(true).when(slowConsumerMock).isWritable(); doReturn(slowChannelMock).when(slowConsumerMock).sendMessages( anyList(), @@ -375,13 +374,12 @@ public void testSkipRedeliverTemporally() { // and then stop to dispatch to slowConsumer persistentDispatcher.readEntriesComplete(redeliverEntries, PersistentDispatcherMultipleConsumers.ReadType.Replay); - verify(consumerMock, times(1)).sendMessages( argThat(arg -> { assertEquals(arg.size(), 1); Entry entry = arg.get(0); assertEquals(entry.getLedgerId(), 1); - assertEquals(entry.getEntryId(), 3); + assertEquals(entry.getEntryId(), 1); return true; }), any(EntryBatchSizes.class), @@ -408,25 +406,9 @@ public void testMessageRedelivery() throws Exception { final Queue actualEntriesToConsumer2 = new ConcurrentLinkedQueue<>(); final Queue expectedEntriesToConsumer1 = new ConcurrentLinkedQueue<>(); - expectedEntriesToConsumer1.add(PositionFactory.create(1, 1)); final Queue expectedEntriesToConsumer2 = new ConcurrentLinkedQueue<>(); - expectedEntriesToConsumer2.add(PositionFactory.create(1, 2)); - expectedEntriesToConsumer2.add(PositionFactory.create(1, 3)); - - final AtomicInteger remainingEntriesNum = new AtomicInteger( - expectedEntriesToConsumer1.size() + expectedEntriesToConsumer2.size()); - - // Messages with key1 are routed to consumer1 and messages with key2 are routed to consumer2 - final List allEntries = new ArrayList<>(); - allEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key2"))); - allEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); - allEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key1"))); - allEntries.forEach(entry -> ((EntryImpl) entry).retain()); - final List redeliverEntries = new ArrayList<>(); - redeliverEntries.add(allEntries.get(0)); // message1 - final List readEntries = new ArrayList<>(); - readEntries.add(allEntries.get(2)); // message3 + final AtomicInteger remainingEntriesNum = new AtomicInteger(0); final Consumer consumer1 = createMockConsumer(); doReturn("consumer1").when(consumer1).consumerName(); @@ -434,8 +416,7 @@ public void testMessageRedelivery() throws Exception { when(consumer1.getAvailablePermits()).thenReturn(0).thenReturn(10); doReturn(true).when(consumer1).isWritable(); doAnswer(invocationOnMock -> { - @SuppressWarnings("unchecked") - List entries = (List) invocationOnMock.getArgument(0); + List entries = invocationOnMock.getArgument(0); for (Entry entry : entries) { remainingEntriesNum.decrementAndGet(); actualEntriesToConsumer1.add(entry.getPosition()); @@ -449,8 +430,7 @@ public void testMessageRedelivery() throws Exception { when(consumer2.getAvailablePermits()).thenReturn(10); doReturn(true).when(consumer2).isWritable(); doAnswer(invocationOnMock -> { - @SuppressWarnings("unchecked") - List entries = (List) invocationOnMock.getArgument(0); + List entries = invocationOnMock.getArgument(0); for (Entry entry : entries) { remainingEntriesNum.decrementAndGet(); actualEntriesToConsumer2.add(entry.getPosition()); @@ -467,44 +447,64 @@ public void testMessageRedelivery() throws Exception { totalAvailablePermitsField.setAccessible(true); totalAvailablePermitsField.set(persistentDispatcher, 1000); - final Field redeliveryMessagesField = PersistentDispatcherMultipleConsumers.class - .getDeclaredField("redeliveryMessages"); - redeliveryMessagesField.setAccessible(true); - MessageRedeliveryController redeliveryMessages = (MessageRedeliveryController) redeliveryMessagesField - .get(persistentDispatcher); - redeliveryMessages.add(allEntries.get(0).getLedgerId(), allEntries.get(0).getEntryId(), - getStickyKeyHash(allEntries.get(0))); // message1 - redeliveryMessages.add(allEntries.get(1).getLedgerId(), allEntries.get(1).getEntryId(), - getStickyKeyHash(allEntries.get(1))); // message2 + StickyKeyConsumerSelector selector = persistentDispatcher.getSelector(); + + String keyForConsumer1 = generateKeyForConsumer(selector, consumer1); + String keyForConsumer2 = generateKeyForConsumer(selector, consumer2); + + // Messages with key1 are routed to consumer1 and messages with key2 are routed to consumer2 + final List allEntries = new ArrayList<>(); + allEntries.add(EntryAndMetadata.create(EntryImpl.create(1, 1, createMessage("message1", 1, keyForConsumer1)))); + allEntries.add(EntryAndMetadata.create(EntryImpl.create(1, 2, createMessage("message2", 2, keyForConsumer1)))); + allEntries.add(EntryAndMetadata.create(EntryImpl.create(1, 3, createMessage("message3", 3, keyForConsumer2)))); + allEntries.forEach(entry -> { + EntryImpl entryImpl = (EntryImpl) ((EntryAndMetadata) entry).unwrap(); + entryImpl.retain(); + // initialize sticky key hash + persistentDispatcher.getStickyKeyHash(entry); + }); + remainingEntriesNum.set(allEntries.size()); + + final List redeliverEntries = new ArrayList<>(); + redeliverEntries.add(allEntries.get(0)); // message1 + final List readEntries = new ArrayList<>(); + readEntries.add(allEntries.get(2)); // message3 + + expectedEntriesToConsumer1.add(allEntries.get(0).getPosition()); + expectedEntriesToConsumer1.add(allEntries.get(1).getPosition()); + expectedEntriesToConsumer2.add(allEntries.get(2).getPosition()); // Mock Cursor#asyncReplayEntries doAnswer(invocationOnMock -> { - @SuppressWarnings("unchecked") - Set positions = (Set) invocationOnMock.getArgument(0); - List entries = allEntries.stream().filter(entry -> positions.contains(entry.getPosition())) + Set positionsArg = invocationOnMock.getArgument(0); + Set positions = new TreeSet<>(positionsArg); + Set alreadyReceived = new TreeSet<>(); + alreadyReceived.addAll(actualEntriesToConsumer1); + alreadyReceived.addAll(actualEntriesToConsumer2); + List entries = allEntries.stream().filter(entry -> positions.contains(entry.getPosition()) + && !alreadyReceived.contains(entry.getPosition())) .collect(Collectors.toList()); - if (!entries.isEmpty()) { - ((PersistentStickyKeyDispatcherMultipleConsumers) invocationOnMock.getArgument(1)) - .readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Replay); - } - return Collections.emptySet(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = invocationOnMock.getArgument(1); + dispatcher.readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Replay); + return alreadyReceived; }).when(cursorMock).asyncReplayEntries(anySet(), any(PersistentStickyKeyDispatcherMultipleConsumers.class), eq(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Replay), anyBoolean()); // Mock Cursor#asyncReadEntriesOrWait - AtomicBoolean asyncReadEntriesOrWaitCalled = new AtomicBoolean(); doAnswer(invocationOnMock -> { - if (asyncReadEntriesOrWaitCalled.compareAndSet(false, true)) { - ((PersistentStickyKeyDispatcherMultipleConsumers) invocationOnMock.getArgument(2)) - .readEntriesComplete(readEntries, PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal); - } else { - ((PersistentStickyKeyDispatcherMultipleConsumers) invocationOnMock.getArgument(2)) - .readEntriesComplete(Collections.emptyList(), PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal); - } + int maxEntries = invocationOnMock.getArgument(0); + Set alreadyReceived = new TreeSet<>(); + alreadyReceived.addAll(actualEntriesToConsumer1); + alreadyReceived.addAll(actualEntriesToConsumer2); + List entries = allEntries.stream() + .filter(entry -> !alreadyReceived.contains(entry.getPosition())) + .limit(maxEntries).collect(Collectors.toList()); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = invocationOnMock.getArgument(2); + dispatcher.readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal); return null; - }).when(cursorMock).asyncReadEntriesOrWait(anyInt(), anyLong(), + }).when(cursorMock).asyncReadEntriesWithSkipOrWait(anyInt(), anyLong(), any(PersistentStickyKeyDispatcherMultipleConsumers.class), - eq(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal), any()); + eq(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Normal), any(), any()); // (1) Run sendMessagesToConsumers // (2) Attempts to send message1 to consumer1 but skipped because availablePermits is 0 @@ -512,6 +512,11 @@ public void testMessageRedelivery() throws Exception { // (4) Run readMoreEntries internally // (5) Run sendMessagesToConsumers internally // (6) Attempts to send message3 to consumer2 but skipped because redeliveryMessages contains message2 + redeliverEntries.forEach(entry -> { + EntryImpl entryImpl = (EntryImpl) ((EntryAndMetadata) entry).unwrap(); + entryImpl.retain(); + persistentDispatcher.addEntryToReplay(entry); + }); persistentDispatcher.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumers.ReadType.Replay, redeliverEntries, true); while (remainingEntriesNum.get() > 0) { @@ -525,313 +530,16 @@ public void testMessageRedelivery() throws Exception { allEntries.forEach(entry -> entry.release()); } - @DataProvider(name = "initializeLastSentPosition") - private Object[][] initialLastSentPositionProvider() { - return new Object[][] { { false }, { true } }; - } - - @Test(dataProvider = "initializeLastSentPosition") - public void testLastSentPositionAndIndividuallySentPositions(final boolean initializeLastSentPosition) throws Exception { - final Position initialLastSentPosition = PositionFactory.create(1, 10); - final LongPairRangeSet expectedIndividuallySentPositions - = new ConcurrentOpenLongPairRangeSet<>(4096, PositionFactory::create); - - final Field lastSentPositionField = PersistentStickyKeyDispatcherMultipleConsumers.class - .getDeclaredField("lastSentPosition"); - lastSentPositionField.setAccessible(true); - final LongPairRangeSet individuallySentPositions = persistentDispatcher.getIndividuallySentPositionsField(); - final Supplier clearPosition = () -> { - try { - lastSentPositionField.set(persistentDispatcher, initializeLastSentPosition ? initialLastSentPosition : null); - individuallySentPositions.clear(); - expectedIndividuallySentPositions.clear(); - } catch (Throwable e) { - return e; + private String generateKeyForConsumer(StickyKeyConsumerSelector selector, Consumer consumer) { + int i = 0; + while (!Thread.currentThread().isInterrupted()) { + String key = "key" + i++; + Consumer selectedConsumer = selector.select(key.getBytes(UTF_8)); + if (selectedConsumer == consumer) { + return key; } - return null; - }; - if (!initializeLastSentPosition) { - doReturn(initialLastSentPosition).when(cursorMock).getMarkDeletedPosition(); - doAnswer(invocationOnMock -> { - // skip copy operation - return initialLastSentPosition; - }).when(cursorMock).processIndividuallyDeletedMessagesAndGetMarkDeletedPosition(any()); } - - // Assume the range sequence is [1:0, 1:19], [2:0, 2:19], ..., [10:0, 10:19] - doAnswer((invocationOnMock -> { - final Position position = invocationOnMock.getArgument(0); - if (position.getEntryId() > 0) { - return PositionFactory.create(position.getLedgerId(), position.getEntryId() - 1); - } else if (position.getLedgerId() > 0) { - return PositionFactory.create(position.getLedgerId() - 1, 19); - } else { - throw new NullPointerException(); - } - })).when(ledgerMock).getPreviousPosition(any(Position.class)); - doAnswer((invocationOnMock -> { - final Position position = invocationOnMock.getArgument(0); - if (position.getEntryId() < 19) { - return PositionFactory.create(position.getLedgerId(), position.getEntryId() + 1); - } else { - return PositionFactory.create(position.getLedgerId() + 1, 0); - } - })).when(ledgerMock).getNextValidPosition(any(Position.class)); - doReturn(PositionFactory.create(10, 19)).when(ledgerMock).getLastConfirmedEntry(); - doAnswer((invocationOnMock -> { - final Range range = invocationOnMock.getArgument(0); - Position fromPosition = range.lowerEndpoint(); - boolean fromIncluded = range.lowerBoundType() == BoundType.CLOSED; - Position toPosition = range.upperEndpoint(); - boolean toIncluded = range.upperBoundType() == BoundType.CLOSED; - - if (fromPosition.getLedgerId() == toPosition.getLedgerId()) { - // If the 2 positions are in the same ledger - long count = toPosition.getEntryId() - fromPosition.getEntryId() - 1; - count += fromIncluded ? 1 : 0; - count += toIncluded ? 1 : 0; - return count; - } else { - long count = 0; - // If the from & to are pointing to different ledgers, then we need to : - // 1. Add the entries in the ledger pointed by toPosition - count += toPosition.getEntryId(); - count += toIncluded ? 1 : 0; - - // 2. Add the entries in the ledger pointed by fromPosition - count += 20 - (fromPosition.getEntryId() + 1); - count += fromIncluded ? 1 : 0; - - // 3. Add the whole ledgers entries in between - for (long i = fromPosition.getLedgerId() + 1; i < toPosition.getLedgerId(); i++) { - count += 20; - } - - return count; - } - })).when(ledgerMock).getNumberOfEntries(any()); - assertEquals(ledgerMock.getNextValidPosition(PositionFactory.create(1, 0)), PositionFactory.create(1, 1)); - assertEquals(ledgerMock.getNextValidPosition(PositionFactory.create(1, 19)), PositionFactory.create(2, 0)); - assertEquals(ledgerMock.getPreviousPosition(PositionFactory.create(2, 0)), PositionFactory.create(1, 19)); - assertThrows(NullPointerException.class, () -> ledgerMock.getPreviousPosition(PositionFactory.create(0, 0))); - assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( - PositionFactory.create(1, 0), PositionFactory.create(1, 0))), 0); - assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( - PositionFactory.create(1, -1), PositionFactory.create(1, 9))), 10); - assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( - PositionFactory.create(1, 19), PositionFactory.create(2, -1))), 0); - assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( - PositionFactory.create(1, 19), PositionFactory.create(2, 9))), 10); - assertEquals(ledgerMock.getNumberOfEntries(Range.openClosed( - PositionFactory.create(1, -1), PositionFactory.create(3, 19))), 60); - - // Add a consumer - final Consumer consumer1 = createMockConsumer(); - doReturn("consumer1").when(consumer1).consumerName(); - when(consumer1.getAvailablePermits()).thenReturn(1000); - doReturn(true).when(consumer1).isWritable(); - doReturn(channelMock).when(consumer1).sendMessages(anyList(), any(EntryBatchSizes.class), - any(EntryBatchIndexesAcks.class), anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); - persistentDispatcher.addConsumer(consumer1); - - /* - On single ledger - */ - - // Expected individuallySentPositions (isp): [(1:-1, 1:8]] (init) -> [(1:-1, 1:9]] (update) -> [] (remove) - // Expected lastSentPosition (lsp): 1:10 (init) -> 1:10 (remove) - // upper bound and the new entry are less than initial last sent position - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, -1, 1, 8); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 9, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - // isp: [(1:-1, 1:9]] -> [(1:-1, 1:10]] -> [] - // lsp: 1:10 -> 1:10 - // upper bound is less than initial last sent position - // upper bound and the new entry are less than or equal to initial last sent position - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, -1, 1, 9); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 10, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - // isp: [(1:-1, 1:2], (1:3, 1:4], (1:5, 1:6]] -> [(1:-1, 1:2], (1:3, 1:4], (1:5, 1:6], (1:9, 1:10]] -> [] - // lsp: 1:10 -> 1:10 - // upper bound and the new entry are less than or equal to initial last sent position - // individually sent positions has multiple ranges - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, -1, 1, 2); - individuallySentPositions.addOpenClosed(1, 3, 1, 4); - individuallySentPositions.addOpenClosed(1, 5, 1, 6); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 10, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - // isp: [(1:-1, 1:10]] -> [(1:-1, 1:11]] -> [] - // lsp: 1:10 -> 1:11 - // upper bound is less than or equal to initial last sent position - // the new entry is next position of initial last sent position - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, -1, 1, 10); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 11).toString()); - - // isp: [(1:-1, 1:9]] -> [(1:-1, 1:9], (1:10, 1:11]] -> [] - // lsp: 1:10 -> 1:11 - // upper bound is less than initial last sent position - // the new entry is next position of initial last sent position - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, -1, 1, 9); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 11).toString()); - - // isp: [(1:11, 1:15]] -> [(1:10, 1:15]] -> [] - // lsp: 1:10 -> 1:15 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry is next position of initial last sent position - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 15); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 15).toString()); - - // isp: [(1:11, 1:15]] -> [(1:10, 1:16]] -> [] - // lsp: 1:10 -> 1:16 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entries contain next position of initial last sent position - // first of the new entries is less than initial last sent position - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 15); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 9, createMessage("test", 1)), - EntryImpl.create(1, 11, createMessage("test", 2)), - EntryImpl.create(1, 16, createMessage("test", 3))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(1, 16).toString()); - - // isp: [(1:11, 1:15]] -> [(1:11, 1:15]] -> [(1:11, 1:15]] - // lsp: 1:10 -> 1:10 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry isn't next position of initial last sent position - // the range contains the new entry - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 15); - expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 15); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 15, createMessage("test", 1))), true); - assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - // isp: [(1:11, 1:15]] -> [(1:11, 1:16]] -> [(1:11, 1:16]] - // lsp: 1:10 -> 1:10 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry isn't next position of initial last sent position - // the range doesn't contain the new entry - // the new entry is next position of upper bound - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 15); - expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 16); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 16, createMessage("test", 1))), true); - assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - // isp: [(1:11, 1:15]] -> [(1:11, 1:15], (1:16, 1:17]] -> [(1:11, 1:15], (1:16, 1:17]] - // lsp: 1:10 -> 1:10 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry isn't next position of initial last sent position - // the range doesn't contain the new entry - // the new entry isn't next position of upper bound - // the new entry is same ledger - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 15); - expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 15); - expectedIndividuallySentPositions.addOpenClosed(1, 16, 1, 17); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 17, createMessage("test", 1))), true); - assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - /* - On multiple contiguous ledgers - */ - - // isp: [(1:11, 1:18]] -> [(1:11, 1:18], (2:-1, 2:0]] -> [(1:11, 1:18], (2:-1, 2:0]] - // lsp: 1:10 -> 1:10 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry isn't next position of initial last sent position - // the range doesn't contain the new entry - // the new entry isn't next position of upper bound - // the new entry isn't same ledger - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 18); - expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 18); - expectedIndividuallySentPositions.addOpenClosed(2, -1, 2, 0); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(2, 0, createMessage("test", 1))), true); - assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); - - // isp: [(1:11, 1:19], (2:-1, 2:0]] -> [(1:10, 1:19], (2:-1, 2:0]] -> [] - // lsp: 1:10 -> 2:0 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry is next position of initial last sent position - // the new entry isn't same ledger - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 19); - individuallySentPositions.addOpenClosed(2, -1, 2, 0); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(2, 0).toString()); - - // isp: [(1:11, 1:19], (2:-1, 2:19], (3:-1, 3:0]] -> [(1:10, 1:19], (2:-1, 2:19], (3:-1, 3:0]] -> [] - // lsp: 1:10 -> 3:0 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry is next position of initial last sent position - // the new entry isn't same ledger - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 19); - individuallySentPositions.addOpenClosed(2, -1, 2, 19); - individuallySentPositions.addOpenClosed(3, -1, 3, 0); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(1, 11, createMessage("test", 1))), true); - assertTrue(individuallySentPositions.isEmpty()); - assertEquals(persistentDispatcher.getLastSentPosition(), PositionFactory.create(3, 0).toString()); - - // isp: [(1:11, 1:19], (2:-1, 2:0]] -> [(1:11, 1:19], (2:-1, 2:1]] -> [(1:11, 1:19], (2:-1, 2:1]] - // lsp: 1:10 -> 1:10 - // upper bound is greater than initial last sent position - // the range doesn't contain next position of initial last sent position - // the new entry isn't next position of initial last sent position - // the new entry isn't same ledger - assertNull(clearPosition.get()); - individuallySentPositions.addOpenClosed(1, 11, 1, 19); - individuallySentPositions.addOpenClosed(2, -1, 2, 0); - expectedIndividuallySentPositions.addOpenClosed(1, 11, 1, 19); - expectedIndividuallySentPositions.addOpenClosed(2, -1, 2, 1); - persistentDispatcher.sendMessagesToConsumers(PersistentDispatcherMultipleConsumers.ReadType.Normal, - Arrays.asList(EntryImpl.create(2, 1, createMessage("test", 1))), true); - assertEquals(individuallySentPositions.toString(), expectedIndividuallySentPositions.toString()); - assertEquals(persistentDispatcher.getLastSentPosition(), initialLastSentPosition.toString()); + return null; } @DataProvider(name = "testBackoffDelayWhenNoMessagesDispatched") @@ -872,7 +580,7 @@ protected void reScheduleReadInMs(long readAfterMs) { // call "readEntriesComplete" directly to test the retry behavior List entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 1); assertEquals(retryDelays.get(0), 10, "Initial retry delay should be 10ms"); @@ -880,7 +588,7 @@ protected void reScheduleReadInMs(long readAfterMs) { ); // test the second retry delay entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 2); double delay = retryDelays.get(1); @@ -890,7 +598,7 @@ protected void reScheduleReadInMs(long readAfterMs) { // verify the max retry delay for (int i = 0; i < 100; i++) { entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); } Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 102); @@ -901,14 +609,14 @@ protected void reScheduleReadInMs(long readAfterMs) { // unblock to check that the retry delay is reset consumerMockAvailablePermits.set(1000); entries = List.of(EntryImpl.create(1, 2, createMessage("message2", 1, "key2"))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); // wait that the possibly async handling has completed Awaitility.await().untilAsserted(() -> assertFalse(dispatcher.isSendInProgress())); // now block again to check the next retry delay so verify it was reset consumerMockAvailablePermits.set(0); entries = List.of(EntryImpl.create(1, 3, createMessage("message3", 1, "key3"))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 103); assertEquals(retryDelays.get(0), 10, "Resetted retry delay should be 10ms"); @@ -955,7 +663,7 @@ protected void reScheduleReadInMs(long readAfterMs) { // call "readEntriesComplete" directly to test the retry behavior List entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 1); assertEquals(retryDelays.get(0), 0, "Initial retry delay should be 0ms"); @@ -963,7 +671,7 @@ protected void reScheduleReadInMs(long readAfterMs) { ); // test the second retry delay entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 2); double delay = retryDelays.get(1); @@ -973,7 +681,7 @@ protected void reScheduleReadInMs(long readAfterMs) { // verify the max retry delay for (int i = 0; i < 100; i++) { entries = List.of(EntryImpl.create(1, 1, createMessage("message1", 1))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); } Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 102); @@ -984,14 +692,14 @@ protected void reScheduleReadInMs(long readAfterMs) { // unblock to check that the retry delay is reset consumerMockAvailablePermits.set(1000); entries = List.of(EntryImpl.create(1, 2, createMessage("message2", 1, "key2"))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); // wait that the possibly async handling has completed Awaitility.await().untilAsserted(() -> assertFalse(dispatcher.isSendInProgress())); // now block again to check the next retry delay so verify it was reset consumerMockAvailablePermits.set(0); entries = List.of(EntryImpl.create(1, 3, createMessage("message3", 1, "key3"))); - dispatcher.readEntriesComplete(entries, PersistentDispatcherMultipleConsumers.ReadType.Normal); + dispatcher.readEntriesComplete(new ArrayList<>(entries), PersistentDispatcherMultipleConsumers.ReadType.Normal); Awaitility.await().untilAsserted(() -> { assertEquals(retryDelays.size(), 103); assertEquals(retryDelays.get(0), 0, "Resetted retry delay should be 0ms"); @@ -1092,9 +800,4 @@ private ByteBuf createMessage(String message, int sequenceId, String key) { .setPublishTime(System.currentTimeMillis()); return serializeMetadataAndPayload(Commands.ChecksumType.Crc32c, messageMetadata, Unpooled.copiedBuffer(message.getBytes(UTF_8))); } - - private int getStickyKeyHash(Entry entry) { - byte[] stickyKey = Commands.peekStickyKey(entry.getDataBuffer(), topicName, subscriptionName); - return StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey); - } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandlerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandlerTest.java new file mode 100644 index 0000000000000..cf404e38b66d3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandlerTest.java @@ -0,0 +1,161 @@ +/* + * 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.service.persistent; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.eq; +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 java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BooleanSupplier; +import java.util.function.LongSupplier; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class RescheduleReadHandlerTest { + private LongSupplier readIntervalMsSupplier; + private ScheduledExecutorService executor; + private Runnable cancelPendingRead; + private Runnable rescheduleReadImmediately; + private BooleanSupplier hasPendingReadRequestThatMightWait; + private LongSupplier readOpCounterSupplier; + private BooleanSupplier hasEntriesInReplayQueue; + private RescheduleReadHandler rescheduleReadHandler; + + @BeforeMethod + public void setUp() { + readIntervalMsSupplier = mock(LongSupplier.class); + executor = mock(ScheduledExecutorService.class); + cancelPendingRead = mock(Runnable.class); + rescheduleReadImmediately = mock(Runnable.class); + hasPendingReadRequestThatMightWait = mock(BooleanSupplier.class); + readOpCounterSupplier = mock(LongSupplier.class); + hasEntriesInReplayQueue = mock(BooleanSupplier.class); + rescheduleReadHandler = new RescheduleReadHandler(readIntervalMsSupplier, executor, cancelPendingRead, + () -> rescheduleReadImmediately.run(), hasPendingReadRequestThatMightWait, readOpCounterSupplier, hasEntriesInReplayQueue); + } + + @Test + public void rescheduleReadImmediately() { + when(readIntervalMsSupplier.getAsLong()).thenReturn(0L); + + rescheduleReadHandler.rescheduleRead(); + + verify(rescheduleReadImmediately).run(); + verify(executor, never()).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + } + + @Test + public void rescheduleReadWithDelay() { + when(readIntervalMsSupplier.getAsLong()).thenReturn(100L); + + rescheduleReadHandler.rescheduleRead(); + + verify(rescheduleReadImmediately, never()).run(); + verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS)); + } + + @Test + public void rescheduleReadWithDelayAndCancelPendingRead() { + when(readIntervalMsSupplier.getAsLong()).thenReturn(100L); + when(hasPendingReadRequestThatMightWait.getAsBoolean()).thenReturn(true); + when(readOpCounterSupplier.getAsLong()).thenReturn(5L); + when(hasEntriesInReplayQueue.getAsBoolean()).thenReturn(true); + doAnswer(invocation -> { + Runnable runnable = invocation.getArgument(0); + runnable.run(); + return null; + }).when(executor).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + + rescheduleReadHandler.rescheduleRead(); + + verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS)); + verify(rescheduleReadImmediately).run(); + verify(cancelPendingRead).run(); + } + + @Test + public void rescheduleReadWithDelayAndDontCancelPendingReadIfNoEntriesInReplayQueue() { + when(readIntervalMsSupplier.getAsLong()).thenReturn(100L); + when(hasPendingReadRequestThatMightWait.getAsBoolean()).thenReturn(true); + when(readOpCounterSupplier.getAsLong()).thenReturn(5L); + when(hasEntriesInReplayQueue.getAsBoolean()).thenReturn(false); + doAnswer(invocation -> { + Runnable runnable = invocation.getArgument(0); + runnable.run(); + return null; + }).when(executor).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + + rescheduleReadHandler.rescheduleRead(); + + verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS)); + verify(rescheduleReadImmediately).run(); + verify(cancelPendingRead, never()).run(); + } + + @Test + public void rescheduleReadBatching() { + when(readOpCounterSupplier.getAsLong()).thenReturn(5L); + when(readIntervalMsSupplier.getAsLong()).thenReturn(100L); + AtomicReference scheduledRunnable = new AtomicReference<>(); + doAnswer(invocation -> { + Runnable runnable = invocation.getArgument(0); + if (!scheduledRunnable.compareAndSet(null, runnable)) { + runnable.run(); + } + return null; + }).when(executor).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + // 3 rescheduleRead calls + rescheduleReadHandler.rescheduleRead(); + rescheduleReadHandler.rescheduleRead(); + rescheduleReadHandler.rescheduleRead(); + // scheduled task runs + scheduledRunnable.get().run(); + // verify that rescheduleReadImmediately is called only once + verify(rescheduleReadImmediately, times(1)).run(); + } + + @Test + public void rescheduleReadWithoutCancelPendingReadWhenReadOpCounterIncrements() { + // given + when(readIntervalMsSupplier.getAsLong()).thenReturn(100L); + when(hasPendingReadRequestThatMightWait.getAsBoolean()).thenReturn(true); + when(readOpCounterSupplier.getAsLong()).thenReturn(5L).thenReturn(6L); + when(hasEntriesInReplayQueue.getAsBoolean()).thenReturn(true); + doAnswer(invocation -> { + Runnable runnable = invocation.getArgument(0); + runnable.run(); + return null; + }).when(executor).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + // when rescheduleRead is called + rescheduleReadHandler.rescheduleRead(); + // then verify calls + verify(executor).schedule(any(Runnable.class), eq(100L), eq(TimeUnit.MILLISECONDS)); + verify(rescheduleReadImmediately).run(); + // verify that cancelPendingRead is not called + verify(cancelPendingRead, never()).run(); + } +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java index bc4cb73e5b6fe..4a8e7077395ed 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/SubscriptionStatsTest.java @@ -38,12 +38,10 @@ import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; -import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.TopicName; @@ -83,48 +81,6 @@ protected void cleanup() throws Exception { super.internalCleanup(); } - @Test - public void testConsumersAfterMarkDelete() throws PulsarClientException, PulsarAdminException { - final String topicName = "persistent://my-property/my-ns/testConsumersAfterMarkDelete-" - + UUID.randomUUID(); - final String subName = "my-sub"; - - Consumer consumer1 = pulsarClient.newConsumer() - .topic(topicName) - .receiverQueueSize(10) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .subscribe(); - - Producer producer = pulsarClient.newProducer() - .topic(topicName) - .create(); - - final int messages = 100; - for (int i = 0; i < messages; i++) { - producer.send(String.valueOf(i).getBytes()); - } - - // Receive by do not ack the message, so that the next consumer can added to the recentJoinedConsumer of the dispatcher. - consumer1.receive(); - - Consumer consumer2 = pulsarClient.newConsumer() - .topic(topicName) - .receiverQueueSize(10) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .subscribe(); - - TopicStats stats = admin.topics().getStats(topicName); - Assert.assertEquals(stats.getSubscriptions().size(), 1); - Assert.assertEquals(stats.getSubscriptions().entrySet().iterator().next().getValue() - .getConsumersAfterMarkDeletePosition().size(), 1); - - consumer1.close(); - consumer2.close(); - producer.close(); - } - @Test public void testNonContiguousDeletedMessagesRanges() throws Exception { final String topicName = "persistent://my-property/my-ns/testNonContiguousDeletedMessagesRanges-" diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index c08c37b413f4f..2b16647f5590c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -19,11 +19,9 @@ package org.apache.pulsar.client.api; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; import static org.apache.pulsar.broker.BrokerTestUtil.receiveMessages; import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -32,7 +30,6 @@ import static org.testng.Assert.fail; import com.google.common.collect.Lists; import java.io.IOException; -import java.lang.reflect.Field; import java.nio.file.Files; import java.nio.file.Paths; import java.time.Duration; @@ -41,7 +38,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -51,6 +47,7 @@ import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -59,33 +56,30 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; +import java.util.function.BiFunction; import java.util.stream.Collectors; import lombok.Cleanup; +import lombok.SneakyThrows; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.DrainingHashesTracker; +import org.apache.pulsar.broker.service.PendingAcksMap; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentStickyKeyDispatcherMultipleConsumers; -import org.apache.pulsar.broker.service.persistent.MessageRedeliveryController; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.impl.ConsumerImpl; -import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.api.proto.KeySharedMode; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.schema.KeyValue; -import org.apache.pulsar.common.util.Murmur3_32Hash; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; -import org.apache.pulsar.common.util.collections.LongPairRangeSet; +import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.mockito.Mockito; import org.slf4j.Logger; @@ -102,6 +96,7 @@ public class KeySharedSubscriptionTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(KeySharedSubscriptionTest.class); private static final List keys = Arrays.asList("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"); + private static final String SUBSCRIPTION_NAME = "key_shared"; @DataProvider(name = "batch") public Object[] batchProvider() { @@ -169,7 +164,9 @@ public void resetDefaultNamespace() throws Exception { defaultConf.getKeySharedLookAheadMsgInReplayThresholdPerConsumer()); } - private static final Random random = new Random(System.nanoTime()); + // Use a fixed seed to make the tests using random values deterministic + // When a test fails, it's possible to re-run it to reproduce the issue + private static final Random random = new Random(1); private static final int NUMBER_OF_KEYS = 300; @Test(dataProvider = "data") @@ -260,6 +257,8 @@ public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(bo Consumer consumer3 = createConsumer(topic, KeySharedPolicy.stickyHashRange() .ranges(Range.of(40001, KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE-1))); + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + @Cleanup Producer producer = createProducer(topic, enableBatch); @@ -269,11 +268,10 @@ public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(bo for (int i = 0; i < 10; i++) { for (String key : keys) { - int slot = Murmur3_32Hash.getInstance().makeHash(key.getBytes()) - % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; - if (slot <= 20000) { + int stickyKeyHash = selector.makeStickyKeyHash(key.getBytes()); + if (stickyKeyHash <= 20000) { consumer1ExpectMessages++; - } else if (slot <= 40000) { + } else if (stickyKeyHash <= 40000) { consumer2ExpectMessages++; } else { consumer3ExpectMessages++; @@ -383,6 +381,8 @@ public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelect Consumer consumer3 = createConsumer(topic, KeySharedPolicy.stickyHashRange() .ranges(Range.of(40001, KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE-1))); + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + @Cleanup Producer producer = createProducer(topic, enableBatch); @@ -396,11 +396,10 @@ public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelect .send(); String fallbackKey = producer.getProducerName() + "-" + producer.getLastSequenceId(); - int slot = Murmur3_32Hash.getInstance().makeHash(fallbackKey.getBytes()) - % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; - if (slot <= 20000) { + int stickyKeyHash = selector.makeStickyKeyHash(fallbackKey.getBytes()); + if (stickyKeyHash <= 20000) { consumer1ExpectMessages++; - } else if (slot <= 40000) { + } else if (stickyKeyHash <= 40000) { consumer2ExpectMessages++; } else { consumer3ExpectMessages++; @@ -460,6 +459,8 @@ public void testOrderingKeyWithHashRangeExclusiveStickyKeyConsumerSelector(boole Consumer consumer3 = createConsumer(topic, KeySharedPolicy.stickyHashRange() .ranges(Range.of(40001, KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE-1))); + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + @Cleanup Producer producer = createProducer(topic, enableBatch); @@ -469,11 +470,10 @@ public void testOrderingKeyWithHashRangeExclusiveStickyKeyConsumerSelector(boole for (int i = 0; i < 10; i++) { for (String key : keys) { - int slot = Murmur3_32Hash.getInstance().makeHash(key.getBytes()) - % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; - if (slot <= 20000) { + int stickyKeyHash = selector.makeStickyKeyHash(key.getBytes()); + if (stickyKeyHash <= 20000) { consumer1ExpectMessages++; - } else if (slot <= 40000) { + } else if (stickyKeyHash <= 40000) { consumer2ExpectMessages++; } else { consumer3ExpectMessages++; @@ -502,7 +502,7 @@ public void testDisableKeySharedSubscription() throws PulsarClientException { @Cleanup Consumer c = pulsarClient.newConsumer() .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .ackTimeout(10, TimeUnit.SECONDS) .subscribe(); @@ -540,7 +540,6 @@ public void testCannotUseAcknowledgeCumulative() throws PulsarClientException { @Test(dataProvider = "batch") public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exception { String topic = "testMakingProgressWithSlowerConsumer-" + UUID.randomUUID(); - String slowKey = "slowKey"; List clients = new ArrayList<>(); @@ -556,16 +555,15 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc Consumer c = client.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) - .receiverQueueSize(1) + .receiverQueueSize(100) .messageListener((consumer, msg) -> { try { if (slowKey.equals(msg.getKey())) { // Block the thread to simulate a slow consumer Thread.sleep(10000); } - receivedMessages.incrementAndGet(); consumer.acknowledge(msg); } catch (Exception e) { @@ -576,6 +574,11 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc consumers.add(c); } + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + + org.apache.pulsar.broker.service.Consumer slowConsumer = + selector.select(selector.makeStickyKeyHash(slowKey.getBytes())); + @Cleanup Producer producer = createProducer(topic, enableBatch); @@ -587,18 +590,24 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc int N = 1000; + int nonSlowMessages = 0; + // Then send all the other keys for (int i = 0; i < N; i++) { + String key = String.valueOf(random.nextInt(NUMBER_OF_KEYS)); + if (selector.select(selector.makeStickyKeyHash(key.getBytes())) != slowConsumer) { + // count messages that are not going to the slow consumer + nonSlowMessages++; + } producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(key) .value(i) .send(); } - // Since only 1 out of 10 consumers is stuck, we should be able to receive ~90% messages, - // plus or minus for some skew in the key distribution. + int finalNonSlowMessages = nonSlowMessages; Awaitility.await().untilAsserted(() -> { - assertEquals((double) receivedMessages.get(), N * 0.9, N * 0.3); + assertThat(receivedMessages.get()).isGreaterThanOrEqualTo(finalNonSlowMessages); }); for (Consumer c : consumers) { @@ -614,6 +623,7 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc @Test public void testOrderingWhenAddingConsumers() throws Exception { String topic = "testOrderingWhenAddingConsumers-" + UUID.randomUUID(); + int numberOfKeys = 10; @Cleanup Producer producer = createProducer(topic, false); @@ -623,12 +633,14 @@ public void testOrderingWhenAddingConsumers() throws Exception { for (int i = 0; i < 10; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } - // All the already published messages will be pre-fetched by C1. + PendingAcksMap c1PendingAcks = getDispatcher(topic, SUBSCRIPTION_NAME).getConsumers().get(0).getPendingAcks(); + // Wait until all the already published messages have been pre-fetched by C1. + Awaitility.await().ignoreExceptions().until(() -> c1PendingAcks.size() == 10); // Adding a new consumer. @Cleanup @@ -636,11 +648,14 @@ public void testOrderingWhenAddingConsumers() throws Exception { for (int i = 10; i < 20; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } + Message message = c2.receive(100, TimeUnit.MILLISECONDS); + assertThat(message).describedAs("All keys should be blocked by ").isNull(); + // Closing c1, would trigger all messages to go to c2 c1.close(); @@ -652,6 +667,12 @@ public void testOrderingWhenAddingConsumers() throws Exception { } } + @SneakyThrows + private PersistentStickyKeyDispatcherMultipleConsumers getDispatcher(String topic, String subscription) { + return (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topic).get() + .get().getSubscription(subscription).getDispatcher(); + } + @Test public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { String topic = "testReadAheadWithConfiguredLookAheadLimit-" + UUID.randomUUID(); @@ -665,7 +686,7 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { @Cleanup Consumer c1 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .receiverQueueSize(10) .subscribe(); @@ -683,7 +704,7 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { @Cleanup Consumer c2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .receiverQueueSize(10) .subscribe(); @@ -701,7 +722,7 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { Thread.sleep(1000); Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); - PersistentSubscription sub = (PersistentSubscription) t.getSubscription("key_shared"); + PersistentSubscription sub = (PersistentSubscription) t.getSubscription(SUBSCRIPTION_NAME); // We need to ensure that dispatcher does not keep to look ahead in the topic, Position readPosition = sub.getCursor().getReadPosition(); @@ -712,6 +733,7 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { @Test public void testRemoveFirstConsumer() throws Exception { String topic = "testReadAheadWhenAddingConsumers-" + UUID.randomUUID(); + int numberOfKeys = 10; @Cleanup Producer producer = createProducer(topic, false); @@ -719,7 +741,7 @@ public void testRemoveFirstConsumer() throws Exception { @Cleanup Consumer c1 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .receiverQueueSize(10) .consumerName("c1") @@ -727,7 +749,7 @@ public void testRemoveFirstConsumer() throws Exception { for (int i = 0; i < 10; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } @@ -740,7 +762,7 @@ public void testRemoveFirstConsumer() throws Exception { @Cleanup Consumer c2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .receiverQueueSize(10) .consumerName("c2") @@ -748,13 +770,13 @@ public void testRemoveFirstConsumer() throws Exception { for (int i = 10; i < 20; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } // C2 will not be able to receive any messages until C1 is done processing whatever he got prefetched - assertNull(c2.receive(100, TimeUnit.MILLISECONDS)); + assertNull(c2.receive(1, TimeUnit.SECONDS)); c1.close(); @@ -777,8 +799,7 @@ public void testHashRangeConflict() throws PulsarClientException { Consumer consumer2 = createFixedHashRangesConsumer(topic, sub, Range.of(100,399)); Assert.assertTrue(consumer2.isConnected()); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = (PersistentStickyKeyDispatcherMultipleConsumers) pulsar - .getBrokerService().getTopicReference(topic).get().getSubscription(sub).getDispatcher(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, sub); Assert.assertEquals(dispatcher.getConsumers().size(), 2); try { @@ -887,6 +908,7 @@ public void testAttachKeyToMessageMetadata() throws PulsarClientException { public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { int defaultTTLSec = 3; int totalMessages = 1000; + int numberOfKeys = 50; this.conf.setTtlDurationDefaultInSeconds(defaultTTLSec); final String topic = "persistent://public/default/key_shared-" + UUID.randomUUID(); final String subName = "my-sub"; @@ -894,11 +916,15 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { @Cleanup Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) + .consumerName("consumer1") .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, subName); + StickyKeyConsumerSelector selector = dispatcher.getSelector(); + @Cleanup Producer producer = pulsarClient.newProducer(Schema.INT32) .topic(topic) @@ -906,20 +932,26 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { for (int i = 0; i < totalMessages; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } - // don't ack the first message - consumer1.receive(); - consumer1.acknowledge(consumer1.receive()); + Set blockedHashes = new HashSet<>(); + // pull up to numberOfKeys messages and don't ack them + for (int i = 0; i < numberOfKeys + 1; i++) { + Message received = consumer1.receive(); + int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); + log.info("Received message {} with sticky key hash: {}", received.getMessageId(), stickyKeyHash); + blockedHashes.add(stickyKeyHash); + } - // The consumer1 and consumer2 should be stuck because of the mark delete position did not move forward. + // The consumer1 and consumer2 should be stuck since all hashes are blocked @Cleanup Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) + .consumerName("consumer2") .subscriptionName(subName) .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); @@ -929,11 +961,19 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { received = consumer2.receive(1, TimeUnit.SECONDS); } catch (PulsarClientException ignore) { } - Assert.assertNull(received); + if (received != null) { + int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); + DrainingHashesTracker.DrainingHashEntry entry = + dispatcher.getDrainingHashesTracker().getEntry(stickyKeyHash); + Assertions.fail("Received message %s with sticky key hash that should have been blocked: %d. entry=%s, " + + "included in blockedHashes=%s", + received.getMessageId(), stickyKeyHash, entry, blockedHashes.contains(stickyKeyHash)); + } @Cleanup Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) + .consumerName("consumer3") .subscriptionName(subName) .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); @@ -942,7 +982,14 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { received = consumer3.receive(1, TimeUnit.SECONDS); } catch (PulsarClientException ignore) { } - Assert.assertNull(received); + if (received != null) { + int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); + DrainingHashesTracker.DrainingHashEntry entry = + dispatcher.getDrainingHashesTracker().getEntry(stickyKeyHash); + Assertions.fail("Received message %s with sticky key hash that should have been blocked: %d. entry=%s, " + + "included in blockedHashes=%s", + received.getMessageId(), stickyKeyHash, entry, blockedHashes.contains(stickyKeyHash)); + } Optional topicRef = pulsar.getBrokerService().getTopic(topic, false).get(); assertTrue(topicRef.isPresent()); @@ -952,14 +999,23 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { // The mark delete position is move forward, so the consumers should receive new messages now. for (int i = 0; i < totalMessages; i++) { producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(i % numberOfKeys)) .value(i) .send(); } - // Wait broker dispatch messages. - Assert.assertNotNull(consumer2.receive(1, TimeUnit.SECONDS)); - Assert.assertNotNull(consumer3.receive(1, TimeUnit.SECONDS)); + Map receivedMessagesCountByConsumer = new ConcurrentHashMap<>(); + receiveMessages((consumer, message) -> { + consumer.acknowledgeAsync(message); + receivedMessagesCountByConsumer.computeIfAbsent(consumer.getConsumerName(), id -> new AtomicInteger(0)) + .incrementAndGet(); + return true; + }, Duration.ofSeconds(2), consumer1, consumer2, consumer3); + + assertThat(receivedMessagesCountByConsumer.values().stream().mapToInt(AtomicInteger::intValue) + .sum()).isGreaterThanOrEqualTo(totalMessages); + assertThat(receivedMessagesCountByConsumer.values()).allSatisfy( + count -> assertThat(count.get()).isGreaterThan(0)); } @Test(dataProvider = "partitioned") @@ -1151,15 +1207,8 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr producer.send("message".getBytes()); Awaitility.await().untilAsserted(() -> assertNotNull(consumer1.receive(100, TimeUnit.MILLISECONDS))); - CompletableFuture> future = pulsar.getBrokerService().getTopicIfExists(topicName); - assertTrue(future.isDone()); - assertTrue(future.get().isPresent()); - Topic topic = future.get().get(); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subName).getDispatcher(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topicName, subName); assertTrue(dispatcher.isAllowOutOfOrderDelivery()); - assertNull(dispatcher.getLastSentPositionField()); - assertNull(dispatcher.getIndividuallySentPositionsField()); consumer1.close(); final Consumer consumer2 = pulsarClient.newConsumer() @@ -1171,14 +1220,8 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr producer.send("message".getBytes()); Awaitility.await().untilAsserted(() -> assertNotNull(consumer2.receive(100, TimeUnit.MILLISECONDS))); - future = pulsar.getBrokerService().getTopicIfExists(topicName); - assertTrue(future.isDone()); - assertTrue(future.get().isPresent()); - topic = future.get().get(); - dispatcher = (PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subName).getDispatcher(); + dispatcher = getDispatcher(topicName, subName); assertFalse(dispatcher.isAllowOutOfOrderDelivery()); - assertNotNull(dispatcher.getLastSentPositionField()); - assertNotNull(dispatcher.getIndividuallySentPositionsField()); consumer2.close(); } @@ -1250,7 +1293,7 @@ public void testCheckConsumersWithSameName() throws Exception { } }); - l.await(); + l.await(10, TimeUnit.SECONDS); } @DataProvider(name = "preSend") @@ -1258,366 +1301,6 @@ private Object[][] preSendProvider() { return new Object[][] { { false }, { true } }; } - @Test(timeOut = 30_000, dataProvider = "preSend") - public void testCheckBetweenSkippingAndRecentlyJoinedConsumers(boolean preSend) throws Exception { - conf.setSubscriptionKeySharedUseConsistentHashing(true); - - final String topicName = "persistent://public/default/recently-joined-consumers-" + UUID.randomUUID(); - final String subName = "my-sub"; - - @Cleanup - final Producer p = pulsarClient.newProducer(Schema.STRING) - .topic(topicName) - .create(); - if (preSend) { - // verify that the test succeeds even if the topic has a message - p.send("msg"); - } - - final Supplier> cb = () -> pulsarClient.newConsumer(Schema.STRING) - .topic(topicName) - .subscriptionInitialPosition(SubscriptionInitialPosition.Latest) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() - .setAllowOutOfOrderDelivery(false)); - - // create 2 consumers - final String c1ConsumerName = "c1"; - @Cleanup - final Consumer c1 = cb.get().consumerName(c1ConsumerName).receiverQueueSize(1).subscribe(); - @Cleanup - final Consumer c2 = cb.get().consumerName("c2").receiverQueueSize(1000).subscribe(); - - final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - final Field recentlyJoinedConsumersField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("recentlyJoinedConsumers"); - recentlyJoinedConsumersField.setAccessible(true); - final LinkedHashMap recentlyJoinedConsumers = (LinkedHashMap) recentlyJoinedConsumersField.get(dispatcher); - final String keyA = "key-a"; - final int hashA = Murmur3_32Hash.getInstance().makeHash(keyA.getBytes()); - final Map hashConsumerMap = new HashMap<>(); - hashConsumerMap.put(hashA, c1.getConsumerName()); - - // enforce the selector will return c1 if keyA - final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); - selectorField.setAccessible(true); - final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); - selectorField.set(dispatcher, selector); - doAnswer((invocationOnMock -> { - final int hash = invocationOnMock.getArgument(0); - final String consumerName = hashConsumerMap.getOrDefault(hash, c2.getConsumerName()); - return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); - })).when(selector).select(anyInt()); - - // send and receive - Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getConsumers().stream().filter(c -> c.getConsumerName().equals(c1ConsumerName)).findFirst().get().getAvailablePermits(), 1)); - final MessageIdImpl msg0Id = (MessageIdImpl) p.newMessage().key(keyA).value("msg-0").send(); - Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getConsumers().stream().filter(c -> c.getConsumerName().equals(c1ConsumerName)).findFirst().get().getAvailablePermits(), 0)); - - final MessageIdImpl msg1Id = (MessageIdImpl) p.newMessage().key(keyA).value("msg-1").send(); - Awaitility.await().untilAsserted(() -> assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getMsgBacklog(), 2)); - - final Field redeliveryMessagesField = PersistentDispatcherMultipleConsumers.class - .getDeclaredField("redeliveryMessages"); - redeliveryMessagesField.setAccessible(true); - final MessageRedeliveryController redeliveryMessages = (MessageRedeliveryController) redeliveryMessagesField.get(dispatcher); - - final Set replayMsgSet = redeliveryMessages.getMessagesToReplayNow(3, item -> true); - assertEquals(replayMsgSet.size(), 1); - final Position replayMsg = replayMsgSet.stream().findAny().get(); - assertEquals(replayMsg, PositionFactory.create(msg1Id.getLedgerId(), msg1Id.getEntryId())); - - // add c3 - final String c3ConsumerName = "c3"; - hashConsumerMap.put(hashA, c3ConsumerName); - @Cleanup - final Consumer c3 = cb.get().consumerName(c3ConsumerName).subscribe(); - final List> c3Msgs = new ArrayList<>(); - final org.apache.pulsar.broker.service.Consumer c3Broker = dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(c3ConsumerName)).findFirst().get(); - assertEquals(recentlyJoinedConsumers.get(c3Broker), PositionFactory.create(msg0Id.getLedgerId(), msg0Id.getEntryId())); - - // None of messages are sent to c3. - Message c3Msg = c3.receive(100, TimeUnit.MILLISECONDS); - assertNull(c3Msg); - - // Disconnect c1 - c1.close(); - - c3Msg = c3.receive(100, TimeUnit.MILLISECONDS); - assertNotNull(c3Msg); - c3Msgs.add(c3Msg); - // The mark delete position will move forward. Then remove c3 from recentlyJoinedConsumers. - c3.acknowledge(c3Msg); - Awaitility.await().untilAsserted(() -> assertNull(recentlyJoinedConsumers.get(c3Broker))); - c3Msg = c3.receive(100, TimeUnit.MILLISECONDS); - assertNotNull(c3Msg); - c3Msgs.add(c3Msg); - c3.acknowledge(c3Msg); - - // check ordering - assertTrue(c3Msgs.get(0).getMessageId().compareTo(c3Msgs.get(1).getMessageId()) < 0); - } - - @Test(timeOut = 30_000) - public void testLastSentPositionWhenRecreatingDispatcher() throws Exception { - // The lastSentPosition and individuallySentPositions should be initialized - // by the markDeletedPosition and individuallyDeletedMessages. - final String topicName = "persistent://public/default/rewind-" + UUID.randomUUID(); - final String subName = "my-sub"; - - final int numMessages = 9; - final List keys = Arrays.asList("key-a", "key-b", "key-c"); - final AtomicInteger receiveCounter = new AtomicInteger(); - final AtomicInteger ackCounter = new AtomicInteger(); - - @Cleanup - final Producer producer = pulsarClient.newProducer(Schema.INT32) - .topic(topicName) - .enableBatching(false) - .create(); - - final Supplier> cb = () -> pulsarClient.newConsumer(Schema.INT32) - .topic(topicName) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() - .setAllowOutOfOrderDelivery(false)); - - @Cleanup - final Consumer c1 = cb.get().messageListener((c, msg) -> { - if (keys.get(0).equals(msg.getKey())) { - try { - c.acknowledge(msg); - ackCounter.getAndIncrement(); - } catch (PulsarClientException e) { - fail(e.getMessage()); - } - } - receiveCounter.getAndIncrement(); - }).subscribe(); - - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - LongPairRangeSet individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); - final ManagedCursorImpl cursor = (ManagedCursorImpl) ((PersistentSubscription) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName)).getCursor(); - final ManagedLedgerImpl ledger = (ManagedLedgerImpl) cursor.getManagedLedger(); - - MessageIdImpl msgId = null; - for (int i = 0; i < numMessages; i++) { - msgId = (MessageIdImpl) producer.newMessage().key(keys.get(i % keys.size())).value(i).send(); - } - - // wait for consumption - Awaitility.await().untilAsserted(() -> assertEquals(receiveCounter.get(), numMessages)); - assertEquals(ackCounter.get(), numMessages / keys.size()); - assertEquals(dispatcher.getLastSentPositionField(), PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId())); - assertTrue(individuallySentPositionsField.isEmpty()); - receiveCounter.set(0); - ackCounter.set(0); - - // create expected values - final Position expectedLastSentPosition = ledger.getNextValidPosition(cursor.getMarkDeletedPosition()); - final ConcurrentOpenLongPairRangeSet - expectedIndividuallySentPositions = new ConcurrentOpenLongPairRangeSet<>(4096, PositionFactory::create); - cursor.getIndividuallyDeletedMessagesSet().forEach(range -> { - final Position lower = range.lowerEndpoint(); - final Position upper = range.upperEndpoint(); - expectedIndividuallySentPositions.addOpenClosed(lower.getLedgerId(), lower.getEntryId(), upper.getLedgerId(), upper.getEntryId()); - return true; - }); - - // modify subscription type to close current dispatcher - admin.topics().createSubscription(topicName, "sub-alt", MessageId.earliest); - c1.close(); - @Cleanup - final Consumer c2 = pulsarClient.newConsumer(Schema.INT32) - .topic(topicName) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Exclusive) - .subscribe(); - c2.close(); - assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName).getType(), SubscriptionType.Exclusive.toString()); - - @Cleanup - final Consumer c3 = cb.get().receiverQueueSize(0).subscribe(); - dispatcher = (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); - - assertNull(dispatcher.getLastSentPositionField()); - assertTrue(individuallySentPositionsField.isEmpty()); - - assertNotNull(c3.receive()); - - // validate the individuallySentPosition is initialized by the individuallyDeletedMessages - // if it is not initialized expectedly, it has sent-hole of key-c messages because key-c messages are not scheduled to be dispatched to some consumer(already acked). - assertEquals(dispatcher.getLastSentPositionField(), expectedLastSentPosition); - assertEquals(individuallySentPositionsField.toString(), expectedIndividuallySentPositions.toString()); - } - - @Test(timeOut = 30_000) - public void testLastSentPositionWhenResettingCursor() throws Exception { - // The lastSentPosition and individuallySentPositions should be cleared if reset-cursor operation is executed. - final String nsName = "public/default"; - final String topicName = "persistent://" + nsName + "/reset-cursor-" + UUID.randomUUID(); - final String subName = "my-sub"; - - final int numMessages = 10; - final List keys = Arrays.asList("key-a", "key-b"); - final AtomicInteger ackCounter = new AtomicInteger(); - - @Cleanup - final Producer producer = pulsarClient.newProducer(Schema.INT32) - .topic(topicName) - .enableBatching(false) - .create(); - - final Supplier> cb = () -> pulsarClient.newConsumer(Schema.INT32) - .topic(topicName) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .receiverQueueSize(0) - .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() - .setAllowOutOfOrderDelivery(false)); - - @Cleanup - final Consumer c1 = cb.get().consumerName("c1").subscribe(); - @Cleanup - final Consumer c2 = cb.get().consumerName("c2").subscribe(); - - // set retention policy - admin.namespaces().setRetention(nsName, new RetentionPolicies(1, 1024 * 1024)); - - // enforce the selector will return c1 if keys.get(0) - final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - final int hashA = Murmur3_32Hash.getInstance().makeHash(keys.get(0).getBytes()); - final Map hashConsumerMap = new HashMap<>(); - hashConsumerMap.put(hashA, c1.getConsumerName()); - final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); - selectorField.setAccessible(true); - final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); - selectorField.set(dispatcher, selector); - doAnswer((invocationOnMock -> { - final int hash = invocationOnMock.getArgument(0); - final String consumerName = hashConsumerMap.getOrDefault(hash, c2.getConsumerName()); - return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); - })).when(selector).select(anyInt()); - - for (int i = 0; i < numMessages; i++) { - producer.newMessage().key(keys.get(i % keys.size())).value(i).send(); - } - - // consume some messages - for (int i = 0; i < numMessages / keys.size(); i++) { - final Message msg = c2.receive(); - if (msg != null) { - c2.acknowledge(msg); - ackCounter.getAndIncrement(); - } - } - assertEquals(ackCounter.get(), numMessages / keys.size()); - - // store current lastSentPosition for comparison - final LongPairRangeSet individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); - assertNotNull(dispatcher.getLastSentPositionField()); - assertFalse(individuallySentPositionsField.isEmpty()); - - // reset cursor and receive a message - admin.topics().resetCursor(topicName, subName, MessageId.earliest, true); - - // validate the lastSentPosition and individuallySentPositions are cleared after resetting cursor - assertNull(dispatcher.getLastSentPositionField()); - assertTrue(individuallySentPositionsField.isEmpty()); - } - - @Test(timeOut = 30_000) - public void testLastSentPositionWhenSkipping() throws Exception { - // The lastSentPosition and individuallySentPositions should be updated if skip operation is executed. - // There are updated to follow the new markDeletedPosition. - final String topicName = "persistent://public/default/skip-" + UUID.randomUUID(); - final String subName = "my-sub"; - - final int numMessages = 10; - final List keys = Arrays.asList("key-a", "key-b"); - final int numSkip = 2; - final AtomicInteger ackCounter = new AtomicInteger(); - - @Cleanup - final Producer producer = pulsarClient.newProducer(Schema.INT32) - .topic(topicName) - .enableBatching(false) - .create(); - - final Supplier> cb = () -> pulsarClient.newConsumer(Schema.INT32) - .topic(topicName) - .subscriptionName(subName) - .subscriptionType(SubscriptionType.Key_Shared) - .keySharedPolicy(KeySharedPolicy.autoSplitHashRange() - .setAllowOutOfOrderDelivery(false)) - .receiverQueueSize(0); - - @Cleanup - final Consumer c1 = cb.get().consumerName("c1").subscribe(); - @Cleanup - final Consumer c2 = cb.get().consumerName("c2").subscribe(); - - // enforce the selector will return c1 if keys.get(0) - final PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName).getDispatcher(); - final int hashA = Murmur3_32Hash.getInstance().makeHash(keys.get(0).getBytes()); - final Map hashConsumerMap = new HashMap<>(); - hashConsumerMap.put(hashA, c1.getConsumerName()); - final Field selectorField = PersistentStickyKeyDispatcherMultipleConsumers.class.getDeclaredField("selector"); - selectorField.setAccessible(true); - final StickyKeyConsumerSelector selector = spy((StickyKeyConsumerSelector) selectorField.get(dispatcher)); - selectorField.set(dispatcher, selector); - doAnswer((invocationOnMock -> { - final int hash = invocationOnMock.getArgument(0); - final String consumerName = hashConsumerMap.getOrDefault(hash, c2.getConsumerName()); - return dispatcher.getConsumers().stream().filter(consumer -> consumer.consumerName().equals(consumerName)).findFirst().get(); - })).when(selector).select(anyInt()); - - final List positionList = new ArrayList<>(); - for (int i = 0; i < numMessages; i++) { - final MessageIdImpl msgId = (MessageIdImpl) producer.newMessage().key(keys.get(i % keys.size())).value(i).send(); - positionList.add(PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId())); - } - - // consume some messages - for (int i = 0; i < numSkip; i++) { - final Message msg = c2.receive(); - if (msg != null) { - c2.acknowledge(msg); - ackCounter.getAndIncrement(); - } - } - assertEquals(ackCounter.get(), numSkip); - final ManagedCursorImpl managedCursor = ((ManagedCursorImpl) ((PersistentSubscription) pulsar.getBrokerService().getTopicIfExists(topicName).get().get().getSubscription(subName)).getCursor()); - Awaitility.await().untilAsserted(() -> assertEquals(managedCursor.getIndividuallyDeletedMessagesSet().size(), 2)); - - // store current lastSentPosition for comparison - final Position lastSentPositionBeforeSkip = dispatcher.getLastSentPositionField(); - final LongPairRangeSet individuallySentPositionsField = dispatcher.getIndividuallySentPositionsField(); - assertNotNull(lastSentPositionBeforeSkip); - assertFalse(individuallySentPositionsField.isEmpty()); - - // skip messages and receive a message - admin.topics().skipMessages(topicName, subName, numSkip); - final MessageIdImpl msgIdAfterSkip = (MessageIdImpl) c1.receive().getMessageId(); - final Position positionAfterSkip = PositionFactory.create(msgIdAfterSkip.getLedgerId(), - msgIdAfterSkip.getEntryId()); - assertEquals(positionAfterSkip, positionList.get(4)); - - // validate the lastSentPosition is updated to the new markDeletedPosition - // validate the individuallySentPositions is updated expectedly (removeAtMost the new markDeletedPosition) - final Position lastSentPosition = dispatcher.getLastSentPositionField(); - assertNotNull(lastSentPosition); - assertTrue(lastSentPosition.compareTo(lastSentPositionBeforeSkip) > 0); - assertEquals(lastSentPosition, positionList.get(4)); - assertTrue(individuallySentPositionsField.isEmpty()); - } - private KeySharedMode getKeySharedModeOfSubscription(Topic topic, String subscription) { if (TopicName.get(topic.getName()).getDomain().equals(TopicDomain.persistent)) { return ((PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subscription) @@ -1665,7 +1348,7 @@ private Consumer createConsumer(String topic, KeySharedPolicy keyShared throws PulsarClientException { ConsumerBuilder builder = pulsarClient.newConsumer(Schema.INT32); builder.topic(topic) - .subscriptionName("key_shared") + .subscriptionName(SUBSCRIPTION_NAME) .subscriptionType(SubscriptionType.Key_Shared) .ackTimeout(3, TimeUnit.SECONDS); if (keySharedPolicy != null) { @@ -1927,8 +1610,8 @@ public void testStickyKeyRangesRestartConsumers() throws Exception { }}); // wait for some messages to be received by both of the consumers - count1.await(); - count2.await(); + count1.await(5, TimeUnit.SECONDS); + count2.await(5, TimeUnit.SECONDS); consumer1.close(); consumer2.close(); @@ -1974,7 +1657,7 @@ public void testStickyKeyRangesRestartConsumers() throws Exception { }) .subscribe(); // wait for all the messages to be delivered - count3.await(); + count3.await(20, TimeUnit.SECONDS); assertTrue(sentMessages.isEmpty(), "didn't receive " + sentMessages); producerFuture.get(); @@ -2085,7 +1768,8 @@ private AtomicInteger injectReplayReadCounter(String topicName, String cursorNam @Test public void testNoRepeatedReadAndDiscard() throws Exception { int delayedMessages = 100; - final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + int numberOfKeys = delayedMessages; + final String topic = newUniqueName("persistent://public/default/tp"); final String subName = "my-sub"; admin.topics().createNonPartitionedTopic(topic); AtomicInteger replyReadCounter = injectReplayReadCounter(topic, subName); @@ -2095,7 +1779,7 @@ public void testNoRepeatedReadAndDiscard() throws Exception { Producer producer = pulsarClient.newProducer(Schema.INT32).topic(topic).enableBatching(false).create(); for (int i = 0; i < delayedMessages; i++) { MessageId messageId = producer.newMessage() - .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .key(String.valueOf(random.nextInt(numberOfKeys))) .value(100 + i) .send(); log.info("Published message :{}", messageId); @@ -2103,12 +1787,14 @@ public void testNoRepeatedReadAndDiscard() throws Exception { producer.close(); // Make ack holes. + @Cleanup Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); + @Cleanup Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) @@ -2136,7 +1822,7 @@ public void testNoRepeatedReadAndDiscard() throws Exception { redeliverConsumer = consumer1; } - // consumer3 will be added to the "recentJoinedConsumers". + @Cleanup Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) @@ -2145,17 +1831,10 @@ public void testNoRepeatedReadAndDiscard() throws Exception { .subscribe(); redeliverConsumer.close(); + Thread.sleep(5000); // Verify: no repeated Read-and-discard. - Thread.sleep(5 * 1000); int maxReplayCount = delayedMessages * 2; - log.info("Reply read count: {}", replyReadCounter.get()); - assertTrue(replyReadCounter.get() < maxReplayCount); - - // cleanup. - consumer1.close(); - consumer2.close(); - consumer3.close(); - admin.topics().delete(topic, false); + assertThat(replyReadCounter.get()).isLessThanOrEqualTo(maxReplayCount); } @DataProvider(name = "allowKeySharedOutOfOrder") @@ -2184,7 +1863,7 @@ public Object[][] allowKeySharedOutOfOrder() { public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedOutOfOrder) throws Exception { final int messagesSentPerTime = 100; final Set totalReceivedMessages = new TreeSet<>(); - final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topic = newUniqueName("persistent://public/default/tp"); final String subName = "my-sub"; admin.topics().createNonPartitionedTopic(topic); AtomicInteger replyReadCounter = injectReplayReadCounter(topic, subName); @@ -2243,7 +1922,7 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO msgList2.add(msg2); } Message msg3 = consumer3.receive(1, TimeUnit.SECONDS); - if (msg2 != null) { + if (msg3 != null) { totalReceivedMessages.add(msg3.getValue()); msgList3.add(msg3); } @@ -2251,23 +1930,35 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO Consumer consumerWillBeClose = null; Consumer consumerAlwaysAck = null; Consumer consumerStuck = null; + Runnable consumerStuckAckHandler; + if (!msgList1.isEmpty()) { msgList1.forEach(msg -> consumer1.acknowledgeAsync(msg)); consumerAlwaysAck = consumer1; consumerWillBeClose = consumer2; consumerStuck = consumer3; + consumerStuckAckHandler = () -> { + msgList3.forEach(msg -> consumer3.acknowledgeAsync(msg)); + }; } else if (!msgList2.isEmpty()){ msgList2.forEach(msg -> consumer2.acknowledgeAsync(msg)); consumerAlwaysAck = consumer2; consumerWillBeClose = consumer3; consumerStuck = consumer1; + consumerStuckAckHandler = () -> { + msgList1.forEach(msg -> consumer1.acknowledgeAsync(msg)); + }; } else { msgList3.forEach(msg -> consumer3.acknowledgeAsync(msg)); consumerAlwaysAck = consumer3; consumerWillBeClose = consumer1; consumerStuck = consumer2; + consumerStuckAckHandler = () -> { + msgList2.forEach(msg -> consumer2.acknowledgeAsync(msg)); + }; } + // 2. Add consumer4 after "consumerWillBeClose" was close, and consumer4 will be stuck due to the mechanism // "recentlyJoinedConsumers". Consumer consumer4 = pulsarClient.newConsumer(Schema.INT32) @@ -2314,6 +2005,7 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO log.info("Reply read count: {}", replyReadCounter.get()); assertTrue(replyReadCounter.get() < maxReplayCount); // Verify: at last, all messages will be received. + consumerStuckAckHandler.run(); ReceivedMessages receivedMessages = ackAllMessages(consumerAlwaysAck, consumerStuck, consumer4); totalReceivedMessages.addAll(receivedMessages.messagesReceived.stream().map(p -> p.getRight()).collect( Collectors.toList())); @@ -2340,7 +2032,7 @@ public void testReadAheadLimit() throws Exception { Producer producer = createProducer(topic, false); // create a consumer and close it to create a subscription - String subscriptionName = "key_shared"; + String subscriptionName = SUBSCRIPTION_NAME; pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subscriptionName) @@ -2348,11 +2040,7 @@ public void testReadAheadLimit() throws Exception { .subscribe() .close(); - Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); - PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscriptionName); - // get the dispatcher reference - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, subscriptionName); // create a function to use for checking the number of messages in replay Runnable checkLimit = () -> { @@ -2394,8 +2082,7 @@ public void testReadAheadLimit() throws Exception { for (int i = 0; i < numberOfKeys; i++) { String key = String.valueOf(i); byte[] keyBytes = key.getBytes(UTF_8); - int hash = StickyKeyConsumerSelector.makeStickyKeyHash(keyBytes); - if (dispatcher.getSelector().select(hash).consumerName().equals("c2")) { + if (dispatcher.getSelector().select(keyBytes).consumerName().equals("c2")) { keysForC2.add(key); } } @@ -2453,4 +2140,171 @@ public void testReadAheadLimit() throws Exception { }, Duration.ofSeconds(2), c1, c2, c3); assertEquals(remainingMessageValues, Collections.emptySet()); } + + @SneakyThrows + private StickyKeyConsumerSelector getSelector(String topic, String subscription) { + Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); + PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + return dispatcher.getSelector(); + } + + // This test case simulates a rolling restart scenario with behaviors that can trigger out-of-order issues. + // In earlier versions of Pulsar, this issue occurred in about 25% of cases. + // To increase the probability of reproducing the issue, use the invocationCount parameter. + @Test//(invocationCount = 50) + public void testOrderingAfterReconnects() throws Exception { + String topic = newUniqueName("testOrderingAfterReconnects"); + int numberOfKeys = 1000; + long pauseTime = 100L; + + @Cleanup + Producer producer = createProducer(topic, false); + + // create a consumer and close it to create a subscription + pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe() + .close(); + + Set remainingMessageValues = new HashSet<>(); + Map> keyPositions = new HashMap<>(); + BiFunction, Message, Boolean> messageHandler = (consumer, msg) -> { + synchronized (this) { + consumer.acknowledgeAsync(msg); + String key = msg.getKey(); + MessageIdAdv msgId = (MessageIdAdv) msg.getMessageId(); + Position currentPosition = PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId()); + Pair prevPair = keyPositions.get(key); + if (prevPair != null && prevPair.getLeft().compareTo(currentPosition) > 0) { + log.error("key: {} value: {} prev: {}/{} current: {}/{}", key, msg.getValue(), prevPair.getLeft(), + prevPair.getRight(), currentPosition, consumer.getConsumerName()); + fail("out of order"); + } + keyPositions.put(key, Pair.of(currentPosition, consumer.getConsumerName())); + boolean removed = remainingMessageValues.remove(msg.getValue()); + if (!removed) { + // duplicates are possible during reconnects, this is not an error + log.warn("Duplicate message: {} value: {}", msg.getMessageId(), msg.getValue()); + } + return true; + } + }; + + // Adding a new consumer. + @Cleanup + Consumer c1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c1") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .startPaused(true) // start paused + .subscribe(); + + @Cleanup + Consumer c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(500) // use large receiver queue size + .subscribe(); + + @Cleanup + Consumer c3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c3") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .startPaused(true) // start paused + .subscribe(); + + StickyKeyConsumerSelector selector = getSelector(topic, SUBSCRIPTION_NAME); + + // find keys that will be assigned to c2 + List keysForC2 = new ArrayList<>(); + for (int i = 0; i < numberOfKeys; i++) { + String key = String.valueOf(i); + byte[] keyBytes = key.getBytes(UTF_8); + int hash = selector.makeStickyKeyHash(keyBytes); + if (selector.select(hash).consumerName().equals("c2")) { + keysForC2.add(key); + } + } + + // produce messages with keys that all get assigned to c2 + for (int i = 0; i < 1000; i++) { + String key = keysForC2.get(random.nextInt(keysForC2.size())); + //log.info("Producing message with key: {} value: {}", key, i); + producer.newMessage() + .key(key) + .value(i) + .send(); + remainingMessageValues.add(i); + } + + Thread.sleep(2 * pauseTime); + // close c2 + c2.close(); + Thread.sleep(pauseTime); + // resume c1 and c3 + c1.resume(); + c3.resume(); + Thread.sleep(pauseTime); + // reconnect c2 + c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .subscribe(); + // close and reconnect c1 + c1.close(); + Thread.sleep(pauseTime); + c1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c1") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .subscribe(); + // close and reconnect c3 + c3.close(); + Thread.sleep(pauseTime); + c3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c3") + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(10) + .subscribe(); + + logTopicStats(topic); + + // produce more messages + for (int i = 1000; i < 2000; i++) { + String key = String.valueOf(random.nextInt(numberOfKeys)); + //log.info("Producing message with key: {} value: {}", key, i); + producer.newMessage() + .key(key) + .value(i) + .send(); + remainingMessageValues.add(i); + } + + // consume the messages + receiveMessages(messageHandler, Duration.ofSeconds(2), c1, c2, c3); + + try { + assertEquals(remainingMessageValues, Collections.emptySet()); + } finally { + logTopicStats(topic); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java index 7889b19e5b29e..704af89777f05 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java @@ -18,18 +18,25 @@ */ package org.apache.pulsar.client.impl; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; +import static org.assertj.core.api.Assertions.assertThat; import com.google.common.collect.Sets; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.lang3.RandomStringUtils; +import lombok.SneakyThrows; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.client.api.BatcherBuilder; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -40,7 +47,6 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Range; import org.apache.pulsar.client.api.SubscriptionType; -import org.apache.pulsar.common.util.Murmur3_32Hash; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -65,33 +71,76 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + enum KeySharedSelectorType { + AutoSplit_ConsistentHashing(true), AutoSplit_Classic(true), Sticky(false); + final boolean autoSplit; + + KeySharedSelectorType(boolean autoSplit) { + this.autoSplit = autoSplit; + } + } + @DataProvider public Object[][] subType() { - return new Object[][] { { SubscriptionType.Shared }, { SubscriptionType.Key_Shared } }; + return new Object[][] { + { SubscriptionType.Shared, null }, + { SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_ConsistentHashing }, + { SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_Classic }, + { SubscriptionType.Key_Shared, KeySharedSelectorType.Sticky } + }; } - @Test(dataProvider = "subType") - public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(SubscriptionType subscriptionType) + @Test(dataProvider = "subType", timeOut = 30000) + public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(SubscriptionType subscriptionType, + KeySharedSelectorType selectorType) throws PulsarClientException { + if (selectorType == KeySharedSelectorType.AutoSplit_Classic) { + conf.setSubscriptionKeySharedUseConsistentHashing(false); + } + final int totalMsg = 1000; - String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5); - Map, List> nameToId = new ConcurrentHashMap<>(); + String topic = newUniqueName("broker-close-test"); + String subscriptionName = "sub-1"; + Map, List> unackedMessages = new ConcurrentHashMap<>(); Set pubMessages = Sets.newConcurrentHashSet(); Set recMessages = Sets.newConcurrentHashSet(); AtomicLong lastActiveTime = new AtomicLong(); AtomicBoolean canAcknowledgement = new AtomicBoolean(false); + if (subscriptionType == SubscriptionType.Key_Shared) { + // create and close consumer to create the dispatcher so that the selector can be used + ConsumerBuilder consumerBuilder = pulsarClient.newConsumer() + .topic(topic) + .subscriptionName(subscriptionName) + .subscriptionType(subscriptionType); + if (subscriptionType == SubscriptionType.Key_Shared) { + if (selectorType.autoSplit) { + consumerBuilder.keySharedPolicy(KeySharedPolicy.autoSplitHashRange()); + } else { + consumerBuilder.keySharedPolicy(KeySharedPolicy.stickyHashRange().ranges(Range.of(0, 65535))); + } + } + consumerBuilder + .subscribe() + .close(); + } + List> consumerList = new ArrayList<>(); - // create 3 consumers - for (int i = 0; i < 3; i++) { + int consumerCount = 3; + + Range[] ranges = null; + if (subscriptionType == SubscriptionType.Key_Shared && !selectorType.autoSplit) { + ranges = splitRange(getSelector(topic, subscriptionName).getKeyHashRange(), consumerCount); + } + + for (int i = 0; i < consumerCount; i++) { ConsumerBuilder builder = pulsarClient.newConsumer() .topic(topic) - .subscriptionName("sub-1") + .consumerName("consumer-" + i) + .subscriptionName(subscriptionName) .subscriptionType(subscriptionType) .messageListener((consumer, msg) -> { lastActiveTime.set(System.currentTimeMillis()); - nameToId.computeIfAbsent(consumer, (k) -> new ArrayList<>()) - .add(msg.getMessageId()); recMessages.add(msg.getMessageId()); if (canAcknowledgement.get()) { try { @@ -99,19 +148,31 @@ public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(Subsc } catch (PulsarClientException e) { throw new RuntimeException(e); } + } else { + unackedMessages.computeIfAbsent(consumer, + (k) -> Collections.synchronizedList(new ArrayList<>())) + .add(msg.getMessageId()); } }); if (subscriptionType == SubscriptionType.Key_Shared) { - // ensure every consumer can be distributed messages - int hash = Murmur3_32Hash.getInstance().makeHash(("key-" + i).getBytes()) - % KeySharedPolicy.DEFAULT_HASH_RANGE_SIZE; - builder.keySharedPolicy(KeySharedPolicy.stickyHashRange().ranges(Range.of(hash, hash))); + if (selectorType.autoSplit) { + builder.keySharedPolicy(KeySharedPolicy.autoSplitHashRange()); + } else { + builder.keySharedPolicy(KeySharedPolicy.stickyHashRange().ranges(ranges[i])); + } } consumerList.add(builder.subscribe()); } + String[] keys = new String[consumerCount]; + for (int i = 0; i < consumerCount; i++) { + keys[i] = subscriptionType == SubscriptionType.Key_Shared ? + generateKeyForConsumer(getSelector(topic, subscriptionName), + consumerList.get(i).getConsumerName()) : "key-" + i; + } + Producer producer = pulsarClient.newProducer() .topic(topic) .enableBatching(true) @@ -122,42 +183,39 @@ public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(Subsc .create(); for (int i = 0; i < totalMsg; i++) { - byte[] msg = UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8); - producer.newMessage().key("key-" + (i % 3)).value(msg) + producer.newMessage() + .key(keys[i % consumerCount]) + .value(("message-" + i).getBytes(StandardCharsets.UTF_8)) .sendAsync().thenAccept(pubMessages::add); } + producer.flush(); + // Wait for all consumers can not read more messages. the consumers are stuck by max unacked messages. - Awaitility.await() - .pollDelay(5, TimeUnit.SECONDS) - .until(() -> - (System.currentTimeMillis() - lastActiveTime.get()) > TimeUnit.SECONDS.toMillis(5)); + waitUntilLastActiveTimeNoLongerGetsUpdated(lastActiveTime); // All consumers can acknowledge messages as they continue to receive messages. canAcknowledgement.set(true); // Acknowledgment of currently received messages to get out of stuck state due to unack message - for (Map.Entry, List> entry : nameToId.entrySet()) { + for (Map.Entry, List> entry : unackedMessages.entrySet()) { Consumer consumer = entry.getKey(); - consumer.acknowledge(entry.getValue()); + List messageIdList = entry.getValue(); + consumer.acknowledge(messageIdList); } + // refresh active time lastActiveTime.set(System.currentTimeMillis()); // Wait for all consumers to continue receiving messages. - Awaitility.await() - .atMost(30, TimeUnit.SECONDS) - .pollDelay(5, TimeUnit.SECONDS) - .until(() -> - (System.currentTimeMillis() - lastActiveTime.get()) > TimeUnit.SECONDS.toMillis(5)); + waitUntilLastActiveTimeNoLongerGetsUpdated(lastActiveTime); logTopicStats(topic); //Determine if all messages have been received. //If the dispatcher is stuck, we can not receive enough messages. Assert.assertEquals(totalMsg, pubMessages.size()); - Assert.assertEquals(recMessages.size(), pubMessages.size()); - Assert.assertTrue(recMessages.containsAll(pubMessages)); + assertThat(recMessages).containsExactlyInAnyOrderElementsOf(pubMessages); // cleanup producer.close(); @@ -165,4 +223,43 @@ public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(Subsc consumer.close(); } } + + private Range[] splitRange(Range keyHashRange, int consumerCount) { + Range[] ranges = new Range[consumerCount]; + int start = keyHashRange.getStart(); + for (int i = 0; i < consumerCount; i++) { + int end = Math.min(start + keyHashRange.size() / consumerCount, keyHashRange.getEnd()); + ranges[i] = Range.of(start, end); + start = end + 1; + } + return ranges; + } + + private String generateKeyForConsumer(StickyKeyConsumerSelector selector, + String consumerName) { + int i = 0; + while (!Thread.currentThread().isInterrupted()) { + String key = "key" + i++; + org.apache.pulsar.broker.service.Consumer selectedConsumer = selector.select(key.getBytes(UTF_8)); + if (selectedConsumer != null && selectedConsumer.consumerName().equals(consumerName)) { + return key; + } + } + return null; + } + + private static void waitUntilLastActiveTimeNoLongerGetsUpdated(AtomicLong lastActiveTime) { + Awaitility.await() + .pollInterval(100, TimeUnit.MILLISECONDS) + .until(() -> System.currentTimeMillis() - lastActiveTime.get() > TimeUnit.SECONDS.toMillis(1)); + } + + @SneakyThrows + private StickyKeyConsumerSelector getSelector(String topic, String subscription) { + Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); + PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); + PersistentStickyKeyDispatcherMultipleConsumers dispatcher = + (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + return dispatcher.getSelector(); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index e307e41862e74..7b7c1f5765cc5 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -118,15 +118,6 @@ public interface SubscriptionStats { /** Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. */ String getKeySharedMode(); - /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ - Map getConsumersAfterMarkDeletePosition(); - - /** The last sent position of the cursor. This is for Key_Shared subscription. */ - String getLastSentPosition(); - - /** Set of individually sent ranges. This is for Key_Shared subscription. */ - String getIndividuallySentPositions(); - /** SubscriptionProperties (key/value strings) associated with this subscribe. */ Map getSubscriptionProperties(); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java index 488083f484b76..cbca1ef8f06bd 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Range.java @@ -93,4 +93,12 @@ public int compareTo(Range o) { } return result; } + + public boolean contains(int value) { + return value >= start && value <= end; + } + + public int size() { + return end - start + 1; + } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java index 610c782518348..50168221fea37 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/api/RangeTest.java @@ -61,4 +61,44 @@ public void testIntersect() { public void testInvalid() { Range.of(0, -5); } + + @Test + public void testCompareTo() { + Range range1 = Range.of(0, 5); + Range range2 = Range.of(0, 5); + Range range3 = Range.of(0, 10); + Range range4 = Range.of(5, 10); + + Assert.assertEquals(0, range1.compareTo(range2)); + Assert.assertTrue(range1.compareTo(range3) < 0); + Assert.assertTrue(range3.compareTo(range1) > 0); + Assert.assertTrue(range1.compareTo(range4) < 0); + Assert.assertTrue(range4.compareTo(range1) > 0); + } + + @Test + public void testContains() { + Range range = Range.of(0, 5); + + Assert.assertTrue(range.contains(0)); + Assert.assertTrue(range.contains(3)); + Assert.assertTrue(range.contains(5)); + Assert.assertFalse(range.contains(-1)); + Assert.assertFalse(range.contains(6)); + } + + @Test + public void testSize() { + Range range = Range.of(0, 0); + Assert.assertEquals(1, range.size()); + + range = Range.of(0, 1); + Assert.assertEquals(2, range.size()); + + range = Range.of(0, 5); + Assert.assertEquals(6, range.size()); + + range = Range.of(3, 3); + Assert.assertEquals(1, range.size()); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java index 977ed28e86814..4206a4aa8d61b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import java.util.ArrayList; import java.util.HashMap; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -126,15 +125,9 @@ public class SubscriptionStatsImpl implements SubscriptionStats { /** Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. */ public String keySharedMode; - /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ - public Map consumersAfterMarkDeletePosition; - /** The last sent position of the cursor. This is for Key_Shared subscription. */ public String lastSentPosition; - /** Set of individually sent ranges. This is for Key_Shared subscription. */ - public String individuallySentPositions; - /** The number of non-contiguous deleted messages ranges. */ public int nonContiguousDeletedMessagesRanges; @@ -160,7 +153,6 @@ public class SubscriptionStatsImpl implements SubscriptionStats { public SubscriptionStatsImpl() { this.consumers = new ArrayList<>(); - this.consumersAfterMarkDeletePosition = new LinkedHashMap<>(); this.subscriptionProperties = new HashMap<>(); this.bucketDelayedIndexStats = new HashMap<>(); } @@ -185,7 +177,6 @@ public void reset() { lastExpireTimestamp = 0L; lastMarkDeleteAdvancedTimestamp = 0L; consumers.clear(); - consumersAfterMarkDeletePosition.clear(); nonContiguousDeletedMessagesRanges = 0; nonContiguousDeletedMessagesRangesSerializedSize = 0; earliestMsgPublishTimeInBacklog = 0L; @@ -231,7 +222,6 @@ public SubscriptionStatsImpl add(SubscriptionStatsImpl stats) { } } this.allowOutOfOrderDelivery |= stats.allowOutOfOrderDelivery; - this.consumersAfterMarkDeletePosition.putAll(stats.consumersAfterMarkDeletePosition); this.nonContiguousDeletedMessagesRanges += stats.nonContiguousDeletedMessagesRanges; this.nonContiguousDeletedMessagesRangesSerializedSize += stats.nonContiguousDeletedMessagesRangesSerializedSize; if (this.earliestMsgPublishTimeInBacklog != 0 && stats.earliestMsgPublishTimeInBacklog != 0) { 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 15b5676094ec1..8635368f00f0b 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 @@ -1970,27 +1970,37 @@ public static MessageMetadata peekAndCopyMessageMetadata( return metadata; } - private static final byte[] NONE_KEY = "NONE_KEY".getBytes(StandardCharsets.UTF_8); + public static final byte[] NONE_KEY = "NONE_KEY".getBytes(StandardCharsets.UTF_8); + public static byte[] peekStickyKey(ByteBuf metadataAndPayload, String topic, String subscription) { try { int readerIdx = metadataAndPayload.readerIndex(); - MessageMetadata metadata = Commands.parseMessageMetadata(metadataAndPayload); + MessageMetadata metadata = parseMessageMetadata(metadataAndPayload); metadataAndPayload.readerIndex(readerIdx); - if (metadata.hasOrderingKey()) { - return metadata.getOrderingKey(); - } else if (metadata.hasPartitionKey()) { - if (metadata.isPartitionKeyB64Encoded()) { - return Base64.getDecoder().decode(metadata.getPartitionKey()); - } - return metadata.getPartitionKey().getBytes(StandardCharsets.UTF_8); - } else if (metadata.hasProducerName() && metadata.hasSequenceId()) { - String fallbackKey = metadata.getProducerName() + "-" + metadata.getSequenceId(); - return fallbackKey.getBytes(StandardCharsets.UTF_8); - } + return resolveStickyKey(metadata); } catch (Throwable t) { log.error("[{}] [{}] Failed to peek sticky key from the message metadata", topic, subscription, t); + return NONE_KEY; + } + } + + public static byte[] resolveStickyKey(MessageMetadata metadata) { + byte[] stickyKey; + if (metadata.hasOrderingKey()) { + stickyKey = metadata.getOrderingKey(); + } else if (metadata.hasPartitionKey()) { + if (metadata.isPartitionKeyB64Encoded()) { + stickyKey = Base64.getDecoder().decode(metadata.getPartitionKey()); + } else { + stickyKey = metadata.getPartitionKey().getBytes(StandardCharsets.UTF_8); + } + } else if (metadata.hasProducerName() && metadata.hasSequenceId()) { + String fallbackKey = metadata.getProducerName() + "-" + metadata.getSequenceId(); + stickyKey = fallbackKey.getBytes(StandardCharsets.UTF_8); + } else { + stickyKey = NONE_KEY; } - return Commands.NONE_KEY; + return stickyKey; } public static int getCurrentProtocolVersion() { diff --git a/pulsar-common/src/main/resources/findbugsExclude.xml b/pulsar-common/src/main/resources/findbugsExclude.xml index df161c4b621a7..b3e511006bce3 100644 --- a/pulsar-common/src/main/resources/findbugsExclude.xml +++ b/pulsar-common/src/main/resources/findbugsExclude.xml @@ -53,4 +53,9 @@ + + + + + From 4d6dee456ca3c255d00d313995f7c1f9e78420f1 Mon Sep 17 00:00:00 2001 From: Jiawen Wang <74594733+summeriiii@users.noreply.github.com> Date: Wed, 9 Oct 2024 02:52:47 +0800 Subject: [PATCH 567/580] [fix][ml] Remove unnecessary return in ManagedLedgerImpl (#23418) --- .../apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index cb19bd94bce01..c1081761b601f 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -2229,13 +2229,11 @@ public void readEntryComplete(Entry entry, Object ctx) { Object cbCtx = this.cntx; if (recycle(reOpCount)) { callback.readEntryComplete(entry, cbCtx); - return; } else { if (log.isDebugEnabled()) { log.debug("[{}] read entry already completed for {}-{}", name, ledgerId, entryId); } entry.release(); - return; } } @@ -2246,7 +2244,6 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { Object cbCtx = this.cntx; if (recycle(reOpCount)) { callback.readEntryFailed(exception, cbCtx); - return; } else { if (log.isDebugEnabled()) { log.debug("[{}] read entry already completed for {}-{}", name, ledgerId, entryId); @@ -2261,13 +2258,11 @@ public void readEntriesComplete(List returnedEntries, Object ctx) { Object cbCtx = this.cntx; if (recycle(reOpCount)) { callback.readEntriesComplete(returnedEntries, cbCtx); - return; } else { if (log.isDebugEnabled()) { log.debug("[{}] read entry already completed for {}-{}", name, ledgerId, entryId); } returnedEntries.forEach(Entry::release); - return; } } @@ -2278,12 +2273,10 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { Object cbCtx = this.cntx; if (recycle(reOpCount)) { callback.readEntriesFailed(exception, cbCtx); - return; } else { if (log.isDebugEnabled()) { log.debug("[{}] read entry already completed for {}-{}", name, ledgerId, entryId); } - return; } } From 9579c4dea9120ed29523d7eb56c1b0637aaa9bc4 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov <8622884+dlg99@users.noreply.github.com> Date: Tue, 8 Oct 2024 15:22:41 -0700 Subject: [PATCH 568/580] [improve][ci] Upgrade/Downgrade test (#22988) --- .github/workflows/pulsar-ci.yaml | 3 + build/run_integration_group.sh | 4 + .../containers/PulsarContainer.java | 5 + .../integration/topologies/PulsarCluster.java | 58 ++++-- .../topologies/PulsarClusterSpec.java | 6 + .../topologies/PulsarClusterTestBase.java | 6 +- .../upgrade/PulsarUpgradeDowngradeTest.java | 175 ++++++++++++++++++ .../src/test/resources/pulsar-upgrade.xml | 2 +- 8 files changed, 243 insertions(+), 16 deletions(-) create mode 100644 tests/integration/src/test/java/org/apache/pulsar/tests/integration/upgrade/PulsarUpgradeDowngradeTest.java diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 091dab25ec696..47a39bef9c908 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -603,6 +603,9 @@ jobs: - name: Metrics group: METRICS + - name: Upgrade + group: UPGRADE + steps: - name: checkout uses: actions/checkout@v4 diff --git a/build/run_integration_group.sh b/build/run_integration_group.sh index 2d82fce08878d..63b92d4e0a798 100755 --- a/build/run_integration_group.sh +++ b/build/run_integration_group.sh @@ -177,6 +177,10 @@ test_group_standalone() { mvn_run_integration_test "$@" -DintegrationTestSuiteFile=pulsar-standalone.xml -DintegrationTests } +test_group_upgrade() { + mvn_run_integration_test "$@" -DintegrationTestSuiteFile=pulsar-upgrade.xml -DintegrationTests +} + test_group_transaction() { mvn_run_integration_test "$@" -DintegrationTestSuiteFile=pulsar-transaction.xml -DintegrationTests } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PulsarContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PulsarContainer.java index 77cdc1bfd28a9..3cdb048aea55f 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PulsarContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/PulsarContainer.java @@ -51,8 +51,13 @@ public abstract class PulsarContainer> exte public static final int BROKER_HTTP_PORT = 8080; public static final int BROKER_HTTPS_PORT = 8081; + public static final String ALPINE_IMAGE_NAME = "alpine:3.20"; public static final String DEFAULT_IMAGE_NAME = System.getenv().getOrDefault("PULSAR_TEST_IMAGE_NAME", "apachepulsar/pulsar-test-latest-version:latest"); + public static final String UPGRADE_TEST_IMAGE_NAME = System.getenv().getOrDefault("PULSAR_UPGRADE_TEST_IMAGE_NAME", + DEFAULT_IMAGE_NAME); + public static final String LAST_RELEASE_IMAGE_NAME = System.getenv().getOrDefault("PULSAR_LAST_RELEASE_IMAGE_NAME", + "apachepulsar/pulsar:3.0.7"); public static final String DEFAULT_HTTP_PATH = "/metrics"; public static final String PULSAR_2_5_IMAGE_NAME = "apachepulsar/pulsar:2.5.0"; public static final String PULSAR_2_4_IMAGE_NAME = "apachepulsar/pulsar:2.4.0"; diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java index 90f08a9639471..35fb453c4bb8e 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarCluster.java @@ -72,22 +72,28 @@ public class PulsarCluster { * @return the built pulsar cluster */ public static PulsarCluster forSpec(PulsarClusterSpec spec) { + return forSpec(spec, Network.newNetwork()); + } + + public static PulsarCluster forSpec(PulsarClusterSpec spec, Network network) { + checkArgument(network != null, "Network should not be null"); CSContainer csContainer = null; if (!spec.enableOxia) { csContainer = new CSContainer(spec.clusterName) - .withNetwork(Network.newNetwork()) + .withNetwork(network) .withNetworkAliases(CSContainer.NAME); } - return new PulsarCluster(spec, csContainer, false); + return new PulsarCluster(spec, network, csContainer, false); } public static PulsarCluster forSpec(PulsarClusterSpec spec, CSContainer csContainer) { - return new PulsarCluster(spec, csContainer, true); + return new PulsarCluster(spec, csContainer.getNetwork(), csContainer, true); } @Getter private final PulsarClusterSpec spec; + public boolean closeNetworkOnExit = true; @Getter private final String clusterName; private final Network network; @@ -108,19 +114,18 @@ public static PulsarCluster forSpec(PulsarClusterSpec spec, CSContainer csContai private final String metadataStoreUrl; private final String configurationMetadataStoreUrl; - private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean sharedCsContainer) { - + private PulsarCluster(PulsarClusterSpec spec, Network network, CSContainer csContainer, boolean sharedCsContainer) { this.spec = spec; this.sharedCsContainer = sharedCsContainer; this.clusterName = spec.clusterName(); - if (csContainer != null ) { + if (network != null) { + this.network = network; + } else if (csContainer != null) { this.network = csContainer.getNetwork(); } else { this.network = Network.newNetwork(); } - - if (spec.enableOxia) { this.zkContainer = null; this.oxiaContainer = new OxiaContainer(clusterName); @@ -203,7 +208,9 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s .withEnv("PULSAR_PREFIX_diskUsageWarnThreshold", "0.95") .withEnv("diskUsageThreshold", "0.99") .withEnv("PULSAR_PREFIX_diskUsageLwmThreshold", "0.97") - .withEnv("nettyMaxFrameSizeBytes", String.valueOf(spec.maxMessageSize)); + .withEnv("nettyMaxFrameSizeBytes", String.valueOf(spec.maxMessageSize)) + .withEnv("ledgerDirectories", "data/bookkeeper/" + name + "/ledgers") + .withEnv("journalDirectory", "data/bookkeeper/" + name + "/journal"); if (spec.bookkeeperEnvs != null) { bookieContainer.withEnv(spec.bookkeeperEnvs); } @@ -262,10 +269,27 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s } )); + if (spec.dataContainer != null) { + if (!sharedCsContainer && csContainer != null) { + csContainer.withVolumesFrom(spec.dataContainer, BindMode.READ_WRITE); + } + if (zkContainer != null) { + zkContainer.withVolumesFrom(spec.dataContainer, BindMode.READ_WRITE); + } + proxyContainer.withVolumesFrom(spec.dataContainer, BindMode.READ_WRITE); + + bookieContainers.values().forEach(c -> c.withVolumesFrom(spec.dataContainer, BindMode.READ_WRITE)); + brokerContainers.values().forEach(c -> c.withVolumesFrom(spec.dataContainer, BindMode.READ_WRITE)); + workerContainers.values().forEach(c -> c.withVolumesFrom(spec.dataContainer, BindMode.READ_WRITE)); + } + spec.classPathVolumeMounts.forEach((key, value) -> { if (zkContainer != null) { zkContainer.withClasspathResourceMapping(key, value, BindMode.READ_WRITE); } + if (!sharedCsContainer && csContainer != null) { + csContainer.withClasspathResourceMapping(key, value, BindMode.READ_WRITE); + } proxyContainer.withClasspathResourceMapping(key, value, BindMode.READ_WRITE); bookieContainers.values().forEach(c -> c.withClasspathResourceMapping(key, value, BindMode.READ_WRITE)); @@ -323,6 +347,10 @@ public Map> getExternalServices() { } public void start() throws Exception { + start(true); + } + + public void start(boolean doInit) throws Exception { if (!spec.enableOxia) { // start the local zookeeper @@ -338,7 +366,7 @@ public void start() throws Exception { oxiaContainer.start(); } - { + if (doInit) { // Run cluster metadata initialization @Cleanup PulsarInitMetadataContainer init = new PulsarInitMetadataContainer( @@ -453,10 +481,12 @@ public synchronized void stop() { oxiaContainer.stop(); } - try { - network.close(); - } catch (Exception e) { - log.info("Failed to shutdown network for pulsar cluster {}", clusterName, e); + if (closeNetworkOnExit) { + try { + network.close(); + } catch (Exception e) { + log.info("Failed to shutdown network for pulsar cluster {}", clusterName, e); + } } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java index 8a991be49fad0..ca45c9b7c9b82 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterSpec.java @@ -124,6 +124,12 @@ public class PulsarClusterSpec { @Builder.Default Map classPathVolumeMounts = new TreeMap<>(); + /** + * Data container + */ + @Builder.Default + GenericContainer dataContainer = null; + /** * Pulsar Test Image Name * diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterTestBase.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterTestBase.java index 93e2221ab2493..8b99f21373560 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterTestBase.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/topologies/PulsarClusterTestBase.java @@ -142,6 +142,10 @@ protected void beforeStartCluster() throws Exception { } protected void setupCluster(PulsarClusterSpec spec) throws Exception { + setupCluster(spec, true); + } + + protected void setupCluster(PulsarClusterSpec spec, boolean doInit) throws Exception { incrementSetupNumber(); log.info("Setting up cluster {} with {} bookies, {} brokers", spec.clusterName(), spec.numBookies(), spec.numBrokers()); @@ -150,7 +154,7 @@ protected void setupCluster(PulsarClusterSpec spec) throws Exception { beforeStartCluster(); - pulsarCluster.start(); + pulsarCluster.start(doInit); pulsarAdmin = PulsarAdmin.builder().serviceHttpUrl(pulsarCluster.getHttpServiceUrl()).build(); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/upgrade/PulsarUpgradeDowngradeTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/upgrade/PulsarUpgradeDowngradeTest.java new file mode 100644 index 0000000000000..ddabd67b2294b --- /dev/null +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/upgrade/PulsarUpgradeDowngradeTest.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.tests.integration.upgrade; + +import com.github.dockerjava.api.model.Bind; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.tests.integration.containers.PulsarContainer; +import org.apache.pulsar.tests.integration.topologies.PulsarCluster; +import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; +import org.apache.pulsar.tests.integration.topologies.PulsarClusterTestBase; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testng.annotations.Test; +import java.util.stream.Stream; +import static java.util.stream.Collectors.joining; +import static org.testng.Assert.assertEquals; + +/** + * Test upgrading/downgrading Pulsar cluster from major releases. + */ +@Slf4j +public class PulsarUpgradeDowngradeTest extends PulsarClusterTestBase { + + @Test(timeOut=600_000) + public void upgradeTest() throws Exception { + testUpgradeDowngrade(PulsarContainer.LAST_RELEASE_IMAGE_NAME, PulsarContainer.UPGRADE_TEST_IMAGE_NAME); + } + + private void testUpgradeDowngrade(String imageOld, String imageNew) throws Exception { + final String clusterName = Stream.of(this.getClass().getSimpleName(), randomName(5)) + .filter(s -> !s.isEmpty()) + .collect(joining("-")); + String topicName = generateTopicName("testupdown", true); + + @Cleanup + Network network = Network.newNetwork(); + @Cleanup + GenericContainer alpine = new GenericContainer<>(PulsarContainer.ALPINE_IMAGE_NAME) + .withExposedPorts(80) + .withNetwork(network) + .withNetworkAliases("shared-storage") + .withEnv("MAGIC_NUMBER", "42") + .withCreateContainerCmdModifier(createContainerCmd -> createContainerCmd + .getHostConfig() + .withBinds(Bind.parse("/pulsar/data:/pulsar/data"))) + .withCommand("/bin/sh", "-c", + "mkdir -p /pulsar/data && " + + "chmod -R ug+rwx /pulsar/data && " + + "chown -R 10000:0 /pulsar/data && " + + "rm -rf /pulsar/data/* && " + + "while true; do echo \"$MAGIC_NUMBER\" | nc -l -p 80; done"); + alpine.start(); + + PulsarClusterSpec specOld = PulsarClusterSpec.builder() + .numBookies(2) + .numBrokers(1) + .clusterName(clusterName) + .dataContainer(alpine) + .pulsarTestImage(imageOld) + .build(); + + PulsarClusterSpec specNew = PulsarClusterSpec.builder() + .numBookies(2) + .numBrokers(1) + .clusterName(clusterName) + .dataContainer(alpine) + .pulsarTestImage(imageNew) + .build(); + + log.info("Setting up OLD cluster {} with {} bookies, {} brokers using {}", + specOld.clusterName(), specOld.numBookies(), specOld.numBrokers(), imageOld); + + pulsarCluster = PulsarCluster.forSpec(specNew, network); + pulsarCluster.closeNetworkOnExit = false; + pulsarCluster.start(true); + + try { + log.info("setting retention"); + pulsarCluster.runAdminCommandOnAnyBroker("namespaces", + "set-retention", "--size", "100M", "--time", "100m", "public/default"); + + publishAndConsume(topicName, pulsarCluster.getPlainTextServiceUrl(), 10, 10); + } finally { + pulsarCluster.stop(); + } + + log.info("Upgrading to NEW cluster {} with {} bookies, {} brokers using {}", + specNew.clusterName(), specNew.numBookies(), specNew.numBrokers(), imageNew); + + pulsarCluster = PulsarCluster.forSpec(specNew, network); + pulsarCluster.closeNetworkOnExit = false; + pulsarCluster.start(false); + + try { + publishAndConsume(topicName, pulsarCluster.getPlainTextServiceUrl(), 10, 20); + } finally { + pulsarCluster.stop(); + } + + log.info("Downgrading to OLD cluster {} with {} bookies, {} brokers using {}", + specOld.clusterName(), specOld.numBookies(), specOld.numBrokers(), imageOld); + + pulsarCluster = PulsarCluster.forSpec(specOld, network); + pulsarCluster.closeNetworkOnExit = false; + pulsarCluster.start(false); + + try { + publishAndConsume(topicName, pulsarCluster.getPlainTextServiceUrl(), 10, 30); + } finally { + pulsarCluster.stop(); + alpine.stop(); + network.close(); + } + } + + private void publishAndConsume(String topicName, String serviceUrl, int numProduce, int numConsume) throws Exception { + log.info("publishAndConsume: topic name: {}", topicName); + + @Cleanup + PulsarClient client = PulsarClient.builder() + .serviceUrl(serviceUrl) + .build(); + + @Cleanup + Producer producer = client.newProducer(Schema.STRING) + .topic(topicName) + .create(); + + log.info("Publishing {} messages", numProduce); + for (int i = numConsume - numProduce; i < numConsume; i++) { + log.info("Publishing message: {}", "smoke-message-" + i); + producer.send("smoke-message-" + i); + } + + @Cleanup + Consumer consumer = client.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName("my-sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + consumer.seek(MessageId.earliest); + + log.info("Consuming {} messages", numConsume); + for (int i = 0; i < numConsume; i++) { + log.info("Waiting for message: {}", i); + Message m = consumer.receive(); + log.info("Received message: {}", m.getValue()); + assertEquals("smoke-message-" + i, m.getValue()); + } + } +} diff --git a/tests/integration/src/test/resources/pulsar-upgrade.xml b/tests/integration/src/test/resources/pulsar-upgrade.xml index a52db54753372..dc966b160ba17 100644 --- a/tests/integration/src/test/resources/pulsar-upgrade.xml +++ b/tests/integration/src/test/resources/pulsar-upgrade.xml @@ -22,7 +22,7 @@ - + From 84b834f95c83e2385d8ca9bccb4cb78120ed582c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 9 Oct 2024 02:34:21 +0300 Subject: [PATCH 569/580] [improve][broker] PIP-379: Snapshot hash range assignments only in AUTO_SPLIT ordered mode (#23423) --- ...stentHashingStickyKeyConsumerSelector.java | 35 ++++++++++++++----- ...ngeAutoSplitStickyKeyConsumerSelector.java | 32 ++++++++++++----- ...ngeExclusiveStickyKeyConsumerSelector.java | 25 ++++--------- .../service/StickyKeyConsumerSelector.java | 5 +-- ...tStickyKeyDispatcherMultipleConsumers.java | 10 +++--- ...tHashingStickyKeyConsumerSelectorTest.java | 6 ++-- ...utoSplitStickyKeyConsumerSelectorTest.java | 6 ++-- 7 files changed, 73 insertions(+), 46 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java index 8381f9543bdc2..fde140a299c27 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; import java.util.NavigableMap; +import java.util.Optional; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.locks.ReadWriteLock; @@ -44,21 +45,32 @@ public class ConsistentHashingStickyKeyConsumerSelector implements StickyKeyCons private final int numberOfPoints; private final Range keyHashRange; + private final boolean addOrRemoveReturnsImpactedConsumersResult; private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot; public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints) { - this(numberOfPoints, DEFAULT_RANGE_SIZE - 1); + this(numberOfPoints, false); } - public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints, int rangeMaxValue) { + public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints, + boolean addOrRemoveReturnsImpactedConsumersResult) { + this(numberOfPoints, addOrRemoveReturnsImpactedConsumersResult, DEFAULT_RANGE_SIZE - 1); + } + + public ConsistentHashingStickyKeyConsumerSelector(int numberOfPoints, + boolean addOrRemoveReturnsImpactedConsumersResult, + int rangeMaxValue) { + this.addOrRemoveReturnsImpactedConsumersResult = addOrRemoveReturnsImpactedConsumersResult; this.hashRing = new TreeMap<>(); this.numberOfPoints = numberOfPoints; this.keyHashRange = Range.of(STICKY_KEY_HASH_NOT_SET + 1, rangeMaxValue); - this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty(); + this.consumerHashAssignmentsSnapshot = addOrRemoveReturnsImpactedConsumersResult + ? ConsumerHashAssignmentsSnapshot.empty() + : null; } @Override - public CompletableFuture addConsumer(Consumer consumer) { + public CompletableFuture> addConsumer(Consumer consumer) { rwLock.writeLock().lock(); try { ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer); @@ -76,11 +88,14 @@ public CompletableFuture addConsumer(Consumer consumer) consumerNameIndexTracker.decreaseConsumerRefCount(removed); } } + if (!addOrRemoveReturnsImpactedConsumersResult) { + return CompletableFuture.completedFuture(Optional.empty()); + } ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); ImpactedConsumersResult impactedConsumers = consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); consumerHashAssignmentsSnapshot = assignmentsAfter; - return CompletableFuture.completedFuture(impactedConsumers); + return CompletableFuture.completedFuture(Optional.of(impactedConsumers)); } finally { rwLock.writeLock().unlock(); } @@ -103,7 +118,7 @@ private int calculateHashForConsumerAndIndex(Consumer consumer, int consumerName } @Override - public ImpactedConsumersResult removeConsumer(Consumer consumer) { + public Optional removeConsumer(Consumer consumer) { rwLock.writeLock().lock(); try { ConsumerIdentityWrapper consumerIdentityWrapper = new ConsumerIdentityWrapper(consumer); @@ -117,11 +132,14 @@ public ImpactedConsumersResult removeConsumer(Consumer consumer) { } } } + if (!addOrRemoveReturnsImpactedConsumersResult) { + return Optional.empty(); + } ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); ImpactedConsumersResult impactedConsumers = consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); consumerHashAssignmentsSnapshot = assignmentsAfter; - return impactedConsumers; + return Optional.of(impactedConsumers); } finally { rwLock.writeLock().unlock(); } @@ -155,7 +173,8 @@ public Range getKeyHashRange() { public ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() { rwLock.readLock().lock(); try { - return consumerHashAssignmentsSnapshot; + return consumerHashAssignmentsSnapshot != null ? consumerHashAssignmentsSnapshot + : internalGetConsumerHashAssignmentsSnapshot(); } finally { rwLock.readLock().unlock(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java index b90aef739f2b1..48d5491d119b2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelector.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentSkipListMap; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException; @@ -59,13 +60,20 @@ public class HashRangeAutoSplitStickyKeyConsumerSelector implements StickyKeyCon private final Range keyHashRange; private final ConcurrentSkipListMap rangeMap; private final Map consumerRange; + private final boolean addOrRemoveReturnsImpactedConsumersResult; private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot; public HashRangeAutoSplitStickyKeyConsumerSelector() { - this(DEFAULT_RANGE_SIZE); + this(false); } - public HashRangeAutoSplitStickyKeyConsumerSelector(int rangeSize) { + public HashRangeAutoSplitStickyKeyConsumerSelector(boolean addOrRemoveReturnsImpactedConsumersResult) { + this(DEFAULT_RANGE_SIZE, addOrRemoveReturnsImpactedConsumersResult); + } + + public HashRangeAutoSplitStickyKeyConsumerSelector(int rangeSize, + boolean addOrRemoveReturnsImpactedConsumersResult) { + this.addOrRemoveReturnsImpactedConsumersResult = addOrRemoveReturnsImpactedConsumersResult; if (rangeSize < 2) { throw new IllegalArgumentException("range size must greater than 2"); } @@ -76,11 +84,12 @@ public HashRangeAutoSplitStickyKeyConsumerSelector(int rangeSize) { this.consumerRange = new HashMap<>(); this.rangeSize = rangeSize; this.keyHashRange = Range.of(0, rangeSize - 1); - this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty(); + this.consumerHashAssignmentsSnapshot = addOrRemoveReturnsImpactedConsumersResult + ? ConsumerHashAssignmentsSnapshot.empty() : null; } @Override - public synchronized CompletableFuture addConsumer(Consumer consumer) { + public synchronized CompletableFuture> addConsumer(Consumer consumer) { if (rangeMap.isEmpty()) { rangeMap.put(rangeSize, consumer); consumerRange.put(consumer, rangeSize); @@ -91,15 +100,18 @@ public synchronized CompletableFuture addConsumer(Consu return CompletableFuture.failedFuture(e); } } + if (!addOrRemoveReturnsImpactedConsumersResult) { + return CompletableFuture.completedFuture(Optional.empty()); + } ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); ImpactedConsumersResult impactedConsumers = consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); consumerHashAssignmentsSnapshot = assignmentsAfter; - return CompletableFuture.completedFuture(impactedConsumers); + return CompletableFuture.completedFuture(Optional.of(impactedConsumers)); } @Override - public synchronized ImpactedConsumersResult removeConsumer(Consumer consumer) { + public synchronized Optional removeConsumer(Consumer consumer) { Integer removeRange = consumerRange.remove(consumer); if (removeRange != null) { if (removeRange == rangeSize && rangeMap.size() > 1) { @@ -111,11 +123,14 @@ public synchronized ImpactedConsumersResult removeConsumer(Consumer consumer) { rangeMap.remove(removeRange); } } + if (!addOrRemoveReturnsImpactedConsumersResult) { + return Optional.empty(); + } ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); ImpactedConsumersResult impactedConsumers = consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); consumerHashAssignmentsSnapshot = assignmentsAfter; - return impactedConsumers; + return Optional.of(impactedConsumers); } @Override @@ -134,7 +149,8 @@ public Range getKeyHashRange() { @Override public synchronized ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() { - return consumerHashAssignmentsSnapshot; + return consumerHashAssignmentsSnapshot != null ? consumerHashAssignmentsSnapshot + : internalGetConsumerHashAssignmentsSnapshot(); } private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java index 7c76d9dca7456..904fb702a943e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/HashRangeExclusiveStickyKeyConsumerSelector.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentSkipListMap; import org.apache.pulsar.client.api.Range; @@ -38,7 +39,6 @@ public class HashRangeExclusiveStickyKeyConsumerSelector implements StickyKeyCon private final int rangeSize; private final Range keyHashRange; private final ConcurrentSkipListMap rangeMap; - private ConsumerHashAssignmentsSnapshot consumerHashAssignmentsSnapshot; public HashRangeExclusiveStickyKeyConsumerSelector() { this(DEFAULT_RANGE_SIZE); @@ -52,11 +52,10 @@ public HashRangeExclusiveStickyKeyConsumerSelector(int rangeSize) { this.rangeSize = rangeSize; this.keyHashRange = Range.of(0, rangeSize - 1); this.rangeMap = new ConcurrentSkipListMap<>(); - this.consumerHashAssignmentsSnapshot = ConsumerHashAssignmentsSnapshot.empty(); } @Override - public synchronized CompletableFuture addConsumer(Consumer consumer) { + public synchronized CompletableFuture> addConsumer(Consumer consumer) { return validateKeySharedMeta(consumer).thenApply(__ -> { try { return internalAddConsumer(consumer); @@ -66,7 +65,7 @@ public synchronized CompletableFuture addConsumer(Consu }); } - private synchronized ImpactedConsumersResult internalAddConsumer(Consumer consumer) + private synchronized Optional internalAddConsumer(Consumer consumer) throws BrokerServiceException.ConsumerAssignException { Consumer conflictingConsumer = findConflictingConsumer(consumer.getKeySharedMeta().getHashRangesList()); if (conflictingConsumer != null) { @@ -77,29 +76,17 @@ private synchronized ImpactedConsumersResult internalAddConsumer(Consumer consum rangeMap.put(intRange.getStart(), consumer); rangeMap.put(intRange.getEnd(), consumer); } - ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); - ImpactedConsumersResult impactedConsumers = - consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); - consumerHashAssignmentsSnapshot = assignmentsAfter; - return impactedConsumers; + return Optional.empty(); } @Override - public synchronized ImpactedConsumersResult removeConsumer(Consumer consumer) { + public synchronized Optional removeConsumer(Consumer consumer) { rangeMap.entrySet().removeIf(entry -> entry.getValue().equals(consumer)); - ConsumerHashAssignmentsSnapshot assignmentsAfter = internalGetConsumerHashAssignmentsSnapshot(); - ImpactedConsumersResult impactedConsumers = - consumerHashAssignmentsSnapshot.resolveImpactedConsumers(assignmentsAfter); - consumerHashAssignmentsSnapshot = assignmentsAfter; - return impactedConsumers; + return Optional.empty(); } @Override public synchronized ConsumerHashAssignmentsSnapshot getConsumerHashAssignmentsSnapshot() { - return consumerHashAssignmentsSnapshot; - } - - private ConsumerHashAssignmentsSnapshot internalGetConsumerHashAssignmentsSnapshot() { List result = new ArrayList<>(); Map.Entry prev = null; for (Map.Entry entry: rangeMap.entrySet()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java index 1ead3f946c24d..099929fd2a696 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.pulsar.client.api.Range; @@ -50,7 +51,7 @@ public interface StickyKeyConsumerSelector { * The result contains information about the existing consumers whose hash ranges were affected * by the addition of the new consumer. */ - CompletableFuture addConsumer(Consumer consumer); + CompletableFuture> addConsumer(Consumer consumer); /** * Remove the consumer. @@ -59,7 +60,7 @@ public interface StickyKeyConsumerSelector { * @return the result of impacted consumers. The result contains information about the existing consumers * whose hash ranges were affected by the removal of the consumer. */ - ImpactedConsumersResult removeConsumer(Consumer consumer); + Optional removeConsumer(Consumer consumer); /** * Select a consumer by sticky key. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index a78e4e46c0e5a..925e99ed699a2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -91,9 +91,9 @@ public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDi case AUTO_SPLIT: if (conf.isSubscriptionKeySharedUseConsistentHashing()) { selector = new ConsistentHashingStickyKeyConsumerSelector( - conf.getSubscriptionKeySharedConsistentHashingReplicaPoints()); + conf.getSubscriptionKeySharedConsistentHashingReplicaPoints(), drainingHashesRequired); } else { - selector = new HashRangeAutoSplitStickyKeyConsumerSelector(); + selector = new HashRangeAutoSplitStickyKeyConsumerSelector(drainingHashesRequired); } break; case STICKY: @@ -155,7 +155,7 @@ public void endBatch() { drainingHashesTracker.endBatch(); } }); - registerDrainingHashes(consumer, impactedConsumers); + registerDrainingHashes(consumer, impactedConsumers.orElseThrow()); } }).exceptionally(ex -> { internalRemoveConsumer(consumer); @@ -184,13 +184,13 @@ private synchronized void registerDrainingHashes(Consumer skipConsumer, @Override public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException { // The consumer must be removed from the selector before calling the superclass removeConsumer method. - ImpactedConsumersResult impactedConsumers = selector.removeConsumer(consumer); + Optional impactedConsumers = selector.removeConsumer(consumer); super.removeConsumer(consumer); if (drainingHashesRequired) { // register draining hashes for the impacted consumers and ranges, in case a hash switched from one // consumer to another. This will handle the case where a hash gets switched from an existing // consumer to another existing consumer during removal. - registerDrainingHashes(consumer, impactedConsumers); + registerDrainingHashes(consumer, impactedConsumers.orElseThrow()); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java index e2feb2050652b..2b01256611b01 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelectorTest.java @@ -476,7 +476,8 @@ public void testShouldNotChangeSelectedConsumerWhenConsumerIsAdded() { @Test public void testShouldContainMinimalMappingChangesWhenConsumerLeavesAndRejoins() { - final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final ConsistentHashingStickyKeyConsumerSelector selector = + new ConsistentHashingStickyKeyConsumerSelector(100, true); final String consumerName = "consumer"; final int numOfInitialConsumers = 10; List consumers = new ArrayList<>(); @@ -563,7 +564,8 @@ public void testPerformanceOfAdding1000ConsumersWith100Points() { // test that adding 1000 consumers with 100 points runs in a reasonable time. // This takes about 1 second on Apple M3 // this unit test can be used for basic profiling - final ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(100); + final ConsistentHashingStickyKeyConsumerSelector selector = + new ConsistentHashingStickyKeyConsumerSelector(100, true); for (int i = 0; i < 1000; i++) { // use real class to avoid Mockito over head final Consumer consumer = new Consumer("consumer" + i, 0) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelectorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelectorTest.java index 98e27ebb9fb83..61fc015cf953e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelectorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/HashRangeAutoSplitStickyKeyConsumerSelectorTest.java @@ -37,7 +37,8 @@ public class HashRangeAutoSplitStickyKeyConsumerSelectorTest { @Test public void testGetConsumerKeyHashRanges() throws BrokerServiceException.ConsumerAssignException { - HashRangeAutoSplitStickyKeyConsumerSelector selector = new HashRangeAutoSplitStickyKeyConsumerSelector(2 << 5); + HashRangeAutoSplitStickyKeyConsumerSelector selector = + new HashRangeAutoSplitStickyKeyConsumerSelector(2 << 5, false); List consumerName = Arrays.asList("consumer1", "consumer2", "consumer3", "consumer4"); List consumers = new ArrayList<>(); for (String s : consumerName) { @@ -61,7 +62,8 @@ public void testGetConsumerKeyHashRanges() throws BrokerServiceException.Consume @Test public void testGetConsumerKeyHashRangesWithSameConsumerName() throws Exception { - HashRangeAutoSplitStickyKeyConsumerSelector selector = new HashRangeAutoSplitStickyKeyConsumerSelector(2 << 5); + HashRangeAutoSplitStickyKeyConsumerSelector selector = + new HashRangeAutoSplitStickyKeyConsumerSelector(2 << 5, false); final String consumerName = "My-consumer"; List consumers = new ArrayList<>(); for (int i = 0; i < 3; i++) { From 5aadec02a3e8767f55d4101a2efde47a86094e68 Mon Sep 17 00:00:00 2001 From: psxjoy Date: Wed, 9 Oct 2024 12:29:13 +0800 Subject: [PATCH 570/580] [fix][broker] Fix typos in pulsar-broker and tiered-storage. (#23415) --- .../java/org/apache/pulsar/broker/web/WebServiceTest.java | 8 ++++---- .../provider/AbstractJCloudBlobStoreFactoryTest.java | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index 08041d72c7e44..e6e792c2f3839 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -176,7 +176,7 @@ public void testDefaultClientVersion() throws Exception { } /** - * Test that if enableTls option is enabled, WebServcie is available both on HTTP and HTTPS. + * Test that if enableTls option is enabled, WebService is available both on HTTP and HTTPS. * * @throws Exception */ @@ -198,7 +198,7 @@ public void testTlsEnabled() throws Exception { } /** - * Test that if enableTls option is disabled, WebServcie is available only on HTTP. + * Test that if enableTls option is disabled, WebService is available only on HTTP. * * @throws Exception */ @@ -221,7 +221,7 @@ public void testTlsDisabled() throws Exception { } /** - * Test that if enableAuth option and allowInsecure option are enabled, WebServcie requires trusted/untrusted client + * Test that if enableAuth option and allowInsecure option are enabled, WebService requires trusted/untrusted client * certificate. * * @throws Exception @@ -245,7 +245,7 @@ public void testTlsAuthAllowInsecure() throws Exception { } /** - * Test that if enableAuth option is enabled, WebServcie requires trusted client certificate. + * Test that if enableAuth option is enabled, WebService requires trusted client certificate. * * @throws Exception */ diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/provider/AbstractJCloudBlobStoreFactoryTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/provider/AbstractJCloudBlobStoreFactoryTest.java index 17a337cc22a3c..93fb6dcc8d52e 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/provider/AbstractJCloudBlobStoreFactoryTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/provider/AbstractJCloudBlobStoreFactoryTest.java @@ -98,9 +98,9 @@ protected void sendMultipartPayload(String containerName, String blobName, Strin blobStore.completeMultipartUpload(mpu, parts); } - protected void deleteBlobAndVerify(String conatinerName, String blobName) { - blobStore.removeBlob(conatinerName, blobName); - Assert.assertFalse(blobStore.blobExists(conatinerName, blobName)); + protected void deleteBlobAndVerify(String containerName, String blobName) { + blobStore.removeBlob(containerName, blobName); + Assert.assertFalse(blobStore.blobExists(containerName, blobName)); } protected void deleteContainerAndVerify(String containerName) { From 676fdb1ffb4392bb7b10b8d1e8ba94b379b25166 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 9 Oct 2024 08:38:14 +0300 Subject: [PATCH 571/580] [improve][broker] PIP-379: Enable the use of the classic implementation of Key_Shared / Shared with feature flag (#23424) --- .../pulsar/broker/ServiceConfiguration.java | 16 + .../AbstractDelayedDeliveryTracker.java | 8 +- .../BucketDelayedDeliveryTrackerFactory.java | 6 +- .../DelayedDeliveryTrackerFactory.java | 4 +- .../InMemoryDelayedDeliveryTracker.java | 13 +- ...InMemoryDelayedDeliveryTrackerFactory.java | 6 +- .../bucket/BucketDelayedDeliveryTracker.java | 6 +- .../AbstractDispatcherMultipleConsumers.java | 6 - .../pulsar/broker/service/BrokerService.java | 16 +- .../pulsar/broker/service/Consumer.java | 10 +- .../pulsar/broker/service/Dispatcher.java | 4 +- .../broker/service/StickyKeyDispatcher.java | 48 + ...PersistentDispatcherMultipleConsumers.java | 67 + .../MessageRedeliveryController.java | 8 +- ...PersistentDispatcherMultipleConsumers.java | 25 +- ...entDispatcherMultipleConsumersClassic.java | 1374 +++++++++++++++++ ...tStickyKeyDispatcherMultipleConsumers.java | 4 +- ...KeyDispatcherMultipleConsumersClassic.java | 583 +++++++ .../persistent/PersistentSubscription.java | 50 +- .../service/persistent/PersistentTopic.java | 16 +- .../apache/pulsar/broker/BrokerTestUtil.java | 25 +- .../delayed/AbstractDeliveryTrackerTest.java | 4 +- .../DelayedDeliveryTrackerFactoryTest.java | 14 +- .../delayed/InMemoryDeliveryTrackerTest.java | 7 +- .../BucketDelayedDeliveryTrackerTest.java | 4 +- .../broker/service/BatchMessageTest.java | 4 +- .../BatchMessageWithBatchIndexLevelTest.java | 16 +- ...sistentDispatcherFailoverConsumerTest.java | 15 +- .../broker/service/PersistentTopicTest.java | 3 +- .../persistent/BucketDelayedDeliveryTest.java | 12 +- .../persistent/DelayedDeliveryTest.java | 8 +- ...ispatcherMultipleConsumersClassicTest.java | 172 +++ ...istentDispatcherMultipleConsumersTest.java | 2 +- ...ispatcherMultipleConsumersClassicTest.java | 482 ++++++ ...ckyKeyDispatcherMultipleConsumersTest.java | 2 +- .../broker/stats/ConsumerStatsTest.java | 2 +- .../client/api/KeySharedSubscriptionTest.java | 281 ++-- ...criptionMessageDispatchThrottlingTest.java | 18 +- ...SubscriptionPauseOnAckStatPersistTest.java | 5 +- ...edSubscriptionMaxUnackedMessagesTest.java} | 42 +- .../tests/KeySharedImplementationType.java | 61 + .../common/policies/data/ConsumerStats.java | 4 +- .../policies/data/SubscriptionStats.java | 3 + .../data/stats/ConsumerStatsImpl.java | 10 +- .../data/stats/SubscriptionStatsImpl.java | 8 +- 45 files changed, 3211 insertions(+), 263 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyDispatcher.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/AbstractPersistentDispatcherMultipleConsumers.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassicTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java rename pulsar-broker/src/test/java/org/apache/pulsar/client/impl/{KeySharedSubscriptionTest.java => KeySharedSubscriptionMaxUnackedMessagesTest.java} (85%) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/tests/KeySharedImplementationType.java 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 1b021bd569969..19e9ff625cada 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 @@ -817,6 +817,22 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece + "The higher the number, the more equal the assignment of keys to consumers") private int subscriptionKeySharedConsistentHashingReplicaPoints = 100; + @FieldContext( + category = CATEGORY_POLICIES, + doc = "For persistent Key_Shared subscriptions, enables the use of the classic implementation of the " + + "Key_Shared subscription that was used before Pulsar 4.0.0 and PIP-379.", + dynamic = true + ) + private boolean subscriptionKeySharedUseClassicPersistentImplementation = false; + + @FieldContext( + category = CATEGORY_POLICIES, + doc = "For persistent Shared subscriptions, enables the use of the classic implementation of the Shared " + + "subscription that was used before Pulsar 4.0.0.", + dynamic = true + ) + private boolean subscriptionSharedUseClassicPersistentImplementation = false; + @FieldContext( category = CATEGORY_POLICIES, doc = "Set the default behavior for message deduplication in the broker.\n\n" diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java index f93a627bca7b8..bec5134c4f79a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/AbstractDelayedDeliveryTracker.java @@ -24,12 +24,12 @@ import java.time.Clock; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; @Slf4j public abstract class AbstractDelayedDeliveryTracker implements DelayedDeliveryTracker, TimerTask { - protected final PersistentDispatcherMultipleConsumers dispatcher; + protected final AbstractPersistentDispatcherMultipleConsumers dispatcher; // Reference to the shared (per-broker) timer for delayed delivery protected final Timer timer; @@ -49,13 +49,13 @@ public abstract class AbstractDelayedDeliveryTracker implements DelayedDeliveryT private final boolean isDelayedDeliveryDeliverAtTimeStrict; - public AbstractDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, + public AbstractDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict) { this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict); } - public AbstractDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, + public AbstractDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, Clock clock, boolean isDelayedDeliveryDeliverAtTimeStrict) { this.dispatcher = dispatcher; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java index 11ad243e0c9d1..c2d002ad19cb0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/BucketDelayedDeliveryTrackerFactory.java @@ -36,7 +36,7 @@ import org.apache.pulsar.broker.delayed.bucket.BucketSnapshotStorage; import org.apache.pulsar.broker.delayed.bucket.RecoverDelayedDeliveryTrackerException; import org.apache.pulsar.broker.service.BrokerService; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,7 +78,7 @@ public void initialize(PulsarService pulsarService) throws Exception { } @Override - public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher) { + public DelayedDeliveryTracker newTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher) { String topicName = dispatcher.getTopic().getName(); String subscriptionName = dispatcher.getSubscription().getName(); BrokerService brokerService = dispatcher.getTopic().getBrokerService(); @@ -97,7 +97,7 @@ public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers d } @VisibleForTesting - BucketDelayedDeliveryTracker newTracker0(PersistentDispatcherMultipleConsumers dispatcher) + BucketDelayedDeliveryTracker newTracker0(AbstractPersistentDispatcherMultipleConsumers dispatcher) throws RecoverDelayedDeliveryTrackerException { return new BucketDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, delayedDeliveryMinIndexCountPerBucket, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java index 5427a46a2e4b3..763b6d66da142 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactory.java @@ -20,7 +20,7 @@ import com.google.common.annotations.Beta; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; /** * Factory of InMemoryDelayedDeliveryTracker objects. This is the entry point for implementations. @@ -42,7 +42,7 @@ public interface DelayedDeliveryTrackerFactory extends AutoCloseable { * @param dispatcher * a multi-consumer dispatcher instance */ - DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher); + DelayedDeliveryTracker newTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher); /** * Close the factory and release all the resources. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java index 8bd9fafa13715..bdc6e4c814e33 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java @@ -28,7 +28,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue; @Slf4j @@ -52,17 +52,18 @@ public class InMemoryDelayedDeliveryTracker extends AbstractDelayedDeliveryTrack // Track whether we have seen all messages with fixed delay so far. private boolean messagesHaveFixedDelay = true; - InMemoryDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, + InMemoryDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, + long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict, long fixedDelayDetectionLookahead) { this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); } - public InMemoryDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, Timer timer, - long tickTimeMillis, Clock clock, - boolean isDelayedDeliveryDeliverAtTimeStrict, - long fixedDelayDetectionLookahead) { + public InMemoryDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, + long tickTimeMillis, Clock clock, + boolean isDelayedDeliveryDeliverAtTimeStrict, + long fixedDelayDetectionLookahead) { super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict); this.fixedDelayDetectionLookahead = fixedDelayDetectionLookahead; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java index 179cf74db4179..f8b8f5a8ba459 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTrackerFactory.java @@ -25,7 +25,7 @@ import java.util.concurrent.TimeUnit; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,7 +51,7 @@ public void initialize(PulsarService pulsarService) { } @Override - public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher) { + public DelayedDeliveryTracker newTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher) { String topicName = dispatcher.getTopic().getName(); String subscriptionName = dispatcher.getSubscription().getName(); DelayedDeliveryTracker tracker = DelayedDeliveryTracker.DISABLE; @@ -66,7 +66,7 @@ public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers d } @VisibleForTesting - InMemoryDelayedDeliveryTracker newTracker0(PersistentDispatcherMultipleConsumers dispatcher) { + InMemoryDelayedDeliveryTracker newTracker0(AbstractPersistentDispatcherMultipleConsumers dispatcher) { return new InMemoryDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, isDelayedDeliveryDeliverAtTimeStrict, fixedDelayDetectionLookahead); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index 47c78fa9ee2ec..0091bf5b9bd30 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -57,7 +57,7 @@ import org.apache.pulsar.broker.delayed.AbstractDelayedDeliveryTracker; import org.apache.pulsar.broker.delayed.proto.DelayedIndex; import org.apache.pulsar.broker.delayed.proto.SnapshotSegment; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue; @@ -105,7 +105,7 @@ public class BucketDelayedDeliveryTracker extends AbstractDelayedDeliveryTracker private CompletableFuture pendingLoad = null; - public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, + public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict, BucketSnapshotStorage bucketSnapshotStorage, @@ -117,7 +117,7 @@ public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispat maxIndexesPerBucketSnapshotSegment, maxNumBuckets); } - public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, + public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, Clock clock, boolean isDelayedDeliveryDeliverAtTimeStrict, BucketSnapshotStorage bucketSnapshotStorage, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java index 9fc6b9581a3ac..e3c2cf40cf318 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherMultipleConsumers.java @@ -24,10 +24,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * @@ -239,7 +236,4 @@ private int getFirstConsumerIndexOfPriority(int targetPriority) { return -1; } - private static final Logger log = LoggerFactory.getLogger(PersistentStickyKeyDispatcherMultipleConsumers.class); - - } 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 8d0b9a4a84e6a..fee5e25647ce6 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 @@ -129,8 +129,8 @@ import org.apache.pulsar.broker.service.TopicEventsListener.EventStage; import org.apache.pulsar.broker.service.TopicEventsListener.TopicEvent; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.persistent.SystemTopic; import org.apache.pulsar.broker.service.plugin.EntryFilterProvider; @@ -301,7 +301,7 @@ public class BrokerService implements Closeable { private final int maxUnackedMessages; public final int maxUnackedMsgsPerDispatcher; private final AtomicBoolean blockedDispatcherOnHighUnackedMsgs = new AtomicBoolean(false); - private final Set blockedDispatchers = ConcurrentHashMap.newKeySet(); + private final Set blockedDispatchers = ConcurrentHashMap.newKeySet(); private final ReadWriteLock lock = new ReentrantReadWriteLock(); @VisibleForTesting private final DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory; @@ -3328,7 +3328,7 @@ public OrderedExecutor getTopicOrderedExecutor() { * @param dispatcher * @param numberOfMessages */ - public void addUnAckedMessages(PersistentDispatcherMultipleConsumers dispatcher, int numberOfMessages) { + public void addUnAckedMessages(AbstractPersistentDispatcherMultipleConsumers dispatcher, int numberOfMessages) { // don't block dispatchers if maxUnackedMessages = 0 if (maxUnackedMessages > 0) { totalUnackedMessages.add(numberOfMessages); @@ -3387,10 +3387,10 @@ private void blockDispatchersWithLargeUnAckMessages() { try { forEachTopic(topic -> { topic.getSubscriptions().forEach((subName, persistentSubscription) -> { - if (persistentSubscription.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) persistentSubscription - .getDispatcher(); + if (persistentSubscription.getDispatcher() + instanceof AbstractPersistentDispatcherMultipleConsumers) { + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) persistentSubscription.getDispatcher(); int dispatcherUnAckMsgs = dispatcher.getTotalUnackedMessages(); if (dispatcherUnAckMsgs > maxUnackedMsgsPerDispatcher) { log.info("[{}] Blocking dispatcher due to reached max broker limit {}", @@ -3411,7 +3411,7 @@ private void blockDispatchersWithLargeUnAckMessages() { * * @param dispatcherList */ - public void unblockDispatchersOnUnAckMessages(List dispatcherList) { + public void unblockDispatchersOnUnAckMessages(List dispatcherList) { lock.writeLock().lock(); try { dispatcherList.forEach(dispatcher -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index c9584f2c1790f..d25ebd0839df1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -146,7 +146,7 @@ public class Consumer { private static final double avgPercent = 0.9; private boolean preciseDispatcherFlowControl; - private Position lastSentPositionWhenJoining; + private Position readPositionWhenJoining; private final String clientAddress; // IP address only, no port number included private final MessageId startMessageId; private final boolean isAcknowledgmentAtBatchIndexLevelEnabled; @@ -973,8 +973,8 @@ public ConsumerStatsImpl getStats() { stats.unackedMessages = unackedMessages; stats.blockedConsumerOnUnackedMsgs = blockedConsumerOnUnackedMsgs; stats.avgMessagesPerEntry = getAvgMessagesPerEntry(); - if (lastSentPositionWhenJoining != null) { - stats.lastSentPositionWhenJoining = lastSentPositionWhenJoining.toString(); + if (readPositionWhenJoining != null) { + stats.readPositionWhenJoining = readPositionWhenJoining.toString(); } return stats; } @@ -1189,8 +1189,8 @@ public boolean isPreciseDispatcherFlowControl() { return preciseDispatcherFlowControl; } - public void setLastSentPositionWhenJoining(Position lastSentPositionWhenJoining) { - this.lastSentPositionWhenJoining = lastSentPositionWhenJoining; + public void setReadPositionWhenJoining(Position readPositionWhenJoining) { + this.readPositionWhenJoining = readPositionWhenJoining; } public int getMaxUnackedMessages() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java index d1d44709a9c52..f68a9a0986b84 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java @@ -153,8 +153,8 @@ default void afterAckMessages(Throwable exOfDeletion, Object ctxOfDeletion){} /** * Trigger a new "readMoreEntries" if the dispatching has been paused before. This method is only implemented in - * {@link org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers} right now, other - * implements are not necessary to implement this method. + * {@link org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers} right now, + * other implementations do not necessary implement this method. * @return did a resume. */ default boolean checkAndResumeIfPaused(){ diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyDispatcher.java new file mode 100644 index 0000000000000..79ce308158422 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyDispatcher.java @@ -0,0 +1,48 @@ +/* + * 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.service; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import org.apache.bookkeeper.mledger.Position; +import org.apache.pulsar.client.api.Range; +import org.apache.pulsar.common.api.proto.KeySharedMeta; +import org.apache.pulsar.common.api.proto.KeySharedMode; + +public interface StickyKeyDispatcher extends Dispatcher { + + boolean hasSameKeySharedPolicy(KeySharedMeta ksm); + + Map> getConsumerKeyHashRanges(); + + boolean isAllowOutOfOrderDelivery(); + + KeySharedMode getKeySharedMode(); + + StickyKeyConsumerSelector getSelector(); + + long getNumberOfMessagesInReplay(); + + default LinkedHashMap getRecentlyJoinedConsumers() { + return null; + } + + boolean isClassic(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/AbstractPersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/AbstractPersistentDispatcherMultipleConsumers.java new file mode 100644 index 0000000000000..79d365b9fee21 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/AbstractPersistentDispatcherMultipleConsumers.java @@ -0,0 +1,67 @@ +/* + * 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.service.persistent; + +import java.util.Map; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; + +public abstract class AbstractPersistentDispatcherMultipleConsumers extends AbstractDispatcherMultipleConsumers + implements Dispatcher, AsyncCallbacks.ReadEntriesCallback { + public AbstractPersistentDispatcherMultipleConsumers(Subscription subscription, + ServiceConfiguration serviceConfig) { + super(subscription, serviceConfig); + } + + public abstract void unBlockDispatcherOnUnackedMsgs(); + + public abstract void readMoreEntriesAsync(); + + public abstract String getName(); + + public abstract boolean isBlockedDispatcherOnUnackedMsgs(); + + public abstract int getTotalUnackedMessages(); + + public abstract void blockDispatcherOnUnackedMsgs(); + + public abstract long getNumberOfMessagesInReplay(); + + public abstract boolean isHavePendingRead(); + + public abstract boolean isHavePendingReplayRead(); + + public abstract ManagedCursor getCursor(); + + public abstract Topic getTopic(); + + public abstract Subscription getSubscription(); + + public abstract long getDelayedTrackerMemoryUsage(); + + public abstract Map getBucketDelayedIndexStats(); + + public abstract boolean isClassic(); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java index b34a0b454385f..46f1f0a535650 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java @@ -41,12 +41,18 @@ public class MessageRedeliveryController { private final boolean allowOutOfOrderDelivery; + private final boolean isClassicDispatcher; private final ConcurrentBitmapSortedLongPairSet messagesToRedeliver; private final ConcurrentLongLongPairHashMap hashesToBeBlocked; private final ConcurrentLongLongHashMap hashesRefCount; public MessageRedeliveryController(boolean allowOutOfOrderDelivery) { + this(allowOutOfOrderDelivery, false); + } + + public MessageRedeliveryController(boolean allowOutOfOrderDelivery, boolean isClassicDispatcher) { this.allowOutOfOrderDelivery = allowOutOfOrderDelivery; + this.isClassicDispatcher = isClassicDispatcher; this.messagesToRedeliver = new ConcurrentBitmapSortedLongPairSet(); if (!allowOutOfOrderDelivery) { this.hashesToBeBlocked = ConcurrentLongLongPairHashMap @@ -65,7 +71,7 @@ public void add(long ledgerId, long entryId) { public void add(long ledgerId, long entryId, long stickyKeyHash) { if (!allowOutOfOrderDelivery) { - if (stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { + if (!isClassicDispatcher && stickyKeyHash == STICKY_KEY_HASH_NOT_SET) { throw new IllegalArgumentException("Sticky key hash is not set. It is required."); } boolean inserted = hashesToBeBlocked.putIfAbsent(ledgerId, entryId, stickyKeyHash, 0); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 73d152bab1a60..b1cd186c31784 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -39,7 +39,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.function.Predicate; -import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; @@ -60,7 +59,6 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; import org.apache.pulsar.broker.service.Consumer; -import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.EntryAndMetadata; import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; import org.apache.pulsar.broker.service.EntryBatchSizes; @@ -85,8 +83,7 @@ /** * */ -public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMultipleConsumers - implements Dispatcher, ReadEntriesCallback { +public class PersistentDispatcherMultipleConsumers extends AbstractPersistentDispatcherMultipleConsumers { protected final PersistentTopic topic; protected final ManagedCursor cursor; protected volatile Range lastIndividualDeletedRangeFromCursorRecovery; @@ -162,7 +159,7 @@ public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCurso this.name = topic.getName() + " / " + Codec.decode(cursor.getName()); this.topic = topic; this.dispatchMessagesThread = topic.getBrokerService().getTopicOrderedExecutor().chooseThread(); - this.redeliveryMessages = new MessageRedeliveryController(allowOutOfOrderDelivery); + this.redeliveryMessages = new MessageRedeliveryController(allowOutOfOrderDelivery, false); this.redeliveryTracker = this.serviceConfig.isSubscriptionRedeliveryTrackerEnabled() ? new InMemoryRedeliveryTracker() : RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED; @@ -320,6 +317,7 @@ private synchronized void internalConsumerFlow(Consumer consumer, int additional * We should not call readMoreEntries() recursively in the same thread as there is a risk of StackOverflowError. * */ + @Override public void readMoreEntriesAsync() { // deduplication for readMoreEntriesAsync calls if (readMoreEntriesAsyncRequested.compareAndSet(false, true)) { @@ -1285,6 +1283,7 @@ public void blockDispatcherOnUnackedMsgs() { blockedDispatcherOnUnackedMsgs = TRUE; } + @Override public void unBlockDispatcherOnUnackedMsgs() { blockedDispatcherOnUnackedMsgs = FALSE; } @@ -1293,6 +1292,7 @@ public int getTotalUnackedMessages() { return totalUnackedMessages; } + @Override public String getName() { return name; } @@ -1488,6 +1488,11 @@ public Map getBucketDelayedIndexStats() { return Collections.emptyMap(); } + @Override + public boolean isClassic() { + return false; + } + public ManagedCursor getCursor() { return cursor; } @@ -1505,5 +1510,15 @@ public long getNumberOfMessagesInReplay() { return redeliveryMessages.size(); } + @Override + public boolean isHavePendingRead() { + return havePendingRead; + } + + @Override + public boolean isHavePendingReplayRead() { + return havePendingReplayRead; + } + private static final Logger log = LoggerFactory.getLogger(PersistentDispatcherMultipleConsumers.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java new file mode 100644 index 0000000000000..6ab7acfa56da8 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java @@ -0,0 +1,1374 @@ +/* + * 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.service.persistent; + +import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Range; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.function.Predicate; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ManagedLedgerException.NoMoreEntriesToReadException; +import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; +import org.apache.pulsar.broker.delayed.DelayedDeliveryTracker; +import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; +import org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker; +import org.apache.pulsar.broker.delayed.bucket.BucketDelayedDeliveryTracker; +import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; +import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.BrokerServiceException; +import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.EntryAndMetadata; +import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; +import org.apache.pulsar.broker.service.EntryBatchSizes; +import org.apache.pulsar.broker.service.InMemoryRedeliveryTracker; +import org.apache.pulsar.broker.service.RedeliveryTracker; +import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; +import org.apache.pulsar.broker.service.SendMessageInfo; +import org.apache.pulsar.broker.service.SharedConsumerAssignor; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; +import org.apache.pulsar.broker.transaction.exception.buffer.TransactionBufferException; +import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; +import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.util.Backoff; +import org.apache.pulsar.common.util.Codec; +import org.apache.pulsar.common.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is the "classic" dispatcher implementation for the Shared subscription that was used before + * Pulsar 4.0.0 and PIP-379. When `subscriptionSharedUseClassicPersistentImplementation=true`, + * this dispatcher will be used. The main purpose is to provide a way to rollback to the old behavior + * in case of issues with the preferred implementation. + */ +public class PersistentDispatcherMultipleConsumersClassic extends AbstractPersistentDispatcherMultipleConsumers { + protected final PersistentTopic topic; + protected final ManagedCursor cursor; + protected volatile Range lastIndividualDeletedRangeFromCursorRecovery; + + private CompletableFuture closeFuture = null; + protected final MessageRedeliveryController redeliveryMessages; + protected final RedeliveryTracker redeliveryTracker; + + private Optional delayedDeliveryTracker = Optional.empty(); + + protected volatile boolean havePendingRead = false; + protected volatile boolean havePendingReplayRead = false; + protected volatile Position minReplayedPosition = null; + protected boolean shouldRewindBeforeReadingOrReplaying = false; + protected final String name; + private boolean sendInProgress = false; + protected static final AtomicIntegerFieldUpdater + TOTAL_AVAILABLE_PERMITS_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumersClassic.class, + "totalAvailablePermits"); + protected volatile int totalAvailablePermits = 0; + protected volatile int readBatchSize; + protected final Backoff readFailureBackoff; + private static final AtomicIntegerFieldUpdater + TOTAL_UNACKED_MESSAGES_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumersClassic.class, + "totalUnackedMessages"); + protected volatile int totalUnackedMessages = 0; + /** + * A signature that relate to the check of "Dispatching has paused on cursor data can fully persist". + * Note: It is a tool that helps determine whether it should trigger a new reading after acknowledgments to avoid + * too many CPU circles, see {@link #afterAckMessages(Throwable, Object)} for more details. Do not use this + * to confirm whether the delivery should be paused, please call {@link #shouldPauseOnAckStatePersist}. + */ + protected static final AtomicIntegerFieldUpdater + BLOCKED_DISPATCHER_ON_CURSOR_DATA_CAN_NOT_FULLY_PERSIST_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumersClassic.class, + "blockedDispatcherOnCursorDataCanNotFullyPersist"); + private volatile int blockedDispatcherOnCursorDataCanNotFullyPersist = FALSE; + private volatile int blockedDispatcherOnUnackedMsgs = FALSE; + protected static final AtomicIntegerFieldUpdater + BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumersClassic.class, + "blockedDispatcherOnUnackedMsgs"); + protected Optional dispatchRateLimiter = Optional.empty(); + private AtomicBoolean isRescheduleReadInProgress = new AtomicBoolean(false); + protected final ExecutorService dispatchMessagesThread; + private final SharedConsumerAssignor assignor; + + protected enum ReadType { + Normal, Replay + } + + public PersistentDispatcherMultipleConsumersClassic(PersistentTopic topic, ManagedCursor cursor, + Subscription subscription) { + this(topic, cursor, subscription, true); + } + + public PersistentDispatcherMultipleConsumersClassic(PersistentTopic topic, ManagedCursor cursor, + Subscription subscription, + boolean allowOutOfOrderDelivery) { + super(subscription, topic.getBrokerService().pulsar().getConfiguration()); + this.cursor = cursor; + this.lastIndividualDeletedRangeFromCursorRecovery = cursor.getLastIndividualDeletedRange(); + this.name = topic.getName() + " / " + Codec.decode(cursor.getName()); + this.topic = topic; + this.dispatchMessagesThread = topic.getBrokerService().getTopicOrderedExecutor().chooseThread(); + this.redeliveryMessages = new MessageRedeliveryController(allowOutOfOrderDelivery, true); + this.redeliveryTracker = this.serviceConfig.isSubscriptionRedeliveryTrackerEnabled() + ? new InMemoryRedeliveryTracker() + : RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED; + this.readBatchSize = serviceConfig.getDispatcherMaxReadBatchSize(); + this.initializeDispatchRateLimiterIfNeeded(); + this.assignor = new SharedConsumerAssignor(this::getNextConsumer, this::addMessageToReplay); + this.readFailureBackoff = new Backoff( + topic.getBrokerService().pulsar().getConfiguration().getDispatcherReadFailureBackoffInitialTimeInMs(), + TimeUnit.MILLISECONDS, + 1, TimeUnit.MINUTES, 0, TimeUnit.MILLISECONDS); + } + + @Override + public synchronized CompletableFuture addConsumer(Consumer consumer) { + if (IS_CLOSED_UPDATER.get(this) == TRUE) { + log.warn("[{}] Dispatcher is already closed. Closing consumer {}", name, consumer); + consumer.disconnect(); + return CompletableFuture.completedFuture(null); + } + if (consumerList.isEmpty()) { + if (havePendingRead || havePendingReplayRead) { + // There is a pending read from previous run. We must wait for it to complete and then rewind + shouldRewindBeforeReadingOrReplaying = true; + } else { + cursor.rewind(); + shouldRewindBeforeReadingOrReplaying = false; + } + redeliveryMessages.clear(); + delayedDeliveryTracker.ifPresent(tracker -> { + // Don't clean up BucketDelayedDeliveryTracker, otherwise we will lose the bucket snapshot + if (tracker instanceof InMemoryDelayedDeliveryTracker) { + tracker.clear(); + } + }); + } + + if (isConsumersExceededOnSubscription()) { + log.warn("[{}] Attempting to add consumer to subscription which reached max consumers limit {}", + name, consumer); + return FutureUtil.failedFuture(new ConsumerBusyException("Subscription reached max consumers limit")); + } + // This is not an expected scenario, it will never happen in expected. Just print a warn log if the unexpected + // scenario happens. See more detail: https://github.com/apache/pulsar/pull/22283. + if (consumerSet.contains(consumer)) { + log.warn("[{}] Attempting to add a consumer that already registered {}", name, consumer); + } + + consumerList.add(consumer); + if (consumerList.size() > 1 + && consumer.getPriorityLevel() < consumerList.get(consumerList.size() - 2).getPriorityLevel()) { + consumerList.sort(Comparator.comparingInt(Consumer::getPriorityLevel)); + } + consumerSet.add(consumer); + + return CompletableFuture.completedFuture(null); + } + + @Override + protected boolean isConsumersExceededOnSubscription() { + return isConsumersExceededOnSubscription(topic, consumerList.size()); + } + + @Override + public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException { + // decrement unack-message count for removed consumer + addUnAckedMessages(-consumer.getUnackedMessages()); + if (consumerSet.removeAll(consumer) == 1) { + consumerList.remove(consumer); + log.info("Removed consumer {} with pending {} acks", consumer, consumer.getPendingAcks().size()); + if (consumerList.isEmpty()) { + clearComponentsAfterRemovedAllConsumers(); + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Consumer are left, reading more entries", name); + } + consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { + addMessageToReplay(ledgerId, entryId, stickyKeyHash); + }); + totalAvailablePermits -= consumer.getAvailablePermits(); + if (log.isDebugEnabled()) { + log.debug("[{}] Decreased totalAvailablePermits by {} in PersistentDispatcherMultipleConsumers. " + + "New dispatcher permit count is {}", name, consumer.getAvailablePermits(), + totalAvailablePermits); + } + readMoreEntries(); + } + } else { + /** + * This is not an expected scenario, it will never happen in expected. + * Just add a defensive code to avoid the topic can not be unloaded anymore: remove the consumers which + * are not mismatch with {@link #consumerSet}. See more detail: https://github.com/apache/pulsar/pull/22270. + */ + log.error("[{}] Trying to remove a non-connected consumer: {}", name, consumer); + consumerList.removeIf(c -> consumer.equals(c)); + if (consumerList.isEmpty()) { + clearComponentsAfterRemovedAllConsumers(); + } + } + } + + private synchronized void clearComponentsAfterRemovedAllConsumers() { + cancelPendingRead(); + + redeliveryMessages.clear(); + redeliveryTracker.clear(); + if (closeFuture != null) { + log.info("[{}] All consumers removed. Subscription is disconnected", name); + closeFuture.complete(null); + } + totalAvailablePermits = 0; + } + + @Override + public void consumerFlow(Consumer consumer, int additionalNumberOfMessages) { + topic.getBrokerService().executor().execute(() -> { + internalConsumerFlow(consumer, additionalNumberOfMessages); + }); + } + + private synchronized void internalConsumerFlow(Consumer consumer, int additionalNumberOfMessages) { + if (!consumerSet.contains(consumer)) { + if (log.isDebugEnabled()) { + log.debug("[{}] Ignoring flow control from disconnected consumer {}", name, consumer); + } + return; + } + + totalAvailablePermits += additionalNumberOfMessages; + + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Trigger new read after receiving flow control message with permits {} " + + "after adding {} permits", name, consumer, + totalAvailablePermits, additionalNumberOfMessages); + } + readMoreEntries(); + } + + /** + * We should not call readMoreEntries() recursively in the same thread as there is a risk of StackOverflowError. + * + */ + public void readMoreEntriesAsync() { + topic.getBrokerService().executor().execute(this::readMoreEntries); + } + + public synchronized void readMoreEntries() { + if (cursor.isClosed()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor is already closed, skipping read more entries.", cursor.getName()); + } + return; + } + if (isSendInProgress()) { + // we cannot read more entries while sending the previous batch + // otherwise we could re-read the same entries and send duplicates + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Skipping read for the topic, Due to sending in-progress.", + topic.getName(), getSubscriptionName()); + } + return; + } + if (shouldPauseDeliveryForDelayTracker()) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Skipping read for the topic, Due to pause delivery for delay tracker.", + topic.getName(), getSubscriptionName()); + } + return; + } + if (topic.isTransferring()) { + // Do not deliver messages for topics that are undergoing transfer, as the acknowledgments would be ignored. + return; + } + + // totalAvailablePermits may be updated by other threads + int firstAvailableConsumerPermits = getFirstAvailableConsumerPermits(); + int currentTotalAvailablePermits = Math.max(totalAvailablePermits, firstAvailableConsumerPermits); + if (currentTotalAvailablePermits > 0 && firstAvailableConsumerPermits > 0) { + Pair calculateResult = calculateToRead(currentTotalAvailablePermits); + int messagesToRead = calculateResult.getLeft(); + long bytesToRead = calculateResult.getRight(); + + if (messagesToRead == -1 || bytesToRead == -1) { + // Skip read as topic/dispatcher has exceed the dispatch rate or previous pending read hasn't complete. + return; + } + + NavigableSet messagesToReplayNow = getMessagesToReplayNow(messagesToRead); + NavigableSet messagesToReplayFiltered = filterOutEntriesWillBeDiscarded(messagesToReplayNow); + if (!messagesToReplayFiltered.isEmpty()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Schedule replay of {} messages for {} consumers", name, + messagesToReplayFiltered.size(), consumerList.size()); + } + + havePendingReplayRead = true; + minReplayedPosition = messagesToReplayNow.first(); + Set deletedMessages = topic.isDelayedDeliveryEnabled() + ? asyncReplayEntriesInOrder(messagesToReplayFiltered) + : asyncReplayEntries(messagesToReplayFiltered); + // clear already acked positions from replay bucket + + deletedMessages.forEach(position -> redeliveryMessages.remove(((Position) position).getLedgerId(), + ((Position) position).getEntryId())); + // if all the entries are acked-entries and cleared up from redeliveryMessages, try to read + // next entries as readCompletedEntries-callback was never called + if ((messagesToReplayFiltered.size() - deletedMessages.size()) == 0) { + havePendingReplayRead = false; + readMoreEntriesAsync(); + } + } else if (BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.get(this) == TRUE) { + if (log.isDebugEnabled()) { + log.debug("[{}] Dispatcher read is blocked due to unackMessages {} reached to max {}", name, + totalUnackedMessages, topic.getMaxUnackedMessagesOnSubscription()); + } + } else if (!havePendingRead && hasConsumersNeededNormalRead()) { + if (shouldPauseOnAckStatePersist(ReadType.Normal)) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Skipping read for the topic, Due to blocked on ack state persistent.", + topic.getName(), getSubscriptionName()); + } + return; + } + if (log.isDebugEnabled()) { + log.debug("[{}] Schedule read of {} messages for {} consumers", name, messagesToRead, + consumerList.size()); + } + havePendingRead = true; + NavigableSet toReplay = getMessagesToReplayNow(1); + if (!toReplay.isEmpty()) { + minReplayedPosition = toReplay.first(); + redeliveryMessages.add(minReplayedPosition.getLedgerId(), minReplayedPosition.getEntryId()); + } else { + minReplayedPosition = null; + } + + // Filter out and skip read delayed messages exist in DelayedDeliveryTracker + if (delayedDeliveryTracker.isPresent()) { + Predicate skipCondition = null; + final DelayedDeliveryTracker deliveryTracker = delayedDeliveryTracker.get(); + if (deliveryTracker instanceof BucketDelayedDeliveryTracker) { + skipCondition = position -> ((BucketDelayedDeliveryTracker) deliveryTracker) + .containsMessage(position.getLedgerId(), position.getEntryId()); + } + cursor.asyncReadEntriesWithSkipOrWait(messagesToRead, bytesToRead, this, ReadType.Normal, + topic.getMaxReadPosition(), skipCondition); + } else { + cursor.asyncReadEntriesOrWait(messagesToRead, bytesToRead, this, ReadType.Normal, + topic.getMaxReadPosition()); + } + } else { + if (log.isDebugEnabled()) { + if (!messagesToReplayNow.isEmpty()) { + log.debug("[{}] [{}] Skipping read for the topic: because all entries in replay queue were" + + " filtered out due to the mechanism of Key_Shared mode, and the left consumers have" + + " no permits now", + topic.getName(), getSubscriptionName()); + } else { + log.debug("[{}] Cannot schedule next read until previous one is done", name); + } + } + } + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Consumer buffer is full, pause reading", name); + } + } + } + + private boolean shouldPauseOnAckStatePersist(ReadType readType) { + // Allows new consumers to consume redelivered messages caused by the just-closed consumer. + if (readType != ReadType.Normal) { + return false; + } + if (!((PersistentTopic) subscription.getTopic()).isDispatcherPauseOnAckStatePersistentEnabled()) { + return false; + } + if (cursor == null) { + return true; + } + return blockedDispatcherOnCursorDataCanNotFullyPersist == TRUE; + } + + @Override + protected void reScheduleRead() { + if (isRescheduleReadInProgress.compareAndSet(false, true)) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Reschedule message read in {} ms", topic.getName(), name, MESSAGE_RATE_BACKOFF_MS); + } + topic.getBrokerService().executor().schedule( + () -> { + isRescheduleReadInProgress.set(false); + readMoreEntries(); + }, + MESSAGE_RATE_BACKOFF_MS, TimeUnit.MILLISECONDS); + } + } + + // left pair is messagesToRead, right pair is bytesToRead + protected Pair calculateToRead(int currentTotalAvailablePermits) { + int messagesToRead = Math.min(currentTotalAvailablePermits, readBatchSize); + long bytesToRead = serviceConfig.getDispatcherMaxReadSizeBytes(); + + Consumer c = getRandomConsumer(); + // if turn on precise dispatcher flow control, adjust the record to read + if (c != null && c.isPreciseDispatcherFlowControl()) { + int avgMessagesPerEntry = Math.max(1, c.getAvgMessagesPerEntry()); + messagesToRead = Math.min( + (int) Math.ceil(currentTotalAvailablePermits * 1.0 / avgMessagesPerEntry), + readBatchSize); + } + + if (!isConsumerWritable()) { + // If the connection is not currently writable, we issue the read request anyway, but for a single + // message. The intent here is to keep use the request as a notification mechanism while avoiding to + // read and dispatch a big batch of messages which will need to wait before getting written to the + // socket. + messagesToRead = 1; + } + + // throttle only if: (1) cursor is not active (or flag for throttle-nonBacklogConsumer is enabled) bcz + // active-cursor reads message from cache rather from bookkeeper (2) if topic has reached message-rate + // threshold: then schedule the read after MESSAGE_RATE_BACKOFF_MS + if (serviceConfig.isDispatchThrottlingOnNonBacklogConsumerEnabled() || !cursor.isActive()) { + if (topic.getBrokerDispatchRateLimiter().isPresent()) { + DispatchRateLimiter brokerRateLimiter = topic.getBrokerDispatchRateLimiter().get(); + Pair calculateToRead = + updateMessagesToRead(brokerRateLimiter, messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] message-read exceeded broker message-rate {}/{}, schedule after a {}", name, + brokerRateLimiter.getDispatchRateOnMsg(), brokerRateLimiter.getDispatchRateOnByte(), + MESSAGE_RATE_BACKOFF_MS); + } + reScheduleRead(); + return Pair.of(-1, -1L); + } + } + + if (topic.getDispatchRateLimiter().isPresent()) { + DispatchRateLimiter topicRateLimiter = topic.getDispatchRateLimiter().get(); + Pair calculateToRead = + updateMessagesToRead(topicRateLimiter, messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] message-read exceeded topic message-rate {}/{}, schedule after a {}", name, + topicRateLimiter.getDispatchRateOnMsg(), topicRateLimiter.getDispatchRateOnByte(), + MESSAGE_RATE_BACKOFF_MS); + } + reScheduleRead(); + return Pair.of(-1, -1L); + } + } + + if (dispatchRateLimiter.isPresent()) { + Pair calculateToRead = + updateMessagesToRead(dispatchRateLimiter.get(), messagesToRead, bytesToRead); + messagesToRead = calculateToRead.getLeft(); + bytesToRead = calculateToRead.getRight(); + if (messagesToRead == 0 || bytesToRead == 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] message-read exceeded subscription message-rate {}/{}, schedule after a {}", + name, dispatchRateLimiter.get().getDispatchRateOnMsg(), + dispatchRateLimiter.get().getDispatchRateOnByte(), + MESSAGE_RATE_BACKOFF_MS); + } + reScheduleRead(); + return Pair.of(-1, -1L); + } + } + } + + if (havePendingReplayRead) { + if (log.isDebugEnabled()) { + log.debug("[{}] Skipping replay while awaiting previous read to complete", name); + } + return Pair.of(-1, -1L); + } + + // If messagesToRead is 0 or less, correct it to 1 to prevent IllegalArgumentException + messagesToRead = Math.max(messagesToRead, 1); + bytesToRead = Math.max(bytesToRead, 1); + return Pair.of(messagesToRead, bytesToRead); + } + + protected Set asyncReplayEntries(Set positions) { + return cursor.asyncReplayEntries(positions, this, ReadType.Replay); + } + + protected Set asyncReplayEntriesInOrder(Set positions) { + return cursor.asyncReplayEntries(positions, this, ReadType.Replay, true); + } + + @Override + public boolean isConsumerConnected() { + return !consumerList.isEmpty(); + } + + @Override + public CopyOnWriteArrayList getConsumers() { + return consumerList; + } + + @Override + public synchronized boolean canUnsubscribe(Consumer consumer) { + return consumerList.size() == 1 && consumerSet.contains(consumer); + } + + @Override + public CompletableFuture close(boolean disconnectConsumers, + Optional assignedBrokerLookupData) { + IS_CLOSED_UPDATER.set(this, TRUE); + + Optional delayedDeliveryTracker; + synchronized (this) { + delayedDeliveryTracker = this.delayedDeliveryTracker; + this.delayedDeliveryTracker = Optional.empty(); + } + + delayedDeliveryTracker.ifPresent(DelayedDeliveryTracker::close); + dispatchRateLimiter.ifPresent(DispatchRateLimiter::close); + + return disconnectConsumers + ? disconnectAllConsumers(false, assignedBrokerLookupData) : CompletableFuture.completedFuture(null); + } + + @Override + public synchronized CompletableFuture disconnectAllConsumers( + boolean isResetCursor, Optional assignedBrokerLookupData) { + closeFuture = new CompletableFuture<>(); + if (consumerList.isEmpty()) { + closeFuture.complete(null); + } else { + // Iterator of CopyOnWriteArrayList uses the internal array to do the for-each, and CopyOnWriteArrayList + // will create a new internal array when adding/removing a new item. So remove items in the for-each + // block is safety when the for-each and add/remove are using a same lock. + consumerList.forEach(consumer -> consumer.disconnect(isResetCursor, assignedBrokerLookupData)); + cancelPendingRead(); + } + return closeFuture; + } + + @Override + protected void cancelPendingRead() { + if ((havePendingRead || havePendingReplayRead) && cursor.cancelPendingReadRequest()) { + havePendingRead = false; + havePendingReplayRead = false; + } + } + + @Override + public CompletableFuture disconnectActiveConsumers(boolean isResetCursor) { + return disconnectAllConsumers(isResetCursor); + } + + @Override + public synchronized void resetCloseFuture() { + closeFuture = null; + } + + @Override + public void reset() { + resetCloseFuture(); + IS_CLOSED_UPDATER.set(this, FALSE); + } + + @Override + public SubType getType() { + return SubType.Shared; + } + + @Override + public final synchronized void readEntriesComplete(List entries, Object ctx) { + ReadType readType = (ReadType) ctx; + if (readType == ReadType.Normal) { + havePendingRead = false; + } else { + havePendingReplayRead = false; + } + + if (readBatchSize < serviceConfig.getDispatcherMaxReadBatchSize()) { + int newReadBatchSize = Math.min(readBatchSize * 2, serviceConfig.getDispatcherMaxReadBatchSize()); + if (log.isDebugEnabled()) { + log.debug("[{}] Increasing read batch size from {} to {}", name, readBatchSize, newReadBatchSize); + } + + readBatchSize = newReadBatchSize; + } + + readFailureBackoff.reduceToHalf(); + + if (shouldRewindBeforeReadingOrReplaying && readType == ReadType.Normal) { + // All consumers got disconnected before the completion of the read operation + entries.forEach(Entry::release); + cursor.rewind(); + shouldRewindBeforeReadingOrReplaying = false; + readMoreEntries(); + return; + } + + if (log.isDebugEnabled()) { + log.debug("[{}] Distributing {} messages to {} consumers", name, entries.size(), consumerList.size()); + } + + long size = entries.stream().mapToLong(Entry::getLength).sum(); + updatePendingBytesToDispatch(size); + + // dispatch messages to a separate thread, but still in order for this subscription + // sendMessagesToConsumers is responsible for running broker-side filters + // that may be quite expensive + if (serviceConfig.isDispatcherDispatchMessagesInSubscriptionThread()) { + // setting sendInProgress here, because sendMessagesToConsumers will be executed + // in a separate thread, and we want to prevent more reads + acquireSendInProgress(); + dispatchMessagesThread.execute(() -> { + if (sendMessagesToConsumers(readType, entries, false)) { + updatePendingBytesToDispatch(-size); + readMoreEntries(); + } else { + updatePendingBytesToDispatch(-size); + } + }); + } else { + if (sendMessagesToConsumers(readType, entries, true)) { + updatePendingBytesToDispatch(-size); + readMoreEntriesAsync(); + } else { + updatePendingBytesToDispatch(-size); + } + } + } + + protected synchronized void acquireSendInProgress() { + sendInProgress = true; + } + + protected synchronized void releaseSendInProgress() { + sendInProgress = false; + } + + protected synchronized boolean isSendInProgress() { + return sendInProgress; + } + + protected final synchronized boolean sendMessagesToConsumers(ReadType readType, List entries, + boolean needAcquireSendInProgress) { + if (needAcquireSendInProgress) { + acquireSendInProgress(); + } + try { + return trySendMessagesToConsumers(readType, entries); + } finally { + releaseSendInProgress(); + } + } + + /** + * Dispatch the messages to the Consumers. + * @return true if you want to trigger a new read. + * This method is overridden by other classes, please take a look to other implementations + * if you need to change it. + */ + protected synchronized boolean trySendMessagesToConsumers(ReadType readType, List entries) { + if (needTrimAckedMessages()) { + cursor.trimDeletedEntries(entries); + } + + int entriesToDispatch = entries.size(); + // Trigger read more messages + if (entriesToDispatch == 0) { + return true; + } + final MessageMetadata[] metadataArray = new MessageMetadata[entries.size()]; + int remainingMessages = 0; + boolean hasChunk = false; + for (int i = 0; i < metadataArray.length; i++) { + final MessageMetadata metadata = Commands.peekAndCopyMessageMetadata( + entries.get(i).getDataBuffer(), subscription.toString(), -1); + if (metadata != null) { + remainingMessages += metadata.getNumMessagesInBatch(); + if (!hasChunk && metadata.hasUuid()) { + hasChunk = true; + } + } + metadataArray[i] = metadata; + } + if (hasChunk) { + return sendChunkedMessagesToConsumers(readType, entries, metadataArray); + } + + int start = 0; + long totalMessagesSent = 0; + long totalBytesSent = 0; + long totalEntries = 0; + int avgBatchSizePerMsg = remainingMessages > 0 ? Math.max(remainingMessages / entries.size(), 1) : 1; + + // If the dispatcher is closed, firstAvailableConsumerPermits will be 0, which skips dispatching the + // messages. + while (entriesToDispatch > 0 && isAtleastOneConsumerAvailable()) { + Consumer c = getNextConsumer(); + if (c == null) { + // Do nothing, cursor will be rewind at reconnection + log.info("[{}] rewind because no available consumer found from total {}", name, consumerList.size()); + entries.subList(start, entries.size()).forEach(Entry::release); + cursor.rewind(); + return false; + } + + // round-robin dispatch batch size for this consumer + int availablePermits = c.isWritable() ? c.getAvailablePermits() : 1; + if (c.getMaxUnackedMessages() > 0) { + // Avoid negative number + int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0); + availablePermits = Math.min(availablePermits, remainUnAckedMessages); + } + if (log.isDebugEnabled() && !c.isWritable()) { + log.debug("[{}-{}] consumer is not writable. dispatching only 1 message to {}; " + + "availablePermits are {}", topic.getName(), name, + c, c.getAvailablePermits()); + } + + int messagesForC = Math.min(Math.min(remainingMessages, availablePermits), + serviceConfig.getDispatcherMaxRoundRobinBatchSize()); + messagesForC = Math.max(messagesForC / avgBatchSizePerMsg, 1); + + int end = Math.min(start + messagesForC, entries.size()); + List entriesForThisConsumer = entries.subList(start, end); + + // remove positions first from replay list first : sendMessages recycles entries + if (readType == ReadType.Replay) { + entriesForThisConsumer.forEach(entry -> { + redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId()); + }); + } + + SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + + EntryBatchSizes batchSizes = EntryBatchSizes.get(entriesForThisConsumer.size()); + EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(entriesForThisConsumer.size()); + totalEntries += filterEntriesForConsumer(metadataArray, start, + entriesForThisConsumer, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, + readType == ReadType.Replay, c); + + c.sendMessages(entriesForThisConsumer, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), + sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), redeliveryTracker); + + int msgSent = sendMessageInfo.getTotalMessages(); + remainingMessages -= msgSent; + start += messagesForC; + entriesToDispatch -= messagesForC; + TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, + -(msgSent - batchIndexesAcks.getTotalAckedIndexCount())); + if (log.isDebugEnabled()) { + log.debug("[{}] Added -({} minus {}) permits to TOTAL_AVAILABLE_PERMITS_UPDATER in " + + "PersistentDispatcherMultipleConsumers", + name, msgSent, batchIndexesAcks.getTotalAckedIndexCount()); + } + totalMessagesSent += sendMessageInfo.getTotalMessages(); + totalBytesSent += sendMessageInfo.getTotalBytes(); + } + + acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); + + if (entriesToDispatch > 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] No consumers found with available permits, storing {} positions for later replay", name, + entries.size() - start); + } + entries.subList(start, entries.size()).forEach(entry -> { + long stickyKeyHash = getStickyKeyHash(entry); + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + }); + } + return true; + } + + private boolean sendChunkedMessagesToConsumers(ReadType readType, + List entries, + MessageMetadata[] metadataArray) { + final List originalEntryAndMetadataList = new ArrayList<>(metadataArray.length); + for (int i = 0; i < metadataArray.length; i++) { + originalEntryAndMetadataList.add(EntryAndMetadata.create(entries.get(i), metadataArray[i])); + } + + final Map> assignResult = + assignor.assign(originalEntryAndMetadataList, consumerList.size()); + long totalMessagesSent = 0; + long totalBytesSent = 0; + long totalEntries = 0; + final AtomicInteger numConsumers = new AtomicInteger(assignResult.size()); + for (Map.Entry> current : assignResult.entrySet()) { + final Consumer consumer = current.getKey(); + final List entryAndMetadataList = current.getValue(); + final int messagesForC = Math.min(consumer.getAvailablePermits(), entryAndMetadataList.size()); + if (log.isDebugEnabled()) { + log.debug("[{}] select consumer {} with messages num {}, read type is {}", + name, consumer.consumerName(), messagesForC, readType); + } + if (messagesForC < entryAndMetadataList.size()) { + for (int i = messagesForC; i < entryAndMetadataList.size(); i++) { + final EntryAndMetadata entry = entryAndMetadataList.get(i); + addMessageToReplay(entry); + entryAndMetadataList.set(i, null); + } + } + if (messagesForC == 0) { + numConsumers.decrementAndGet(); + continue; + } + if (readType == ReadType.Replay) { + entryAndMetadataList.stream().limit(messagesForC) + .forEach(e -> redeliveryMessages.remove(e.getLedgerId(), e.getEntryId())); + } + final SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + final EntryBatchSizes batchSizes = EntryBatchSizes.get(messagesForC); + final EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(messagesForC); + + totalEntries += filterEntriesForConsumer(entryAndMetadataList, batchSizes, sendMessageInfo, + batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); + consumer.sendMessages(entryAndMetadataList, batchSizes, batchIndexesAcks, + sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), + sendMessageInfo.getTotalChunkedMessages(), getRedeliveryTracker() + ).addListener(future -> { + if (future.isDone() && numConsumers.decrementAndGet() == 0) { + readMoreEntries(); + } + }); + + TOTAL_AVAILABLE_PERMITS_UPDATER.getAndAdd(this, + -(sendMessageInfo.getTotalMessages() - batchIndexesAcks.getTotalAckedIndexCount())); + totalMessagesSent += sendMessageInfo.getTotalMessages(); + totalBytesSent += sendMessageInfo.getTotalBytes(); + } + + acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); + + return numConsumers.get() == 0; // trigger a new readMoreEntries() call + } + + @Override + public synchronized void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + + ReadType readType = (ReadType) ctx; + long waitTimeMillis = readFailureBackoff.next(); + + // Do not keep reading more entries if the cursor is already closed. + if (exception instanceof ManagedLedgerException.CursorAlreadyClosedException) { + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor is already closed, skipping read more entries", cursor.getName()); + } + // Set the wait time to -1 to avoid rescheduling the read. + waitTimeMillis = -1; + } else if (exception instanceof NoMoreEntriesToReadException) { + if (cursor.getNumberOfEntriesInBacklog(false) == 0) { + // Topic has been terminated and there are no more entries to read + // Notify the consumer only if all the messages were already acknowledged + checkAndApplyReachedEndOfTopicOrTopicMigration(consumerList); + } + } else if (exception.getCause() instanceof TransactionBufferException.TransactionNotSealedException + || exception.getCause() instanceof ManagedLedgerException.OffloadReadHandleClosedException) { + waitTimeMillis = 1; + if (log.isDebugEnabled()) { + log.debug("[{}] Error reading transaction entries : {}, Read Type {} - Retrying to read in {} seconds", + name, exception.getMessage(), readType, waitTimeMillis / 1000.0); + } + } else if (!(exception instanceof TooManyRequestsException)) { + log.error("[{}] Error reading entries at {} : {}, Read Type {} - Retrying to read in {} seconds", name, + cursor.getReadPosition(), exception.getMessage(), readType, waitTimeMillis / 1000.0); + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Error reading entries at {} : {}, Read Type {} - Retrying to read in {} seconds", name, + cursor.getReadPosition(), exception.getMessage(), readType, waitTimeMillis / 1000.0); + } + } + + if (shouldRewindBeforeReadingOrReplaying) { + shouldRewindBeforeReadingOrReplaying = false; + cursor.rewind(); + } + + if (readType == ReadType.Normal) { + havePendingRead = false; + } else { + havePendingReplayRead = false; + if (exception instanceof ManagedLedgerException.InvalidReplayPositionException) { + Position markDeletePosition = (Position) cursor.getMarkDeletedPosition(); + redeliveryMessages.removeAllUpTo(markDeletePosition.getLedgerId(), markDeletePosition.getEntryId()); + } + } + + readBatchSize = serviceConfig.getDispatcherMinReadBatchSize(); + // Skip read if the waitTimeMillis is a nagetive value. + if (waitTimeMillis >= 0) { + scheduleReadEntriesWithDelay(exception, readType, waitTimeMillis); + } + } + + @VisibleForTesting + void scheduleReadEntriesWithDelay(Exception e, ReadType readType, long waitTimeMillis) { + topic.getBrokerService().executor().schedule(() -> { + synchronized (PersistentDispatcherMultipleConsumersClassic.this) { + // If it's a replay read we need to retry even if there's already + // another scheduled read, otherwise we'd be stuck until + // more messages are published. + if (!havePendingRead || readType == ReadType.Replay) { + log.info("[{}] Retrying read operation", name); + readMoreEntries(); + } else { + log.info("[{}] Skipping read retry: havePendingRead {}", name, havePendingRead, e); + } + } + }, waitTimeMillis, TimeUnit.MILLISECONDS); + } + + private boolean needTrimAckedMessages() { + if (lastIndividualDeletedRangeFromCursorRecovery == null) { + return false; + } else { + return lastIndividualDeletedRangeFromCursorRecovery.upperEndpoint() + .compareTo((Position) cursor.getReadPosition()) > 0; + } + } + + /** + * returns true only if {@link AbstractDispatcherMultipleConsumers#consumerList} + * has atleast one unblocked consumer and have available permits. + * + * @return + */ + protected boolean isAtleastOneConsumerAvailable() { + return getFirstAvailableConsumerPermits() > 0; + } + + protected int getFirstAvailableConsumerPermits() { + if (consumerList.isEmpty() || IS_CLOSED_UPDATER.get(this) == TRUE) { + // abort read if no consumers are connected or if disconnect is initiated + return 0; + } + for (Consumer consumer : consumerList) { + if (consumer != null && !consumer.isBlocked()) { + int availablePermits = consumer.getAvailablePermits(); + if (availablePermits > 0) { + return availablePermits; + } + } + } + return 0; + } + + private boolean isConsumerWritable() { + for (Consumer consumer : consumerList) { + if (consumer.isWritable()) { + return true; + } + } + if (log.isDebugEnabled()) { + log.debug("[{}-{}] consumer is not writable", topic.getName(), name); + } + return false; + } + + @Override + public boolean isConsumerAvailable(Consumer consumer) { + return consumer != null && !consumer.isBlocked() && consumer.getAvailablePermits() > 0; + } + + @Override + public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, long consumerEpoch) { + consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, stickyKeyHash) -> { + if (addMessageToReplay(ledgerId, entryId, stickyKeyHash)) { + redeliveryTracker.incrementAndGetRedeliveryCount((PositionFactory.create(ledgerId, entryId))); + } + }); + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, + redeliveryMessages); + } + readMoreEntries(); + } + + @Override + public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List positions) { + positions.forEach(position -> { + // TODO: We want to pass a sticky key hash as a third argument to guarantee the order of the messages + // on Key_Shared subscription, but it's difficult to get the sticky key here + if (addMessageToReplay(position.getLedgerId(), position.getEntryId())) { + redeliveryTracker.incrementAndGetRedeliveryCount(position); + } + }); + if (log.isDebugEnabled()) { + log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, positions); + } + readMoreEntries(); + } + + @Override + public void addUnAckedMessages(int numberOfMessages) { + int maxUnackedMessages = topic.getMaxUnackedMessagesOnSubscription(); + // don't block dispatching if maxUnackedMessages = 0 + if (maxUnackedMessages <= 0 && blockedDispatcherOnUnackedMsgs == TRUE + && BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.compareAndSet(this, TRUE, FALSE)) { + log.info("[{}] Dispatcher is unblocked, since maxUnackedMessagesPerSubscription=0", name); + readMoreEntriesAsync(); + } + + int unAckedMessages = TOTAL_UNACKED_MESSAGES_UPDATER.addAndGet(this, numberOfMessages); + if (unAckedMessages >= maxUnackedMessages && maxUnackedMessages > 0 + && BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.compareAndSet(this, FALSE, TRUE)) { + // block dispatcher if it reaches maxUnAckMsg limit + log.debug("[{}] Dispatcher is blocked due to unackMessages {} reached to max {}", name, + unAckedMessages, maxUnackedMessages); + } else if (topic.getBrokerService().isBrokerDispatchingBlocked() + && blockedDispatcherOnUnackedMsgs == TRUE) { + // unblock dispatcher: if dispatcher is blocked due to broker-unackMsg limit and if it ack back enough + // messages + if (totalUnackedMessages < (topic.getBrokerService().maxUnackedMsgsPerDispatcher / 2)) { + if (BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.compareAndSet(this, TRUE, FALSE)) { + // it removes dispatcher from blocked list and unblocks dispatcher by scheduling read + topic.getBrokerService().unblockDispatchersOnUnAckMessages(Lists.newArrayList(this)); + } + } + } else if (blockedDispatcherOnUnackedMsgs == TRUE && unAckedMessages < maxUnackedMessages / 2) { + // unblock dispatcher if it acks back enough messages + if (BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER.compareAndSet(this, TRUE, FALSE)) { + log.debug("[{}] Dispatcher is unblocked", name); + readMoreEntriesAsync(); + } + } + // increment broker-level count + topic.getBrokerService().addUnAckedMessages(this, numberOfMessages); + } + + @Override + public void afterAckMessages(Throwable exOfDeletion, Object ctxOfDeletion) { + boolean unPaused = blockedDispatcherOnCursorDataCanNotFullyPersist == FALSE; + // Trigger a new read if needed. + boolean shouldPauseNow = !checkAndResumeIfPaused(); + // Switch stat to "paused" if needed. + if (unPaused && shouldPauseNow) { + if (!BLOCKED_DISPATCHER_ON_CURSOR_DATA_CAN_NOT_FULLY_PERSIST_UPDATER + .compareAndSet(this, FALSE, TRUE)) { + // Retry due to conflict update. + afterAckMessages(exOfDeletion, ctxOfDeletion); + } + } + } + + @Override + public boolean checkAndResumeIfPaused() { + boolean paused = blockedDispatcherOnCursorDataCanNotFullyPersist == TRUE; + // Calling "cursor.isCursorDataFullyPersistable()" will loop the collection "individualDeletedMessages". It is + // not a light method. + // If never enabled "dispatcherPauseOnAckStatePersistentEnabled", skip the following checks to improve + // performance. + if (!paused && !topic.isDispatcherPauseOnAckStatePersistentEnabled()){ + // "true" means no need to pause. + return true; + } + // Enabled "dispatcherPauseOnAckStatePersistentEnabled" before. + boolean shouldPauseNow = !cursor.isCursorDataFullyPersistable() + && topic.isDispatcherPauseOnAckStatePersistentEnabled(); + // No need to change. + if (paused == shouldPauseNow) { + return !shouldPauseNow; + } + // Should change to "un-pause". + if (paused && !shouldPauseNow) { + // If there was no previous pause due to cursor data is too large to persist, we don't need to manually + // trigger a new read. This can avoid too many CPU circles. + if (BLOCKED_DISPATCHER_ON_CURSOR_DATA_CAN_NOT_FULLY_PERSIST_UPDATER.compareAndSet(this, TRUE, FALSE)) { + readMoreEntriesAsync(); + } else { + // Retry due to conflict update. + checkAndResumeIfPaused(); + } + } + return !shouldPauseNow; + } + + public boolean isBlockedDispatcherOnUnackedMsgs() { + return blockedDispatcherOnUnackedMsgs == TRUE; + } + + public void blockDispatcherOnUnackedMsgs() { + blockedDispatcherOnUnackedMsgs = TRUE; + } + + public void unBlockDispatcherOnUnackedMsgs() { + blockedDispatcherOnUnackedMsgs = FALSE; + } + + public int getTotalUnackedMessages() { + return totalUnackedMessages; + } + + public String getName() { + return name; + } + + @Override + public RedeliveryTracker getRedeliveryTracker() { + return redeliveryTracker; + } + + @Override + public Optional getRateLimiter() { + return dispatchRateLimiter; + } + + @Override + public boolean initializeDispatchRateLimiterIfNeeded() { + if (!dispatchRateLimiter.isPresent() && DispatchRateLimiter.isDispatchRateEnabled( + topic.getSubscriptionDispatchRate(getSubscriptionName()))) { + this.dispatchRateLimiter = + Optional.of(new DispatchRateLimiter(topic, getSubscriptionName(), Type.SUBSCRIPTION)); + return true; + } + return false; + } + + @Override + public boolean trackDelayedDelivery(long ledgerId, long entryId, MessageMetadata msgMetadata) { + if (!topic.isDelayedDeliveryEnabled()) { + // If broker has the feature disabled, always deliver messages immediately + return false; + } + + synchronized (this) { + if (delayedDeliveryTracker.isEmpty()) { + if (!msgMetadata.hasDeliverAtTime()) { + // No need to initialize the tracker here + return false; + } + + // Initialize the tracker the first time we need to use it + delayedDeliveryTracker = Optional.of( + topic.getBrokerService().getDelayedDeliveryTrackerFactory().newTracker(this)); + } + + delayedDeliveryTracker.get().resetTickTime(topic.getDelayedDeliveryTickTimeMillis()); + + long deliverAtTime = msgMetadata.hasDeliverAtTime() ? msgMetadata.getDeliverAtTime() : -1L; + return delayedDeliveryTracker.get().addMessage(ledgerId, entryId, deliverAtTime); + } + } + + protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { + if (delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().hasMessageAvailable()) { + delayedDeliveryTracker.get().resetTickTime(topic.getDelayedDeliveryTickTimeMillis()); + NavigableSet messagesAvailableNow = + delayedDeliveryTracker.get().getScheduledMessages(maxMessagesToRead); + messagesAvailableNow.forEach(p -> redeliveryMessages.add(p.getLedgerId(), p.getEntryId())); + } + + if (!redeliveryMessages.isEmpty()) { + return redeliveryMessages.getMessagesToReplayNow(maxMessagesToRead, position -> true); + } else { + return Collections.emptyNavigableSet(); + } + } + + /** + * This is a mode method designed for Key_Shared mode. + * Filter out the entries that will be discarded due to the order guarantee mechanism of Key_Shared mode. + * This method is in order to avoid the scenario below: + * - Get positions from the Replay queue. + * - Read entries from BK. + * - The order guarantee mechanism of Key_Shared mode filtered out all the entries. + * - Delivery non entry to the client, but we did a BK read. + */ + protected NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + return src; + } + + /** + * This is a mode method designed for Key_Shared mode, to avoid unnecessary stuck. + * See detail {@link PersistentStickyKeyDispatcherMultipleConsumersClassic#hasConsumersNeededNormalRead}. + */ + protected boolean hasConsumersNeededNormalRead() { + return true; + } + + protected synchronized boolean shouldPauseDeliveryForDelayTracker() { + return delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().shouldPauseAllDeliveries(); + } + + @Override + public long getNumberOfDelayedMessages() { + return delayedDeliveryTracker.map(DelayedDeliveryTracker::getNumberOfDelayedMessages).orElse(0L); + } + + @Override + public CompletableFuture clearDelayedMessages() { + if (!topic.isDelayedDeliveryEnabled()) { + return CompletableFuture.completedFuture(null); + } + + if (delayedDeliveryTracker.isPresent()) { + return this.delayedDeliveryTracker.get().clear(); + } else { + DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory = + topic.getBrokerService().getDelayedDeliveryTrackerFactory(); + if (delayedDeliveryTrackerFactory instanceof BucketDelayedDeliveryTrackerFactory + bucketDelayedDeliveryTrackerFactory) { + return bucketDelayedDeliveryTrackerFactory.cleanResidualSnapshots(cursor); + } + return CompletableFuture.completedFuture(null); + } + } + + @Override + public void cursorIsReset() { + if (this.lastIndividualDeletedRangeFromCursorRecovery != null) { + this.lastIndividualDeletedRangeFromCursorRecovery = null; + } + } + + private void addMessageToReplay(Entry entry) { + addMessageToReplay(entry.getLedgerId(), entry.getEntryId()); + entry.release(); + } + + protected boolean addMessageToReplay(long ledgerId, long entryId, long stickyKeyHash) { + if (checkIfMessageIsUnacked(ledgerId, entryId)) { + redeliveryMessages.add(ledgerId, entryId, stickyKeyHash); + return true; + } else { + return false; + } + } + + protected boolean addMessageToReplay(long ledgerId, long entryId) { + if (checkIfMessageIsUnacked(ledgerId, entryId)) { + redeliveryMessages.add(ledgerId, entryId); + return true; + } else { + return false; + } + } + + private boolean checkIfMessageIsUnacked(long ledgerId, long entryId) { + Position markDeletePosition = cursor.getMarkDeletedPosition(); + return (markDeletePosition == null || ledgerId > markDeletePosition.getLedgerId() + || (ledgerId == markDeletePosition.getLedgerId() && entryId > markDeletePosition.getEntryId())); + } + + @Override + public boolean checkAndUnblockIfStuck() { + if (cursor.checkAndUpdateReadPositionChanged()) { + return false; + } + // consider dispatch is stuck if : dispatcher has backlog, available-permits and there is no pending read + if (totalAvailablePermits > 0 && !havePendingReplayRead && !havePendingRead + && cursor.getNumberOfEntriesInBacklog(false) > 0) { + log.warn("{}-{} Dispatcher is stuck and unblocking by issuing reads", topic.getName(), name); + readMoreEntries(); + return true; + } + return false; + } + + public PersistentTopic getTopic() { + return topic; + } + + + public long getDelayedTrackerMemoryUsage() { + return delayedDeliveryTracker.map(DelayedDeliveryTracker::getBufferMemoryUsage).orElse(0L); + } + + public Map getBucketDelayedIndexStats() { + if (delayedDeliveryTracker.isEmpty()) { + return Collections.emptyMap(); + } + + if (delayedDeliveryTracker.get() instanceof BucketDelayedDeliveryTracker) { + return ((BucketDelayedDeliveryTracker) delayedDeliveryTracker.get()).genTopicMetricMap(); + } + + return Collections.emptyMap(); + } + + @Override + public boolean isClassic() { + return true; + } + + public ManagedCursor getCursor() { + return cursor; + } + + protected int getStickyKeyHash(Entry entry) { + return StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; + } + + public Subscription getSubscription() { + return subscription; + } + + public long getNumberOfMessagesInReplay() { + return redeliveryMessages.size(); + } + + @Override + public boolean isHavePendingRead() { + return havePendingRead; + } + + @Override + public boolean isHavePendingReplayRead() { + return havePendingReplayRead; + } + + private static final Logger log = LoggerFactory.getLogger(PersistentDispatcherMultipleConsumersClassic.class); +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index 925e99ed699a2..df053e6d8a549 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -50,6 +50,7 @@ import org.apache.pulsar.broker.service.PendingAcksMap; import org.apache.pulsar.broker.service.SendMessageInfo; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.client.api.Range; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; @@ -60,7 +61,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDispatcherMultipleConsumers { +public class PersistentStickyKeyDispatcherMultipleConsumers extends PersistentDispatcherMultipleConsumers implements + StickyKeyDispatcher { private final boolean allowOutOfOrderDelivery; private final StickyKeyConsumerSelector selector; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java new file mode 100644 index 0000000000000..c227bf5b435bc --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassic.java @@ -0,0 +1,583 @@ +/* + * 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.service.persistent; + +import com.google.common.annotations.VisibleForTesting; +import io.netty.util.concurrent.FastThreadLocal; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.commons.collections4.MapUtils; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerServiceException; +import org.apache.pulsar.broker.service.ConsistentHashingStickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.EntryAndMetadata; +import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; +import org.apache.pulsar.broker.service.EntryBatchSizes; +import org.apache.pulsar.broker.service.HashRangeAutoSplitStickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.HashRangeExclusiveStickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.SendMessageInfo; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.client.api.Range; +import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; +import org.apache.pulsar.common.api.proto.KeySharedMeta; +import org.apache.pulsar.common.api.proto.KeySharedMode; +import org.apache.pulsar.common.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is the "classic" dispatcher implementation for the Key_Shared subscription that was used before + * Pulsar 4.0.0 and PIP-379. When `subscriptionKeySharedUseClassicPersistentImplementation=true`, + * this dispatcher will be used. The main purpose is to provide a way to rollback to the old behavior + * in case of issues with the preferred implementation. + */ +public class PersistentStickyKeyDispatcherMultipleConsumersClassic + extends PersistentDispatcherMultipleConsumersClassic implements StickyKeyDispatcher { + + private final boolean allowOutOfOrderDelivery; + private final StickyKeyConsumerSelector selector; + + private boolean isDispatcherStuckOnReplays = false; + private final KeySharedMode keySharedMode; + + /** + * When a consumer joins, it will be added to this map with the current read position. + * This means that, in order to preserve ordering, new consumers can only receive old + * messages, until the mark-delete position will move past this point. + */ + private final LinkedHashMap recentlyJoinedConsumers; + + PersistentStickyKeyDispatcherMultipleConsumersClassic(PersistentTopic topic, ManagedCursor cursor, + Subscription subscription, ServiceConfiguration conf, + KeySharedMeta ksm) { + super(topic, cursor, subscription, ksm.isAllowOutOfOrderDelivery()); + + this.allowOutOfOrderDelivery = ksm.isAllowOutOfOrderDelivery(); + this.recentlyJoinedConsumers = allowOutOfOrderDelivery ? null : new LinkedHashMap<>(); + this.keySharedMode = ksm.getKeySharedMode(); + switch (this.keySharedMode) { + case AUTO_SPLIT: + if (conf.isSubscriptionKeySharedUseConsistentHashing()) { + selector = new ConsistentHashingStickyKeyConsumerSelector( + conf.getSubscriptionKeySharedConsistentHashingReplicaPoints(), + false, + // Classic implementation uses Integer.MAX_VALUE - 1 as the range end value + Integer.MAX_VALUE - 1); + } else { + selector = new HashRangeAutoSplitStickyKeyConsumerSelector(); + } + break; + + case STICKY: + this.selector = new HashRangeExclusiveStickyKeyConsumerSelector(); + break; + + default: + throw new IllegalArgumentException("Invalid key-shared mode: " + keySharedMode); + } + } + + @VisibleForTesting + public StickyKeyConsumerSelector getSelector() { + return selector; + } + + @Override + public synchronized CompletableFuture addConsumer(Consumer consumer) { + if (IS_CLOSED_UPDATER.get(this) == TRUE) { + log.warn("[{}] Dispatcher is already closed. Closing consumer {}", name, consumer); + consumer.disconnect(); + return CompletableFuture.completedFuture(null); + } + return super.addConsumer(consumer).thenCompose(__ -> + selector.addConsumer(consumer).handle((result, ex) -> { + if (ex != null) { + synchronized (PersistentStickyKeyDispatcherMultipleConsumersClassic.this) { + consumerSet.removeAll(consumer); + consumerList.remove(consumer); + } + throw FutureUtil.wrapToCompletionException(ex); + } + return result; + }) + ).thenRun(() -> { + synchronized (PersistentStickyKeyDispatcherMultipleConsumersClassic.this) { + Position readPositionWhenJoining = (Position) cursor.getReadPosition(); + consumer.setReadPositionWhenJoining(readPositionWhenJoining); + // If this was the 1st consumer, or if all the messages are already acked, then we + // don't need to do anything special + if (!allowOutOfOrderDelivery + && recentlyJoinedConsumers != null + && consumerList.size() > 1 + && cursor.getNumberOfEntriesSinceFirstNotAckedMessage() > 1) { + recentlyJoinedConsumers.put(consumer, readPositionWhenJoining); + } + } + }); + } + + @Override + public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException { + // The consumer must be removed from the selector before calling the superclass removeConsumer method. + // In the superclass removeConsumer method, the pending acks that the consumer has are added to + // redeliveryMessages. If the consumer has not been removed from the selector at this point, + // the broker will try to redeliver the messages to the consumer that has already been closed. + // As a result, the messages are not redelivered to any consumer, and the mark-delete position does not move, + // eventually causing all consumers to get stuck. + selector.removeConsumer(consumer); + super.removeConsumer(consumer); + if (recentlyJoinedConsumers != null) { + recentlyJoinedConsumers.remove(consumer); + if (consumerList.size() == 1) { + recentlyJoinedConsumers.clear(); + } + if (removeConsumersFromRecentJoinedConsumers() || !redeliveryMessages.isEmpty()) { + readMoreEntries(); + } + } + } + + private static final FastThreadLocal>> localGroupedEntries = + new FastThreadLocal>>() { + @Override + protected Map> initialValue() throws Exception { + return new HashMap<>(); + } + }; + + private static final FastThreadLocal>> localGroupedPositions = + new FastThreadLocal>>() { + @Override + protected Map> initialValue() throws Exception { + return new HashMap<>(); + } + }; + + @Override + protected synchronized boolean trySendMessagesToConsumers(ReadType readType, List entries) { + long totalMessagesSent = 0; + long totalBytesSent = 0; + long totalEntries = 0; + int entriesCount = entries.size(); + + // Trigger read more messages + if (entriesCount == 0) { + return true; + } + + if (consumerSet.isEmpty()) { + entries.forEach(Entry::release); + cursor.rewind(); + return false; + } + + // A corner case that we have to retry a readMoreEntries in order to preserver order delivery. + // This may happen when consumer closed. See issue #12885 for details. + if (!allowOutOfOrderDelivery) { + NavigableSet messagesToReplayNow = this.getMessagesToReplayNow(1); + if (messagesToReplayNow != null && !messagesToReplayNow.isEmpty()) { + Position replayPosition = messagesToReplayNow.first(); + + // We have received a message potentially from the delayed tracker and, since we're not using it + // right now, it needs to be added to the redelivery tracker or we won't attempt anymore to + // resend it (until we disconnect consumer). + redeliveryMessages.add(replayPosition.getLedgerId(), replayPosition.getEntryId()); + + if (this.minReplayedPosition != null) { + // If relayPosition is a new entry wither smaller position is inserted for redelivery during this + // async read, it is possible that this relayPosition should dispatch to consumer first. So in + // order to preserver order delivery, we need to discard this read result, and try to trigger a + // replay read, that containing "relayPosition", by calling readMoreEntries. + if (replayPosition.compareTo(minReplayedPosition) < 0) { + if (log.isDebugEnabled()) { + log.debug("[{}] Position {} (<{}) is inserted for relay during current {} read, " + + "discard this read and retry with readMoreEntries.", + name, replayPosition, minReplayedPosition, readType); + } + if (readType == ReadType.Normal) { + entries.forEach(entry -> { + long stickyKeyHash = getStickyKeyHash(entry); + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + }); + } else if (readType == ReadType.Replay) { + entries.forEach(Entry::release); + } + return true; + } + } + } + } + + final Map> groupedEntries = localGroupedEntries.get(); + groupedEntries.clear(); + final Map> consumerStickyKeyHashesMap = new HashMap<>(); + + for (Entry entry : entries) { + int stickyKeyHash = getStickyKeyHash(entry); + Consumer c = selector.select(stickyKeyHash); + if (c != null) { + groupedEntries.computeIfAbsent(c, k -> new ArrayList<>()).add(entry); + consumerStickyKeyHashesMap.computeIfAbsent(c, k -> new HashSet<>()).add(stickyKeyHash); + } else { + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + } + } + + AtomicInteger keyNumbers = new AtomicInteger(groupedEntries.size()); + + int currentThreadKeyNumber = groupedEntries.size(); + if (currentThreadKeyNumber == 0) { + currentThreadKeyNumber = -1; + } + for (Map.Entry> current : groupedEntries.entrySet()) { + Consumer consumer = current.getKey(); + assert consumer != null; // checked when added to groupedEntries + List entriesWithSameKey = current.getValue(); + int entriesWithSameKeyCount = entriesWithSameKey.size(); + int availablePermits = getAvailablePermits(consumer); + int messagesForC = getRestrictedMaxEntriesForConsumer(consumer, + entriesWithSameKey.stream().map(Entry::getPosition).collect(Collectors.toList()), availablePermits, + readType, consumerStickyKeyHashesMap.get(consumer)); + if (log.isDebugEnabled()) { + log.debug("[{}] select consumer {} with messages num {}, read type is {}", + name, consumer.consumerName(), messagesForC, readType); + } + + if (messagesForC < entriesWithSameKeyCount) { + // We are not able to push all the messages with given key to its consumer, + // so we discard for now and mark them for later redelivery + for (int i = messagesForC; i < entriesWithSameKeyCount; i++) { + Entry entry = entriesWithSameKey.get(i); + long stickyKeyHash = getStickyKeyHash(entry); + addMessageToReplay(entry.getLedgerId(), entry.getEntryId(), stickyKeyHash); + entry.release(); + entriesWithSameKey.set(i, null); + } + } + + if (messagesForC > 0) { + // remove positions first from replay list first : sendMessages recycles entries + if (readType == ReadType.Replay) { + for (int i = 0; i < messagesForC; i++) { + Entry entry = entriesWithSameKey.get(i); + redeliveryMessages.remove(entry.getLedgerId(), entry.getEntryId()); + } + } + + SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); + EntryBatchSizes batchSizes = EntryBatchSizes.get(messagesForC); + EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(messagesForC); + totalEntries += filterEntriesForConsumer(entriesWithSameKey, batchSizes, sendMessageInfo, + batchIndexesAcks, cursor, readType == ReadType.Replay, consumer); + consumer.sendMessages(entriesWithSameKey, batchSizes, batchIndexesAcks, + sendMessageInfo.getTotalMessages(), + sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), + getRedeliveryTracker()).addListener(future -> { + if (future.isDone() && keyNumbers.decrementAndGet() == 0) { + readMoreEntries(); + } + }); + + TOTAL_AVAILABLE_PERMITS_UPDATER.getAndAdd(this, + -(sendMessageInfo.getTotalMessages() - batchIndexesAcks.getTotalAckedIndexCount())); + totalMessagesSent += sendMessageInfo.getTotalMessages(); + totalBytesSent += sendMessageInfo.getTotalBytes(); + } else { + currentThreadKeyNumber = keyNumbers.decrementAndGet(); + } + } + + // acquire message-dispatch permits for already delivered messages + acquirePermitsForDeliveredMessages(topic, cursor, totalEntries, totalMessagesSent, totalBytesSent); + + if (totalMessagesSent == 0 && (recentlyJoinedConsumers == null || recentlyJoinedConsumers.isEmpty())) { + // This means, that all the messages we've just read cannot be dispatched right now. + // This condition can only happen when: + // 1. We have consumers ready to accept messages (otherwise the would not haven been triggered) + // 2. All keys in the current set of messages are routing to consumers that are currently busy + // + // The solution here is to move on and read next batch of messages which might hopefully contain + // also keys meant for other consumers. + // + // We do it unless that are "recently joined consumers". In that case, we would be looking + // ahead in the stream while the new consumers are not ready to accept the new messages, + // therefore would be most likely only increase the distance between read-position and mark-delete + // position. + isDispatcherStuckOnReplays = true; + return true; + } else if (currentThreadKeyNumber == 0) { + return true; + } + return false; + } + + private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List entries, + int availablePermits, ReadType readType, Set stickyKeyHashes) { + int maxMessages = Math.min(entries.size(), availablePermits); + if (maxMessages == 0) { + return 0; + } + if (readType == ReadType.Normal && stickyKeyHashes != null + && redeliveryMessages.containsStickyKeyHashes(stickyKeyHashes)) { + // If redeliveryMessages contains messages that correspond to the same hash as the messages + // that the dispatcher is trying to send, do not send those messages for order guarantee + return 0; + } + if (recentlyJoinedConsumers == null) { + return maxMessages; + } + removeConsumersFromRecentJoinedConsumers(); + Position maxReadPosition = recentlyJoinedConsumers.get(consumer); + // At this point, all the old messages were already consumed and this consumer + // is now ready to receive any message + if (maxReadPosition == null) { + // The consumer has not recently joined, so we can send all messages + return maxMessages; + } + + // If the read type is Replay, we should avoid send messages that hold by other consumer to the new consumers, + // For example, we have 10 messages [0,1,2,3,4,5,6,7,8,9] + // If the consumer0 get message 0 and 1, and does not acked message 0, then consumer1 joined, + // when consumer1 get message 2,3, the broker will not dispatch messages to consumer1 + // because of the mark delete position did not move forward. + // So message 2,3 will stored in the redeliver tracker. + // Now, consumer2 joined, it will read new messages from the cursor, + // so the recentJoinedPosition is 4 for consumer2 + // Because of there are messages need to redeliver, so the broker will read the redelivery message first [2,3] + // message [2,3] is lower than the recentJoinedPosition 4, + // so the message [2,3] will dispatched to the consumer2 + // But the message [2,3] should not dispatch to consumer2. + + if (readType == ReadType.Replay) { + Position minReadPositionForRecentJoinedConsumer = recentlyJoinedConsumers.values().iterator().next(); + if (minReadPositionForRecentJoinedConsumer != null + && minReadPositionForRecentJoinedConsumer.compareTo(maxReadPosition) < 0) { + maxReadPosition = minReadPositionForRecentJoinedConsumer; + } + } + // Here, the consumer is one that has recently joined, so we can only send messages that were + // published before it has joined. + for (int i = 0; i < maxMessages; i++) { + if (((Position) entries.get(i)).compareTo(maxReadPosition) >= 0) { + // We have already crossed the divider line. All messages in the list are now + // newer than what we can currently dispatch to this consumer + return i; + } + } + + return maxMessages; + } + + @Override + public void markDeletePositionMoveForward() { + // Execute the notification in different thread to avoid a mutex chain here + // from the delete operation that was completed + topic.getBrokerService().getTopicOrderedExecutor().execute(() -> { + synchronized (PersistentStickyKeyDispatcherMultipleConsumersClassic.this) { + if (recentlyJoinedConsumers != null && !recentlyJoinedConsumers.isEmpty() + && removeConsumersFromRecentJoinedConsumers()) { + // After we process acks, we need to check whether the mark-delete position was advanced and we + // can finally read more messages. It's safe to call readMoreEntries() multiple times. + readMoreEntries(); + } + } + }); + } + + private boolean removeConsumersFromRecentJoinedConsumers() { + if (MapUtils.isEmpty(recentlyJoinedConsumers)) { + return false; + } + Iterator> itr = recentlyJoinedConsumers.entrySet().iterator(); + boolean hasConsumerRemovedFromTheRecentJoinedConsumers = false; + Position mdp = (Position) cursor.getMarkDeletedPosition(); + if (mdp != null) { + Position nextPositionOfTheMarkDeletePosition = + ((ManagedLedgerImpl) cursor.getManagedLedger()).getNextValidPosition(mdp); + while (itr.hasNext()) { + Map.Entry entry = itr.next(); + if (entry.getValue().compareTo(nextPositionOfTheMarkDeletePosition) <= 0) { + itr.remove(); + hasConsumerRemovedFromTheRecentJoinedConsumers = true; + } else { + break; + } + } + } + return hasConsumerRemovedFromTheRecentJoinedConsumers; + } + + @Override + protected synchronized NavigableSet getMessagesToReplayNow(int maxMessagesToRead) { + if (isDispatcherStuckOnReplays) { + // If we're stuck on replay, we want to move forward reading on the topic (until the overall max-unacked + // messages kicks in), instead of keep replaying the same old messages, since the consumer that these + // messages are routing to might be busy at the moment + this.isDispatcherStuckOnReplays = false; + return Collections.emptyNavigableSet(); + } else { + return super.getMessagesToReplayNow(maxMessagesToRead); + } + } + + private int getAvailablePermits(Consumer c) { + int availablePermits = Math.max(c.getAvailablePermits(), 0); + if (c.getMaxUnackedMessages() > 0) { + // Avoid negative number + int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0); + availablePermits = Math.min(availablePermits, remainUnAckedMessages); + } + return availablePermits; + } + + @Override + protected synchronized NavigableSet filterOutEntriesWillBeDiscarded(NavigableSet src) { + // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", + // So skip this filter out. + if (isAllowOutOfOrderDelivery()) { + return src; + } + if (src.isEmpty()) { + return src; + } + NavigableSet res = new TreeSet<>(); + // Group positions. + final Map> groupedPositions = localGroupedPositions.get(); + groupedPositions.clear(); + for (Position pos : src) { + Long stickyKeyHash = redeliveryMessages.getHash(pos.getLedgerId(), pos.getEntryId()); + if (stickyKeyHash == null) { + res.add(pos); + continue; + } + Consumer c = selector.select(stickyKeyHash.intValue()); + if (c == null) { + // Maybe using HashRangeExclusiveStickyKeyConsumerSelector. + continue; + } + groupedPositions.computeIfAbsent(c, k -> new ArrayList<>()).add(pos); + } + // Filter positions by the Recently Joined Position rule. + for (Map.Entry> item : groupedPositions.entrySet()) { + int availablePermits = getAvailablePermits(item.getKey()); + if (availablePermits == 0) { + continue; + } + int posCountToRead = getRestrictedMaxEntriesForConsumer(item.getKey(), item.getValue(), availablePermits, + ReadType.Replay, null); + if (posCountToRead > 0) { + res.addAll(item.getValue().subList(0, posCountToRead)); + } + } + return res; + } + + /** + * In Key_Shared mode, the consumer will not receive any entries from a normal reading if it is included in + * {@link #recentlyJoinedConsumers}, they can only receive entries from replay reads. + * If all entries in {@link #redeliveryMessages} have been filtered out due to the order guarantee mechanism, + * Broker need a normal read to make the consumers not included in @link #recentlyJoinedConsumers} will not be + * stuck. See https://github.com/apache/pulsar/pull/7105. + */ + @Override + protected boolean hasConsumersNeededNormalRead() { + // The variable "hashesToBeBlocked" and "recentlyJoinedConsumers" will be null if "isAllowOutOfOrderDelivery()", + // So the method "filterOutEntriesWillBeDiscarded" will filter out nothing, just return "true" here. + if (isAllowOutOfOrderDelivery()) { + return true; + } + for (Consumer consumer : consumerList) { + if (consumer == null || consumer.isBlocked()) { + continue; + } + if (recentlyJoinedConsumers.containsKey(consumer)) { + continue; + } + if (consumer.getAvailablePermits() > 0) { + return true; + } + } + return false; + } + + @Override + public SubType getType() { + return SubType.Key_Shared; + } + + @Override + protected Set asyncReplayEntries(Set positions) { + return cursor.asyncReplayEntries(positions, this, ReadType.Replay, true); + } + + public KeySharedMode getKeySharedMode() { + return this.keySharedMode; + } + + public boolean isAllowOutOfOrderDelivery() { + return this.allowOutOfOrderDelivery; + } + + public boolean hasSameKeySharedPolicy(KeySharedMeta ksm) { + return (ksm.getKeySharedMode() == this.keySharedMode + && ksm.isAllowOutOfOrderDelivery() == this.allowOutOfOrderDelivery); + } + + public LinkedHashMap getRecentlyJoinedConsumers() { + return recentlyJoinedConsumers; + } + + public Map> getConsumerKeyHashRanges() { + return selector.getConsumerKeyHashRanges(); + } + + @Override + protected int getStickyKeyHash(Entry entry) { + if (entry instanceof EntryAndMetadata entryAndMetadata) { + // use the cached sticky key hash if available, otherwise calculate the sticky key hash and cache it + return entryAndMetadata.getOrUpdateCachedStickyKeyHash(selector::makeStickyKeyHash); + } + return selector.makeStickyKeyHash(peekStickyKey(entry.getDataBuffer())); + } + + private static final Logger log = + LoggerFactory.getLogger(PersistentStickyKeyDispatcherMultipleConsumersClassic.class); + +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index b8d351bddf839..eaa147b81b126 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -25,6 +25,7 @@ import io.netty.buffer.ByteBuf; import java.io.IOException; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -69,6 +70,7 @@ import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.EntryFilterSupport; import org.apache.pulsar.broker.service.GetStatsOptions; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.plugin.EntryFilter; @@ -250,7 +252,12 @@ private CompletableFuture addConsumerInternal(Consumer consumer) { case Shared: if (dispatcher == null || dispatcher.getType() != SubType.Shared) { previousDispatcher = dispatcher; - dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, this); + ServiceConfiguration config = topic.getBrokerService().getPulsar().getConfig(); + if (config.isSubscriptionSharedUseClassicPersistentImplementation()) { + dispatcher = new PersistentDispatcherMultipleConsumersClassic(topic, cursor, this); + } else { + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, this); + } } break; case Failover: @@ -271,11 +278,19 @@ private CompletableFuture addConsumerInternal(Consumer consumer) { case Key_Shared: KeySharedMeta ksm = consumer.getKeySharedMeta(); if (dispatcher == null || dispatcher.getType() != SubType.Key_Shared - || !((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher) + || !((StickyKeyDispatcher) dispatcher) .hasSameKeySharedPolicy(ksm)) { previousDispatcher = dispatcher; - dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers(topic, cursor, this, - topic.getBrokerService().getPulsar().getConfiguration(), ksm); + ServiceConfiguration config = topic.getBrokerService().getPulsar().getConfig(); + if (config.isSubscriptionKeySharedUseClassicPersistentImplementation()) { + dispatcher = + new PersistentStickyKeyDispatcherMultipleConsumersClassic(topic, cursor, + this, + topic.getBrokerService().getPulsar().getConfiguration(), ksm); + } else { + dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers(topic, cursor, this, + topic.getBrokerService().getPulsar().getConfiguration(), ksm); + } } break; default: @@ -1221,7 +1236,7 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge Dispatcher dispatcher = this.dispatcher; if (dispatcher != null) { Map> consumerKeyHashRanges = getType() == SubType.Key_Shared - ? ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher).getConsumerKeyHashRanges() : null; + ? ((StickyKeyDispatcher) dispatcher).getConsumerKeyHashRanges() : null; dispatcher.getConsumers().forEach(consumer -> { ConsumerStatsImpl consumerStats = consumer.getStats(); if (!getStatsOptions.isExcludeConsumers()) { @@ -1260,17 +1275,18 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge } } - if (dispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (dispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subStats.delayedMessageIndexSizeInBytes = - ((PersistentDispatcherMultipleConsumers) dispatcher).getDelayedTrackerMemoryUsage(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getDelayedTrackerMemoryUsage(); subStats.bucketDelayedIndexStats = - ((PersistentDispatcherMultipleConsumers) dispatcher).getBucketDelayedIndexStats(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getBucketDelayedIndexStats(); } if (Subscription.isIndividualAckMode(subType)) { - if (dispatcher instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers d = (PersistentDispatcherMultipleConsumers) dispatcher; + if (dispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { + AbstractPersistentDispatcherMultipleConsumers d = + (AbstractPersistentDispatcherMultipleConsumers) dispatcher; subStats.unackedMessages = d.getTotalUnackedMessages(); subStats.blockedSubscriptionOnUnackedMsgs = d.isBlockedDispatcherOnUnackedMsgs(); subStats.msgDelayed = d.getNumberOfDelayedMessages(); @@ -1290,12 +1306,18 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge subStats.isReplicated = isReplicated(); subStats.subscriptionProperties = subscriptionProperties; subStats.isDurable = cursor.isDurable(); - if (getType() == SubType.Key_Shared && dispatcher instanceof PersistentStickyKeyDispatcherMultipleConsumers) { - PersistentStickyKeyDispatcherMultipleConsumers keySharedDispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) dispatcher; - + if (getType() == SubType.Key_Shared && dispatcher instanceof StickyKeyDispatcher) { + StickyKeyDispatcher keySharedDispatcher = (StickyKeyDispatcher) dispatcher; subStats.allowOutOfOrderDelivery = keySharedDispatcher.isAllowOutOfOrderDelivery(); subStats.keySharedMode = keySharedDispatcher.getKeySharedMode().toString(); + + LinkedHashMap recentlyJoinedConsumers = keySharedDispatcher + .getRecentlyJoinedConsumers(); + if (recentlyJoinedConsumers != null && recentlyJoinedConsumers.size() > 0) { + recentlyJoinedConsumers.forEach((k, v) -> { + subStats.consumersAfterMarkDeletePosition.put(k.consumerName(), v.toString()); + }); + } } subStats.nonContiguousDeletedMessagesRanges = cursor.getTotalNonContiguousDeletedMessagesRange(); subStats.nonContiguousDeletedMessagesRangesSerializedSize = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 3cce175660e70..9c86a99de0f14 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -2449,9 +2449,9 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats } if (Subscription.isIndividualAckMode(subscription.getType())) { - if (subscription.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) subscription.getDispatcher(); + if (subscription.getDispatcher() instanceof AbstractPersistentDispatcherMultipleConsumers) { + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) subscription.getDispatcher(); topicStatsStream.writePair("blockedSubscriptionOnUnackedMsgs", dispatcher.isBlockedDispatcherOnUnackedMsgs()); topicStatsStream.writePair("unackedMessages", @@ -2758,11 +2758,11 @@ public CompletableFuture getInternalStats(boolean // subscription metrics PersistentSubscription sub = subscriptions.get(Codec.decode(c.getName())); if (sub != null) { - if (sub.getDispatcher() instanceof PersistentDispatcherMultipleConsumers) { - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) sub.getDispatcher(); - cs.subscriptionHavePendingRead = dispatcher.havePendingRead; - cs.subscriptionHavePendingReplayRead = dispatcher.havePendingReplayRead; + if (sub.getDispatcher() instanceof AbstractPersistentDispatcherMultipleConsumers) { + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) sub.getDispatcher(); + cs.subscriptionHavePendingRead = dispatcher.isHavePendingRead(); + cs.subscriptionHavePendingReplayRead = dispatcher.isHavePendingReplayRead(); } else if (sub.getDispatcher() instanceof PersistentDispatcherSingleActiveConsumer) { PersistentDispatcherSingleActiveConsumer dispatcher = (PersistentDispatcherSingleActiveConsumer) sub.getDispatcher(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java index ffcc3bf0881db..7ed4542b2505f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java @@ -36,6 +36,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.BiFunction; +import java.util.stream.Stream; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -207,14 +208,28 @@ public static void receiveMessagesN(BiConsumer, Message> mess * Receive messages concurrently from multiple consumers and handles them using the provided message handler. * * @param messageHandler the message handler - * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages - * @param consumers the consumers to receive messages from - * @param the message value type + * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages + * @param consumers the consumers to receive messages from + * @param the message value type + */ + public static void receiveMessagesInThreads(BiFunction, Message, Boolean> messageHandler, + final Duration quietTimeout, + Consumer... consumers) { + receiveMessagesInThreads(messageHandler, quietTimeout, Arrays.stream(consumers).sequential()); + } + + /** + * Receive messages concurrently from multiple consumers and handles them using the provided message handler. + * + * @param messageHandler the message handler + * @param quietTimeout the duration of quiet time after which the method will stop waiting for more messages + * @param consumers the consumers to receive messages from + * @param the message value type */ public static void receiveMessagesInThreads(BiFunction, Message, Boolean> messageHandler, final Duration quietTimeout, - Consumer... consumers) { - FutureUtil.waitForAll(Arrays.stream(consumers).sequential().map(consumer -> { + Stream> consumers) { + FutureUtil.waitForAll(consumers.map(consumer -> { return CompletableFuture.runAsync(() -> { try { while (!Thread.currentThread().isInterrupted()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java index ea6ffa2d70dba..cdd0be58b34d5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/AbstractDeliveryTrackerTest.java @@ -38,7 +38,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.mledger.Position; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; @@ -49,7 +49,7 @@ public abstract class AbstractDeliveryTrackerTest { protected final Timer timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-in-memory-delayed-delivery-test"), 500, TimeUnit.MILLISECONDS); - protected PersistentDispatcherMultipleConsumers dispatcher; + protected AbstractPersistentDispatcherMultipleConsumers dispatcher; protected Clock clock; protected AtomicLong clockTime; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java index 9861ab5723732..ff6bf534129c1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.java @@ -24,6 +24,7 @@ import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -67,10 +68,10 @@ public void cleanup() throws Exception { @Test public void testFallbackToInMemoryTracker() throws Exception { - Pair pair = + Pair pair = mockDelayedDeliveryTrackerFactoryAndDispatcher(); BrokerService brokerService = pair.getLeft(); - PersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); // Since Mocked BucketDelayedDeliveryTrackerFactory.newTracker0() throws RecoverDelayedDeliveryTrackerException, // the factory should be fallback to InMemoryDelayedDeliveryTrackerFactory @@ -83,12 +84,13 @@ public void testFallbackToInMemoryTracker() throws Exception { } - private Pair mockDelayedDeliveryTrackerFactoryAndDispatcher() + private Pair mockDelayedDeliveryTrackerFactoryAndDispatcher() throws Exception { BrokerService brokerService = Mockito.spy(pulsar.getBrokerService()); // Mock dispatcher - PersistentDispatcherMultipleConsumers dispatcher = Mockito.mock(PersistentDispatcherMultipleConsumers.class); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + Mockito.mock(AbstractPersistentDispatcherMultipleConsumers.class); Mockito.doReturn("test").when(dispatcher).getName(); // Mock BucketDelayedDeliveryTrackerFactory @Cleanup @@ -113,10 +115,10 @@ private Pair mockDelayedDe @Test public void testFallbackToInMemoryTrackerFactoryFailed() throws Exception { - Pair pair = + Pair pair = mockDelayedDeliveryTrackerFactoryAndDispatcher(); BrokerService brokerService = pair.getLeft(); - PersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); // Mock InMemoryDelayedDeliveryTrackerFactory @Cleanup diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java index 6711aed924c20..ff7763927d888 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/InMemoryDeliveryTrackerTest.java @@ -37,7 +37,7 @@ import java.util.TreeMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -46,7 +46,7 @@ public class InMemoryDeliveryTrackerTest extends AbstractDeliveryTrackerTest { @DataProvider(name = "delayedTracker") public Object[][] provider(Method method) throws Exception { - dispatcher = mock(PersistentDispatcherMultipleConsumers.class); + dispatcher = mock(AbstractPersistentDispatcherMultipleConsumers.class); clock = mock(Clock.class); clockTime = new AtomicLong(); when(clock.millis()).then(x -> clockTime.get()); @@ -212,7 +212,8 @@ public void testClose() throws Exception { Timer timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-in-memory-delayed-delivery-test"), 1, TimeUnit.MILLISECONDS); - PersistentDispatcherMultipleConsumers dispatcher = mock(PersistentDispatcherMultipleConsumers.class); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + mock(AbstractPersistentDispatcherMultipleConsumers.class); AtomicLong clockTime = new AtomicLong(); Clock clock = mock(Clock.class); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java index bf5a282a4ee6d..426bd50c96bbb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java @@ -51,7 +51,7 @@ import org.apache.pulsar.broker.delayed.AbstractDeliveryTrackerTest; import org.apache.pulsar.broker.delayed.MockBucketSnapshotStorage; import org.apache.pulsar.broker.delayed.MockManagedCursor; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.awaitility.Awaitility; import org.roaringbitmap.RoaringBitmap; import org.roaringbitmap.buffer.ImmutableRoaringBitmap; @@ -74,7 +74,7 @@ public void clean() throws Exception { @DataProvider(name = "delayedTracker") public Object[][] provider(Method method) throws Exception { - dispatcher = mock(PersistentDispatcherMultipleConsumers.class); + dispatcher = mock(AbstractPersistentDispatcherMultipleConsumers.class); clock = mock(Clock.class); clockTime = new AtomicLong(); when(clock.millis()).then(x -> clockTime.get()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java index c66eff2c8a180..2fd288239e362 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java @@ -37,7 +37,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import lombok.Cleanup; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.BatcherBuilder; @@ -786,7 +786,7 @@ public void testConcurrentBatchMessageAck(BatcherBuilder builder) throws Excepti } latch.await(); - PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers) topic + AbstractPersistentDispatcherMultipleConsumers dispatcher = (AbstractPersistentDispatcherMultipleConsumers) topic .getSubscription(subscriptionName).getDispatcher(); // check strategically to let ack-message receive by broker retryStrategically((test) -> dispatcher.getConsumers().get(0).getUnackedMessages() == 0, 50, 150); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java index ed7f6974dd26f..21a843a3efc22 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageWithBatchIndexLevelTest.java @@ -41,7 +41,7 @@ import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.pulsar.broker.BrokerTestUtil; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -106,7 +106,7 @@ public void testBatchMessageAck() { } FutureUtil.waitForAll(sendFutureList).get(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); - PersistentDispatcherMultipleConsumers dispatcher = (PersistentDispatcherMultipleConsumers) topic + AbstractPersistentDispatcherMultipleConsumers dispatcher = (AbstractPersistentDispatcherMultipleConsumers) topic .getSubscription(subscriptionName).getDispatcher(); Message receive1 = consumer.receive(); Message receive2 = consumer.receive(); @@ -515,8 +515,8 @@ private BitSetRecyclable createBitSetRecyclable(int batchSize) { private ManagedCursorImpl getCursor(String topic, String sub) { PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); return (ManagedCursorImpl) dispatcher.getCursor(); } @@ -528,8 +528,8 @@ private org.apache.pulsar.broker.service.Consumer makeConsumerReceiveMessagesDel CompletableFuture signal) throws Exception { PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); org.apache.pulsar.broker.service.Consumer serviceConsumer = null; for (org.apache.pulsar.broker.service.Consumer c : dispatcher.getConsumers()){ if (c.consumerName().equals(consumerName)) { @@ -664,8 +664,8 @@ public void testPermitsIfHalfAckBatchMessage() throws Exception { private org.apache.pulsar.broker.service.Consumer getTheUniqueServiceConsumer(String topic, String sub) { PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService(). getTopic(topic, false).join().get(); - PersistentDispatcherMultipleConsumers dispatcher = - (PersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); + AbstractPersistentDispatcherMultipleConsumers dispatcher = + (AbstractPersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); return dispatcher.getConsumers().iterator().next(); } } 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 69f3e2e4d3917..2899e9f2d67db 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 @@ -66,6 +66,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -526,7 +527,8 @@ public void testMultipleDispatcherGetNextConsumerWithDifferentPriorityLevel() th PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, pulsarTestContext.getBrokerService()); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); + AbstractPersistentDispatcherMultipleConsumers + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(topic, 0, 2, false, 1); Consumer consumer2 = createConsumer(topic, 0, 2, false, 2); Consumer consumer3 = createConsumer(topic, 0, 2, false, 3); @@ -571,7 +573,8 @@ public void testMultipleDispatcherGetNextConsumerWithDifferentPriorityLevel() th public void testFewBlockedConsumerSamePriority() throws Exception{ PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, pulsarTestContext.getBrokerService()); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); + AbstractPersistentDispatcherMultipleConsumers + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(topic, 0, 2, false, 1); Consumer consumer2 = createConsumer(topic, 0, 2, false, 2); Consumer consumer3 = createConsumer(topic, 0, 2, false, 3); @@ -599,7 +602,8 @@ public void testFewBlockedConsumerSamePriority() throws Exception{ public void testFewBlockedConsumerDifferentPriority() throws Exception { PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, pulsarTestContext.getBrokerService()); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); + AbstractPersistentDispatcherMultipleConsumers + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(topic, 0, 2, false, 1); Consumer consumer2 = createConsumer(topic, 0, 2, false, 2); Consumer consumer3 = createConsumer(topic, 0, 2, false, 3); @@ -654,7 +658,8 @@ public void testFewBlockedConsumerDifferentPriority() throws Exception { public void testFewBlockedConsumerDifferentPriority2() throws Exception { PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, pulsarTestContext.getBrokerService()); - PersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); + AbstractPersistentDispatcherMultipleConsumers + dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursorMock, null); Consumer consumer1 = createConsumer(topic, 0, 2, true, 1); Consumer consumer2 = createConsumer(topic, 0, 2, true, 2); Consumer consumer3 = createConsumer(topic, 0, 2, true, 3); @@ -677,7 +682,7 @@ public void testFewBlockedConsumerDifferentPriority2() throws Exception { } @SuppressWarnings("unchecked") - private Consumer getNextConsumer(PersistentDispatcherMultipleConsumers dispatcher) throws Exception { + private Consumer getNextConsumer(AbstractPersistentDispatcherMultipleConsumers dispatcher) throws Exception { Consumer consumer = dispatcher.getNextConsumer(); 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 1e96da737dd51..92b767104f6cf 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 @@ -99,6 +99,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; @@ -346,7 +347,7 @@ public void testDispatcherMultiConsumerReadFailed() { when(cursor.getName()).thenReturn("cursor"); Subscription subscription = mock(Subscription.class); when(subscription.getName()).thenReturn("sub"); - PersistentDispatcherMultipleConsumers dispatcher = + AbstractPersistentDispatcherMultipleConsumers dispatcher = new PersistentDispatcherMultipleConsumers(topic, cursor, subscription); dispatcher.readEntriesFailed(new ManagedLedgerException.InvalidCursorPositionException("failed"), null); verify(topic, atLeast(1)).getBrokerService(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index 20ea33fb3e1ed..4c8e6897df3fc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -102,8 +102,8 @@ public void testBucketDelayedDeliveryWithAllConsumersDisconnecting() throws Exce Dispatcher dispatcher = pulsar.getBrokerService().getTopicReference(topic).get().getSubscription("sub").getDispatcher(); Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher.getNumberOfDelayedMessages(), 1000)); List bucketKeys = - ((PersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties().keySet().stream() - .filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties().keySet() + .stream().filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); c1.close(); @@ -117,8 +117,8 @@ public void testBucketDelayedDeliveryWithAllConsumersDisconnecting() throws Exce Dispatcher dispatcher2 = pulsar.getBrokerService().getTopicReference(topic).get().getSubscription("sub").getDispatcher(); List bucketKeys2 = - ((PersistentDispatcherMultipleConsumers) dispatcher2).getCursor().getCursorProperties().keySet().stream() - .filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher2).getCursor().getCursorProperties().keySet() + .stream().filter(x -> x.startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX)).toList(); Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher2.getNumberOfDelayedMessages(), 1000)); Assert.assertEquals(bucketKeys, bucketKeys2); @@ -152,7 +152,7 @@ public void testUnsubscribe() throws Exception { Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher.getNumberOfDelayedMessages(), 1000)); Map cursorProperties = - ((PersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties(); List bucketIds = cursorProperties.entrySet().stream() .filter(x -> x.getKey().startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX + "delayed.bucket")).map( x -> Long.valueOf(x.getValue())).toList(); @@ -339,7 +339,7 @@ public void testDelete() throws Exception { Awaitility.await().untilAsserted(() -> Assert.assertEquals(dispatcher.getNumberOfDelayedMessages(), 1000)); Map cursorProperties = - ((PersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties(); + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).getCursor().getCursorProperties(); List bucketIds = cursorProperties.entrySet().stream() .filter(x -> x.getKey().startsWith(CURSOR_INTERNAL_PROPERTY_PREFIX + "delayed.bucket")).map( x -> Long.valueOf(x.getValue())).toList(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java index 3ca966d210886..e47857e8ec60f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java @@ -259,12 +259,14 @@ public void testDelayedDeliveryWithMultipleConcurrentReadEntries() .subscribe(); // Simulate race condition with high frequency of calls to dispatcher.readMoreEntries() - PersistentDispatcherMultipleConsumers d = (PersistentDispatcherMultipleConsumers) ((PersistentTopic) pulsar - .getBrokerService().getTopicReference(topic).get()).getSubscription("shared-sub").getDispatcher(); + AbstractPersistentDispatcherMultipleConsumers d = + (AbstractPersistentDispatcherMultipleConsumers) ((PersistentTopic) pulsar + .getBrokerService().getTopicReference(topic).get()).getSubscription("shared-sub") + .getDispatcher(); Thread t = new Thread(() -> { while (true) { synchronized (d) { - d.readMoreEntries(); + d.readMoreEntriesAsync(); } try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassicTest.java new file mode 100644 index 0000000000000..487d99891fd3a --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassicTest.java @@ -0,0 +1,172 @@ +/* + * 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.service.persistent; + +import com.carrotsearch.hppc.ObjectSet; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.Subscription; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.awaitility.reflect.WhiteboxImpl; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class PersistentDispatcherMultipleConsumersClassicTest extends ProducerConsumerBase { + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test(timeOut = 30 * 1000) + public void testTopicDeleteIfConsumerSetMismatchConsumerList() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subscription, MessageId.earliest); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).subscribe(); + // Make an error that "consumerSet" is mismatch with "consumerList". + Dispatcher dispatcher = pulsar.getBrokerService() + .getTopic(topicName, false).join().get() + .getSubscription(subscription).getDispatcher(); + ObjectSet consumerSet = + WhiteboxImpl.getInternalState(dispatcher, "consumerSet"); + List consumerList = + WhiteboxImpl.getInternalState(dispatcher, "consumerList"); + + org.apache.pulsar.broker.service.Consumer serviceConsumer = consumerList.get(0); + consumerSet.add(serviceConsumer); + consumerList.add(serviceConsumer); + + // Verify: the topic can be deleted successfully. + consumer.close(); + admin.topics().delete(topicName, false); + } + + @Test(timeOut = 30 * 1000) + public void testTopicDeleteIfConsumerSetMismatchConsumerList2() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subscription, MessageId.earliest); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).subscribe(); + // Make an error that "consumerSet" is mismatch with "consumerList". + Dispatcher dispatcher = pulsar.getBrokerService() + .getTopic(topicName, false).join().get() + .getSubscription(subscription).getDispatcher(); + ObjectSet consumerSet = + WhiteboxImpl.getInternalState(dispatcher, "consumerSet"); + consumerSet.clear(); + + // Verify: the topic can be deleted successfully. + consumer.close(); + admin.topics().delete(topicName, false); + } + + @Test + public void testSkipReadEntriesFromCloseCursor() throws Exception { + final String topicName = + BrokerTestUtil.newUniqueName("persistent://public/default/testSkipReadEntriesFromCloseCursor"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topicName); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + for (int i = 0; i < 10; i++) { + producer.send("message-" + i); + } + producer.close(); + + // Get the dispatcher of the topic. + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService() + .getTopic(topicName, false).join().get(); + + ManagedCursor cursor = Mockito.mock(ManagedCursorImpl.class); + Mockito.doReturn(subscription).when(cursor).getName(); + Subscription sub = Mockito.mock(PersistentSubscription.class); + Mockito.doReturn(topic).when(sub).getTopic(); + // Mock the dispatcher. + PersistentDispatcherMultipleConsumersClassic dispatcher = + Mockito.spy(new PersistentDispatcherMultipleConsumersClassic(topic, cursor, sub)); + // Return 10 permits to make the dispatcher can read more entries. + Mockito.doReturn(10).when(dispatcher).getFirstAvailableConsumerPermits(); + + // Make the count + 1 when call the scheduleReadEntriesWithDelay(...). + AtomicInteger callScheduleReadEntriesWithDelayCnt = new AtomicInteger(0); + Mockito.doAnswer(inv -> { + callScheduleReadEntriesWithDelayCnt.getAndIncrement(); + return inv.callRealMethod(); + }).when(dispatcher).scheduleReadEntriesWithDelay(Mockito.any(), Mockito.any(), Mockito.anyLong()); + + // Make the count + 1 when call the readEntriesFailed(...). + AtomicInteger callReadEntriesFailed = new AtomicInteger(0); + Mockito.doAnswer(inv -> { + callReadEntriesFailed.getAndIncrement(); + return inv.callRealMethod(); + }).when(dispatcher).readEntriesFailed(Mockito.any(), Mockito.any()); + + Mockito.doReturn(false).when(cursor).isClosed(); + + // Mock the readEntriesOrWait(...) to simulate the cursor is closed. + Mockito.doAnswer(inv -> { + PersistentDispatcherMultipleConsumersClassic dispatcher1 = inv.getArgument(2); + dispatcher1.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException("cursor closed"), + null); + return null; + }).when(cursor).asyncReadEntriesOrWait(Mockito.anyInt(), Mockito.anyLong(), Mockito.eq(dispatcher), + Mockito.any(), Mockito.any()); + + dispatcher.readMoreEntries(); + + // Verify: the readEntriesFailed should be called once and the scheduleReadEntriesWithDelay should not be called. + Assert.assertTrue(callReadEntriesFailed.get() == 1 && callScheduleReadEntriesWithDelayCnt.get() == 0); + + // Verify: the topic can be deleted successfully. + admin.topics().delete(topicName, false); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java index 052c5ceb5cdde..772b1843d2894 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersTest.java @@ -154,7 +154,7 @@ public void testSkipReadEntriesFromCloseCursor() throws Exception { // Mock the readEntriesOrWait(...) to simulate the cursor is closed. Mockito.doAnswer(inv -> { - PersistentDispatcherMultipleConsumers dispatcher1 = inv.getArgument(2); + AbstractPersistentDispatcherMultipleConsumers dispatcher1 = inv.getArgument(2); dispatcher1.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException("cursor closed"), null); return null; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java new file mode 100644 index 0000000000000..1f40fd46aa344 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersClassicTest.java @@ -0,0 +1,482 @@ +/* + * 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.service.persistent; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.pulsar.common.protocol.Commands.serializeMetadataAndPayload; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.anySet; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +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.assertTrue; +import static org.testng.Assert.fail; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelPromise; +import io.netty.channel.EventLoopGroup; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.EntryImpl; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; +import org.apache.pulsar.broker.service.EntryBatchSizes; +import org.apache.pulsar.broker.service.RedeliveryTracker; +import org.apache.pulsar.broker.service.plugin.EntryFilterProvider; +import org.apache.pulsar.common.api.proto.KeySharedMeta; +import org.apache.pulsar.common.api.proto.KeySharedMode; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.policies.data.HierarchyTopicPolicies; +import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.protocol.Markers; +import org.awaitility.Awaitility; +import org.mockito.ArgumentCaptor; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class PersistentStickyKeyDispatcherMultipleConsumersClassicTest { + + private PulsarService pulsarMock; + private BrokerService brokerMock; + private ManagedCursorImpl cursorMock; + private Consumer consumerMock; + private PersistentTopic topicMock; + private PersistentSubscription subscriptionMock; + private ServiceConfiguration configMock; + private ChannelPromise channelMock; + private OrderedExecutor orderedExecutor; + + private PersistentStickyKeyDispatcherMultipleConsumersClassic persistentDispatcher; + + final String topicName = "persistent://public/default/testTopic"; + final String subscriptionName = "testSubscription"; + + @BeforeMethod + public void setup() throws Exception { + configMock = mock(ServiceConfiguration.class); + doReturn(true).when(configMock).isSubscriptionRedeliveryTrackerEnabled(); + doReturn(100).when(configMock).getDispatcherMaxReadBatchSize(); + doReturn(true).when(configMock).isSubscriptionKeySharedUseConsistentHashing(); + doReturn(1).when(configMock).getSubscriptionKeySharedConsistentHashingReplicaPoints(); + doReturn(true).when(configMock).isDispatcherDispatchMessagesInSubscriptionThread(); + doReturn(false).when(configMock).isAllowOverrideEntryFilters(); + + pulsarMock = mock(PulsarService.class); + doReturn(configMock).when(pulsarMock).getConfiguration(); + + EntryFilterProvider mockEntryFilterProvider = mock(EntryFilterProvider.class); + when(mockEntryFilterProvider.getBrokerEntryFilters()).thenReturn(Collections.emptyList()); + + brokerMock = mock(BrokerService.class); + doReturn(pulsarMock).when(brokerMock).pulsar(); + when(brokerMock.getEntryFilterProvider()).thenReturn(mockEntryFilterProvider); + + HierarchyTopicPolicies topicPolicies = new HierarchyTopicPolicies(); + topicPolicies.getMaxConsumersPerSubscription().updateBrokerValue(0); + + orderedExecutor = OrderedExecutor.newBuilder().build(); + doReturn(orderedExecutor).when(brokerMock).getTopicOrderedExecutor(); + + EventLoopGroup eventLoopGroup = mock(EventLoopGroup.class); + doReturn(eventLoopGroup).when(brokerMock).executor(); + doAnswer(invocation -> { + orderedExecutor.execute(((Runnable)invocation.getArguments()[0])); + return null; + }).when(eventLoopGroup).execute(any(Runnable.class)); + + topicMock = mock(PersistentTopic.class); + doReturn(brokerMock).when(topicMock).getBrokerService(); + doReturn(topicName).when(topicMock).getName(); + doReturn(topicPolicies).when(topicMock).getHierarchyTopicPolicies(); + + cursorMock = mock(ManagedCursorImpl.class); + doReturn(null).when(cursorMock).getLastIndividualDeletedRange(); + doReturn(subscriptionName).when(cursorMock).getName(); + + consumerMock = mock(Consumer.class); + channelMock = mock(ChannelPromise.class); + doReturn("consumer1").when(consumerMock).consumerName(); + doReturn(1000).when(consumerMock).getAvailablePermits(); + doReturn(true).when(consumerMock).isWritable(); + doReturn(channelMock).when(consumerMock).sendMessages( + anyList(), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + + subscriptionMock = mock(PersistentSubscription.class); + when(subscriptionMock.getTopic()).thenReturn(topicMock); + persistentDispatcher = new PersistentStickyKeyDispatcherMultipleConsumersClassic( + topicMock, cursorMock, subscriptionMock, configMock, + new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)); + } + + @AfterMethod(alwaysRun = true) + public void cleanup() { + if (persistentDispatcher != null && !persistentDispatcher.isClosed()) { + persistentDispatcher.close(); + } + if (orderedExecutor != null) { + orderedExecutor.shutdownNow(); + orderedExecutor = null; + } + } + + @Test(timeOut = 10000) + public void testAddConsumerWhenClosed() throws Exception { + persistentDispatcher.close().get(); + Consumer consumer = mock(Consumer.class); + persistentDispatcher.addConsumer(consumer); + verify(consumer, times(1)).disconnect(); + assertEquals(0, persistentDispatcher.getConsumers().size()); + assertTrue(persistentDispatcher.getSelector().getConsumerKeyHashRanges().isEmpty()); + } + + @Test + public void testSendMarkerMessage() { + try { + persistentDispatcher.addConsumer(consumerMock); + persistentDispatcher.consumerFlow(consumerMock, 1000); + } catch (Exception e) { + fail("Failed to add mock consumer", e); + } + + List entries = new ArrayList<>(); + ByteBuf markerMessage = Markers.newReplicatedSubscriptionsSnapshotRequest("testSnapshotId", "testSourceCluster"); + entries.add(EntryImpl.create(1, 1, markerMessage)); + entries.add(EntryImpl.create(1, 2, createMessage("message1", 1))); + entries.add(EntryImpl.create(1, 3, createMessage("message2", 2))); + entries.add(EntryImpl.create(1, 4, createMessage("message3", 3))); + entries.add(EntryImpl.create(1, 5, createMessage("message4", 4))); + entries.add(EntryImpl.create(1, 6, createMessage("message5", 5))); + + try { + persistentDispatcher.readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + } catch (Exception e) { + fail("Failed to readEntriesComplete.", e); + } + + Awaitility.await().untilAsserted(() -> { + ArgumentCaptor totalMessagesCaptor = ArgumentCaptor.forClass(Integer.class); + verify(consumerMock, times(1)).sendMessages( + anyList(), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + totalMessagesCaptor.capture(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + + List allTotalMessagesCaptor = totalMessagesCaptor.getAllValues(); + Assert.assertEquals(allTotalMessagesCaptor.get(0).intValue(), 5); + }); + } + + @Test(timeOut = 10000) + public void testSendMessage() { + KeySharedMeta keySharedMeta = new KeySharedMeta().setKeySharedMode(KeySharedMode.STICKY); + PersistentStickyKeyDispatcherMultipleConsumersClassic + persistentDispatcher = new PersistentStickyKeyDispatcherMultipleConsumersClassic( + topicMock, cursorMock, subscriptionMock, configMock, keySharedMeta); + try { + keySharedMeta.addHashRange() + .setStart(0) + .setEnd(9); + + Consumer consumerMock = mock(Consumer.class); + doReturn(keySharedMeta).when(consumerMock).getKeySharedMeta(); + persistentDispatcher.addConsumer(consumerMock); + persistentDispatcher.consumerFlow(consumerMock, 1000); + } catch (Exception e) { + fail("Failed to add mock consumer", e); + } + + List entries = new ArrayList<>(); + entries.add(EntryImpl.create(1, 1, createMessage("message1", 1))); + entries.add(EntryImpl.create(1, 2, createMessage("message2", 2))); + + try { + //Should success,see issue #8960 + persistentDispatcher.readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + } catch (Exception e) { + fail("Failed to readEntriesComplete.", e); + } + } + + @Test + public void testSkipRedeliverTemporally() { + final Consumer slowConsumerMock = mock(Consumer.class); + final ChannelPromise slowChannelMock = mock(ChannelPromise.class); + // add entries to redeliver and read target + final List redeliverEntries = new ArrayList<>(); + redeliverEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key1"))); + final List readEntries = new ArrayList<>(); + readEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); + readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key2"))); + + try { + Field totalAvailablePermitsField = PersistentDispatcherMultipleConsumersClassic.class.getDeclaredField("totalAvailablePermits"); + totalAvailablePermitsField.setAccessible(true); + totalAvailablePermitsField.set(persistentDispatcher, 1000); + + doAnswer(invocationOnMock -> { + ((PersistentStickyKeyDispatcherMultipleConsumersClassic) invocationOnMock.getArgument(2)) + .readEntriesComplete(readEntries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + return null; + }).when(cursorMock).asyncReadEntriesOrWait( + anyInt(), anyLong(), any(PersistentStickyKeyDispatcherMultipleConsumersClassic.class), + eq(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal), any()); + } catch (Exception e) { + fail("Failed to set to field", e); + } + + // Create 2Consumers + try { + doReturn("consumer2").when(slowConsumerMock).consumerName(); + // Change slowConsumer availablePermits to 0 and back to normal + when(slowConsumerMock.getAvailablePermits()) + .thenReturn(0) + .thenReturn(1); + doReturn(true).when(slowConsumerMock).isWritable(); + doReturn(slowChannelMock).when(slowConsumerMock).sendMessages( + anyList(), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + + persistentDispatcher.addConsumer(consumerMock); + persistentDispatcher.addConsumer(slowConsumerMock); + } catch (Exception e) { + fail("Failed to add mock consumer", e); + } + + // run PersistentStickyKeyDispatcherMultipleConsumers#sendMessagesToConsumers + // run readMoreEntries internally (and skip internally) + // Change slowConsumer availablePermits to 1 + // run PersistentStickyKeyDispatcherMultipleConsumers#sendMessagesToConsumers internally + // and then stop to dispatch to slowConsumer + if (persistentDispatcher.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal, + redeliverEntries, true)) { + persistentDispatcher.readMoreEntriesAsync(); + } + + Awaitility.await().untilAsserted(() -> { + verify(consumerMock, times(1)).sendMessages( + argThat(arg -> { + assertEquals(arg.size(), 1); + Entry entry = arg.get(0); + assertEquals(entry.getLedgerId(), 1); + assertEquals(entry.getEntryId(), 3); + return true; + }), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + }); + verify(slowConsumerMock, times(0)).sendMessages( + anyList(), + any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), + anyInt(), + anyLong(), + anyLong(), + any(RedeliveryTracker.class) + ); + } + + @Test(timeOut = 30000) + public void testMessageRedelivery() throws Exception { + final Queue actualEntriesToConsumer1 = new ConcurrentLinkedQueue<>(); + final Queue actualEntriesToConsumer2 = new ConcurrentLinkedQueue<>(); + + final Queue expectedEntriesToConsumer1 = new ConcurrentLinkedQueue<>(); + expectedEntriesToConsumer1.add(PositionFactory.create(1, 1)); + final Queue expectedEntriesToConsumer2 = new ConcurrentLinkedQueue<>(); + expectedEntriesToConsumer2.add(PositionFactory.create(1, 2)); + expectedEntriesToConsumer2.add(PositionFactory.create(1, 3)); + + final AtomicInteger remainingEntriesNum = new AtomicInteger( + expectedEntriesToConsumer1.size() + expectedEntriesToConsumer2.size()); + + // Messages with key1 are routed to consumer1 and messages with key2 are routed to consumer2 + final List allEntries = new ArrayList<>(); + allEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key2"))); + allEntries.add(EntryImpl.create(1, 2, createMessage("message2", 2, "key1"))); + allEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key1"))); + allEntries.forEach(entry -> ((EntryImpl) entry).retain()); + + final List redeliverEntries = new ArrayList<>(); + redeliverEntries.add(allEntries.get(0)); // message1 + final List readEntries = new ArrayList<>(); + readEntries.add(allEntries.get(2)); // message3 + + final Consumer consumer1 = mock(Consumer.class); + doReturn("consumer1").when(consumer1).consumerName(); + // Change availablePermits of consumer1 to 0 and then back to normal + when(consumer1.getAvailablePermits()).thenReturn(0).thenReturn(10); + doReturn(true).when(consumer1).isWritable(); + doAnswer(invocationOnMock -> { + @SuppressWarnings("unchecked") + List entries = (List) invocationOnMock.getArgument(0); + for (Entry entry : entries) { + remainingEntriesNum.decrementAndGet(); + actualEntriesToConsumer1.add(entry.getPosition()); + } + return channelMock; + }).when(consumer1).sendMessages(anyList(), any(EntryBatchSizes.class), any(EntryBatchIndexesAcks.class), + anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); + + final Consumer consumer2 = mock(Consumer.class); + doReturn("consumer2").when(consumer2).consumerName(); + when(consumer2.getAvailablePermits()).thenReturn(10); + doReturn(true).when(consumer2).isWritable(); + doAnswer(invocationOnMock -> { + @SuppressWarnings("unchecked") + List entries = (List) invocationOnMock.getArgument(0); + for (Entry entry : entries) { + remainingEntriesNum.decrementAndGet(); + actualEntriesToConsumer2.add(entry.getPosition()); + } + return channelMock; + }).when(consumer2).sendMessages(anyList(), any(EntryBatchSizes.class), any(EntryBatchIndexesAcks.class), + anyInt(), anyLong(), anyLong(), any(RedeliveryTracker.class)); + + persistentDispatcher.addConsumer(consumer1); + persistentDispatcher.addConsumer(consumer2); + + final Field totalAvailablePermitsField = PersistentDispatcherMultipleConsumersClassic.class + .getDeclaredField("totalAvailablePermits"); + totalAvailablePermitsField.setAccessible(true); + totalAvailablePermitsField.set(persistentDispatcher, 1000); + + final Field redeliveryMessagesField = PersistentDispatcherMultipleConsumersClassic.class + .getDeclaredField("redeliveryMessages"); + redeliveryMessagesField.setAccessible(true); + MessageRedeliveryController redeliveryMessages = (MessageRedeliveryController) redeliveryMessagesField + .get(persistentDispatcher); + redeliveryMessages.add(allEntries.get(0).getLedgerId(), allEntries.get(0).getEntryId(), + persistentDispatcher.getStickyKeyHash(allEntries.get(0))); // message1 + redeliveryMessages.add(allEntries.get(1).getLedgerId(), allEntries.get(1).getEntryId(), + persistentDispatcher.getStickyKeyHash(allEntries.get(1))); // message2 + + // Mock Cursor#asyncReplayEntries + doAnswer(invocationOnMock -> { + @SuppressWarnings("unchecked") + Set positions = (Set) invocationOnMock.getArgument(0); + List entries = allEntries.stream().filter(entry -> positions.contains(entry.getPosition())) + .collect(Collectors.toList()); + if (!entries.isEmpty()) { + ((PersistentStickyKeyDispatcherMultipleConsumersClassic) invocationOnMock.getArgument(1)) + .readEntriesComplete(entries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Replay); + } + return Collections.emptySet(); + }).when(cursorMock).asyncReplayEntries(anySet(), any(PersistentStickyKeyDispatcherMultipleConsumersClassic.class), + eq(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Replay), anyBoolean()); + + // Mock Cursor#asyncReadEntriesOrWait + AtomicBoolean asyncReadEntriesOrWaitCalled = new AtomicBoolean(); + doAnswer(invocationOnMock -> { + if (asyncReadEntriesOrWaitCalled.compareAndSet(false, true)) { + ((PersistentStickyKeyDispatcherMultipleConsumersClassic) invocationOnMock.getArgument(2)) + .readEntriesComplete(readEntries, PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + } else { + ((PersistentStickyKeyDispatcherMultipleConsumersClassic) invocationOnMock.getArgument(2)) + .readEntriesComplete(Collections.emptyList(), PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal); + } + return null; + }).when(cursorMock).asyncReadEntriesOrWait(anyInt(), anyLong(), + any(PersistentStickyKeyDispatcherMultipleConsumersClassic.class), + eq(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Normal), any()); + + // (1) Run sendMessagesToConsumers + // (2) Attempts to send message1 to consumer1 but skipped because availablePermits is 0 + // (3) Change availablePermits of consumer1 to 10 + // (4) Run readMoreEntries internally + // (5) Run sendMessagesToConsumers internally + // (6) Attempts to send message3 to consumer2 but skipped because redeliveryMessages contains message2 + persistentDispatcher.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumersClassic.ReadType.Replay, + redeliverEntries, true); + while (remainingEntriesNum.get() > 0) { + // (7) Run readMoreEntries and resend message1 to consumer1 and message2-3 to consumer2 + persistentDispatcher.readMoreEntries(); + } + + assertThat(actualEntriesToConsumer1).containsExactlyElementsOf(expectedEntriesToConsumer1); + assertThat(actualEntriesToConsumer2).containsExactlyElementsOf(expectedEntriesToConsumer2); + + allEntries.forEach(entry -> entry.release()); + } + + private ByteBuf createMessage(String message, int sequenceId) { + return createMessage(message, sequenceId, "testKey"); + } + + private ByteBuf createMessage(String message, int sequenceId, String key) { + MessageMetadata messageMetadata = new MessageMetadata() + .setSequenceId(sequenceId) + .setProducerName("testProducer") + .setPartitionKey(key) + .setPartitionKeyB64Encoded(false) + .setPublishTime(System.currentTimeMillis()); + return serializeMetadataAndPayload(Commands.ChecksumType.Crc32c, messageMetadata, Unpooled.copiedBuffer(message.getBytes(UTF_8))); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index 4b29ead984e7a..7234f0caefc63 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -720,7 +720,7 @@ public void testNoBackoffDelayWhenDelayedMessages(boolean dispatchMessagesInSubs AtomicInteger reScheduleReadInMsCalled = new AtomicInteger(0); AtomicBoolean delayAllMessages = new AtomicBoolean(true); - PersistentDispatcherMultipleConsumers dispatcher; + AbstractPersistentDispatcherMultipleConsumers dispatcher; if (isKeyShared) { dispatcher = new PersistentStickyKeyDispatcherMultipleConsumers( topicMock, cursorMock, subscriptionMock, configMock, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index 14403765105b9..5b2998216e8e1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -233,7 +233,7 @@ public void testConsumerStatsOutput() throws Exception { "unackedMessages", "avgMessagesPerEntry", "blockedConsumerOnUnackedMsgs", - "lastSentPositionWhenJoining", + "readPositionWhenJoining", "lastAckedTime", "lastAckedTimestamp", "lastConsumedTime", diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java index 2b16647f5590c..08efb6d9583ef 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/KeySharedSubscriptionTest.java @@ -65,10 +65,13 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.DrainingHashesTracker; import org.apache.pulsar.broker.service.PendingAcksMap; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; @@ -79,6 +82,7 @@ import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.tests.KeySharedImplementationType; import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.mockito.Mockito; @@ -89,53 +93,80 @@ import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "broker-impl") public class KeySharedSubscriptionTest extends ProducerConsumerBase { - private static final Logger log = LoggerFactory.getLogger(KeySharedSubscriptionTest.class); private static final List keys = Arrays.asList("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"); private static final String SUBSCRIPTION_NAME = "key_shared"; + private final KeySharedImplementationType implementationType; + + // Comment out the next line (Factory annotation) to run tests manually in IntelliJ, one-by-one + @Factory + public static Object[] createTestInstances() { + return KeySharedImplementationType.generateTestInstances(KeySharedSubscriptionTest::new); + } + + public KeySharedSubscriptionTest() { + // set the default implementation type for manual running in IntelliJ + this(KeySharedImplementationType.DEFAULT); + } + + public KeySharedSubscriptionTest(KeySharedImplementationType implementationType) { + this.implementationType = implementationType; + } + + private Object[][] prependImplementationTypeToData(Object[][] data) { + return implementationType.prependImplementationTypeToData(data); + } + + @DataProvider(name = "currentImplementationType") + public Object[] currentImplementationType() { + return new Object[]{ implementationType }; + } @DataProvider(name = "batch") - public Object[] batchProvider() { - return new Object[] { - false, - true - }; + public Object[][] batchProvider() { + return prependImplementationTypeToData(new Object[][]{ + {false}, + {true} + }); } @DataProvider(name = "partitioned") public Object[][] partitionedProvider() { - return new Object[][] { - { false }, - { true } - }; + return prependImplementationTypeToData(new Object[][]{ + {false}, + {true} + }); } @DataProvider(name = "data") public Object[][] dataProvider() { - return new Object[][] { + return prependImplementationTypeToData(new Object[][]{ // Topic-Type and "Batching" - { "persistent", false }, - { "persistent", true }, - { "non-persistent", false }, - { "non-persistent", true }, - }; + {"persistent", false}, + {"persistent", true}, + {"non-persistent", false}, + {"non-persistent", true}, + }); } @DataProvider(name = "topicDomain") public Object[][] topicDomainProvider() { - return new Object[][] { - { "persistent" }, - { "non-persistent" } - }; + return prependImplementationTypeToData(new Object[][]{ + {"persistent"}, + {"non-persistent"} + }); } @BeforeClass(alwaysRun = true) @Override protected void setup() throws Exception { + conf.setSubscriptionKeySharedUseClassicPersistentImplementation(implementationType.classic); + conf.setSubscriptionSharedUseClassicPersistentImplementation(implementationType.classic); this.conf.setUnblockStuckSubscriptionEnabled(true); super.internalSetup(); super.producerBaseSetup(); @@ -170,7 +201,8 @@ public void resetDefaultNamespace() throws Exception { private static final int NUMBER_OF_KEYS = 300; @Test(dataProvider = "data") - public void testSendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector(String topicType, boolean enableBatch) + public void testSendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector(KeySharedImplementationType impl, + String topicType, boolean enableBatch) throws PulsarClientException { String topic = topicType + "://public/default/key_shared-" + UUID.randomUUID(); @@ -197,7 +229,7 @@ public void testSendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector(St } @Test(dataProvider = "data") - public void testSendAndReceiveWithBatching(String topicType, boolean enableBatch) throws Exception { + public void testSendAndReceiveWithBatching(KeySharedImplementationType impl, String topicType, boolean enableBatch) throws Exception { String topic = topicType + "://public/default/key_shared-" + UUID.randomUUID(); @Cleanup @@ -242,7 +274,9 @@ public void testSendAndReceiveWithBatching(String topicType, boolean enableBatch } @Test(dataProvider = "batch") - public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) throws PulsarClientException { + public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(KeySharedImplementationType impl, + boolean enableBatch) + throws PulsarClientException { String topic = "persistent://public/default/key_shared_exclusive-" + UUID.randomUUID(); @Cleanup @@ -294,8 +328,9 @@ public void testSendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(bo @Test(dataProvider = "data") public void testConsumerCrashSendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector( - String topicType, - boolean enableBatch + KeySharedImplementationType impl, + String topicType, + boolean enableBatch ) throws PulsarClientException, InterruptedException { String topic = topicType + "://public/default/key_shared_consumer_crash-" + UUID.randomUUID(); @@ -338,8 +373,9 @@ public void testConsumerCrashSendAndReceiveWithHashRangeAutoSplitStickyKeyConsum @Test(dataProvider = "data") public void testNoKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelector( - String topicType, - boolean enableBatch + KeySharedImplementationType impl, + String topicType, + boolean enableBatch ) throws PulsarClientException { String topic = topicType + "://public/default/key_shared_no_key-" + UUID.randomUUID(); @@ -365,7 +401,8 @@ public void testNoKeySendAndReceiveWithHashRangeAutoSplitStickyKeyConsumerSelect } @Test(dataProvider = "batch") - public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) + public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelector(KeySharedImplementationType impl, + boolean enableBatch) throws PulsarClientException { String topic = "persistent://public/default/key_shared_no_key_exclusive-" + UUID.randomUUID(); @@ -415,7 +452,8 @@ public void testNoKeySendAndReceiveWithHashRangeExclusiveStickyKeyConsumerSelect } @Test(dataProvider = "batch") - public void testOrderingKeyWithHashRangeAutoSplitStickyKeyConsumerSelector(boolean enableBatch) + public void testOrderingKeyWithHashRangeAutoSplitStickyKeyConsumerSelector(KeySharedImplementationType impl, + boolean enableBatch) throws PulsarClientException { String topic = "persistent://public/default/key_shared_ordering_key-" + UUID.randomUUID(); @@ -443,7 +481,8 @@ public void testOrderingKeyWithHashRangeAutoSplitStickyKeyConsumerSelector(boole } @Test(dataProvider = "batch") - public void testOrderingKeyWithHashRangeExclusiveStickyKeyConsumerSelector(boolean enableBatch) + public void testOrderingKeyWithHashRangeExclusiveStickyKeyConsumerSelector(KeySharedImplementationType impl, + boolean enableBatch) throws PulsarClientException { String topic = "persistent://public/default/key_shared_exclusive_ordering_key-" + UUID.randomUUID(); @@ -512,8 +551,8 @@ public void testDisableKeySharedSubscription() throws PulsarClientException { } } - @Test - public void testCannotUseAcknowledgeCumulative() throws PulsarClientException { + @Test(dataProvider = "currentImplementationType") + public void testCannotUseAcknowledgeCumulative(KeySharedImplementationType impl) throws PulsarClientException { String topic = "persistent://public/default/key_shared_ack_cumulative-" + UUID.randomUUID(); @Cleanup @@ -538,7 +577,7 @@ public void testCannotUseAcknowledgeCumulative() throws PulsarClientException { } @Test(dataProvider = "batch") - public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exception { + public void testMakingProgressWithSlowerConsumer(KeySharedImplementationType impl, boolean enableBatch) throws Exception { String topic = "testMakingProgressWithSlowerConsumer-" + UUID.randomUUID(); String slowKey = "slowKey"; @@ -620,8 +659,8 @@ public void testMakingProgressWithSlowerConsumer(boolean enableBatch) throws Exc } } - @Test - public void testOrderingWhenAddingConsumers() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testOrderingWhenAddingConsumers(KeySharedImplementationType impl) throws Exception { String topic = "testOrderingWhenAddingConsumers-" + UUID.randomUUID(); int numberOfKeys = 10; @@ -668,13 +707,13 @@ public void testOrderingWhenAddingConsumers() throws Exception { } @SneakyThrows - private PersistentStickyKeyDispatcherMultipleConsumers getDispatcher(String topic, String subscription) { - return (PersistentStickyKeyDispatcherMultipleConsumers) pulsar.getBrokerService().getTopicIfExists(topic).get() + private StickyKeyDispatcher getDispatcher(String topic, String subscription) { + return (StickyKeyDispatcher) pulsar.getBrokerService().getTopicIfExists(topic).get() .get().getSubscription(subscription).getDispatcher(); } - @Test - public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testReadAheadWithConfiguredLookAheadLimit(KeySharedImplementationType impl) throws Exception { String topic = "testReadAheadWithConfiguredLookAheadLimit-" + UUID.randomUUID(); // Set the look ahead limit to 50 for subscriptions @@ -730,8 +769,8 @@ public void testReadAheadWithConfiguredLookAheadLimit() throws Exception { assertTrue(entryId < 100); } - @Test - public void testRemoveFirstConsumer() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testRemoveFirstConsumer(KeySharedImplementationType impl) throws Exception { String topic = "testReadAheadWhenAddingConsumers-" + UUID.randomUUID(); int numberOfKeys = 10; @@ -788,8 +827,8 @@ public void testRemoveFirstConsumer() throws Exception { } } - @Test - public void testHashRangeConflict() throws PulsarClientException { + @Test(dataProvider = "currentImplementationType") + public void testHashRangeConflict(KeySharedImplementationType impl) throws PulsarClientException { final String topic = "persistent://public/default/testHashRangeConflict-" + UUID.randomUUID().toString(); final String sub = "test"; @@ -799,7 +838,7 @@ public void testHashRangeConflict() throws PulsarClientException { Consumer consumer2 = createFixedHashRangesConsumer(topic, sub, Range.of(100,399)); Assert.assertTrue(consumer2.isConnected()); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, sub); + StickyKeyDispatcher dispatcher = getDispatcher(topic, sub); Assert.assertEquals(dispatcher.getConsumers().size(), 2); try { @@ -849,8 +888,8 @@ public void testHashRangeConflict() throws PulsarClientException { Assert.assertFalse(dispatcher.isConsumerConnected()); } - @Test - public void testWithMessageCompression() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testWithMessageCompression(KeySharedImplementationType impl) throws Exception { final String topic = "testWithMessageCompression" + UUID.randomUUID().toString(); Producer producer = pulsarClient.newProducer() .topic(topic) @@ -876,8 +915,8 @@ public void testWithMessageCompression() throws Exception { consumer.close(); } - @Test - public void testAttachKeyToMessageMetadata() throws PulsarClientException { + @Test(dataProvider = "currentImplementationType") + public void testAttachKeyToMessageMetadata(KeySharedImplementationType impl) throws PulsarClientException { String topic = "persistent://public/default/key_shared-" + UUID.randomUUID(); @Cleanup @@ -904,8 +943,8 @@ public void testAttachKeyToMessageMetadata() throws PulsarClientException { receiveAndCheckDistribution(Lists.newArrayList(consumer1, consumer2, consumer3), 1000); } - @Test - public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testContinueDispatchMessagesWhenMessageTTL(KeySharedImplementationType impl) throws Exception { int defaultTTLSec = 3; int totalMessages = 1000; int numberOfKeys = 50; @@ -922,7 +961,7 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { .subscriptionType(SubscriptionType.Key_Shared) .subscribe(); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, subName); + StickyKeyDispatcher dispatcher = getDispatcher(topic, subName); StickyKeyConsumerSelector selector = dispatcher.getSelector(); @Cleanup @@ -964,7 +1003,7 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { if (received != null) { int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); DrainingHashesTracker.DrainingHashEntry entry = - dispatcher.getDrainingHashesTracker().getEntry(stickyKeyHash); + !impl.classic ? getDrainingHashesTracker(dispatcher).getEntry(stickyKeyHash) : null; Assertions.fail("Received message %s with sticky key hash that should have been blocked: %d. entry=%s, " + "included in blockedHashes=%s", received.getMessageId(), stickyKeyHash, entry, blockedHashes.contains(stickyKeyHash)); @@ -982,10 +1021,10 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { received = consumer3.receive(1, TimeUnit.SECONDS); } catch (PulsarClientException ignore) { } - if (received != null) { + if (received != null && !impl.classic) { int stickyKeyHash = selector.makeStickyKeyHash(received.getKeyBytes()); DrainingHashesTracker.DrainingHashEntry entry = - dispatcher.getDrainingHashesTracker().getEntry(stickyKeyHash); + !impl.classic ? getDrainingHashesTracker(dispatcher).getEntry(stickyKeyHash) : null; Assertions.fail("Received message %s with sticky key hash that should have been blocked: %d. entry=%s, " + "included in blockedHashes=%s", received.getMessageId(), stickyKeyHash, entry, blockedHashes.contains(stickyKeyHash)); @@ -1018,8 +1057,12 @@ public void testContinueDispatchMessagesWhenMessageTTL() throws Exception { count -> assertThat(count.get()).isGreaterThan(0)); } + private DrainingHashesTracker getDrainingHashesTracker(Dispatcher dispatcher) { + return ((PersistentStickyKeyDispatcherMultipleConsumers) dispatcher).getDrainingHashesTracker(); + } + @Test(dataProvider = "partitioned") - public void testOrderingWithConsumerListener(boolean partitioned) throws Exception { + public void testOrderingWithConsumerListener(KeySharedImplementationType impl, boolean partitioned) throws Exception { final String topic = "persistent://public/default/key_shared-" + UUID.randomUUID(); if (partitioned) { admin.topics().createPartitionedTopic(topic, 3); @@ -1075,8 +1118,8 @@ public void testOrderingWithConsumerListener(boolean partitioned) throws Excepti consumer.close(); } - @Test - public void testKeySharedConsumerWithEncrypted() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testKeySharedConsumerWithEncrypted(KeySharedImplementationType impl) throws Exception { final String topic = "persistent://public/default/key_shared-" + UUID.randomUUID(); final int totalMessages = 100; @@ -1142,7 +1185,7 @@ public void testKeySharedConsumerWithEncrypted() throws Exception { } @Test(dataProvider = "topicDomain") - public void testSelectorChangedAfterAllConsumerDisconnected(String topicDomain) throws PulsarClientException, + public void testSelectorChangedAfterAllConsumerDisconnected(KeySharedImplementationType impl, String topicDomain) throws PulsarClientException, ExecutionException, InterruptedException { final String topicName = TopicName.get(topicDomain, "public", "default", "testSelectorChangedAfterAllConsumerDisconnected" + UUID.randomUUID()).toString(); @@ -1187,8 +1230,8 @@ public void testSelectorChangedAfterAllConsumerDisconnected(String topicDomain) consumer1.close(); } - @Test - public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected(KeySharedImplementationType impl) throws Exception { final String topicName = "persistent://public/default/change-allow-ooo-delivery-" + UUID.randomUUID(); final String subName = "my-sub"; @@ -1207,7 +1250,7 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr producer.send("message".getBytes()); Awaitility.await().untilAsserted(() -> assertNotNull(consumer1.receive(100, TimeUnit.MILLISECONDS))); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topicName, subName); + StickyKeyDispatcher dispatcher = getDispatcher(topicName, subName); assertTrue(dispatcher.isAllowOutOfOrderDelivery()); consumer1.close(); @@ -1225,8 +1268,8 @@ public void testAllowOutOfOrderDeliveryChangedAfterAllConsumerDisconnected() thr consumer2.close(); } - @Test(timeOut = 30_000) - public void testCheckConsumersWithSameName() throws Exception { + @Test(timeOut = 30_000, dataProvider = "currentImplementationType") + public void testCheckConsumersWithSameName(KeySharedImplementationType impl) throws Exception { final String topicName = "persistent://public/default/same-name-" + UUID.randomUUID(); final String subName = "my-sub"; final String consumerName = "name"; @@ -1270,25 +1313,37 @@ public void testCheckConsumersWithSameName() throws Exception { @Cleanup("shutdownNow") ExecutorService e = Executors.newCachedThreadPool(); e.submit(() -> { - while (l.getCount() > 0) { + while (l.getCount() > 0 && !Thread.currentThread().isInterrupted()) { try { Message msg = c2.receive(1, TimeUnit.SECONDS); + if (msg == null) { + continue; + } c2.acknowledge(msg); l.countDown(); } catch (PulsarClientException ex) { ex.printStackTrace(); + if (ex instanceof PulsarClientException.AlreadyClosedException) { + break; + } } } }); e.submit(() -> { - while (l.getCount() > 0) { + while (l.getCount() > 0 && !Thread.currentThread().isInterrupted()) { try { Message msg = c3.receive(1, TimeUnit.SECONDS); + if (msg == null) { + continue; + } c3.acknowledge(msg); l.countDown(); } catch (PulsarClientException ex) { ex.printStackTrace(); + if (ex instanceof PulsarClientException.AlreadyClosedException) { + break; + } } } }); @@ -1303,7 +1358,7 @@ private Object[][] preSendProvider() { private KeySharedMode getKeySharedModeOfSubscription(Topic topic, String subscription) { if (TopicName.get(topic.getName()).getDomain().equals(TopicDomain.persistent)) { - return ((PersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subscription) + return ((StickyKeyDispatcher) topic.getSubscription(subscription) .getDispatcher()).getKeySharedMode(); } else if (TopicName.get(topic.getName()).getDomain().equals(TopicDomain.non_persistent)) { return ((NonPersistentStickyKeyDispatcherMultipleConsumers) topic.getSubscription(subscription) @@ -1390,45 +1445,37 @@ private void receive(List> consumers) throws PulsarClientException { */ private void receiveAndCheckDistribution(List> consumers, int expectedTotalMessage) throws PulsarClientException { // Add a key so that we know this key was already assigned to one consumer - Map> keyToConsumer = new HashMap<>(); - Map, Integer> messagesPerConsumer = new HashMap<>(); + Map> keyToConsumer = new ConcurrentHashMap<>(); + Map, AtomicInteger> messagesPerConsumer = new ConcurrentHashMap<>(); + AtomicInteger totalMessages = new AtomicInteger(); - int totalMessages = 0; + BiFunction, Message, Boolean> messageHandler = (consumer, msg) -> { + totalMessages.incrementAndGet(); + messagesPerConsumer.computeIfAbsent(consumer, k -> new AtomicInteger()).incrementAndGet(); + try { + consumer.acknowledge(msg); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } - for (Consumer c : consumers) { - int messagesForThisConsumer = 0; - while (true) { - Message msg = c.receive(100, TimeUnit.MILLISECONDS); - if (msg == null) { - // Go to next consumer - messagesPerConsumer.put(c, messagesForThisConsumer); - break; - } - - ++totalMessages; - ++messagesForThisConsumer; - c.acknowledge(msg); - - if (msg.hasKey() || msg.hasOrderingKey()) { - String key = msg.hasOrderingKey() ? new String(msg.getOrderingKey()) : msg.getKey(); - Consumer assignedConsumer = keyToConsumer.get(key); - if (assignedConsumer == null) { - // This is a new key - keyToConsumer.put(key, c); - } else { - // The consumer should be the same - assertEquals(c, assignedConsumer); - } + if (msg.hasKey() || msg.hasOrderingKey()) { + String key = msg.hasOrderingKey() ? new String(msg.getOrderingKey()) : msg.getKey(); + Consumer assignedConsumer = keyToConsumer.putIfAbsent(key, consumer); + if (assignedConsumer != null && !assignedConsumer.equals(consumer)) { + assertEquals(consumer, assignedConsumer); } } - } + return true; + }; - final double PERCENT_ERROR = 0.40; // 40 % + BrokerTestUtil.receiveMessagesInThreads(messageHandler, Duration.ofMillis(250), + consumers.stream().map(Consumer.class::cast)); - double expectedMessagesPerConsumer = totalMessages / consumers.size(); - Assert.assertEquals(expectedTotalMessage, totalMessages); - for (int count : messagesPerConsumer.values()) { - Assert.assertEquals(count, expectedMessagesPerConsumer, expectedMessagesPerConsumer * PERCENT_ERROR); + final double PERCENT_ERROR = 0.40; // 40 % + double expectedMessagesPerConsumer = totalMessages.get() / (double) consumers.size(); + Assert.assertEquals(expectedTotalMessage, totalMessages.get()); + for (AtomicInteger count : messagesPerConsumer.values()) { + Assert.assertEquals(count.get(), expectedMessagesPerConsumer, expectedMessagesPerConsumer * PERCENT_ERROR); } } @@ -1531,8 +1578,8 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe } } - @Test - public void testStickyKeyRangesRestartConsumers() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testStickyKeyRangesRestartConsumers(KeySharedImplementationType impl) throws Exception { final String topic = TopicName.get("persistent", "public", "default", "testStickyKeyRangesRestartConsumers" + UUID.randomUUID()).toString(); @@ -1663,8 +1710,8 @@ public void testStickyKeyRangesRestartConsumers() throws Exception { producerFuture.get(); } - @Test - public void testContinueDispatchMessagesWhenMessageDelayed() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testContinueDispatchMessagesWhenMessageDelayed(KeySharedImplementationType impl) throws Exception { int delayedMessages = 40; int messages = 40; int sum = 0; @@ -1765,8 +1812,8 @@ private AtomicInteger injectReplayReadCounter(String topicName, String cursorNam return replyReadCounter; } - @Test - public void testNoRepeatedReadAndDiscard() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testNoRepeatedReadAndDiscard(KeySharedImplementationType impl) throws Exception { int delayedMessages = 100; int numberOfKeys = delayedMessages; final String topic = newUniqueName("persistent://public/default/tp"); @@ -1839,10 +1886,10 @@ public void testNoRepeatedReadAndDiscard() throws Exception { @DataProvider(name = "allowKeySharedOutOfOrder") public Object[][] allowKeySharedOutOfOrder() { - return new Object[][]{ + return prependImplementationTypeToData(new Object[][]{ {true}, {false} - }; + }); } /** @@ -1860,7 +1907,7 @@ public Object[][] allowKeySharedOutOfOrder() { * - at last, all messages will be received. */ @Test(timeOut = 180 * 1000, dataProvider = "allowKeySharedOutOfOrder") // the test will be finished in 60s. - public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedOutOfOrder) throws Exception { + public void testRecentJoinedPosWillNotStuckOtherConsumer(KeySharedImplementationType impl, boolean allowKeySharedOutOfOrder) throws Exception { final int messagesSentPerTime = 100; final Set totalReceivedMessages = new TreeSet<>(); final String topic = newUniqueName("persistent://public/default/tp"); @@ -2020,8 +2067,10 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer(boolean allowKeySharedO admin.topics().delete(topic, false); } - @Test - public void testReadAheadLimit() throws Exception { + @Test(dataProvider = "currentImplementationType") + public void testReadAheadLimit(KeySharedImplementationType impl) throws Exception { + // skip for classic implementation since the feature is not implemented + impl.skipIfClassic(); String topic = "testReadAheadLimit-" + UUID.randomUUID(); int numberOfKeys = 1000; long pauseTime = 100L; @@ -2040,7 +2089,7 @@ public void testReadAheadLimit() throws Exception { .subscribe() .close(); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = getDispatcher(topic, subscriptionName); + StickyKeyDispatcher dispatcher = getDispatcher(topic, subscriptionName); // create a function to use for checking the number of messages in replay Runnable checkLimit = () -> { @@ -2145,16 +2194,18 @@ public void testReadAheadLimit() throws Exception { private StickyKeyConsumerSelector getSelector(String topic, String subscription) { Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + StickyKeyDispatcher dispatcher = (StickyKeyDispatcher) sub.getDispatcher(); return dispatcher.getSelector(); } // This test case simulates a rolling restart scenario with behaviors that can trigger out-of-order issues. // In earlier versions of Pulsar, this issue occurred in about 25% of cases. // To increase the probability of reproducing the issue, use the invocationCount parameter. - @Test//(invocationCount = 50) - public void testOrderingAfterReconnects() throws Exception { + @Test(dataProvider = "currentImplementationType")//(invocationCount = 50) + public void testOrderingAfterReconnects(KeySharedImplementationType impl) throws Exception { + // skip for classic implementation since this fails + impl.skipIfClassic(); + String topic = newUniqueName("testOrderingAfterReconnects"); int numberOfKeys = 1000; long pauseTime = 100L; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java index 02de11a2bcc95..ce554ab2d9c00 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionMessageDispatchThrottlingTest.java @@ -25,8 +25,8 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter; -import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -93,7 +93,7 @@ public void testMessageRateLimitingNotReceiveAllMessages(SubscriptionType subscr DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -180,7 +180,7 @@ public void testMessageRateLimitingReceiveAllMessagesAfterThrottling(Subscriptio DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -264,7 +264,7 @@ private void testMessageNotDuplicated(SubscriptionType subscription) throws Exce DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -356,7 +356,7 @@ public void testBytesRateLimitingReceiveAllMessagesAfterThrottling(SubscriptionT DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -442,7 +442,7 @@ private void testDispatchRate(SubscriptionType subscription, DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -649,7 +649,7 @@ public void testRateLimitingMultipleConsumers() throws Exception { DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -805,7 +805,7 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { DispatchRateLimiter subRateLimiter = null; Dispatcher subDispatcher = topic.getSubscription(subName1).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); @@ -855,7 +855,7 @@ public void testClusterPolicyOverrideConfiguration() throws Exception { .subscribe(); subDispatcher = topic2.getSubscription(subName2).getDispatcher(); - if (subDispatcher instanceof PersistentDispatcherMultipleConsumers) { + if (subDispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { subRateLimiter = subDispatcher.getRateLimiter().get(); } else if (subDispatcher instanceof PersistentDispatcherSingleActiveConsumer) { subRateLimiter = subDispatcher.getRateLimiter().get(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java index 390e81ad664f9..8fe1f3e58d96d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SubscriptionPauseOnAckStatPersistTest.java @@ -32,6 +32,7 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService; +import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -104,8 +105,8 @@ private void triggerNewReadMoreEntries(String tpName, String cursorName) throws PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService().getTopic(tpName, false).join().get(); Dispatcher dispatcher = persistentTopic.getSubscription(cursorName).getDispatcher(); - if (dispatcher instanceof PersistentDispatcherMultipleConsumers) { - ((PersistentDispatcherMultipleConsumers) dispatcher).readMoreEntries(); + if (dispatcher instanceof AbstractPersistentDispatcherMultipleConsumers) { + ((AbstractPersistentDispatcherMultipleConsumers) dispatcher).readMoreEntriesAsync(); } else if (dispatcher instanceof PersistentDispatcherSingleActiveConsumer) { PersistentDispatcherSingleActiveConsumer persistentDispatcherSingleActiveConsumer = ((PersistentDispatcherSingleActiveConsumer) dispatcher); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionMaxUnackedMessagesTest.java similarity index 85% rename from pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java rename to pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionMaxUnackedMessagesTest.java index 704af89777f05..b3ef641ca1979 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionMaxUnackedMessagesTest.java @@ -34,8 +34,8 @@ import java.util.concurrent.atomic.AtomicLong; import lombok.SneakyThrows; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Topic; -import org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.client.api.BatcherBuilder; import org.apache.pulsar.client.api.Consumer; @@ -47,19 +47,39 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Range; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.tests.KeySharedImplementationType; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; +import org.testng.annotations.Factory; import org.testng.annotations.Test; @Test(groups = "broker-impl") -public class KeySharedSubscriptionTest extends ProducerConsumerBase { +public class KeySharedSubscriptionMaxUnackedMessagesTest extends ProducerConsumerBase { + private final KeySharedImplementationType implementationType; + + // Comment out the next line (Factory annotation) to run tests manually in IntelliJ, one-by-one + @Factory + public static Object[] createTestInstances() { + return KeySharedImplementationType.generateTestInstances(KeySharedSubscriptionMaxUnackedMessagesTest::new); + } + + public KeySharedSubscriptionMaxUnackedMessagesTest() { + // set the default implementation type for manual running in IntelliJ + this(KeySharedImplementationType.DEFAULT); + } + + public KeySharedSubscriptionMaxUnackedMessagesTest(KeySharedImplementationType implementationType) { + this.implementationType = implementationType; + } @Override @BeforeMethod protected void setup() throws Exception { + conf.setSubscriptionKeySharedUseClassicPersistentImplementation(implementationType.classic); + conf.setSubscriptionSharedUseClassicPersistentImplementation(implementationType.classic); conf.setMaxUnackedMessagesPerConsumer(10); super.internalSetup(); super.producerBaseSetup(); @@ -82,16 +102,17 @@ enum KeySharedSelectorType { @DataProvider public Object[][] subType() { - return new Object[][] { - { SubscriptionType.Shared, null }, - { SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_ConsistentHashing }, - { SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_Classic }, - { SubscriptionType.Key_Shared, KeySharedSelectorType.Sticky } - }; + return implementationType.prependImplementationTypeToData(new Object[][]{ + {SubscriptionType.Shared, null}, + {SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_ConsistentHashing}, + {SubscriptionType.Key_Shared, KeySharedSelectorType.AutoSplit_Classic}, + {SubscriptionType.Key_Shared, KeySharedSelectorType.Sticky} + }); } @Test(dataProvider = "subType", timeOut = 30000) - public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(SubscriptionType subscriptionType, + public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction(KeySharedImplementationType impl, + SubscriptionType subscriptionType, KeySharedSelectorType selectorType) throws PulsarClientException { if (selectorType == KeySharedSelectorType.AutoSplit_Classic) { @@ -258,8 +279,7 @@ private static void waitUntilLastActiveTimeNoLongerGetsUpdated(AtomicLong lastAc private StickyKeyConsumerSelector getSelector(String topic, String subscription) { Topic t = pulsar.getBrokerService().getTopicIfExists(topic).get().get(); PersistentSubscription sub = (PersistentSubscription) t.getSubscription(subscription); - PersistentStickyKeyDispatcherMultipleConsumers dispatcher = - (PersistentStickyKeyDispatcherMultipleConsumers) sub.getDispatcher(); + StickyKeyDispatcher dispatcher = (StickyKeyDispatcher) sub.getDispatcher(); return dispatcher.getSelector(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/tests/KeySharedImplementationType.java b/pulsar-broker/src/test/java/org/apache/pulsar/tests/KeySharedImplementationType.java new file mode 100644 index 0000000000000..39b504131fcee --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/tests/KeySharedImplementationType.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.tests; + +import java.util.Arrays; +import java.util.function.Function; +import org.testng.SkipException; + +/** + * KeyShared implementation type used in test. + */ +public enum KeySharedImplementationType { + // default implementation, PIP-379 + PIP379(false), + // classic implementation before PIP-282 and PIP-379 + Classic(true); + + public static final KeySharedImplementationType DEFAULT = PIP379; + public final boolean classic; + + KeySharedImplementationType(boolean classic) { + this.classic = classic; + } + + public void skipIfClassic() { + if (classic) { + throw new SkipException("Test is not applicable for classic implementation"); + } + } + + public Object[][] prependImplementationTypeToData(Object[][] data) { + return Arrays.stream(data) + .map(array -> { + Object[] newArray = new Object[array.length + 1]; + newArray[0] = this; + System.arraycopy(array, 0, newArray, 1, array.length); + return newArray; + }) + .toArray(Object[][]::new); + } + + public static Object[] generateTestInstances(Function testInstanceFactory) { + return Arrays.stream(KeySharedImplementationType.values()).map(testInstanceFactory).toArray(); + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java index 5f2cf7b209ee9..d2d3600df96ed 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java @@ -72,8 +72,8 @@ public interface ConsumerStats { /** Flag to verify if consumer is blocked due to reaching threshold of unacked messages. */ boolean isBlockedConsumerOnUnackedMsgs(); - /** The last sent position of the cursor when the consumer joining. */ - String getLastSentPositionWhenJoining(); + /** The read position of the cursor when the consumer joining. */ + String getReadPositionWhenJoining(); /** Address of this consumer. */ String getAddress(); diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index 7b7c1f5765cc5..ce3a080a855da 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -118,6 +118,9 @@ public interface SubscriptionStats { /** Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. */ String getKeySharedMode(); + /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ + Map getConsumersAfterMarkDeletePosition(); + /** SubscriptionProperties (key/value strings) associated with this subscribe. */ Map getSubscriptionProperties(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java index b4c5d21e6926e..de36b330b7f1a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java @@ -77,8 +77,8 @@ public class ConsumerStatsImpl implements ConsumerStats { /** Flag to verify if consumer is blocked due to reaching threshold of unacked messages. */ public boolean blockedConsumerOnUnackedMsgs; - /** The last sent position of the cursor when the consumer joining. */ - public String lastSentPositionWhenJoining; + /** The read position of the cursor when the consumer joining. */ + public String readPositionWhenJoining; /** Address of this consumer. */ private String address; @@ -113,7 +113,7 @@ public ConsumerStatsImpl add(ConsumerStatsImpl stats) { this.availablePermits += stats.availablePermits; this.unackedMessages += stats.unackedMessages; this.blockedConsumerOnUnackedMsgs = stats.blockedConsumerOnUnackedMsgs; - this.lastSentPositionWhenJoining = stats.lastSentPositionWhenJoining; + this.readPositionWhenJoining = stats.readPositionWhenJoining; return this; } @@ -141,8 +141,8 @@ public void setClientVersion(String clientVersion) { this.clientVersion = clientVersion; } - public String getLastSentPositionWhenJoining() { - return lastSentPositionWhenJoining; + public String getReadPositionWhenJoining() { + return readPositionWhenJoining; } public String getLastAckedTime() { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java index 4206a4aa8d61b..12734a5586cef 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import java.util.ArrayList; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -125,8 +126,8 @@ public class SubscriptionStatsImpl implements SubscriptionStats { /** Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. */ public String keySharedMode; - /** The last sent position of the cursor. This is for Key_Shared subscription. */ - public String lastSentPosition; + /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ + public Map consumersAfterMarkDeletePosition; /** The number of non-contiguous deleted messages ranges. */ public int nonContiguousDeletedMessagesRanges; @@ -153,6 +154,7 @@ public class SubscriptionStatsImpl implements SubscriptionStats { public SubscriptionStatsImpl() { this.consumers = new ArrayList<>(); + this.consumersAfterMarkDeletePosition = new LinkedHashMap<>(); this.subscriptionProperties = new HashMap<>(); this.bucketDelayedIndexStats = new HashMap<>(); } @@ -177,6 +179,7 @@ public void reset() { lastExpireTimestamp = 0L; lastMarkDeleteAdvancedTimestamp = 0L; consumers.clear(); + consumersAfterMarkDeletePosition.clear(); nonContiguousDeletedMessagesRanges = 0; nonContiguousDeletedMessagesRangesSerializedSize = 0; earliestMsgPublishTimeInBacklog = 0L; @@ -222,6 +225,7 @@ public SubscriptionStatsImpl add(SubscriptionStatsImpl stats) { } } this.allowOutOfOrderDelivery |= stats.allowOutOfOrderDelivery; + this.consumersAfterMarkDeletePosition.putAll(stats.consumersAfterMarkDeletePosition); this.nonContiguousDeletedMessagesRanges += stats.nonContiguousDeletedMessagesRanges; this.nonContiguousDeletedMessagesRangesSerializedSize += stats.nonContiguousDeletedMessagesRangesSerializedSize; if (this.earliestMsgPublishTimeInBacklog != 0 && stats.earliestMsgPublishTimeInBacklog != 0) { From 1d83a534cf1874113eafe1c0ede301996279a3bb Mon Sep 17 00:00:00 2001 From: Jiawen Wang <74594733+summeriiii@users.noreply.github.com> Date: Wed, 9 Oct 2024 21:15:25 +0800 Subject: [PATCH 572/580] [fix][doc] Fix some typos in pip (#23288) --- pip/pip-307.md | 2 +- pip/pip-324-Alpine Docker images.md | 4 ++-- pip/pip-337.md | 6 +++--- pip/pip-352.md | 2 +- pip/pip-359.md | 4 ++-- pip/pip-368.md | 2 +- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/pip/pip-307.md b/pip/pip-307.md index a919991d08991..51ab77ba29595 100644 --- a/pip/pip-307.md +++ b/pip/pip-307.md @@ -80,7 +80,7 @@ sequenceDiagram Leader Broker ->> Owner Broker: "state:Releasing:" close topic Owner Broker ->> Owner Broker: close broker topic sessions Owner Broker ->> Clients: close producers and consumers - Clients ->> Clients: reconnecting (inital delay 100ms) + Clients ->> Clients: reconnecting (initial delay 100ms) Owner Broker ->> New Owner Broker: "state:Assign:" assign new ownership New Owner Broker ->> Owner Broker: "state:Owned:" ack new ownership Clients ->> Owner Broker: lookup diff --git a/pip/pip-324-Alpine Docker images.md b/pip/pip-324-Alpine Docker images.md index c7fcc1903a93d..8d74aa41c8857 100644 --- a/pip/pip-324-Alpine Docker images.md +++ b/pip/pip-324-Alpine Docker images.md @@ -6,8 +6,8 @@ Pulsar Docker images are currently based on Ubuntu base images. While these images has served us well in the past years, there are few shortcomings. -Alpine Linux is a Linux distribution designed explicitely to work well in container environments and has strong -focus on security and a minimalistic set of included depedendencies. +Alpine Linux is a Linux distribution designed explicitly to work well in container environments and has strong +focus on security and a minimalistic set of included dependencies. ### Size of the image diff --git a/pip/pip-337.md b/pip/pip-337.md index 283bb9710de84..a50130623d758 100644 --- a/pip/pip-337.md +++ b/pip/pip-337.md @@ -184,7 +184,7 @@ public interface PulsarSslFactory extends AutoCloseable { /* * Returns the internally stored ssl context - * @throws IllegalStateException If the SSL Context has not be created before this call, then it wil throw this + * @throws IllegalStateException If the SSL Context has not be created before this call, then it will throw this * exception. */ SSLContext getInternalSslContext(); @@ -214,7 +214,7 @@ public class DefaultPulsarSslFactory implements PulsarSslFactory { } ``` -### Pulsar Commmon Changes +### Pulsar Common Changes 4 new configurations will need to be added into the Configurations like `ServiceConfiguration`, `ClientConfigurationData`, `ProxyConfiguration`, etc. All of the below will be optional. It will use the default values @@ -360,7 +360,7 @@ the DefaultAsyncHttpClient. This pattern will be common across all HTTP Clients ### Configuration -Same as [Broker Common Changes](#pulsar-commmon-changes) +Same as [Broker Common Changes](#pulsar-common-changes) ### CLI CLI tools like `PulsarClientTool` and `PulsarAdminTool` will need to be modified to support the new configurations. diff --git a/pip/pip-352.md b/pip/pip-352.md index 31641e7e1e1b5..2e43991a44c04 100644 --- a/pip/pip-352.md +++ b/pip/pip-352.md @@ -29,7 +29,7 @@ the Pulsar topic in a different order than intended. Implementing event time-based checks could mitigate this inconvenience. # Goals -* No impact on current topic compation behavior +* No impact on current topic compaction behavior * Preserve the order of messages during compaction regardless of network latencies ## In Scope diff --git a/pip/pip-359.md b/pip/pip-359.md index 52a76193d6cf2..3c7425dd77308 100644 --- a/pip/pip-359.md +++ b/pip/pip-359.md @@ -4,7 +4,7 @@ Implementation PR: [#22861](https://github.com/apache/pulsar/pull/22861) # Background knowledge In the current Pulsar client versions, from the user's perspective, when using a Pulsar Consumer, we have two main options to consume messages: -1. Pull mode, by calling `consumer.recieve()`(or `consumer.recieveAsync()`) +1. Pull mode, by calling `consumer.receive()`(or `consumer.receiveAsync()`) ```java public class ConsumerExample { public static void main(String[] args) throws PulsarClientException { @@ -25,7 +25,7 @@ public class ConsumerExample { ``` 2. Push mode, by registering a `MessageListener` interface, when building the Consumer. -When this method is used, we can't also use `consumer.receive()`(or `consumer.recieveAsync()`). +When this method is used, we can't also use `consumer.receive()`(or `consumer.receiveAsync()`). In the push mode, the MessageListener instance is called by the consumer, hence it is doing that with a thread taken from its own internal `ExecutorService` (i.e. thread pool). The problem comes when we build and use multiple Consumers from the same PulsarClient. It diff --git a/pip/pip-368.md b/pip/pip-368.md index 06bba2c12761c..f22c5b6c26cb3 100644 --- a/pip/pip-368.md +++ b/pip/pip-368.md @@ -125,7 +125,7 @@ message CommandLookupTopic { } ``` -When the client lookups a topic, it will set the client `lookupPorperties` to the `CommandLookupTopic.properties`. +When the client lookups a topic, it will set the client `lookupProperties` to the `CommandLookupTopic.properties`. ### Public API From ed01b0e14646fb4dc4e6f026059151e2d8b4d0f3 Mon Sep 17 00:00:00 2001 From: Jiawen Wang <74594733+summeriiii@users.noreply.github.com> Date: Wed, 9 Oct 2024 22:17:27 +0800 Subject: [PATCH 573/580] [fix][ml][PIP-327] fix recover from ledger when ledgerForceRecovery is true (#23426) --- .../mledger/impl/ManagedCursorImpl.java | 8 +++--- .../mledger/impl/ManagedCursorTest.java | 28 +++++++++++++++++++ 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index f469b88cae8e6..7c0d13108b1c4 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -549,10 +549,10 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (log.isInfoEnabled()) { log.info("[{}] Opened ledger {} for cursor {}. rc={}", ledger.getName(), ledgerId, name, rc); } - if (isBkErrorNotRecoverable(rc) || ledgerForceRecovery) { + if (isBkErrorNotRecoverable(rc) || (rc != BKException.Code.OK && ledgerForceRecovery)) { log.error("[{}] Error opening metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc)); - // Rewind to oldest entry available + // Rewind to the oldest entry available initialize(getRollbackPosition(info), Collections.emptyMap(), cursorProperties, callback); return; } else if (rc != BKException.Code.OK) { @@ -577,10 +577,10 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (log.isDebugEnabled()) { log.debug("[{}} readComplete rc={} entryId={}", ledger.getName(), rc1, lh1.getLastAddConfirmed()); } - if (isBkErrorNotRecoverable(rc1) || ledgerForceRecovery) { + if (isBkErrorNotRecoverable(rc1) || (rc1 != BKException.Code.OK && ledgerForceRecovery)) { log.error("[{}] Error reading from metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc1)); - // Rewind to oldest entry available + // Rewind to the oldest entry available initialize(getRollbackPosition(info), Collections.emptyMap(), cursorProperties, callback); return; } else if (rc1 != BKException.Code.OK) { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 8ae5a04a507b1..587f87a7d1d38 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -1255,6 +1255,34 @@ void cursorPersistence() throws Exception { assertEquals(c2.getMarkDeletedPosition(), p2); } + @Test(timeOut = 20000) + public void cursorPersistenceWithLedgerForceRecovery() throws Exception { + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setLedgerForceRecovery(true); + + ManagedLedger ledger = factory.open("my_test_ledger", config); + ManagedCursor c1 = ledger.openCursor("c1"); + ledger.addEntry("dummy-entry-1".getBytes(Encoding)); + ledger.addEntry("dummy-entry-2".getBytes(Encoding)); + ledger.addEntry("dummy-entry-3".getBytes(Encoding)); + + List entries = c1.readEntries(2); + Position p1 = entries.get(1).getPosition(); + c1.markDelete(p1); + entries.forEach(Entry::release); + + entries = c1.readEntries(1); + entries.forEach(Entry::release); + + // Reopen + @Cleanup("shutdown") + ManagedLedgerFactory factory2 = new ManagedLedgerFactoryImpl(metadataStore, bkc); + ledger = factory2.open("my_test_ledger", config); + c1 = ledger.openCursor("c1"); + + assertEquals(c1.getMarkDeletedPosition(), p1); + } + @Test(timeOut = 20000) void cursorPersistence2() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger", From c891eaa38a02dbbdb855b5d5d1de609f1ceb6132 Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 9 Oct 2024 22:46:47 +0800 Subject: [PATCH 574/580] [fix][test] Fix flaky GetPartitionMetadataMultiBrokerTest.testCompatibilityDifferentBrokersForNonPersistentTopic (#23259) --- .../broker/admin/GetPartitionMetadataMultiBrokerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java index 60691203e777d..d1eeabdb3d7cc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java @@ -256,8 +256,8 @@ public void testCompatibilityDifferentBrokersForNonPersistentTopic(boolean confi // Initialize the connections of internal Pulsar Client. PulsarClientImpl client1 = (PulsarClientImpl) pulsar1.getClient(); PulsarClientImpl client2 = (PulsarClientImpl) pulsar2.getClient(); - client1.getLookup(pulsar2.getBrokerServiceUrl()).getBroker(TopicName.get(DEFAULT_NS + "/tp1")); - client2.getLookup(pulsar1.getBrokerServiceUrl()).getBroker(TopicName.get(DEFAULT_NS + "/tp1")); + client1.getLookup(pulsar2.getBrokerServiceUrl()).getBroker(TopicName.get(DEFAULT_NS + "/tp1")).join(); + client2.getLookup(pulsar1.getBrokerServiceUrl()).getBroker(TopicName.get(DEFAULT_NS + "/tp1")).join(); // Inject a not support flag into the connections initialized. Field field = ClientCnx.class.getDeclaredField("supportsGetPartitionedMetadataWithoutAutoCreation"); From b051dcd71d1a032353950705a1c52b53117adcd2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 9 Oct 2024 20:12:02 +0300 Subject: [PATCH 575/580] [fix] Reapply shell script parameter passthrough fix #22867 reverted in #22921 (#22923) --- bin/bookkeeper | 12 ++++++------ bin/pulsar | 38 +++++++++++++++++++------------------- 2 files changed, 25 insertions(+), 25 deletions(-) diff --git a/bin/bookkeeper b/bin/bookkeeper index 668c5d4db70a8..ac7b622ac2363 100755 --- a/bin/bookkeeper +++ b/bin/bookkeeper @@ -232,20 +232,20 @@ OPTS="$OPTS $BK_METADATA_OPTIONS" #Change to BK_HOME to support relative paths cd "$BK_HOME" if [ $COMMAND == "bookie" ]; then - exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.server.Main --conf $BOOKIE_CONF $@ + exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.server.Main --conf $BOOKIE_CONF "$@" elif [ $COMMAND == "autorecovery" ]; then - exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.replication.AutoRecoveryMain --conf $BOOKIE_CONF $@ + exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.replication.AutoRecoveryMain --conf $BOOKIE_CONF "$@" elif [ $COMMAND == "localbookie" ]; then NUMBER=$1 shift - exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.util.LocalBookKeeper $NUMBER $BOOKIE_CONF $@ + exec $JAVA $OPTS $JMX_ARGS org.apache.bookkeeper.util.LocalBookKeeper $NUMBER $BOOKIE_CONF "$@" elif [ $COMMAND == "upgrade" ]; then - exec $JAVA $OPTS org.apache.bookkeeper.bookie.FileSystemUpgrade --conf $BOOKIE_CONF $@ + exec $JAVA $OPTS org.apache.bookkeeper.bookie.FileSystemUpgrade --conf $BOOKIE_CONF "$@" elif [ $COMMAND == "shell" ]; then ENTRY_FORMATTER_ARG="-DentryFormatterClass=${ENTRY_FORMATTER_CLASS:-org.apache.bookkeeper.util.StringEntryFormatter}" - exec $JAVA $OPTS $ENTRY_FORMATTER_ARG org.apache.bookkeeper.bookie.BookieShell -conf $BOOKIE_CONF $@ + exec $JAVA $OPTS $ENTRY_FORMATTER_ARG org.apache.bookkeeper.bookie.BookieShell -conf $BOOKIE_CONF "$@" elif [ $COMMAND == "help" -o $COMMAND == "--help" -o $COMMAND == "-h" ]; then bookkeeper_help; else - exec $JAVA $OPTS $COMMAND $@ + exec $JAVA $OPTS $COMMAND "$@" fi diff --git a/bin/pulsar b/bin/pulsar index 09be2ac50e279..0125dd81e8bb4 100755 --- a/bin/pulsar +++ b/bin/pulsar @@ -346,56 +346,56 @@ fi cd "$PULSAR_HOME" if [ $COMMAND == "broker" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-broker.log"} - exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.PulsarBrokerStarter --broker-conf $PULSAR_BROKER_CONF $@ + exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.PulsarBrokerStarter --broker-conf $PULSAR_BROKER_CONF "$@" elif [ $COMMAND == "bookie" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"bookkeeper.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.server.Main --conf $PULSAR_BOOKKEEPER_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.server.Main --conf $PULSAR_BOOKKEEPER_CONF "$@" elif [ $COMMAND == "zookeeper" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"zookeeper.log"} - exec $JAVA ${ZK_OPTS} $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_ZK_CONF $@ + exec $JAVA ${ZK_OPTS} $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_ZK_CONF "$@" elif [ $COMMAND == "global-zookeeper" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"global-zookeeper.log"} # Allow global ZK to turn into read-only mode when it cannot reach the quorum OPTS="${OPTS} ${ZK_OPTS} -Dreadonlymode.enabled=true" - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_GLOBAL_ZK_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_GLOBAL_ZK_CONF "$@" elif [ $COMMAND == "configuration-store" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"configuration-store.log"} # Allow global ZK to turn into read-only mode when it cannot reach the quorum OPTS="${OPTS} ${ZK_OPTS} -Dreadonlymode.enabled=true" - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_CONFIGURATION_STORE_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.zookeeper.server.quorum.QuorumPeerMain $PULSAR_CONFIGURATION_STORE_CONF "$@" elif [ $COMMAND == "proxy" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-proxy.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.proxy.server.ProxyServiceStarter --config $PULSAR_PROXY_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.proxy.server.ProxyServiceStarter --config $PULSAR_PROXY_CONF "$@" elif [ $COMMAND == "websocket" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-websocket.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.websocket.service.WebSocketServiceStarter $PULSAR_WEBSOCKET_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.websocket.service.WebSocketServiceStarter $PULSAR_WEBSOCKET_CONF "$@" elif [ $COMMAND == "functions-worker" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-functions-worker.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.functions.worker.FunctionWorkerStarter -c $PULSAR_WORKER_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.pulsar.functions.worker.FunctionWorkerStarter -c $PULSAR_WORKER_CONF "$@" elif [ $COMMAND == "standalone" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-standalone.log"} - exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS ${ZK_OPTS} -Dpulsar.log.file=$PULSAR_LOG_FILE -Dpulsar.config.file=$PULSAR_STANDALONE_CONF org.apache.pulsar.PulsarStandaloneStarter $@ + exec $JAVA $LOG4J2_SHUTDOWN_HOOK_DISABLED $OPTS ${ZK_OPTS} -Dpulsar.log.file=$PULSAR_LOG_FILE -Dpulsar.config.file=$PULSAR_STANDALONE_CONF org.apache.pulsar.PulsarStandaloneStarter "$@" elif [ ${COMMAND} == "autorecovery" ]; then PULSAR_LOG_FILE=${PULSAR_LOG_FILE:-"pulsar-autorecovery.log"} - exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.replication.AutoRecoveryMain --conf $PULSAR_BOOKKEEPER_CONF $@ + exec $JAVA $OPTS -Dpulsar.log.file=$PULSAR_LOG_FILE org.apache.bookkeeper.replication.AutoRecoveryMain --conf $PULSAR_BOOKKEEPER_CONF "$@" elif [ $COMMAND == "initialize-cluster-metadata" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataSetup $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataSetup "$@" elif [ $COMMAND == "delete-cluster-metadata" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataTeardown $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarClusterMetadataTeardown "$@" elif [ $COMMAND == "initialize-transaction-coordinator-metadata" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarTransactionCoordinatorMetadataSetup $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarTransactionCoordinatorMetadataSetup "$@" elif [ $COMMAND == "initialize-namespace" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarInitialNamespaceSetup $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarInitialNamespaceSetup "$@" elif [ $COMMAND == "zookeeper-shell" ]; then - exec $JAVA $OPTS org.apache.zookeeper.ZooKeeperMain $@ + exec $JAVA $OPTS org.apache.zookeeper.ZooKeeperMain "$@" elif [ $COMMAND == "broker-tool" ]; then - exec $JAVA $OPTS org.apache.pulsar.broker.tools.BrokerTool $@ + exec $JAVA $OPTS org.apache.pulsar.broker.tools.BrokerTool "$@" elif [ $COMMAND == "compact-topic" ]; then - exec $JAVA $OPTS org.apache.pulsar.compaction.CompactorTool --broker-conf $PULSAR_BROKER_CONF $@ + exec $JAVA $OPTS org.apache.pulsar.compaction.CompactorTool --broker-conf $PULSAR_BROKER_CONF "$@" elif [ $COMMAND == "tokens" ]; then - exec $JAVA $OPTS org.apache.pulsar.utils.auth.tokens.TokensCliUtils $@ + exec $JAVA $OPTS org.apache.pulsar.utils.auth.tokens.TokensCliUtils "$@" elif [ $COMMAND == "version" ]; then - exec $JAVA $OPTS org.apache.pulsar.PulsarVersionStarter $@ + exec $JAVA $OPTS org.apache.pulsar.PulsarVersionStarter "$@" elif [ $COMMAND == "help" -o $COMMAND == "--help" -o $COMMAND == "-h" ]; then pulsar_help; else From 3dc0adec208100f1cc95865a0ae51fbb4e9dd360 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 10 Oct 2024 09:37:04 +0300 Subject: [PATCH 576/580] [fix][ci] Pin aquasecurity/trivy-action@0.26.0 since master is broken (#23431) --- .github/workflows/pulsar-ci.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 47a39bef9c908..bf44c51b6ad02 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -894,7 +894,7 @@ jobs: - name: Run Trivy container scan id: trivy_scan - uses: aquasecurity/trivy-action@master + uses: aquasecurity/trivy-action@0.26.0 if: ${{ github.repository == 'apache/pulsar' && github.event_name != 'pull_request' }} continue-on-error: true with: From acac72ea03f7c38cab99ec011b309d5e6bb4fe9d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 10 Oct 2024 12:46:42 +0300 Subject: [PATCH 577/580] [improve][broker][PIP-379] Add observability stats for "draining hashes" (#23429) --- .../pulsar/broker/service/Consumer.java | 7 + .../broker/service/DrainingHashesTracker.java | 112 +++++++ ...tStickyKeyDispatcherMultipleConsumers.java | 6 +- .../persistent/PersistentSubscription.java | 20 +- .../persistent/RescheduleReadHandler.java | 14 + .../apache/pulsar/broker/BrokerTestUtil.java | 78 ++++- .../stats/AuthenticatedConsumerStatsTest.java | 57 +--- .../broker/stats/ConsumerStatsTest.java | 276 ++++++++++++++++-- pulsar-broker/src/test/resources/log4j2.xml | 11 + .../common/policies/data/ConsumerStats.java | 48 ++- .../common/policies/data/DrainingHash.java | 41 +++ .../policies/data/SubscriptionStats.java | 24 ++ .../data/stats/ConsumerStatsImpl.java | 43 ++- .../policies/data/stats/DrainingHashImpl.java | 46 +++ .../data/stats/SubscriptionStatsImpl.java | 22 ++ .../common/util/ObjectMapperFactory.java | 3 + 16 files changed, 734 insertions(+), 74 deletions(-) create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DrainingHash.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/DrainingHashImpl.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index d25ebd0839df1..bcd29d86490cf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -174,6 +174,10 @@ public class Consumer { @Setter private volatile PendingAcksMap.PendingAcksRemoveHandler pendingAcksRemoveHandler; + @Getter + @Setter + private volatile java.util.function.BiConsumer drainingHashesConsumerStatsUpdater; + public Consumer(Subscription subscription, SubType subType, String topicName, long consumerId, int priorityLevel, String consumerName, boolean isDurable, TransportCnx cnx, String appId, @@ -976,6 +980,9 @@ public ConsumerStatsImpl getStats() { if (readPositionWhenJoining != null) { stats.readPositionWhenJoining = readPositionWhenJoining.toString(); } + if (drainingHashesConsumerStatsUpdater != null) { + drainingHashesConsumerStatsUpdater.accept(this, stats); + } return stats; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java index 3521fa197a13d..46762c844db6c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/DrainingHashesTracker.java @@ -20,8 +20,18 @@ import static org.apache.pulsar.broker.service.StickyKeyConsumerSelector.STICKY_KEY_HASH_NOT_SET; import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.PrimitiveIterator; +import java.util.concurrent.ConcurrentHashMap; import lombok.ToString; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.common.policies.data.DrainingHash; +import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; +import org.apache.pulsar.common.policies.data.stats.DrainingHashImpl; +import org.roaringbitmap.RoaringBitmap; /** * A thread-safe map to store draining hashes in the consumer. @@ -34,6 +44,8 @@ public class DrainingHashesTracker { private final Int2ObjectOpenHashMap drainingHashes = new Int2ObjectOpenHashMap<>(); int batchLevel; boolean unblockedWhileBatching; + private final Map consumerDrainingHashesStatsMap = + new ConcurrentHashMap<>(); /** * Represents an entry in the draining hashes tracker. @@ -98,6 +110,52 @@ boolean isBlocking() { } } + private class ConsumerDrainingHashesStats { + private final RoaringBitmap drainingHashes = new RoaringBitmap(); + long drainingHashesClearedTotal; + + public synchronized void addHash(int stickyHash) { + drainingHashes.add(stickyHash); + } + + public synchronized boolean clearHash(int hash) { + drainingHashes.remove(hash); + drainingHashesClearedTotal++; + boolean empty = drainingHashes.isEmpty(); + if (log.isDebugEnabled()) { + log.debug("[{}] Cleared hash {} in stats. empty={} totalCleared={} hashes={}", + dispatcherName, hash, empty, drainingHashesClearedTotal, drainingHashes.getCardinality()); + } + return empty; + } + + public synchronized void updateConsumerStats(Consumer consumer, ConsumerStatsImpl consumerStats) { + int drainingHashesUnackedMessages = 0; + List drainingHashesStats = new ArrayList<>(); + PrimitiveIterator.OfInt hashIterator = drainingHashes.stream().iterator(); + while (hashIterator.hasNext()) { + int hash = hashIterator.nextInt(); + DrainingHashEntry entry = getEntry(hash); + if (entry == null) { + log.warn("[{}] Draining hash {} not found in the tracker for consumer {}", dispatcherName, hash, + consumer); + continue; + } + int unackedMessages = entry.refCount; + DrainingHashImpl drainingHash = new DrainingHashImpl(); + drainingHash.hash = hash; + drainingHash.unackMsgs = unackedMessages; + drainingHash.blockedAttempts = entry.blockedCount; + drainingHashesStats.add(drainingHash); + drainingHashesUnackedMessages += unackedMessages; + } + consumerStats.drainingHashesCount = drainingHashesStats.size(); + consumerStats.drainingHashesClearedTotal = drainingHashesClearedTotal; + consumerStats.drainingHashesUnackedMessages = drainingHashesUnackedMessages; + consumerStats.drainingHashes = drainingHashesStats; + } + } + /** * Interface for handling the unblocking of sticky key hashes. */ @@ -127,13 +185,25 @@ public synchronized void addEntry(Consumer consumer, int stickyHash) { } DrainingHashEntry entry = drainingHashes.get(stickyHash); if (entry == null) { + if (log.isDebugEnabled()) { + log.debug("[{}] Adding and incrementing draining hash {} for consumer id:{} name:{}", dispatcherName, + stickyHash, consumer.consumerId(), consumer.consumerName()); + } entry = new DrainingHashEntry(consumer); drainingHashes.put(stickyHash, entry); + // update the consumer specific stats + consumerDrainingHashesStatsMap.computeIfAbsent(new ConsumerIdentityWrapper(consumer), + k -> new ConsumerDrainingHashesStats()).addHash(stickyHash); } else if (entry.getConsumer() != consumer) { throw new IllegalStateException( "Consumer " + entry.getConsumer() + " is already draining hash " + stickyHash + " in dispatcher " + dispatcherName + ". Same hash being used for consumer " + consumer + "."); + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Draining hash {} incrementing {} consumer id:{} name:{}", dispatcherName, stickyHash, + entry.refCount + 1, consumer.consumerId(), consumer.consumerName()); + } } entry.incrementRefCount(); } @@ -178,7 +248,17 @@ public synchronized void reduceRefCount(Consumer consumer, int stickyHash, boole + "."); } if (entry.decrementRefCount()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Draining hash {} removing consumer id:{} name:{}", dispatcherName, stickyHash, + consumer.consumerId(), consumer.consumerName()); + } DrainingHashEntry removed = drainingHashes.remove(stickyHash); + // update the consumer specific stats + ConsumerDrainingHashesStats drainingHashesStats = + consumerDrainingHashesStatsMap.get(new ConsumerIdentityWrapper(consumer)); + if (drainingHashesStats != null) { + drainingHashesStats.clearHash(stickyHash); + } if (!closing && removed.isBlocking()) { if (batchLevel > 0) { unblockedWhileBatching = true; @@ -186,6 +266,11 @@ public synchronized void reduceRefCount(Consumer consumer, int stickyHash, boole unblockingHandler.stickyKeyHashUnblocked(stickyHash); } } + } else { + if (log.isDebugEnabled()) { + log.debug("[{}] Draining hash {} decrementing {} consumer id:{} name:{}", dispatcherName, stickyHash, + entry.refCount, consumer.consumerId(), consumer.consumerName()); + } } } @@ -237,5 +322,32 @@ public synchronized DrainingHashEntry getEntry(int stickyKeyHash) { */ public synchronized void clear() { drainingHashes.clear(); + consumerDrainingHashesStatsMap.clear(); + } + + /** + * Update the consumer specific stats to the target {@link ConsumerStatsImpl}. + * + * @param consumer the consumer + * @param consumerStats the consumer stats to update the values to + */ + public void updateConsumerStats(Consumer consumer, ConsumerStatsImpl consumerStats) { + consumerStats.drainingHashesCount = 0; + consumerStats.drainingHashesClearedTotal = 0; + consumerStats.drainingHashesUnackedMessages = 0; + consumerStats.drainingHashes = Collections.emptyList(); + ConsumerDrainingHashesStats consumerDrainingHashesStats = + consumerDrainingHashesStatsMap.get(new ConsumerIdentityWrapper(consumer)); + if (consumerDrainingHashesStats != null) { + consumerDrainingHashesStats.updateConsumerStats(consumer, consumerStats); + } + } + + /** + * Remove the consumer specific stats from the draining hashes tracker. + * @param consumer the consumer + */ + public void consumerRemoved(Consumer consumer) { + consumerDrainingHashesStatsMap.remove(new ConsumerIdentityWrapper(consumer)); } } \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index df053e6d8a549..1a3e2f706cba8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -157,6 +157,7 @@ public void endBatch() { drainingHashesTracker.endBatch(); } }); + consumer.setDrainingHashesConsumerStatsUpdater(drainingHashesTracker::updateConsumerStats); registerDrainingHashes(consumer, impactedConsumers.orElseThrow()); } }).exceptionally(ex -> { @@ -193,6 +194,7 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE // consumer to another. This will handle the case where a hash gets switched from an existing // consumer to another existing consumer during removal. registerDrainingHashes(consumer, impactedConsumers.orElseThrow()); + drainingHashesTracker.consumerRemoved(consumer); } } @@ -349,8 +351,8 @@ private boolean handleAddingPendingAck(Consumer consumer, long ledgerId, long en return false; } if (log.isDebugEnabled()) { - log.debug("[{}] Adding {}:{} to pending acks for consumer {} with sticky key hash {}", - getName(), ledgerId, entryId, consumer, stickyKeyHash); + log.debug("[{}] Adding {}:{} to pending acks for consumer id:{} name:{} with sticky key hash {}", + getName(), ledgerId, entryId, consumer.consumerId(), consumer.consumerName(), stickyKeyHash); } // allow adding the message to pending acks and sending the message to the consumer return true; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index eaa147b81b126..df1c23cbbcb30 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -1253,11 +1253,23 @@ public CompletableFuture getStatsAsync(GetStatsOptions ge subStats.lastConsumedTimestamp = Math.max(subStats.lastConsumedTimestamp, consumerStats.lastConsumedTimestamp); subStats.lastAckedTimestamp = Math.max(subStats.lastAckedTimestamp, consumerStats.lastAckedTimestamp); - if (consumerKeyHashRanges != null && consumerKeyHashRanges.containsKey(consumer)) { - consumerStats.keyHashRanges = consumerKeyHashRanges.get(consumer).stream() - .map(Range::toString) - .collect(Collectors.toList()); + List keyRanges = consumerKeyHashRanges != null ? consumerKeyHashRanges.get(consumer) : null; + if (keyRanges != null) { + if (((StickyKeyDispatcher) dispatcher).isClassic()) { + // Use string representation for classic mode + consumerStats.keyHashRanges = keyRanges.stream() + .map(Range::toString) + .collect(Collectors.toList()); + } else { + // Use array representation for PIP-379 stats + consumerStats.keyHashRangeArrays = keyRanges.stream() + .map(range -> new int[]{range.getStart(), range.getEnd()}) + .collect(Collectors.toList()); + } } + subStats.drainingHashesCount += consumerStats.drainingHashesCount; + subStats.drainingHashesClearedTotal += consumerStats.drainingHashesClearedTotal; + subStats.drainingHashesUnackedMessages += consumerStats.drainingHashesUnackedMessages; }); subStats.filterProcessedMsgCount = dispatcher.getFilterProcessedMsgCount(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java index 3554f29255227..4812be58cdc78 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/RescheduleReadHandler.java @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.BooleanSupplier; import java.util.function.LongSupplier; +import lombok.extern.slf4j.Slf4j; /** * Reschedules reads so that the possible pending read is cancelled if it's waiting for more entries. @@ -30,6 +31,7 @@ * that should be handled. This will also batch multiple calls together to reduce the number of * operations. */ +@Slf4j class RescheduleReadHandler { private static final int UNSET = -1; private static final int NO_PENDING_READ = 0; @@ -70,15 +72,27 @@ public void rescheduleRead() { // are entries in the replay queue. if (maxReadOpCount != NO_PENDING_READ && readOpCounterSupplier.getAsLong() == maxReadOpCount && hasEntriesInReplayQueue.getAsBoolean()) { + if (log.isDebugEnabled()) { + log.debug("Cancelling pending read request because it's waiting for more entries"); + } cancelPendingRead.run(); } // Re-schedule read immediately, or join the next scheduled read + if (log.isDebugEnabled()) { + log.debug("Triggering read"); + } rescheduleReadImmediately.run(); }; long rescheduleDelay = readIntervalMsSupplier.getAsLong(); if (rescheduleDelay > 0) { + if (log.isDebugEnabled()) { + log.debug("Scheduling after {} ms", rescheduleDelay); + } executor.schedule(runnable, rescheduleDelay, TimeUnit.MILLISECONDS); } else { + if (log.isDebugEnabled()) { + log.debug("Running immediately"); + } runnable.run(); } } else { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java index 7ed4542b2505f..6a41e86f8934e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BrokerTestUtil.java @@ -21,10 +21,15 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; +import java.io.BufferedReader; import java.io.IOException; +import java.io.InputStreamReader; import java.io.StringWriter; import java.io.UncheckedIOException; +import java.net.HttpURLConnection; +import java.net.URL; import java.time.Duration; import java.util.Arrays; import java.util.UUID; @@ -37,6 +42,7 @@ import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.stream.Stream; +import lombok.SneakyThrows; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -46,7 +52,6 @@ import org.apache.pulsar.common.util.ObjectMapperFactory; import org.mockito.Mockito; import org.slf4j.Logger; - /** * Holds util methods used in test. */ @@ -136,6 +141,77 @@ public static void logTopicStats(Logger logger, PulsarAdmin pulsarAdmin, String } } + /** + * Logs the topic stats and internal stats for the given topic + * @param logger logger to use + * @param baseUrl Pulsar service URL + * @param topic topic name + */ + public static void logTopicStats(Logger logger, String baseUrl, String topic) { + logTopicStats(logger, baseUrl, "public", "default", topic); + } + + /** + * Logs the topic stats and internal stats for the given topic + * @param logger logger to use + * @param baseUrl Pulsar service URL + * @param tenant tenant name + * @param namespace namespace name + * @param topic topic name + */ + public static void logTopicStats(Logger logger, String baseUrl, String tenant, String namespace, String topic) { + String topicStatsUri = + String.format("%s/admin/v2/persistent/%s/%s/%s/stats", baseUrl, tenant, namespace, topic); + logger.info("[{}] stats: {}", topic, jsonPrettyPrint(getJsonResourceAsString(topicStatsUri))); + String topicStatsInternalUri = + String.format("%s/admin/v2/persistent/%s/%s/%s/internalStats", baseUrl, tenant, namespace, topic); + logger.info("[{}] internalStats: {}", topic, jsonPrettyPrint(getJsonResourceAsString(topicStatsInternalUri))); + } + + /** + * Pretty print the given JSON string + * @param jsonString JSON string to pretty print + * @return pretty printed JSON string + */ + public static String jsonPrettyPrint(String jsonString) { + try { + ObjectMapper mapper = new ObjectMapper(); + Object json = mapper.readValue(jsonString, Object.class); + ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter(); + return writer.writeValueAsString(json); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + /** + * Get the resource as a string from the given URI + */ + @SneakyThrows + public static String getJsonResourceAsString(String uri) { + URL url = new URL(uri); + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod("GET"); + connection.setRequestProperty("Accept", "application/json"); + try { + int responseCode = connection.getResponseCode(); + if (responseCode == 200) { + try (BufferedReader in = new BufferedReader(new InputStreamReader(connection.getInputStream()))) { + String inputLine; + StringBuilder content = new StringBuilder(); + while ((inputLine = in.readLine()) != null) { + content.append(inputLine); + } + return content.toString(); + } + } else { + throw new IOException("Failed to get resource: " + uri + ", status: " + responseCode); + } + } finally { + connection.disconnect(); + } + } + /** * Receive messages concurrently from multiple consumers and handles them using the provided message handler. * The message handler should return true if it wants to continue receiving more messages, false otherwise. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/AuthenticatedConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/AuthenticatedConsumerStatsTest.java index e8cadb72e1e04..20c1c5498ce6d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/AuthenticatedConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/AuthenticatedConsumerStatsTest.java @@ -18,11 +18,19 @@ */ package org.apache.pulsar.broker.stats; -import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.Sets; import io.jsonwebtoken.Jwts; import io.jsonwebtoken.SignatureAlgorithm; +import java.security.KeyPair; +import java.security.KeyPairGenerator; +import java.security.NoSuchAlgorithmException; +import java.security.PrivateKey; +import java.time.Duration; +import java.util.Base64; +import java.util.Date; +import java.util.HashSet; +import java.util.Properties; +import java.util.Set; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; import org.apache.pulsar.client.admin.PulsarAdminBuilder; import org.apache.pulsar.client.api.AuthenticationFactory; @@ -37,18 +45,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.security.KeyPair; -import java.security.KeyPairGenerator; -import java.security.NoSuchAlgorithmException; -import java.security.PrivateKey; -import java.time.Duration; -import java.util.Base64; -import java.util.Date; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Properties; -import java.util.Set; - public class AuthenticatedConsumerStatsTest extends ConsumerStatsTest{ private final String ADMIN_TOKEN; private final String TOKEN_PUBLIC_KEY; @@ -115,32 +111,6 @@ protected void setup() throws Exception { @Test public void testConsumerStatsOutput() throws Exception { - Set allowedFields = Sets.newHashSet( - "msgRateOut", - "msgThroughputOut", - "bytesOutCounter", - "msgOutCounter", - "messageAckRate", - "msgRateRedeliver", - "chunkedMessageRate", - "consumerName", - "availablePermits", - "unackedMessages", - "avgMessagesPerEntry", - "blockedConsumerOnUnackedMsgs", - "readPositionWhenJoining", - "lastAckedTime", - "lastAckedTimestamp", - "lastConsumedTime", - "lastConsumedTimestamp", - "lastConsumedFlowTimestamp", - "keyHashRanges", - "metadata", - "address", - "connectedSince", - "clientVersion", - "appId"); - final String topicName = "persistent://public/default/testConsumerStatsOutput"; final String subName = "my-subscription"; @@ -154,13 +124,6 @@ public void testConsumerStatsOutput() throws Exception { ObjectMapper mapper = ObjectMapperFactory.create(); ConsumerStats consumerStats = stats.getSubscriptions() .get(subName).getConsumers().get(0); - Assert.assertTrue(consumerStats.getLastConsumedFlowTimestamp() > 0); - JsonNode node = mapper.readTree(mapper.writer().writeValueAsString(consumerStats)); - Iterator itr = node.fieldNames(); - while (itr.hasNext()) { - String field = itr.next(); - Assert.assertTrue(allowedFields.contains(field), field + " should not be exposed"); - } // assert that role is exposed Assert.assertEquals(consumerStats.getAppId(), "admin"); consumer.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index 5b2998216e8e1..59a911500e5d9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -18,9 +18,12 @@ */ package org.apache.pulsar.broker.stats; +import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.InstanceOfAssertFactories.INTEGER; import static org.mockito.Mockito.mock; import static org.testng.Assert.assertNotEquals; import static org.testng.AssertJUnit.assertEquals; @@ -31,22 +34,29 @@ import java.io.ByteArrayOutputStream; import java.lang.reflect.Field; import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; -import java.util.Iterator; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.PrometheusMetricsTestUtil; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.PendingAcksMap; +import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; +import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -67,13 +77,19 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.nar.NarClassLoader; import org.apache.pulsar.common.policies.data.ConsumerStats; +import org.apache.pulsar.common.policies.data.DrainingHash; +import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.assertj.core.groups.Tuple; +import org.awaitility.Awaitility; import org.testng.Assert; +import org.testng.SkipException; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -218,9 +234,24 @@ public void testUpdateStatsForActiveConsumerAndSubscription() throws Exception { Assert.assertEquals(updatedStats.getBytesOutCounter(), 1280); } - @Test - public void testConsumerStatsOutput() throws Exception { - Set allowedFields = Sets.newHashSet( + @DataProvider(name = "classicAndSubscriptionType") + public Object[][] classicAndSubscriptionType() { + return new Object[][]{ + {false, SubscriptionType.Shared}, + {true, SubscriptionType.Key_Shared}, + {false, SubscriptionType.Key_Shared} + }; + } + + @Test(dataProvider = "classicAndSubscriptionType") + public void testConsumerStatsOutput(boolean classicDispatchers, SubscriptionType subscriptionType) + throws Exception { + if (this instanceof AuthenticatedConsumerStatsTest) { + throw new SkipException("Skip test for AuthenticatedConsumerStatsTest"); + } + conf.setSubscriptionSharedUseClassicPersistentImplementation(classicDispatchers); + conf.setSubscriptionKeySharedUseClassicPersistentImplementation(classicDispatchers); + Set expectedFields = Sets.newHashSet( "msgRateOut", "msgThroughputOut", "bytesOutCounter", @@ -233,21 +264,56 @@ public void testConsumerStatsOutput() throws Exception { "unackedMessages", "avgMessagesPerEntry", "blockedConsumerOnUnackedMsgs", - "readPositionWhenJoining", "lastAckedTime", "lastAckedTimestamp", "lastConsumedTime", "lastConsumedTimestamp", "lastConsumedFlowTimestamp", - "keyHashRanges", "metadata", "address", "connectedSince", - "clientVersion"); + "clientVersion", + "drainingHashesCount", + "drainingHashesClearedTotal", + "drainingHashesUnackedMessages" + ); + if (subscriptionType == SubscriptionType.Key_Shared) { + if (classicDispatchers) { + expectedFields.addAll(List.of( + "readPositionWhenJoining", + "keyHashRanges" + )); + } else { + expectedFields.addAll(List.of( + "drainingHashes", + "keyHashRangeArrays" + )); + } + } + final String topicName = newUniqueName("persistent://my-property/my-ns/testConsumerStatsOutput"); + final String subName = "my-subscription"; + + @Cleanup + Consumer consumer = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionType(subscriptionType) + .subscriptionName(subName) + .subscribe(); + + String topicStatsUri = + String.format("%s/admin/v2/%s/stats", pulsar.getWebServiceAddress(), topicName.replace("://", "/")); + String topicStatsJson = BrokerTestUtil.getJsonResourceAsString(topicStatsUri); + ObjectMapper mapper = ObjectMapperFactory.create(); + JsonNode node = mapper.readTree(topicStatsJson).get("subscriptions").get(subName).get("consumers").get(0); + assertThat(node.fieldNames()).toIterable().containsExactlyInAnyOrderElementsOf(expectedFields); + } - final String topicName = "persistent://prop/use/ns-abc/testConsumerStatsOutput"; + @Test + public void testLastConsumerFlowTimestamp() throws PulsarClientException, PulsarAdminException { + final String topicName = newUniqueName("persistent://my-property/my-ns/testLastConsumerFlowTimestamp"); final String subName = "my-subscription"; + @Cleanup Consumer consumer = pulsarClient.newConsumer() .topic(topicName) .subscriptionType(SubscriptionType.Shared) @@ -255,18 +321,9 @@ public void testConsumerStatsOutput() throws Exception { .subscribe(); TopicStats stats = admin.topics().getStats(topicName); - ObjectMapper mapper = ObjectMapperFactory.create(); ConsumerStats consumerStats = stats.getSubscriptions() .get(subName).getConsumers().get(0); Assert.assertTrue(consumerStats.getLastConsumedFlowTimestamp() > 0); - JsonNode node = mapper.readTree(mapper.writer().writeValueAsString(consumerStats)); - Iterator itr = node.fieldNames(); - while (itr.hasNext()) { - String field = itr.next(); - Assert.assertTrue(allowedFields.contains(field), field + " should not be exposed"); - } - - consumer.close(); } @@ -481,4 +538,189 @@ public void testNonPersistentTopicSharedSubscriptionUnackedMessages() throws Exc assertEquals(0, consumers.get(0).getUnackedMessages()); } + @Test + public void testKeySharedDrainingHashesConsumerStats() throws Exception { + String topic = newUniqueName("testKeySharedDrainingHashesConsumerStats"); + String subscriptionName = "sub"; + int numberOfKeys = 10; + + // Create a producer for the topic + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .enableBatching(false) + .create(); + + // Create the first consumer (c1) for the topic + @Cleanup + Consumer c1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c1") + .receiverQueueSize(100) + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + + // Get the dispatcher and selector for the topic + StickyKeyDispatcher dispatcher = getDispatcher(topic, subscriptionName); + StickyKeyConsumerSelector selector = dispatcher.getSelector(); + + // Send 20 messages with keys cycling from 0 to numberOfKeys-1 + for (int i = 0; i < 20; i++) { + String key = String.valueOf(i % numberOfKeys); + int stickyKeyHash = selector.makeStickyKeyHash(key.getBytes(StandardCharsets.UTF_8)); + log.info("Sending message with value {} key {} hash {}", key, i, stickyKeyHash); + producer.newMessage() + .key(key) + .value(i) + .send(); + } + + // Wait until all the already published messages have been pre-fetched by c1 + PendingAcksMap c1PendingAcks = dispatcher.getConsumers().get(0).getPendingAcks(); + Awaitility.await().ignoreExceptions().until(() -> c1PendingAcks.size() == 20); + + // Add a new consumer (c2) for the topic + @Cleanup + Consumer c2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .consumerName("c2") + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + + // Get the subscription stats and consumer stats + SubscriptionStats subscriptionStats = admin.topics().getStats(topic).getSubscriptions().get(subscriptionName); + ConsumerStats c1Stats = subscriptionStats.getConsumers().get(0); + ConsumerStats c2Stats = subscriptionStats.getConsumers().get(1); + + Set c2HashesByStats = new HashSet<>(); + Set c2HashesByDispatcher = new HashSet<>(); + Map c1DrainingHashesExpected = new HashMap<>(); + + int expectedDrainingHashesUnackMessages = 0; + // Determine which hashes are assigned to c2 and which are draining from c1 + // run for the same keys as the sent messages + for (int i = 0; i < 20; i++) { + // use the same key as in the sent messages + String key = String.valueOf(i % numberOfKeys); + int hash = selector.makeStickyKeyHash(key.getBytes(StandardCharsets.UTF_8)); + // Validate that the hash is assigned to c2 in stats + if ("c2".equals(findConsumerNameForHash(subscriptionStats, hash))) { + c2HashesByStats.add(hash); + } + // use the selector to determine the expected draining hashes for c1 + org.apache.pulsar.broker.service.Consumer selected = selector.select(hash); + if ("c2".equals(selected.consumerName())) { + c2HashesByDispatcher.add(hash); + c1DrainingHashesExpected.compute(hash, (k, v) -> v == null ? 1 : v + 1); + expectedDrainingHashesUnackMessages++; + } + } + + // Validate that the hashes assigned to c2 match between stats and dispatcher + assertThat(c2HashesByStats).containsExactlyInAnyOrderElementsOf(c2HashesByDispatcher); + + // Validate the draining hashes for c1 + assertThat(c1Stats.getDrainingHashes()).extracting(DrainingHash::getHash) + .containsExactlyInAnyOrderElementsOf(c2HashesByStats); + assertThat(c1Stats.getDrainingHashes()).extracting(DrainingHash::getHash, DrainingHash::getUnackMsgs) + .containsExactlyInAnyOrderElementsOf(c1DrainingHashesExpected.entrySet().stream() + .map(e -> Tuple.tuple(e.getKey(), e.getValue())).toList()); + + // Validate that c2 has no draining hashes + assertThat(c2Stats.getDrainingHashes()).isEmpty(); + + // Validate counters + assertThat(c1Stats.getDrainingHashesCount()).isEqualTo(c2HashesByStats.size()); + assertThat(c1Stats.getDrainingHashesClearedTotal()).isEqualTo(0); + assertThat(c1Stats.getDrainingHashesUnackedMessages()).isEqualTo(expectedDrainingHashesUnackMessages); + assertThat(c2Stats.getDrainingHashesCount()).isEqualTo(0); + assertThat(c2Stats.getDrainingHashesClearedTotal()).isEqualTo(0); + assertThat(c2Stats.getDrainingHashesUnackedMessages()).isEqualTo(0); + + // Send another 20 messages + for (int i = 0; i < 20; i++) { + producer.newMessage() + .key(String.valueOf(i % numberOfKeys)) + .value(i) + .send(); + } + + // Validate blocked attempts for c1 + Awaitility.await().ignoreExceptions().untilAsserted(() -> { + SubscriptionStats stats = admin.topics().getStats(topic).getSubscriptions().get(subscriptionName); + assertThat(stats.getConsumers().get(0).getDrainingHashes()).isNotEmpty().allSatisfy(dh -> { + assertThat(dh).extracting(DrainingHash::getBlockedAttempts) + .asInstanceOf(INTEGER) + .isGreaterThan(0); + }); + }); + + // Acknowledge messages that were sent before c2 joined, to clear all draining hashes + for (int i = 0; i < 20; i++) { + Message message = c1.receive(1, TimeUnit.SECONDS); + log.info("Acking message with value {} key {}", message.getValue(), message.getKey()); + c1.acknowledge(message); + + if (i == 18) { + // Validate that there is one draining hash left + Awaitility.await().pollInterval(Duration.ofSeconds(1)).atMost(Duration.ofSeconds(3)) + .untilAsserted(() -> { + SubscriptionStats stats = + admin.topics().getStats(topic).getSubscriptions().get(subscriptionName); + assertThat(stats.getConsumers().get(0)).satisfies(consumerStats -> { + assertThat(consumerStats) + .describedAs("Consumer stats should have one draining hash %s", consumerStats) + .extracting(ConsumerStats::getDrainingHashes) + .asList().hasSize(1); + }); + }); + } + + if (i == 19) { + // Validate that there are no draining hashes left + Awaitility.await().pollInterval(Duration.ofSeconds(1)).atMost(Duration.ofSeconds(3)) + .untilAsserted(() -> { + SubscriptionStats stats = + admin.topics().getStats(topic).getSubscriptions().get(subscriptionName); + assertThat(stats.getConsumers().get(0)).satisfies(consumerStats -> { + assertThat(consumerStats).extracting(ConsumerStats::getDrainingHashes) + .asList().isEmpty(); + }); + }); + } + } + + // Get the subscription stats and consumer stats + subscriptionStats = admin.topics().getStats(topic).getSubscriptions().get(subscriptionName); + c1Stats = subscriptionStats.getConsumers().get(0); + c2Stats = subscriptionStats.getConsumers().get(1); + + // Validate counters + assertThat(c1Stats.getDrainingHashesCount()).isEqualTo(0); + assertThat(c1Stats.getDrainingHashesClearedTotal()).isEqualTo(c2HashesByStats.size()); + assertThat(c1Stats.getDrainingHashesUnackedMessages()).isEqualTo(0); + assertThat(c2Stats.getDrainingHashesCount()).isEqualTo(0); + assertThat(c2Stats.getDrainingHashesClearedTotal()).isEqualTo(0); + assertThat(c2Stats.getDrainingHashesUnackedMessages()).isEqualTo(0); + + } + + private String findConsumerNameForHash(SubscriptionStats subscriptionStats, int hash) { + return findConsumerForHash(subscriptionStats, hash).map(ConsumerStats::getConsumerName).orElse(null); + } + + private Optional findConsumerForHash(SubscriptionStats subscriptionStats, int hash) { + return subscriptionStats.getConsumers().stream() + .filter(consumerStats -> consumerStats.getKeyHashRangeArrays().stream() + .anyMatch(hashRanges -> hashRanges[0] <= hash && hashRanges[1] >= hash)) + .findFirst(); + } + + @SneakyThrows + private StickyKeyDispatcher getDispatcher(String topic, String subscription) { + return (StickyKeyDispatcher) pulsar.getBrokerService().getTopicIfExists(topic).get() + .get().getSubscription(subscription).getDispatcher(); + } } diff --git a/pulsar-broker/src/test/resources/log4j2.xml b/pulsar-broker/src/test/resources/log4j2.xml index 09a89702ee2ac..a0732096f2845 100644 --- a/pulsar-broker/src/test/resources/log4j2.xml +++ b/pulsar-broker/src/test/resources/log4j2.xml @@ -36,5 +36,16 @@ + diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java index d2d3600df96ed..16dce5903f492 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/ConsumerStats.java @@ -73,8 +73,41 @@ public interface ConsumerStats { boolean isBlockedConsumerOnUnackedMsgs(); /** The read position of the cursor when the consumer joining. */ + @Deprecated String getReadPositionWhenJoining(); + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the current number of hashes in the draining state for this consumer. + * + * @return the current number of hashes in the draining state for this consumer + */ + int getDrainingHashesCount(); + + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the total number of hashes cleared from the draining state since the consumer connected. + * + * @return the total number of hashes cleared from the draining state since the consumer connected + */ + long getDrainingHashesClearedTotal(); + + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the total number of unacked messages for all draining hashes for this consumer. + * + * @return the total number of unacked messages for all draining hashes for this consumer + */ + int getDrainingHashesUnackedMessages(); + + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the draining hashes for this consumer. + * + * @return a list of draining hashes for this consumer + */ + List getDrainingHashes(); + /** Address of this consumer. */ String getAddress(); @@ -88,9 +121,20 @@ public interface ConsumerStats { long getLastConsumedTimestamp(); long getLastConsumedFlowTimestamp(); - /** Hash ranges assigned to this consumer if is Key_Shared sub mode. **/ + /** + * Hash ranges assigned to this consumer if in Key_Shared subscription mode. + * This format and field is used when `subscriptionKeySharedUseClassicPersistentImplementation` is set to `false` + * (default). + */ + List getKeyHashRangeArrays(); + + /** + * Hash ranges assigned to this consumer if in Key_Shared subscription mode. + * This format and field is used when `subscriptionKeySharedUseClassicPersistentImplementation` is set to `true`. + */ + @Deprecated List getKeyHashRanges(); /** Metadata (key/value strings) associated with this consumer. */ Map getMetadata(); -} +} \ No newline at end of file diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DrainingHash.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DrainingHash.java new file mode 100644 index 0000000000000..685b0b74e64b9 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/DrainingHash.java @@ -0,0 +1,41 @@ +/* + * 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.policies.data; + +/** + * Contains information about a draining hash in a Key_Shared subscription. + * @see ConsumerStats + */ +public interface DrainingHash { + /** + * Get the sticky key hash value of the draining hash. + * @return the sticky hash value + */ + int getHash(); + /** + * Get number of unacknowledged messages for the draining hash. + * @return number of unacknowledged messages + */ + int getUnackMsgs(); + /** + * Get the number of times the hash has blocked an attempted delivery of a message. + * @return number of times the hash has blocked an attempted delivery of a message + */ + int getBlockedAttempts(); +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index ce3a080a855da..95e7c65266bff 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -121,6 +121,30 @@ public interface SubscriptionStats { /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ Map getConsumersAfterMarkDeletePosition(); + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the current number of hashes in the draining state. + * + * @return the current number of hashes in the draining state + */ + int getDrainingHashesCount(); + + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the total number of hashes cleared from the draining state for the connected consumers. + * + * @return the total number of hashes cleared from the draining state for the connected consumers + */ + long getDrainingHashesClearedTotal(); + + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the total number of unacked messages for all draining hashes. + * + * @return the total number of unacked messages for all draining hashes + */ + int getDrainingHashesUnackedMessages(); + /** SubscriptionProperties (key/value strings) associated with this subscribe. */ Map getSubscriptionProperties(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java index de36b330b7f1a..8811247cb2de3 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/ConsumerStatsImpl.java @@ -23,6 +23,7 @@ import java.util.Objects; import lombok.Data; import org.apache.pulsar.common.policies.data.ConsumerStats; +import org.apache.pulsar.common.policies.data.DrainingHash; import org.apache.pulsar.common.util.DateFormatter; /** @@ -80,6 +81,30 @@ public class ConsumerStatsImpl implements ConsumerStats { /** The read position of the cursor when the consumer joining. */ public String readPositionWhenJoining; + /** + * For Key_Shared AUTO_SPLIT ordered subscriptions: The current number of hashes in the draining state. + */ + public int drainingHashesCount; + + /** + * For Key_Shared AUTO_SPLIT ordered subscriptions: The total number of hashes cleared from the draining state for + * the consumer. + */ + public long drainingHashesClearedTotal; + + /** + * For Key_Shared AUTO_SPLIT ordered subscriptions: The total number of unacked messages for all draining hashes. + */ + public int drainingHashesUnackedMessages; + + /** + * For Key_Shared subscription in AUTO_SPLIT ordered mode: + * Retrieves the draining hashes for this consumer. + * + * @return a list of draining hashes for this consumer + */ + public List drainingHashes; + /** Address of this consumer. */ private String address; /** Timestamp of connection. */ @@ -96,7 +121,17 @@ public class ConsumerStatsImpl implements ConsumerStats { public long lastConsumedFlowTimestamp; - /** Hash ranges assigned to this consumer if is Key_Shared sub mode. **/ + /** + * Hash ranges assigned to this consumer if in Key_Shared subscription mode. + * This format and field is used when `subscriptionKeySharedUseClassicPersistentImplementation` is set to `false` + * (default). + */ + public List keyHashRangeArrays; + + /** + * Hash ranges assigned to this consumer if in Key_Shared subscription mode. + * This format and field is used when `subscriptionKeySharedUseClassicPersistentImplementation` is set to `true`. + */ public List keyHashRanges; /** Metadata (key/value strings) associated with this consumer. */ @@ -114,6 +149,12 @@ public ConsumerStatsImpl add(ConsumerStatsImpl stats) { this.unackedMessages += stats.unackedMessages; this.blockedConsumerOnUnackedMsgs = stats.blockedConsumerOnUnackedMsgs; this.readPositionWhenJoining = stats.readPositionWhenJoining; + this.drainingHashesCount = stats.drainingHashesCount; + this.drainingHashesClearedTotal += stats.drainingHashesClearedTotal; + this.drainingHashesUnackedMessages = stats.drainingHashesUnackedMessages; + this.drainingHashes = stats.drainingHashes; + this.keyHashRanges = stats.keyHashRanges; + this.keyHashRangeArrays = stats.keyHashRangeArrays; return this; } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/DrainingHashImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/DrainingHashImpl.java new file mode 100644 index 0000000000000..134bdac597b7c --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/DrainingHashImpl.java @@ -0,0 +1,46 @@ +/* + * 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.policies.data.stats; + +import lombok.Data; +import org.apache.pulsar.common.policies.data.ConsumerStats; +import org.apache.pulsar.common.policies.data.DrainingHash; + +/** + * Contains information about a draining hash in a Key_Shared subscription. + * @see ConsumerStats + */ +@Data +public class DrainingHashImpl implements DrainingHash { + /** + * Get the sticky key hash value of the draining hash. + * @return the sticky hash value + */ + public int hash; + /** + * Get number of unacknowledged messages for the draining hash. + * @return number of unacknowledged messages + */ + public int unackMsgs; + /** + * Get the number of times the hash has blocked an attempted delivery of a message. + * @return number of times the hash has blocked an attempted delivery of a message + */ + public int blockedAttempts; +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java index 12734a5586cef..02df9b7870023 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/SubscriptionStatsImpl.java @@ -129,6 +129,22 @@ public class SubscriptionStatsImpl implements SubscriptionStats { /** This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. */ public Map consumersAfterMarkDeletePosition; + /** + * For Key_Shared AUTO_SPLIT ordered subscriptions: The current number of hashes in the draining state. + */ + public int drainingHashesCount; + + /** + * For Key_Shared AUTO_SPLIT ordered subscriptions: The total number of hashes cleared from the draining state + * for the connected consumers. + */ + public long drainingHashesClearedTotal; + + /** + * For Key_Shared AUTO_SPLIT ordered subscriptions: The total number of unacked messages for all draining hashes. + */ + public int drainingHashesUnackedMessages; + /** The number of non-contiguous deleted messages ranges. */ public int nonContiguousDeletedMessagesRanges; @@ -180,6 +196,9 @@ public void reset() { lastMarkDeleteAdvancedTimestamp = 0L; consumers.clear(); consumersAfterMarkDeletePosition.clear(); + drainingHashesCount = 0; + drainingHashesClearedTotal = 0L; + drainingHashesUnackedMessages = 0; nonContiguousDeletedMessagesRanges = 0; nonContiguousDeletedMessagesRangesSerializedSize = 0; earliestMsgPublishTimeInBacklog = 0L; @@ -226,6 +245,9 @@ public SubscriptionStatsImpl add(SubscriptionStatsImpl stats) { } this.allowOutOfOrderDelivery |= stats.allowOutOfOrderDelivery; this.consumersAfterMarkDeletePosition.putAll(stats.consumersAfterMarkDeletePosition); + this.drainingHashesCount += stats.drainingHashesCount; + this.drainingHashesClearedTotal += stats.drainingHashesClearedTotal; + this.drainingHashesUnackedMessages += stats.drainingHashesUnackedMessages; this.nonContiguousDeletedMessagesRanges += stats.nonContiguousDeletedMessagesRanges; this.nonContiguousDeletedMessagesRangesSerializedSize += stats.nonContiguousDeletedMessagesRangesSerializedSize; if (this.earliestMsgPublishTimeInBacklog != 0 && stats.earliestMsgPublishTimeInBacklog != 0) { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java index 7b235cfa341d1..b737d68d5ea9f 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/ObjectMapperFactory.java @@ -56,6 +56,7 @@ import org.apache.pulsar.common.policies.data.ConsumerStats; import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; import org.apache.pulsar.common.policies.data.DispatchRate; +import org.apache.pulsar.common.policies.data.DrainingHash; import org.apache.pulsar.common.policies.data.FailureDomain; import org.apache.pulsar.common.policies.data.FailureDomainImpl; import org.apache.pulsar.common.policies.data.FunctionInstanceStats; @@ -96,6 +97,7 @@ import org.apache.pulsar.common.policies.data.impl.DelayedDeliveryPoliciesImpl; import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; +import org.apache.pulsar.common.policies.data.stats.DrainingHashImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentPartitionedTopicStatsImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentPublisherStatsImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentReplicatorStatsImpl; @@ -243,6 +245,7 @@ private static void setAnnotationsModule(ObjectMapper mapper) { resolver.addMapping(DispatchRate.class, DispatchRateImpl.class); resolver.addMapping(TopicStats.class, TopicStatsImpl.class); resolver.addMapping(ConsumerStats.class, ConsumerStatsImpl.class); + resolver.addMapping(DrainingHash.class, DrainingHashImpl.class); resolver.addMapping(NonPersistentPublisherStats.class, NonPersistentPublisherStatsImpl.class); resolver.addMapping(NonPersistentReplicatorStats.class, NonPersistentReplicatorStatsImpl.class); resolver.addMapping(NonPersistentSubscriptionStats.class, NonPersistentSubscriptionStatsImpl.class); From 9f8b4a6d15899c7255c525299e1d6011a09b940e Mon Sep 17 00:00:00 2001 From: Masahiro Sakamoto Date: Thu, 10 Oct 2024 19:03:56 +0900 Subject: [PATCH 578/580] [fix][broker] Fix ack hole in cursor for geo-replication (#20931) Co-authored-by: Masahiro Sakamoto --- .../persistent/PersistentReplicator.java | 25 ++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index b3d7546beed81..bcb1f759540b4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -116,6 +116,7 @@ public abstract class PersistentReplicator extends AbstractReplicator protected final ReplicatorStatsImpl stats = new ReplicatorStatsImpl(); protected volatile boolean fetchSchemaInProgress = false; + private volatile boolean waitForCursorRewinding = false; public PersistentReplicator(String localCluster, PersistentTopic localTopic, ManagedCursor cursor, String remoteCluster, String remoteTopic, @@ -143,9 +144,15 @@ public PersistentReplicator(String localCluster, PersistentTopic localTopic, Man @Override protected void setProducerAndTriggerReadEntries(Producer producer) { - // Rewind the cursor to be sure to read again all non-acked messages sent while restarting. - cursor.rewind(); - cursor.cancelPendingReadRequest(); + waitForCursorRewinding = true; + + // Repeat until there are no read operations in progress + if (STATE_UPDATER.get(this) == State.Starting && HAVE_PENDING_READ_UPDATER.get(this) == TRUE + && !cursor.cancelPendingReadRequest()) { + brokerService.getPulsar().getExecutor() + .schedule(() -> setProducerAndTriggerReadEntries(producer), 10, TimeUnit.MILLISECONDS); + return; + } /** * 1. Try change state to {@link Started}. @@ -158,6 +165,7 @@ protected void setProducerAndTriggerReadEntries(Producer producer) { if (!(producer instanceof ProducerImpl)) { log.error("[{}] The partitions count between two clusters is not the same, the replicator can not be" + " created successfully: {}", replicatorId, state); + waitForCursorRewinding = false; doCloseProducerAsync(producer, () -> {}); throw new ClassCastException(producer.getClass().getName() + " can not be cast to ProducerImpl"); } @@ -168,6 +176,11 @@ protected void setProducerAndTriggerReadEntries(Producer producer) { backOff.reset(); // activate cursor: so, entries can be cached. this.cursor.setActive(); + + // Rewind the cursor to be sure to read again all non-acked messages sent while restarting + cursor.rewind(); + waitForCursorRewinding = false; + // read entries readMoreEntries(); } else { @@ -183,6 +196,7 @@ protected void setProducerAndTriggerReadEntries(Producer producer) { log.error("[{}] Replicator state is not expected, so close the producer. Replicator state: {}", replicatorId, changeStateRes.getRight()); } + waitForCursorRewinding = false; // Close the producer if change the state fail. doCloseProducerAsync(producer, () -> {}); } @@ -296,6 +310,11 @@ protected void readMoreEntries() { // Schedule read if (HAVE_PENDING_READ_UPDATER.compareAndSet(this, FALSE, TRUE)) { + if (waitForCursorRewinding) { + log.info("[{}] Skip the reading because repl producer is starting", replicatorId); + HAVE_PENDING_READ_UPDATER.set(this, FALSE); + return; + } if (log.isDebugEnabled()) { log.debug("[{}] Schedule read of {} messages or {} bytes", replicatorId, messagesToRead, bytesToRead); From 8600316113662460fcd559bae85a1646f4862083 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 10 Oct 2024 13:18:43 +0300 Subject: [PATCH 579/580] [improve][client] Increase default Java client connectionMaxIdleSeconds to 60 seconds (#23430) --- .../apache/pulsar/client/impl/conf/ClientConfigurationData.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java index c1c2e75925502..c4fbf1e1744d6 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ClientConfigurationData.java @@ -139,7 +139,7 @@ public class ClientConfigurationData implements Serializable, Cloneable { value = "Release the connection if it is not used for more than [connectionMaxIdleSeconds] seconds. " + "If [connectionMaxIdleSeconds] < 0, disabled the feature that auto release the idle connections" ) - private int connectionMaxIdleSeconds = 25; + private int connectionMaxIdleSeconds = 60; @ApiModelProperty( name = "useTcpNoDelay", From b334c4f637bdd32787494c16e9d34169f1a25812 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Thu, 10 Oct 2024 18:37:27 +0800 Subject: [PATCH 580/580] [improve][broker] PIP-383: Support granting/revoking permissions for multiple topics (#23372) --- .../authorization/AuthorizationProvider.java | 13 ++++ .../authorization/AuthorizationService.java | 11 +++ .../PulsarAuthorizationProvider.java | 78 +++++++++++++++++++ .../pulsar/broker/admin/AdminResource.java | 11 +++ .../broker/admin/impl/NamespacesBase.java | 75 ++++++++++++++++++ .../pulsar/broker/admin/v2/Namespaces.java | 44 +++++++++++ .../pulsar/broker/admin/AdminApi2Test.java | 59 ++++++++++++++ .../admin/GrantTopicPermissionOptions.java | 36 +++++++++ .../pulsar/client/admin/Namespaces.java | 28 +++++++ .../admin/RevokeTopicPermissionOptions.java | 32 ++++++++ .../client/admin/internal/NamespacesImpl.java | 26 +++++++ 11 files changed, 413 insertions(+) create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/GrantTopicPermissionOptions.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/RevokeTopicPermissionOptions.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java index 7d25580ff92bb..ffb38f770a9cc 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java @@ -20,12 +20,15 @@ import java.io.Closeable; import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.resources.PulsarResources; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; @@ -223,6 +226,16 @@ CompletableFuture revokeSubscriptionPermissionAsync(NamespaceName namespac CompletableFuture grantPermissionAsync(TopicName topicName, Set actions, String role, String authDataJson); + default CompletableFuture grantPermissionAsync(List options) { + return FutureUtil.failedFuture(new IllegalStateException( + String.format("grantPermissionAsync is not supported by the Authorization"))); + } + + default CompletableFuture revokePermissionAsync(List options) { + return FutureUtil.failedFuture(new IllegalStateException( + String.format("revokePermissionAsync is not supported by the Authorization"))); + } + /** * Revoke authorization-action permission on a topic to the given client. diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java index c121d93b9b750..2951eb1f2973f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java @@ -20,6 +20,7 @@ import static java.util.concurrent.TimeUnit.SECONDS; import java.net.SocketAddress; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -32,6 +33,8 @@ import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.AuthenticationParameters; import org.apache.pulsar.broker.resources.PulsarResources; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; @@ -181,6 +184,14 @@ public CompletableFuture grantPermissionAsync(TopicName topicName, Set grantPermissionAsync(List options) { + return provider.grantPermissionAsync(options); + } + + public CompletableFuture revokePermissionAsync(List options) { + return provider.revokePermissionAsync(options); + } + /** * Revoke authorization-action permission on a topic to the given client. * diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java index a39c3d0560760..0af63724cc812 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/PulsarAuthorizationProvider.java @@ -24,14 +24,18 @@ import java.io.IOException; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.stream.Stream; import javax.ws.rs.core.Response; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.resources.PulsarResources; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; @@ -251,6 +255,80 @@ public CompletableFuture grantPermissionAsync(TopicName topicName, Set grantPermissionAsync(List options) { + return checkNamespace(options.stream().map(o -> TopicName.get(o.getTopic()).getNamespace())) + .thenCompose(__ -> getPoliciesReadOnlyAsync()) + .thenCompose(readonly -> { + if (readonly) { + if (log.isDebugEnabled()) { + log.debug("Policies are read-only. Broker cannot do read-write operations"); + } + throw new IllegalStateException("policies are in readonly mode"); + } + TopicName topicName = TopicName.get(options.get(0).getTopic()); + return pulsarResources.getNamespaceResources() + .setPoliciesAsync(topicName.getNamespaceObject(), policies -> { + options.stream().forEach(o -> { + final String topicUri = TopicName.get(o.getTopic()).toString(); + policies.auth_policies.getTopicAuthentication() + .computeIfAbsent(topicUri, __ -> new HashMap<>()) + .put(o.getRole(), o.getActions()); + }); + return policies; + }).whenComplete((__, ex) -> { + if (ex != null) { + log.error("Failed to grant permissions for {}", options); + } else { + log.info("Successfully granted access for {}", options); + } + }); + }); + } + + @Override + public CompletableFuture revokePermissionAsync(List options) { + return checkNamespace(options.stream().map(o -> TopicName.get(o.getTopic()).getNamespace())) + .thenCompose(__ -> getPoliciesReadOnlyAsync()) + .thenCompose(readonly -> { + if (readonly) { + if (log.isDebugEnabled()) { + log.debug("Policies are read-only. Broker cannot do read-write operations"); + } + throw new IllegalStateException("policies are in readonly mode"); + } + TopicName topicName = TopicName.get(options.get(0).getTopic()); + return pulsarResources.getNamespaceResources() + .setPoliciesAsync(topicName.getNamespaceObject(), policies -> { + options.stream().forEach(o -> { + final String topicUri = TopicName.get(o.getTopic()).toString(); + policies.auth_policies.getTopicAuthentication() + .computeIfPresent(topicUri, (topicNameUri, roles) -> { + roles.remove(o.getRole()); + if (roles.isEmpty()) { + return null; + } + return roles; + }); + }); + return policies; + }).whenComplete((__, ex) -> { + if (ex != null) { + log.error("Failed to revoke permissions for {}", options, ex); + } else { + log.info("Successfully revoke permissions for {}", options); + } + }); + }); + } + + private CompletableFuture checkNamespace(Stream namespaces) { + boolean sameNamespace = namespaces.distinct().count() == 1; + if (!sameNamespace) { + throw new IllegalArgumentException("The namespace should be the same"); + } + return CompletableFuture.completedFuture(null); + } + @Override public CompletableFuture revokePermissionAsync(TopicName topicName, String role) { return getPoliciesReadOnlyAsync().thenCompose(readonly -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index 3268f07b13d88..45772dc279bab 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -924,4 +924,15 @@ protected void validateOffloadPolicies(OffloadPoliciesImpl offloadPolicies) { "The bucket must be specified for namespace offload."); } } + + protected CompletableFuture internalCheckTopicExists(TopicName topicName) { + return pulsar().getNamespaceService().checkTopicExists(topicName) + .thenAccept(info -> { + boolean exists = info.isExists(); + info.recycle(); + if (!exists) { + throw new RestException(Status.NOT_FOUND, getTopicNotFoundErrorMessage(topicName.toString())); + } + }); + } } 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 18c80d6bef4bf..d80e2487b4f1c 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 @@ -42,6 +42,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.Stream; import javax.annotation.Nonnull; import javax.ws.rs.WebApplicationException; import javax.ws.rs.container.AsyncResponse; @@ -65,8 +66,10 @@ import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.naming.NamedEntity; @@ -613,6 +616,78 @@ protected CompletableFuture internalGrantPermissionOnNamespaceAsync(String }); } + protected CompletableFuture internalGrantPermissionOnTopicsAsync(List options) { + return checkNamespace(options.stream().map(o -> TopicName.get(o.getTopic()).getNamespace())) + .thenCompose(__ -> validateAdminAccessForTenantAsync( + TopicName.get(options.get(0).getTopic()).getTenant()) + ).thenCompose(__ -> internalCheckTopicExists(options.stream().map(o -> TopicName.get(o.getTopic())))) + .thenCompose(__ -> getAuthorizationService().grantPermissionAsync(options)) + .thenAccept(unused -> log.info("[{}] Successfully granted access for {}", clientAppId(), options)) + .exceptionally(ex -> { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + //The IllegalArgumentException and the IllegalStateException were historically thrown by the + // grantPermissionAsync method, so we catch them here to ensure backwards compatibility. + if (realCause instanceof MetadataStoreException.NotFoundException + || realCause instanceof IllegalArgumentException) { + log.warn("[{}] Failed to grant permissions for namespace {}: does not exist", clientAppId(), + namespaceName, ex); + throw new RestException(Status.NOT_FOUND, "Topic's namespace does not exist"); + } else if (realCause instanceof MetadataStoreException.BadVersionException + || realCause instanceof IllegalStateException) { + log.warn("[{}] Failed to grant permissions for namespace {}: {}", + clientAppId(), namespaceName, ex.getCause().getMessage(), ex); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } else { + log.error("[{}] Failed to grant permissions for namespace {}", + clientAppId(), namespaceName, ex); + throw new RestException(realCause); + } + }); + } + + protected CompletableFuture internalRevokePermissionOnTopicsAsync( + List options) { + return checkNamespace(options.stream().map(o -> TopicName.get(o.getTopic()).getNamespace())) + .thenCompose(__ -> validateAdminAccessForTenantAsync( + TopicName.get(options.get(0).getTopic()).getTenant())) + .thenCompose(__ -> internalCheckTopicExists(options.stream().map(o -> TopicName.get(o.getTopic())))) + .thenCompose(__ -> getAuthorizationService().revokePermissionAsync(options)) + .thenAccept(unused -> log.info("[{}] Successfully revoke access for {}", clientAppId(), options)) + .exceptionally(ex -> { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + //The IllegalArgumentException and the IllegalStateException were historically thrown by the + // grantPermissionAsync method, so we catch them here to ensure backwards compatibility. + if (realCause instanceof MetadataStoreException.NotFoundException + || realCause instanceof IllegalArgumentException) { + log.warn("[{}] Failed to revoke permissions for namespace {}: does not exist", clientAppId(), + namespaceName, ex); + throw new RestException(Status.NOT_FOUND, "Topic's namespace does not exist"); + } else if (realCause instanceof MetadataStoreException.BadVersionException + || realCause instanceof IllegalStateException) { + log.warn("[{}] Failed to revoke permissions for namespace {}: {}", + clientAppId(), namespaceName, ex.getCause().getMessage(), ex); + throw new RestException(Status.CONFLICT, "Concurrent modification"); + } else { + log.error("[{}] Failed to revoke permissions for namespace {}", + clientAppId(), namespaceName, ex); + throw new RestException(realCause); + } + }); + } + + private CompletableFuture checkNamespace(Stream namespaces) { + boolean sameNamespace = namespaces.distinct().count() == 1; + if (!sameNamespace) { + throw new RestException(Status.BAD_REQUEST, "The namespace should be the same"); + } + return CompletableFuture.completedFuture(null); + } + + private CompletableFuture internalCheckTopicExists(Stream topicNameStream) { + List topicNames = topicNameStream.collect(Collectors.toList()); + return CompletableFuture.allOf(topicNames.stream().map(topic -> internalCheckTopicExists(topic)) + .toArray(CompletableFuture[]::new)); + } protected CompletableFuture internalGrantPermissionOnSubscriptionAsync(String subscription, Set roles) { 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 54cceaf09e9fe..36150ee21b32c 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 @@ -50,6 +50,8 @@ import org.apache.pulsar.broker.admin.impl.NamespacesBase; import org.apache.pulsar.broker.admin.impl.OffloaderObjectsScannerUtils; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace.Mode; import org.apache.pulsar.common.naming.NamespaceName; @@ -314,6 +316,48 @@ public void grantPermissionOnNamespace(@Suspended AsyncResponse asyncResponse, }); } + @POST + @Path("/grantPermissionsOnTopics") + @ApiOperation(value = "Grant new permissions to a role on multi-topics.") + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), + @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "tenant/namespace/topic doesn't exit"), + @ApiResponse(code = 500, message = "Internal server error") }) + public void grantPermissionsOnTopics(@Suspended final AsyncResponse asyncResponse, + List options) { + internalGrantPermissionOnTopicsAsync(options) + .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) + .exceptionally(ex -> { + log.error("[{}] Failed to grant permissions {}", + clientAppId(), options, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); + } + + @POST + @Path("/revokePermissionsOnTopics") + @ApiOperation(value = "Revoke new permissions to a role on multi-topics.") + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), + @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "tenant/namespace/topic doesn't exit"), + @ApiResponse(code = 500, message = "Internal server error") }) + public void revokePermissionsOnTopics(@Suspended final AsyncResponse asyncResponse, + List options) { + internalRevokePermissionOnTopicsAsync(options) + .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) + .exceptionally(ex -> { + log.error("[{}] Failed to revoke permissions {}", + clientAppId(), options, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); + } + @POST @Path("/{property}/{namespace}/permissions/subscription/{subscription}") @ApiOperation(hidden = true, value = "Grant a new permission to roles for a subscription." diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java index 900babbecf4ad..3f5ee721a7e6b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApi2Test.java @@ -87,11 +87,13 @@ import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; import org.apache.pulsar.broker.testcontext.MockEntryFilterProvider; import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; import org.apache.pulsar.client.admin.ListNamespaceTopicsOptions; import org.apache.pulsar.client.admin.Mode; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.PulsarAdminException.PreconditionFailedException; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.client.admin.Topics.QueryParam; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -3685,4 +3687,61 @@ public void testIsolationPolicyUnloadsNSWithPrimaryChanged(final String topicTyp List.of(".*", "broker.*") ); } + + @Test + public void testGrantAndRevokePermissions() throws Exception { + + String namespace = newUniqueName(defaultTenant + "/") + "-unload-test-"; + String namespace2 = newUniqueName(defaultTenant + "/") + "-unload-test-"; + admin.namespaces().createNamespace(namespace, Set.of("test")); + admin.namespaces().createNamespace(namespace2, Set.of("test")); + // + final String topic1 = "persistent://" + namespace + "/test1"; + final String topic2 = "persistent://" + namespace + "/test2"; + final String topic3 = "non-persistent://" + namespace + "/test3"; + final String topic4 = "persistent://" + namespace2 + "/test4";; + + admin.topics().createPartitionedTopic(topic1, 3); + admin.topics().createPartitionedTopic(topic2, 3); + admin.topics().createPartitionedTopic(topic3, 3); + admin.topics().createPartitionedTopic(topic4, 3); + pulsarClient.newProducer().topic(topic1).create().close(); + pulsarClient.newProducer().topic(topic2).create().close(); + pulsarClient.newProducer().topic(topic3).create().close(); + pulsarClient.newProducer().topic(topic4).create().close(); + + List grantPermissionOptions = new ArrayList<>(); + grantPermissionOptions.add(GrantTopicPermissionOptions.builder().topic(topic1).role("role1").actions(Set.of(AuthAction.produce)).build()); + grantPermissionOptions.add(GrantTopicPermissionOptions.builder().topic(topic4).role("role4").actions(Set.of(AuthAction.produce)).build()); + try { + admin.namespaces().grantPermissionOnTopics(grantPermissionOptions); + fail("Should go here, because there are two namespaces"); + } catch (Exception ex) { + Assert.assertTrue(ex != null); + } + grantPermissionOptions.clear(); + grantPermissionOptions.add(GrantTopicPermissionOptions.builder().topic(topic1).role("role1").actions(Set.of(AuthAction.produce)).build()); + grantPermissionOptions.add(GrantTopicPermissionOptions.builder().topic(topic2).role("role2").actions(Set.of(AuthAction.consume)).build()); + grantPermissionOptions.add(GrantTopicPermissionOptions.builder().topic(topic3).role("role3").actions(Set.of(AuthAction.produce, AuthAction.consume)).build()); + admin.namespaces().grantPermissionOnTopics(grantPermissionOptions); + + final Map> permissions1 = admin.topics().getPermissions(topic1); + final Map> permissions2 = admin.topics().getPermissions(topic2); + final Map> permissions3 = admin.topics().getPermissions(topic3); + + Assert.assertEquals(permissions1.get("role1"), Set.of(AuthAction.produce)); + Assert.assertEquals(permissions2.get("role2"), Set.of(AuthAction.consume)); + Assert.assertEquals(permissions3.get("role3"), Set.of(AuthAction.produce, AuthAction.consume)); + // + List revokePermissionOptions = new ArrayList<>(); + revokePermissionOptions.add(RevokeTopicPermissionOptions.builder().topic(topic1).role("role1").build()); + revokePermissionOptions.add(RevokeTopicPermissionOptions.builder().topic(topic2).role("role2").build()); + admin.namespaces().revokePermissionOnTopics(revokePermissionOptions); + + final Map> permissions11 = admin.topics().getPermissions(topic1); + final Map> permissions22 = admin.topics().getPermissions(topic2); + + Assert.assertTrue(permissions11.isEmpty()); + Assert.assertTrue(permissions22.isEmpty()); + } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/GrantTopicPermissionOptions.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/GrantTopicPermissionOptions.java new file mode 100644 index 0000000000000..e365a086a771f --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/GrantTopicPermissionOptions.java @@ -0,0 +1,36 @@ +/* + * 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.admin; + +import java.util.Set; +import lombok.Builder; +import lombok.Data; +import org.apache.pulsar.common.policies.data.AuthAction; + +@Data +@Builder +public class GrantTopicPermissionOptions { + + private final String topic; + + private final String role; + + private final Set actions; + +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java index 65124a6a76a8f..28ad852064b4f 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java @@ -703,6 +703,34 @@ public interface Namespaces { */ CompletableFuture grantPermissionOnNamespaceAsync(String namespace, String role, Set actions); + /** + * Grant permissions on topics asynchronously. + * @param options + * @return + */ + CompletableFuture grantPermissionOnTopicsAsync(List options); + + /** + * Grant permissions on topics. + * @param options + * @throws PulsarAdminException + */ + void grantPermissionOnTopics(List options) throws PulsarAdminException; + + /** + * Revoke permissions on topics asynchronously. + * @param options + * @return + */ + CompletableFuture revokePermissionOnTopicsAsync(List options); + + /** + * Revoke permissions on topics. + * @param options + * @throws PulsarAdminException + */ + void revokePermissionOnTopics(List options) throws PulsarAdminException; + /** * Revoke permissions on a namespace. *

diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/RevokeTopicPermissionOptions.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/RevokeTopicPermissionOptions.java new file mode 100644 index 0000000000000..38e33c966b284 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/RevokeTopicPermissionOptions.java @@ -0,0 +1,32 @@ +/* + * 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.admin; + +import lombok.Builder; +import lombok.Data; + +@Data +@Builder +public class RevokeTopicPermissionOptions { + + private final String topic; + + private final String role; + +} diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java index 7d41c7203d2c7..7695abdd4809b 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java @@ -28,9 +28,11 @@ import javax.ws.rs.client.WebTarget; import javax.ws.rs.core.MediaType; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.admin.GrantTopicPermissionOptions; import org.apache.pulsar.client.admin.ListNamespaceTopicsOptions; import org.apache.pulsar.client.admin.Namespaces; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.admin.RevokeTopicPermissionOptions; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.NamespaceName; @@ -288,6 +290,30 @@ public CompletableFuture grantPermissionOnNamespaceAsync( return asyncPostRequest(path, Entity.entity(actions, MediaType.APPLICATION_JSON)); } + @Override + public CompletableFuture grantPermissionOnTopicsAsync(List options) { + final WebTarget base = adminV2Namespaces; + WebTarget path = base.path("/grantPermissionsOnTopics"); + return asyncPostRequest(path, Entity.entity(options, MediaType.APPLICATION_JSON)); + } + + @Override + public void grantPermissionOnTopics(List options) throws PulsarAdminException { + sync(() -> grantPermissionOnTopicsAsync(options)); + } + + @Override + public CompletableFuture revokePermissionOnTopicsAsync(List options) { + final WebTarget base = adminV2Namespaces; + WebTarget path = base.path("/revokePermissionsOnTopics"); + return asyncPostRequest(path, Entity.entity(options, MediaType.APPLICATION_JSON)); + } + + @Override + public void revokePermissionOnTopics(List options) throws PulsarAdminException { + sync(() -> revokePermissionOnTopicsAsync(options)); + } + @Override public void revokePermissionsOnNamespace(String namespace, String role) throws PulsarAdminException { sync(() -> revokePermissionsOnNamespaceAsync(namespace, role));